From 8e8a903e7e1a97cc83afb760151301d4a85042f9 Mon Sep 17 00:00:00 2001 From: "chancelq" Date: Tue, 3 Apr 2018 14:20:07 +0800 Subject: [PATCH] HBASE-20312 A faster, GC-friendly, less memory Concurrent Map for memstore --- .../org/apache/hadoop/hbase/util/UnsafeAccess.java | 1 + .../hadoop/hbase/io/util/MemorySizeUtil.java | 4 +- .../org/apache/hadoop/hbase/ipc/RpcServer.java | 13 +- .../hbase/regionserver/AbstractMemStore.java | 23 +- .../apache/hadoop/hbase/regionserver/CellSet.java | 2 +- .../apache/hadoop/hbase/regionserver/HRegion.java | 1 + .../hadoop/hbase/regionserver/HRegionServer.java | 22 +- .../apache/hadoop/hbase/regionserver/HStore.java | 43 +- .../hadoop/hbase/regionserver/MemStoreFlusher.java | 3 +- .../hadoop/hbase/regionserver/MutableSegment.java | 4 +- .../hbase/regionserver/RegionServerAccounting.java | 11 +- .../apache/hadoop/hbase/regionserver/Segment.java | 10 +- .../hadoop/hbase/regionserver/SegmentFactory.java | 18 + .../regionserver/skiplist/core/AbstractCCSMap.java | 719 +++++++++++++ .../regionserver/skiplist/core/AbstractChunk.java | 188 ++++ .../skiplist/core/AllocatorHandlerRegister.java | 166 +++ .../regionserver/skiplist/core/BaseCCSMap.java | 152 +++ .../regionserver/skiplist/core/BaseTwinCCSMap.java | 126 +++ .../skiplist/core/ByteBufferUtils.java | 92 ++ .../skiplist/core/CCSMapChunkPool.java | 256 +++++ .../regionserver/skiplist/core/CCSMapMetrics.java | 60 ++ .../regionserver/skiplist/core/ChunkUtil.java | 148 +++ .../skiplist/core/CompactedConcurrentSkipList.java | 921 +++++++++++++++++ .../hbase/regionserver/skiplist/core/Constant.java | 38 + .../hbase/regionserver/skiplist/core/HeapMode.java | 25 + .../skiplist/core/HugeOnHeapChunk.java | 48 + .../skiplist/core/IAllocatorHandler.java | 54 + .../hbase/regionserver/skiplist/core/ICCSList.java | 249 +++++ .../hbase/regionserver/skiplist/core/ICCSMap.java | 36 + .../hbase/regionserver/skiplist/core/IChunk.java | 105 ++ .../regionserver/skiplist/core/IChunkVisible.java | 31 + .../regionserver/skiplist/core/IIterCCSList.java | 62 ++ .../skiplist/core/ILevelIndexHeader.java | 57 ++ .../hbase/regionserver/skiplist/core/INode.java | 42 + .../skiplist/core/INodeComparator.java | 83 ++ .../hbase/regionserver/skiplist/core/ISerde.java | 60 ++ .../regionserver/skiplist/core/NodeForKey.java | 51 + .../hbase/regionserver/skiplist/core/NodeStat.java | 35 + .../hbase/regionserver/skiplist/core/NodeUtil.java | 173 ++++ .../regionserver/skiplist/core/OffHeapChunk.java | 41 + .../regionserver/skiplist/core/OnHeapChunk.java | 41 + .../skiplist/core/OnHeapIndexHeader.java | 106 ++ .../regionserver/skiplist/core/SchemaEnum.java | 28 + .../core/SubCompactedConcurrentSkipList.java | 402 ++++++++ .../regionserver/skiplist/core/UnsafeHelp.java | 93 ++ .../skiplist/exception/CCSMapException.java | 42 + .../skiplist/exception/ChunkMissingException.java | 42 + .../skiplist/exception/HeapFullException.java | 42 + .../skiplist/exception/SerdeException.java | 42 + .../hbase/CCSMapCellComparatorDefault.java | 81 ++ .../hbase/CCSMapCellComparatorDirectly.java | 258 +++++ .../skiplist/hbase/CCSMapMemStore.java | 112 +++ .../hbase/CCSMapMetaCellComparatorDefault.java | 81 ++ .../regionserver/skiplist/hbase/CellCCSMap.java | 72 ++ .../regionserver/skiplist/hbase/CellSerde.java | 70 ++ .../skiplist/hbase/ImmutableSegmentOnCCSMap.java | 39 + .../skiplist/hbase/MemstoreLABProxyForCCSMap.java | 102 ++ .../skiplist/hbase/MutableSegmentOnCCSMap.java | 71 ++ .../hbase/io/hfile/TestScannerFromBucketCache.java | 3 + .../hbase/regionserver/TestDefaultMemStore.java | 1 + .../hadoop/hbase/regionserver/TestHStore.java | 3 +- .../hbase/regionserver/TestMemStoreChunkPool.java | 7 +- .../regionserver/TestRegionServerAccounting.java | 32 +- .../regionserver/skiplist/core/MockBaseCCSMap.java | 101 ++ .../skiplist/core/TestAbstractChunk.java | 303 ++++++ .../skiplist/core/TestBaseTwinCCSMap.java | 126 +++ .../skiplist/core/TestByteBufferUtils.java | 153 +++ .../regionserver/skiplist/core/TestCCSMap.java | 387 +++++++ .../skiplist/core/TestCCSMapChunkPool.java | 255 +++++ .../regionserver/skiplist/core/TestChunkUtil.java | 142 +++ .../core/TestCompactedConcurrentSkipList.java | 1062 ++++++++++++++++++++ .../regionserver/skiplist/core/TestNodeUtil.java | 175 ++++ .../skiplist/core/TestOffHeapChunk.java | 39 + .../skiplist/core/TestOnHeapChunk.java | 40 + .../skiplist/core/TestOnHeapHeadIndex.java | 200 ++++ .../regionserver/skiplist/core/TestRandomHelp.java | 96 ++ .../core/TestSubCompactedConcurrentSkipList.java | 403 ++++++++ .../skiplist/hbase/TestCCSMapMemStore.java | 76 ++ .../skiplist/hbase/TestCellCCSMap.java | 460 +++++++++ 79 files changed, 9923 insertions(+), 38 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/AbstractCCSMap.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/AbstractChunk.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/AllocatorHandlerRegister.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/BaseCCSMap.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/BaseTwinCCSMap.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/ByteBufferUtils.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/CCSMapChunkPool.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/CCSMapMetrics.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/ChunkUtil.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/CompactedConcurrentSkipList.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/Constant.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/HeapMode.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/HugeOnHeapChunk.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/IAllocatorHandler.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/ICCSList.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/ICCSMap.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/IChunk.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/IChunkVisible.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/IIterCCSList.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/ILevelIndexHeader.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/INode.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/INodeComparator.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/ISerde.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/NodeForKey.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/NodeStat.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/NodeUtil.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/OffHeapChunk.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/OnHeapChunk.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/OnHeapIndexHeader.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/SchemaEnum.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/SubCompactedConcurrentSkipList.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/UnsafeHelp.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/exception/CCSMapException.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/exception/ChunkMissingException.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/exception/HeapFullException.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/exception/SerdeException.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/CCSMapCellComparatorDefault.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/CCSMapCellComparatorDirectly.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/CCSMapMemStore.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/CCSMapMetaCellComparatorDefault.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/CellCCSMap.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/CellSerde.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/ImmutableSegmentOnCCSMap.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/MemstoreLABProxyForCCSMap.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/MutableSegmentOnCCSMap.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/MockBaseCCSMap.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestAbstractChunk.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestBaseTwinCCSMap.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestByteBufferUtils.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestCCSMap.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestCCSMapChunkPool.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestChunkUtil.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestCompactedConcurrentSkipList.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestNodeUtil.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestOffHeapChunk.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestOnHeapChunk.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestOnHeapHeadIndex.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestRandomHelp.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestSubCompactedConcurrentSkipList.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/TestCCSMapMemStore.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/TestCellCCSMap.java diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAccess.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAccess.java index 486f81beb6..ffba328ebc 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAccess.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAccess.java @@ -431,6 +431,7 @@ public final class UnsafeAccess { } return offset + Bytes.SIZEOF_LONG; } + /** * Put a byte value out to the specified BB position in big-endian format. * @param buf the byte buffer diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/util/MemorySizeUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/util/MemorySizeUtil.java index 471eb469b7..5a1e4460c9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/util/MemorySizeUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/util/MemorySizeUtil.java @@ -23,6 +23,7 @@ import java.lang.management.MemoryUsage; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.regionserver.skiplist.hbase.CCSMapMemStore; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -166,7 +167,8 @@ public class MemorySizeUtil { // this entire size split into Chunks and pooling them in MemstoreLABPoool. We dont want to // create so many on demand off heap chunks. In fact when this off heap size is configured, we // will go with 100% of this size as the pool size - if (MemStoreLAB.isEnabled(conf)) { + // For simplify config, CCSMap use same config with MemStoreLAB + if (MemStoreLAB.isEnabled(conf) || CCSMapMemStore.isEnabled(conf)) { // We are in offheap Memstore use long globalMemStoreLimit = (long) (offheapMSGlobal * 1024 * 1024); // Size in bytes return new Pair<>(globalMemStoreLimit, MemoryType.NON_HEAP); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java index 686d5785cc..9dc2ff603d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java @@ -462,10 +462,15 @@ public abstract class RpcServer implements RpcServerInterface, // increment the number of requests that were exceptions. metrics.exception(e); - if (e instanceof LinkageError) throw new DoNotRetryIOException(e); - if (e instanceof IOException) throw (IOException)e; - LOG.error("Unexpected throwable object ", e); - throw new IOException(e.getMessage(), e); + if (e instanceof LinkageError) { + throw new DoNotRetryIOException(e); + } + if (e instanceof IOException) { + throw (IOException) e; + } + String errorMsg = (e == null ? "nullException" : e.getMessage()); + LOG.error("Unexpected throwable object {}", errorMsg, e); + throw new IOException(errorMsg, e); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java index 9f4fd2f8c4..0bb5c5737b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java @@ -27,6 +27,7 @@ import java.util.SortedSet; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.ExtendedCell; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; @@ -43,8 +44,8 @@ public abstract class AbstractMemStore implements MemStore { private static final long NO_SNAPSHOT_ID = -1; - private final Configuration conf; - private final CellComparator comparator; + protected final Configuration conf; + protected final CellComparator comparator; // active segment absorbs write operations protected volatile MutableSegment active; @@ -52,7 +53,8 @@ public abstract class AbstractMemStore implements MemStore { protected volatile ImmutableSegment snapshot; protected volatile long snapshotId; // Used to track when to flush - private volatile long timeOfOldestEdit; + protected volatile long timeOfOldestEdit; + protected final boolean isMeta; public final static long FIXED_OVERHEAD = (long) ClassSize.OBJECT + (4 * ClassSize.REFERENCE) @@ -75,8 +77,13 @@ public abstract class AbstractMemStore implements MemStore { protected AbstractMemStore(final Configuration conf, final CellComparator c) { this.conf = conf; this.comparator = c; + if (c instanceof CellComparatorImpl.MetaCellComparator) { + isMeta = true; + } else { + isMeta = false; + } resetActive(); - this.snapshot = SegmentFactory.instance().createImmutableSegment(c); + initSnapshot(c); this.snapshotId = NO_SNAPSHOT_ID; } @@ -86,6 +93,10 @@ public abstract class AbstractMemStore implements MemStore { this.timeOfOldestEdit = Long.MAX_VALUE; } + protected void initSnapshot(final CellComparator c) { + this.snapshot = SegmentFactory.instance().createImmutableSegment(c); + } + /** * Updates the wal with the lowest sequence id (oldest entry) that is still in memory * @param onlyIfMoreRecent a flag that marks whether to update the sequence id no matter what or @@ -214,7 +225,7 @@ public abstract class AbstractMemStore implements MemStore { * @param readpoint readpoint below which we can safely remove duplicate KVs * @param memstoreSize */ - private void upsert(Cell cell, long readpoint, MemStoreSizing memstoreSizing) { + protected void upsert(Cell cell, long readpoint, MemStoreSizing memstoreSizing) { // Add the Cell to the MemStore // Use the internalAdd method here since we (a) already have a lock // and (b) cannot safely use the MSLAB here without potentially @@ -300,7 +311,7 @@ public abstract class AbstractMemStore implements MemStore { checkActiveSize(); } - private void setOldestEditTimeToNow() { + protected void setOldestEditTimeToNow() { if (timeOfOldestEdit == Long.MAX_VALUE) { timeOfOldestEdit = EnvironmentEdgeManager.currentTime(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSet.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSet.java index a4fe883668..0a76b06467 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSet.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSet.java @@ -57,7 +57,7 @@ public class CellSet implements NavigableSet { this.numUniqueKeys = UNKNOWN_NUM_UNIQUES; } - CellSet(final NavigableMap m, int numUniqueKeys) { + public CellSet(final NavigableMap m, int numUniqueKeys) { this.delegatee = m; this.numUniqueKeys = numUniqueKeys; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 1fb6afe7ec..b38ff04284 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -1582,6 +1582,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi writestate.writesEnabled = true; } // Have to throw to upper layers. I can't abort server from here. + LOG.debug("Failed flush {} ", this, ioe); throw ioe; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 240de85e77..f1a67e97cd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -129,6 +129,8 @@ import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler; import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler; import org.apache.hadoop.hbase.regionserver.handler.RSProcedureHandler; import org.apache.hadoop.hbase.regionserver.handler.RegionReplicaFlushHandler; +import org.apache.hadoop.hbase.regionserver.skiplist.core.CCSMapChunkPool; +import org.apache.hadoop.hbase.regionserver.skiplist.hbase.CCSMapMemStore; import org.apache.hadoop.hbase.regionserver.throttle.FlushThroughputControllerFactory; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; import org.apache.hadoop.hbase.replication.ReplicationUtils; @@ -1562,6 +1564,18 @@ public class HRegionServer extends HasThread implements } } + private void initializeCCSMapChunkPool() { + Pair pair = MemorySizeUtil.getGlobalMemStoreSize(conf); + boolean offheap = this.regionServerAccounting.isOffheap(); + long globalMemStoreSize = pair.getFirst(); + + CCSMapChunkPool.initialize(conf, globalMemStoreSize, + conf.getInt(CCSMapMemStore.CHUNK_SIZE_KEY, CCSMapMemStore.CHUNK_SIZE_DEFAULT), + conf.getInt(CCSMapMemStore.INITIAL_CHUNK_COUNT_KEY, Integer.MAX_VALUE), offheap); + + // not need Register with Heap Memory manager + } + protected void initializeMemStoreChunkCreator() { if (MemStoreLAB.isEnabled(conf)) { // MSLAB is enabled. So initialize MemStoreChunkPool @@ -1961,8 +1975,14 @@ public class HRegionServer extends HasThread implements // Memstore services. startHeapMemoryManager(); + // Call it after starting HeapMemoryManager. - initializeMemStoreChunkCreator(); + // for simplify the configuration, if we use CCSMap, the chunkPool will auto be closed. + if (CCSMapMemStore.isEnabled(conf)) { + initializeCCSMapChunkPool(); + } else if (MemStoreLAB.isEnabled(conf)) { + initializeMemStoreChunkCreator(); + } } private void initializeThreads() throws IOException { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java index 9e89e5e9cc..05e7df0f5e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java @@ -89,6 +89,7 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl; import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor; import org.apache.hadoop.hbase.regionserver.compactions.OffPeakHours; import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher; +import org.apache.hadoop.hbase.regionserver.skiplist.hbase.CCSMapMemStore; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; import org.apache.hadoop.hbase.regionserver.wal.WALUtil; import org.apache.hadoop.hbase.security.EncryptionUtil; @@ -132,6 +133,9 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDes @InterfaceAudience.Private public class HStore implements Store, HeapSize, StoreConfigInformation, PropagatingConfigurationObserver { public static final String MEMSTORE_CLASS_NAME = "hbase.regionserver.memstore.class"; + public static final String COMPACTING_MEMSTORE_CLASS_NAME = + "hbase.regionserver.compacting.memstore.class"; + public static final String DEFAULT_MEMSTORE_CLASS_NAME = CCSMapMemStore.class.getName(); public static final String COMPACTCHECKER_INTERVAL_MULTIPLIER_KEY = "hbase.server.compactchecker.interval.multiplier"; public static final String BLOCKING_STOREFILES_KEY = "hbase.hstore.blockingStoreFiles"; @@ -336,29 +340,38 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat private MemStore getMemstore() { MemStore ms = null; // Check if in-memory-compaction configured. Note MemoryCompactionPolicy is an enum! - MemoryCompactionPolicy inMemoryCompaction = null; - if (this.getTableName().isSystemTable()) { - inMemoryCompaction = MemoryCompactionPolicy.valueOf( - conf.get("hbase.systemtables.compacting.memstore.type", "NONE")); - } else { - inMemoryCompaction = family.getInMemoryCompaction(); - } + MemoryCompactionPolicy inMemoryCompaction = family.getInMemoryCompaction(); + if (inMemoryCompaction == null) { - inMemoryCompaction = - MemoryCompactionPolicy.valueOf(conf.get(CompactingMemStore.COMPACTING_MEMSTORE_TYPE_KEY, + inMemoryCompaction = MemoryCompactionPolicy.valueOf( + conf.get(CompactingMemStore.COMPACTING_MEMSTORE_TYPE_KEY, CompactingMemStore.COMPACTING_MEMSTORE_TYPE_DEFAULT)); } switch (inMemoryCompaction) { case NONE: - ms = ReflectionUtils.newInstance(DefaultMemStore.class, - new Object[]{conf, this.comparator}); + String className = conf.get(MEMSTORE_CLASS_NAME, DEFAULT_MEMSTORE_CLASS_NAME); + ms = ReflectionUtils.instantiateWithCustomCtor(className, + new Class[] { Configuration.class, CellComparator.class }, + new Object[] { conf, this.comparator }); + break; default: - Class clz = conf.getClass(MEMSTORE_CLASS_NAME, - CompactingMemStore.class, CompactingMemStore.class); - ms = ReflectionUtils.newInstance(clz, new Object[]{conf, this.comparator, this, - this.getHRegion().getRegionServicesForStores(), inMemoryCompaction}); + if (this.getTableName().isSystemTable() && MemoryCompactionPolicy + .valueOf(conf.get("hbase.systemtables.compacting.memstore.type", "NONE")) + == MemoryCompactionPolicy.NONE) { + ms = ReflectionUtils.instantiateWithCustomCtor(DefaultMemStore.class.getName(), + new Class[] { Configuration.class, CellComparator.class }, + new Object[] { conf, this.comparator }); + } else { + Class clz = + conf.getClass(COMPACTING_MEMSTORE_CLASS_NAME, CompactingMemStore.class, + CompactingMemStore.class); + ms = ReflectionUtils.newInstance(clz, new Object[] { conf, this.comparator, this, + this.getHRegion().getRegionServicesForStores(), inMemoryCompaction }); + } } + LOG.debug("{}/{} use memStore class = {}", this.getRegionInfo().getRegionNameAsString(), + family.getNameAsString(), ms.getClass().getName()); return ms; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java index ad8aa46d81..d0266e4dc5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java @@ -699,7 +699,8 @@ class MemStoreFlusher implements FlushRequester { server.cacheFlusher.setFlushType(flushType); if (!blocked) { startTime = EnvironmentEdgeManager.currentTime(); - if (!server.getRegionServerAccounting().isOffheap()) { + if (!server.getRegionServerAccounting().isOffheap() && !server + .getRegionServerAccounting().isCCSMapEnable()) { logMsg("global memstore heapsize", server.getRegionServerAccounting().getGlobalMemStoreHeapSize(), server.getRegionServerAccounting().getGlobalMemStoreLimit()); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableSegment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableSegment.java index 1349921a8d..9024521769 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableSegment.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableSegment.java @@ -86,8 +86,8 @@ public class MutableSegment extends Segment { // area. Only the Cell object as such going way. We need to consider cellLen to be // decreased there as 0 only. Just keeping it as existing code now. We need to know the // removed cell is from MSLAB or not. Will do once HBASE-16438 is in - int cellLen = getCellLength(cur); - long heapSize = heapSizeChange(cur, true); + int cellLen = getDataSizeByCell(cur); + long heapSize = getHeapSizeByCell(cur); long offHeapSize = offHeapSizeChange(cur, true); this.incSize(-cellLen, -heapSize, -offHeapSize); if (memStoreSizing != null) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerAccounting.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerAccounting.java index 1c627f74b1..8832109458 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerAccounting.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerAccounting.java @@ -24,6 +24,7 @@ import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.LongAdder; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.regionserver.skiplist.hbase.CCSMapMemStore; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.io.util.MemorySizeUtil; import org.apache.hadoop.hbase.util.Bytes; @@ -52,6 +53,7 @@ public class RegionServerAccounting { private final float globalMemStoreLimitLowMarkPercent; private long globalMemStoreLimitLowMark; private final MemoryType memType; + private final boolean isCCSMapEnable; private long globalOnHeapMemstoreLimit; private long globalOnHeapMemstoreLimitLowMark; @@ -59,6 +61,7 @@ public class RegionServerAccounting { Pair globalMemstoreSizePair = MemorySizeUtil.getGlobalMemStoreSize(conf); this.globalMemStoreLimit = globalMemstoreSizePair.getFirst(); this.memType = globalMemstoreSizePair.getSecond(); + isCCSMapEnable = CCSMapMemStore.isEnabled(conf); this.globalMemStoreLimitLowMarkPercent = MemorySizeUtil.getGlobalMemStoreHeapLowerMark(conf, this.memType == MemoryType.HEAP); // When off heap memstore in use we configure the global off heap space for memstore as bytes @@ -77,6 +80,10 @@ public class RegionServerAccounting { (long) (this.globalOnHeapMemstoreLimit * this.globalMemStoreLimitLowMarkPercent); } + public boolean isCCSMapEnable() { + return isCCSMapEnable; + } + long getGlobalMemStoreLimit() { return this.globalMemStoreLimit; } @@ -154,7 +161,7 @@ public class RegionServerAccounting { public FlushType isAboveHighWaterMark() { // for onheap memstore we check if the global memstore size and the // global heap overhead is greater than the global memstore limit - if (memType == MemoryType.HEAP) { + if (memType == MemoryType.HEAP && !isCCSMapEnable) { if (getGlobalMemStoreHeapSize() >= globalMemStoreLimit) { return FlushType.ABOVE_ONHEAP_HIGHER_MARK; } @@ -185,7 +192,7 @@ public class RegionServerAccounting { public FlushType isAboveLowWaterMark() { // for onheap memstore we check if the global memstore size and the // global heap overhead is greater than the global memstore lower mark limit - if (memType == MemoryType.HEAP) { + if (memType == MemoryType.HEAP && !isCCSMapEnable) { if (getGlobalMemStoreHeapSize() >= globalMemStoreLimitLowMark) { return FlushType.ABOVE_ONHEAP_LOWER_MARK; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java index 517f537d12..325d48633b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java @@ -176,10 +176,18 @@ public abstract class Segment { * Get cell length after serialized in {@link KeyValue} */ @VisibleForTesting - static int getCellLength(Cell cell) { + protected static int getCellLength(Cell cell) { return KeyValueUtil.length(cell); } + protected int getDataSizeByCell(Cell cell) { + return getCellLength(cell); + } + + protected long getHeapSizeByCell(Cell cell) { + return heapSizeChange(cell, true); + } + public boolean shouldSeek(TimeRange tr, long oldestUnexpiredTS) { return !isEmpty() && (tr.isAllTime() || timeRangeTracker.includesTimeRange(tr)) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SegmentFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SegmentFactory.java index db0b319cc7..cbf7736a7f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SegmentFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SegmentFactory.java @@ -20,6 +20,10 @@ package org.apache.hadoop.hbase.regionserver; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.regionserver.skiplist.hbase.CellCCSMap; +import org.apache.hadoop.hbase.regionserver.skiplist.hbase.ImmutableSegmentOnCCSMap; +import org.apache.hadoop.hbase.regionserver.skiplist.hbase.MemstoreLABProxyForCCSMap; +import org.apache.hadoop.hbase.regionserver.skiplist.hbase.MutableSegmentOnCCSMap; import org.apache.yetus.audience.InterfaceAudience; import java.io.IOException; @@ -151,4 +155,18 @@ public final class SegmentFactory { } return new ImmutableMemStoreLAB(mslabs); } + + // create mutable segment + public MutableSegment createMutableSegmentCCSMap(final Configuration conf, + CellComparator comparator, boolean isMeta) { + CellCCSMap cellCCSMap = CellCCSMap.buildCellCCSMap(conf, comparator, isMeta); + CellSet set = new CellSet(cellCCSMap); + MemStoreLAB memStoreLAB = new MemstoreLABProxyForCCSMap(cellCCSMap); + return new MutableSegmentOnCCSMap(set, comparator, memStoreLAB); + } + + // create immutable segment from mutable segment + public ImmutableSegment createImmutableSegmentCCSMap(MutableSegment segment) { + return new ImmutableSegmentOnCCSMap(segment); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/AbstractCCSMap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/AbstractCCSMap.java new file mode 100644 index 0000000000..5ff0a9011e --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/AbstractCCSMap.java @@ -0,0 +1,719 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.NIL_NODE_ID; +import java.util.AbstractCollection; +import java.util.AbstractSet; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.NavigableSet; +import java.util.Set; +import java.util.SortedMap; +import org.apache.hadoop.hbase.regionserver.skiplist.exception.CCSMapException; +import org.apache.hadoop.hbase.regionserver.skiplist.exception.SerdeException; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +abstract class AbstractCCSMap implements ICCSMap { + + static final int EQ = 1; + static final int LT = 2; + static final int GT = 0; // Actually checked as !LT + + protected final ICCSList ccsl; + protected final IAllocatorHandler allocatorHandler; + protected final Class keyClass; + + private transient Values values; + private transient KeySet keySet; + private transient EntrySet entrySet; + + public AbstractCCSMap(AllocatorHandlerRegister.AllocatorHandlerBuilder builder, + INodeComparator comparator, Class keyClass, SchemaEnum schema) { + this.allocatorHandler = AllocatorHandlerRegister.getAllocatorHandler(); + if (this.allocatorHandler == null) { + throw new IllegalStateException("not any allocatorHandler."); + } + this.keyClass = keyClass; + this.ccsl = new CompactedConcurrentSkipList<>(builder, comparator, schema); + } + + public AbstractCCSMap(ICCSList subCCSL, Class keyClass) { + this.allocatorHandler = AllocatorHandlerRegister.getAllocatorHandler(); + if (this.allocatorHandler == null) { + throw new IllegalStateException("not any allocatorHandler."); + } + this.keyClass = keyClass; + this.ccsl = subCCSL; + } + + @Override + public V get(Object key) { + try { + long node = this.ccsl.get(keyClass.cast(key)); + if (node != NIL_NODE_ID) { + return doGetValue(node); + } + } catch (CCSMapException e) { + throw new IllegalStateException(e); + } + return null; + } + + @Override + public V remove(Object key) { + try { + long node = this.ccsl.remove(keyClass.cast(key)); + if (node != NIL_NODE_ID) { + return doGetValue(node); + } + } catch (CCSMapException e) { + throw new IllegalStateException(e); + } + return null; + } + + @Override + public Entry lowerEntry(K key) { + try { + long nodeId = this.ccsl.findNear(key, LT); + if (nodeId == NIL_NODE_ID) { + return null; + } + return doGetEntry(nodeId); + } catch (CCSMapException e) { + throw new IllegalStateException(e); + } + } + + @Override + public K lowerKey(K key) { + try { + long nodeId = this.ccsl.findNear(key, LT); + if (nodeId == NIL_NODE_ID) { + return null; + } + return doGetKey(nodeId); + } catch (CCSMapException e) { + throw new IllegalStateException(e); + } + } + + @Override + public Entry floorEntry(K key) { + try { + long nodeId = this.ccsl.findNear(key, LT | EQ); + if (nodeId == NIL_NODE_ID) { + return null; + } + return doGetEntry(nodeId); + } catch (CCSMapException e) { + throw new IllegalStateException(e); + } + } + + @Override + public K floorKey(K key) { + try { + long nodeId = this.ccsl.findNear(key, LT | EQ); + if (nodeId == NIL_NODE_ID) { + return null; + } + return doGetKey(nodeId); + } catch (CCSMapException e) { + throw new IllegalStateException(e); + } + } + + @Override + public Entry ceilingEntry(K key) { + try { + long nodeId = this.ccsl.findNear(key, GT|EQ); + if (nodeId == NIL_NODE_ID) { + return null; + } + return doGetEntry(nodeId); + } catch (CCSMapException e) { + throw new IllegalStateException(e); + } + } + + @Override + public K ceilingKey(K key) { + try { + long nodeId = this.ccsl.findNear(key, GT|EQ); + if (nodeId == NIL_NODE_ID) { + return null; + } + return doGetKey(nodeId); + } catch (CCSMapException e) { + throw new IllegalStateException(e); + } + } + + @Override + public Entry higherEntry(K key) { + try { + long nodeId = this.ccsl.findNear(key, GT); + if (nodeId == NIL_NODE_ID) { + return null; + } + return doGetEntry(nodeId); + } catch (CCSMapException e) { + throw new IllegalStateException(e); + } + } + + @Override + public K higherKey(K key) { + try { + long nodeId = this.ccsl.findNear(key, GT); + if (nodeId == NIL_NODE_ID) { + return null; + } + return doGetKey(nodeId); + } catch (CCSMapException e) { + throw new IllegalStateException(e); + } + } + + @Override + public Entry firstEntry() { + try { + long nodeId = this.ccsl.findFirst(); + if (nodeId == NIL_NODE_ID) { + return null; + } + return doGetEntry(nodeId); + } catch (CCSMapException e) { + throw new IllegalStateException(e); + } + } + + @Override + public Entry lastEntry() { + try { + long nodeId = this.ccsl.findLast(); + if (nodeId == NIL_NODE_ID) { + return null; + } + return doGetEntry(nodeId); + } catch (CCSMapException e) { + throw new IllegalStateException(e); + } + } + + @Override + public Entry pollFirstEntry() { + try { + long nodeId = this.ccsl.findFirst(); + if (nodeId == NIL_NODE_ID) { + return null; + } + Entry snapshot = doGetEntry(nodeId); + this.ccsl.remove(nodeId); + return snapshot; + } catch (CCSMapException e) { + throw new IllegalStateException(e); + } + } + + @Override + public Entry pollLastEntry() { + try { + long nodeId = this.ccsl.findLast(); + if (nodeId == NIL_NODE_ID) { + return null; + } + Entry snapshot = doGetEntry(nodeId); + this.ccsl.remove(nodeId); + return snapshot; + } catch (CCSMapException e) { + throw new IllegalStateException(e); + } + } + + @Override + public NavigableMap descendingMap() { + throw new UnsupportedOperationException("CCSMap Not implemented"); + } + + @Override + public NavigableSet navigableKeySet() { + throw new UnsupportedOperationException("CCSMap Not implemented"); + } + + @Override + public NavigableSet descendingKeySet() { + throw new UnsupportedOperationException("CCSMap Not implemented"); + } + + @Override + public Comparator comparator() { + throw new UnsupportedOperationException("CCSMap Not implemented"); + } + + @Override + public SortedMap subMap(K fromKey, K toKey) { + return subMap(fromKey, true, toKey, false); + } + + @Override + public SortedMap headMap(K toKey) { + return headMap(toKey, false); + } + + @Override + public SortedMap tailMap(K fromKey) { + return tailMap(fromKey, true); + } + + @Override + public K firstKey() { + try { + long nodeId = this.ccsl.findFirst(); + if (nodeId == NIL_NODE_ID) { + return null; + } + return doGetKey(nodeId); + } catch (CCSMapException e) { + throw new IllegalStateException(e); + } + } + + @Override + public K lastKey() { + try { + long nodeId = this.ccsl.findLast(); + if (nodeId == NIL_NODE_ID) { + return null; + } + return doGetKey(nodeId); + } catch (CCSMapException e) { + throw new IllegalStateException(e); + } + } + + @Override + public int size() { + return this.ccsl.getSize(); + } + + @Override + public boolean isEmpty() { + return size() == 0; + } + + @Override + public boolean containsKey(Object key) { + return get(key) != null; + } + + @Override + public boolean containsValue(Object value) { + throw new UnsupportedOperationException("CCSMap Not implemented"); + } + + @Override + public void putAll(Map m) { + for (Map.Entry e : m.entrySet()) { + put(e.getKey(), e.getValue()); + } + } + + @Override + public void clear() { + throw new UnsupportedOperationException("CCSMap Not implemented"); + } + + @Override + public Set keySet() { + KeySet ks = keySet; + if (ks != null) { + return ks; + } + keySet = new KeySet<>(this, ccsl); + return keySet; + } + + @Override + public Collection values() { + Values vs = values; + if (vs != null) { + return vs; + } + values = new Values<>(this, ccsl); + return values; + } + + @Override + public Set> entrySet() { + EntrySet es = entrySet; + if (es != null) { + return es; + } + entrySet = new EntrySet<>(this, ccsl); + return entrySet; + } + + @Override + public void close() { + this.ccsl.close(); + } + + @Override + public CompactedConcurrentSkipList.Stat getCcslStat() { + return this.ccsl.getCcslStat(); + } + + static final class KeySet extends AbstractSet { + final AbstractCCSMap m; + final ICCSList ccsl; + + KeySet(AbstractCCSMap m, ICCSList ccsl) { + this.m = m; + this.ccsl = ccsl; + } + + @Override + public Iterator iterator() { + try { + return m.new KeyIterator(ccsl); + } catch (CCSMapException e) { + throw new IllegalStateException(e); + } + } + + @Override + public int size() { + return m.size(); + } + + @Override + public boolean isEmpty() { + return m.isEmpty(); + } + + @Override + public boolean contains(Object o) { + return m.containsKey(o); + } + + @Override + public boolean remove(Object o) { + return m.remove(o) != null; + } + + @Override + public void clear() { + m.clear(); + } + + @Override + public boolean equals(Object o) { + if (o == this) { + return true; + } + if (!(o instanceof Set)) { + return false; + } + Collection c = (Collection) o; + try { + return containsAll(c) && c.containsAll(this); + } catch (ClassCastException unused) { + return false; + } catch (NullPointerException unused) { + return false; + } + } + + @Override + public int hashCode() { + int h = 17; + Iterator i = iterator(); + while (i.hasNext()) { + E obj = i.next(); + if (obj != null) { + h = h * 31 + obj.hashCode(); + } + } + return h; + } + + @Override + public Object[] toArray() { + return toList(this).toArray(); + } + + @Override + public T[] toArray(T[] a) { + return toList(this).toArray(a); + } + + //default Spliterator + } + + static final class Values extends AbstractCollection { + final AbstractCCSMap m; + final ICCSList ccsl; + + Values(AbstractCCSMap m, ICCSList ccsl) { + this.m = m; + this.ccsl = ccsl; + } + + @Override + public Iterator iterator() { + try { + return m.new ValueIterator(ccsl); + } catch (CCSMapException e) { + throw new IllegalStateException(e); + } + } + + @Override + public boolean isEmpty() { + return m.isEmpty(); + } + + @Override + public int size() { + return m.size(); + } + + @Override + public boolean contains(Object o) { + return m.containsValue(o); + } + + @Override + public void clear() { + m.clear(); + } + + @Override + public Object[] toArray() { + return toList(this).toArray(); + } + + @Override + public T[] toArray(T[] a) { + return toList(this).toArray(a); + } + //default Spliterator + } + + static final class EntrySet extends AbstractSet> { + final AbstractCCSMap m; + final ICCSList ccsl; + + EntrySet(AbstractCCSMap m, ICCSList ccsl) { + this.m = m; + this.ccsl = ccsl; + } + + @Override + public Iterator> iterator() { + try { + return m.new EntryIterator(ccsl); + } catch (CCSMapException e) { + throw new IllegalStateException(e); + } + } + + @Override + public boolean contains(Object o) { + if (!(o instanceof Map.Entry)) { + return false; + } + Map.Entry e = (Map.Entry) o; + V1 v = m.get(e.getKey()); + return v != null && v.equals(e.getValue()); + } + + @Override + public boolean remove(Object o) { + if (!(o instanceof Map.Entry)) { + return false; + } + Map.Entry e = (Map.Entry) o; + return m.remove(e.getKey(), e.getValue()); + } + + @Override + public boolean isEmpty() { + return m.isEmpty(); + } + + @Override + public int size() { + return m.size(); + } + + @Override + public void clear() { + m.clear(); + } + + @Override + public boolean equals(Object o) { + if (o == this) { + return true; + } + if (!(o instanceof Set)) { + return false; + } + Collection c = (Collection) o; + try { + return containsAll(c) && c.containsAll(this); + } catch (ClassCastException unused) { + return false; + } catch (NullPointerException unused) { + return false; + } + } + + @Override + public int hashCode() { + int h = 17; + Iterator> i = iterator(); + while (i.hasNext()) { + Entry obj = i.next(); + if (obj != null) { + h = h * 31 + obj.hashCode(); + } + } + return h; + } + + @Override + public Object[] toArray() { + return toList(this).toArray(); + } + + @Override + public T[] toArray(T[] a) { + return toList(this).toArray(a); + } + //default Spliterator + } + + abstract static class AbstractIterator implements Iterator { + private final IIterCCSList iter; + + /** + * Initializes ascending iterator for entire range. + */ + AbstractIterator(ICCSList ccsl) throws CCSMapException { + iter = ccsl.nodeIdIter(); + } + + @Override + public final boolean hasNext() { + return iter.hasNext(); + } + + @Override + public T next() { + try { + long nextId = iter.next(); + return getObject(nextId); + } catch (SerdeException e) { + throw new IllegalStateException(e); + } + } + + @Override + public void remove() { + this.iter.remove(); + } + + abstract T getObject(long nextId) throws SerdeException; + } + + class KeyIterator extends AbstractIterator { + + /** + * Initializes ascending iterator for entire range. + * + * @param ccsl ccslist + */ + KeyIterator(ICCSList ccsl) throws CCSMapException { + super(ccsl); + } + + @Override + K getObject(long nodeId) throws SerdeException { + return doGetKey(nodeId); + } + } + + class ValueIterator extends AbstractIterator { + + /** + * Initializes ascending iterator for entire range. + * + * @param ccsl ccslist + */ + ValueIterator(ICCSList ccsl) throws CCSMapException { + super(ccsl); + } + + @Override + V getObject(long nodeId) throws SerdeException { + return doGetValue(nodeId); + } + } + + class EntryIterator extends AbstractIterator> { + + /** + * Initializes ascending iterator for entire range. + * + * @param ccsl ccslist + */ + EntryIterator(ICCSList ccsl) throws CCSMapException { + super(ccsl); + } + + @Override + Map.Entry getObject(long nodeId) throws SerdeException { + return doGetEntry(nodeId); + } + } + + private static List toList(Collection c) { + // Using size() here would be a pessimization. + ArrayList list = new ArrayList<>(); + for (E e : c) { + list.add(e); + } + return list; + } + + abstract Map.Entry doGetEntry(long nodeId) throws SerdeException; + + abstract K doGetKey(long nodeId) throws SerdeException; + + abstract V doGetValue(long nodeId) throws SerdeException; + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/AbstractChunk.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/AbstractChunk.java new file mode 100644 index 0000000000..e4ffc575b7 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/AbstractChunk.java @@ -0,0 +1,188 @@ +/** + * 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.hadoop.hbase.regionserver.skiplist.core; + +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.FOUR_BYTES_MARK; +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; + +/** + * Default chunk size is 4M,we suggest 2~8M. + * not thread safe + */ +@InterfaceAudience.Private +abstract class AbstractChunk implements IChunk { + + private final long chunkId; + private final long capacity; + protected final AtomicInteger nextFreeOffset = new AtomicInteger(0); + private final AtomicInteger alignOccupancy = new AtomicInteger(0); + protected ByteBuffer data; + private final long initialTime = System.currentTimeMillis(); + private volatile long lastAllocateTime = -1; + private final AtomicLong usedTime = new AtomicLong(0); + private final boolean isPooled; + + public AbstractChunk(long chunkId, long capacity, boolean isPooled) { + this.chunkId = chunkId; + this.capacity = capacity; + this.isPooled = isPooled; + } + + /** + * the removed node's offset and len + */ + private final Map removedMap = new ConcurrentHashMap<>(); + + /** + * when reuse the removed node, we need keep the fragment. offset->fragment + */ + private final Map coverMap = new ConcurrentHashMap<>(); + + @Override + public long getChunkId() { + return this.chunkId; + } + + @Override + public long getPosition() { + return this.nextFreeOffset.get(); + } + + @Override + public long allocate(int len) { + int oldLen = len; + //TODO reuse the removed node's space. + //align + //TODO add config for support unalign + if (len % 8 != 0) { + len = (len / 8 + 1) * 8; + } + while (true) { + int oldOffset = nextFreeOffset.get(); + if (oldOffset + len > getLimit()) { + return -1; // alloc doesn't fit + } + + // Try to atomically claim this chunk + if (nextFreeOffset.compareAndSet(oldOffset, oldOffset + len)) { + // we got the alloc + this.alignOccupancy.addAndGet(oldLen - len); + return oldOffset; + } + } + } + + @Override + public void reclaim(long offset, int len) { + removedMap.put(offset, len); + } + + @Override + public long getLimit() { + return this.capacity; + } + + @Override + public ByteBuffer getByteBuffer() { + return this.data; + } + + @Override + public ByteBuffer asSubByteBuffer(long offset, int len) { + ByteBuffer duplicate = this.data.duplicate(); + duplicate.limit((int) offset + len); + duplicate.position((int) offset); + return duplicate.slice(); + } + + @Override + public int occupancy() { + return (int) (getLimit() - getPosition()) + calFragment(); + } + + public int alignOccupancy() { + return this.alignOccupancy.get(); + } + + private int calFragment() { + Integer totalFragment = 0; + for (Map.Entry removeEntry : removedMap.entrySet()) { + if (coverMap.containsKey(removeEntry.getKey())) { + totalFragment += coverMap.get(removeEntry.getKey()); + } else { + totalFragment += removeEntry.getValue(); + } + } + return totalFragment; + } + + @Override + public void setLastAllocateStartTime() { + Preconditions.checkArgument(this.lastAllocateTime == -1, + "chunk stat error. existed allocateTime, cur=" + this.lastAllocateTime); + this.nextFreeOffset.set(0); + this.lastAllocateTime = System.currentTimeMillis(); + } + + @Override + public void setLastAllocateEndTime() { + Preconditions.checkArgument(this.lastAllocateTime > 0, + "chunk stat error. no allocateTime, cur=" + this.lastAllocateTime); + long now = System.currentTimeMillis(); + if (now > this.lastAllocateTime) { + this.usedTime.addAndGet(now - this.lastAllocateTime); + } + this.nextFreeOffset.set(0); + this.lastAllocateTime = -1; + } + + @Override + public long getUsedPercent() { + long now = System.currentTimeMillis(); + return now - initialTime <= 0 ? 0 : this.usedTime.get() * 100 / (now - initialTime); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (!(obj instanceof AbstractChunk)) { + return false; + } + + AbstractChunk that = (AbstractChunk) obj; + return getChunkId() == that.getChunkId(); + } + + @Override + public boolean isPooledChunk() { + return this.isPooled; + } + + @Override + public int hashCode() { + return (int) (getChunkId() & FOUR_BYTES_MARK); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/AllocatorHandlerRegister.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/AllocatorHandlerRegister.java new file mode 100644 index 0000000000..b71224c3d2 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/AllocatorHandlerRegister.java @@ -0,0 +1,166 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.regionserver.skiplist.hbase.CCSMapMemStore; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + +/** + * Global Register for IAllocatorHandler. + * CCSMap will get IAllocatorHandler from the Register to allocate the mem. + */ +@InterfaceAudience.Private +public final class AllocatorHandlerRegister { + private static final Logger LOG = LoggerFactory.getLogger(AllocatorHandlerRegister.class); + + private static final int INITIAL_COUNT_DEFAULT = 100; + private static final int USE_OLD_CHUNK_THRESHOLD = 3; + private static final int CHUNK_SIZE_DEFAULT = 4 * 1024 * 1024; + private static final long CAPACITY_DEFAULT = 10 * 1024 * 1024 * 1024L; + + private static IAllocatorHandler INSTANCE; + + private AllocatorHandlerRegister(){} + + public static void register(IAllocatorHandler allocatorHandler) { + //provided extended capabilities. such as extend this INSTANCE to chain or decorate pattern + INSTANCE = allocatorHandler; + } + + public static IAllocatorHandler getAllocatorHandler() { + if (INSTANCE == null) { + throw new IllegalStateException("not any registered allocatorHandler"); + } + return INSTANCE; + } + + public static class AllocatorHandlerBuilder { + private static AllocatorHandlerBuilder INSTANCE = null; + private long capacity = CAPACITY_DEFAULT; + private int chunkSize = CHUNK_SIZE_DEFAULT; + private int initialCount = INITIAL_COUNT_DEFAULT; + private int useOldChunkThreshold = USE_OLD_CHUNK_THRESHOLD; + private IAllocatorHandler.IChunkAllocator chunkAllocator; + private HeapMode heapMode = HeapMode.offHeap; + private long extraChunkWarnCapacity = CAPACITY_DEFAULT; + + public long getCapacity() { + return capacity; + } + + public int getChunkSize() { + return chunkSize; + } + + public int getInitialCount() { + return initialCount; + } + + public IAllocatorHandler.IChunkAllocator getChunkAllocator() { + return chunkAllocator; + } + + public HeapMode getHeapMode() { + return heapMode; + } + + public int getUseOldChunkThreshold() { + return useOldChunkThreshold; + } + + public AllocatorHandlerBuilder setCapacity(long capacity) { + this.capacity = capacity; + return this; + } + + public AllocatorHandlerBuilder setChunkSize(int chunkSize) { + this.chunkSize = chunkSize; + return this; + } + + public AllocatorHandlerBuilder setInitialCount(int initialCount) { + this.initialCount = initialCount; + return this; + } + + public AllocatorHandlerBuilder setChunkAllocator( + IAllocatorHandler.IChunkAllocator chunkAllocator) { + this.chunkAllocator = chunkAllocator; + return this; + } + + public AllocatorHandlerBuilder setHeapMode(HeapMode heapMode) { + this.heapMode = heapMode; + return this; + } + + public AllocatorHandlerBuilder setUseOldChunkThreshold(int useOldChunkThreshold) { + if (useOldChunkThreshold < 1) { + useOldChunkThreshold = 1; + } + this.useOldChunkThreshold = useOldChunkThreshold; + return this; + } + + public void setExtraChunkWarnCapacity(long extraChunkWarnCapacity) { + this.extraChunkWarnCapacity = extraChunkWarnCapacity; + } + + public long getExtraChunkWarnCapacity() { + return extraChunkWarnCapacity; + } + + public static synchronized AllocatorHandlerBuilder initialize(Configuration conf, + long confCapacity, int confChunkSize, int initialCount, boolean offheap) { + if (INSTANCE == null) { + AllocatorHandlerBuilder builder = new AllocatorHandlerBuilder(); + + long capacity = conf.getLong(CCSMapMemStore.CHUNK_CAPACITY_KEY, confCapacity); + builder.setCapacity(capacity); + + int chunkSize = conf.getInt(CCSMapMemStore.CHUNK_SIZE_KEY, confChunkSize); + builder.setChunkSize(chunkSize); + builder.setInitialCount(initialCount); + offheap = conf.getBoolean(CCSMapMemStore.CCSMAP_USE_OFFHEAP, offheap); + builder.setHeapMode(offheap ? HeapMode.offHeap : HeapMode.onHeap); + long extraChunkCapacity = + conf.getLong(CCSMapMemStore.EXTRA_CHUNK_WARN_CAPACITY, confCapacity); + builder.setExtraChunkWarnCapacity(extraChunkCapacity); + INSTANCE = builder; + LOG.debug("CCSMap AllocatorHandlerBuilder initInstance, capacity={}; " + + "extraTrunkCapacity={}; chunkSize={}; offHeap={}", capacity, extraChunkCapacity, + chunkSize, offheap); + } + return INSTANCE; + } + + public static AllocatorHandlerBuilder getINSTANCE() { + return INSTANCE; + } + + @VisibleForTesting + public static synchronized void setINSTANCE(AllocatorHandlerBuilder INSTANCE) { + AllocatorHandlerBuilder.INSTANCE = INSTANCE; + } + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/BaseCCSMap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/BaseCCSMap.java new file mode 100644 index 0000000000..6cd8bf8087 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/BaseCCSMap.java @@ -0,0 +1,152 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.KEYVALUE_INFRASTRUCTURE_SIZE; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.NIL_NODE_ID; +import java.util.AbstractMap; +import java.util.NavigableMap; +import org.apache.hadoop.hbase.regionserver.skiplist.core.AllocatorHandlerRegister.AllocatorHandlerBuilder; +import org.apache.hadoop.hbase.regionserver.skiplist.exception.CCSMapException; +import org.apache.hadoop.hbase.regionserver.skiplist.exception.SerdeException; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * BaseCCSMap is the map who have the diff key and value. + * for BaseCCSMap: + * - meta: + * - int level && lock: + * -- byte1 level(Level is not more than 127) + * -- byte2 NodeStat + * -- byte3 preserve (for NodeLock) + * -- byte4 preserve + * - int dataLen :total data len + * - long getNextNode: the pointer to next Node. + * - long[] levelIndex; The pointer to the corresponding level layer next Node. + * - data: + * -- int keyLen + * -- int valueLen + * -- byte[] key + * -- byte[] value + */ +@InterfaceAudience.Private +public class BaseCCSMap extends AbstractCCSMap { + + private final ISerde keySerde; + private final ISerde valueSerde; + + public BaseCCSMap(AllocatorHandlerBuilder builder, INodeComparator comparator, + ISerde keySerde, ISerde valueSerde, Class keyClass) { + super(builder, comparator, keyClass, SchemaEnum.diff); + this.keySerde = keySerde; + this.valueSerde = valueSerde; + } + + public BaseCCSMap(ICCSList ccsl, ISerde keySerde, ISerde valueSerde, Class keyClass) { + super(ccsl, keyClass); + this.keySerde = keySerde; + this.valueSerde = valueSerde; + } + + @Override + public V put(K key, V value) { + try { + if (key == null) { + throw new NullPointerException("can't put null key"); + } + //1.flip a coin to get the level + int level = this.ccsl.generateRandomIndexLevel(); + int keyLen = this.keySerde.getSerializedSize(key); + + int valueLen = this.valueSerde.getSerializedSize(value); + int metaLen = NodeUtil.getNodeMetaLenByLevel(level); + + //2. allocate the chunk. + int dataLen = NodeUtil.getDataLen(keyLen, valueLen); + long newNodeId = this.ccsl.getNodeAndWriteMeta(level, metaLen, dataLen); + IChunk chunk = allocatorHandler.getChunkById(NodeUtil.getChunkIdByNodeId(newNodeId)); + //3.copy the data to chunk. + ChunkUtil. + writeNodeData(chunk, NodeUtil.getChunkOffsetByNodeId(newNodeId), metaLen, key, value, + this.keySerde, this.valueSerde, keyLen, valueLen); + + long node = this.ccsl.put(newNodeId, level); + return node == NIL_NODE_ID ? null : doGetValue(node); + } catch (CCSMapException e) { + throw new IllegalStateException(e); + } + } + + @Override + Entry doGetEntry(long nodeId) throws SerdeException { + IChunk chunk = this.ccsl.getChunkById(NodeUtil.getChunkIdByNodeId(nodeId)); + int baseOffset = (int) NodeUtil.getChunkOffsetByNodeId(nodeId); + int keyOffset = ChunkUtil.getKeyOffset(chunk, baseOffset, SchemaEnum.diff); + int keyLen = ChunkUtil.getKeyLen(chunk, baseOffset, keyOffset, SchemaEnum.diff); + int valueOffset = keyOffset + keyLen; + int dataLen = ChunkUtil.getNodeDataLen(chunk, baseOffset); + int valueLen = dataLen - keyLen - KEYVALUE_INFRASTRUCTURE_SIZE; + //only key byte[] + K key = this.keySerde.deserialize(chunk.getByteBuffer(), keyOffset, keyLen); + //only value byte[] + V value = this.valueSerde.deserialize(chunk.getByteBuffer(), valueOffset, valueLen); + return new AbstractMap.SimpleImmutableEntry<>(key, value); + } + + @Override + K doGetKey(long nodeId) throws SerdeException { + IChunk chunk = this.ccsl.getChunkById(NodeUtil.getChunkIdByNodeId(nodeId)); + int baseOffset = (int) NodeUtil.getChunkOffsetByNodeId(nodeId); + int keyOffset = ChunkUtil.getKeyOffset(chunk, baseOffset, SchemaEnum.diff); + int keyLen = ChunkUtil.getKeyLen(chunk, baseOffset, keyOffset, SchemaEnum.diff); + //only key byte[] + return this.keySerde.deserialize(chunk.getByteBuffer(), keyOffset, keyLen); + } + + @Override + V doGetValue(long nodeId) throws SerdeException { + IChunk chunk = this.ccsl.getChunkById(NodeUtil.getChunkIdByNodeId(nodeId)); + int baseOffset = (int) NodeUtil.getChunkOffsetByNodeId(nodeId); + int keyOffset = ChunkUtil.getKeyOffset(chunk, baseOffset, SchemaEnum.diff); + int keyLen = ChunkUtil.getKeyLen(chunk, baseOffset, keyOffset, SchemaEnum.diff); + int valueOffset = keyOffset + keyLen; + int dataLen = ChunkUtil.getNodeDataLen(chunk, baseOffset); + int valueLen = dataLen - keyLen - KEYVALUE_INFRASTRUCTURE_SIZE; + //only value byte[] + return this.valueSerde.deserialize(chunk.getByteBuffer(), valueOffset, valueLen); + } + + @Override + public NavigableMap subMap(K fromKey, boolean fromInclusive, K toKey, boolean toInclusive) { + ICCSList subCCSL = + new SubCompactedConcurrentSkipList<>(this.ccsl, fromKey, toKey, fromInclusive, toInclusive); + return new BaseCCSMap<>(subCCSL, keySerde, valueSerde, keyClass); + } + + @Override + public NavigableMap headMap(K toKey, boolean inclusive) { + ICCSList subCCSL = + new SubCompactedConcurrentSkipList<>(this.ccsl, null, toKey, false, inclusive); + return new BaseCCSMap<>(subCCSL, keySerde, valueSerde, keyClass); + } + + @Override + public NavigableMap tailMap(K fromKey, boolean inclusive) { + ICCSList subCCSL = + new SubCompactedConcurrentSkipList<>(this.ccsl, fromKey, null, inclusive, false); + return new BaseCCSMap<>(subCCSL, keySerde, valueSerde, keyClass); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/BaseTwinCCSMap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/BaseTwinCCSMap.java new file mode 100644 index 0000000000..54a5fc6ffd --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/BaseTwinCCSMap.java @@ -0,0 +1,126 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.NIL_NODE_ID; +import java.util.AbstractMap; +import java.util.NavigableMap; +import org.apache.hadoop.hbase.regionserver.skiplist.core.AllocatorHandlerRegister.AllocatorHandlerBuilder; +import org.apache.hadoop.hbase.regionserver.skiplist.exception.CCSMapException; +import org.apache.hadoop.hbase.regionserver.skiplist.exception.SerdeException; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * BaseTwinCCSMap is the map who have the same key and value. + * BaseTwinCCSMap only stores this object once. + * for BaseTwinCCSMap: + * - meta: + * - int level && lock: + * -- byte1 level(Level is not more than 127) + * -- byte2 NodeStat + * -- byte3 preserve (for NodeLock) + * -- byte4 preserve + * - int dataLen :total data len + * - long getNextNode: the pointer to next Node. + * - long[] levelIndex; The pointer to the corresponding level layer next Node. + * - data: + * -- all data byte[] + */ +@InterfaceAudience.Private +public class BaseTwinCCSMap extends AbstractCCSMap { + private final ISerde keyvalueSerde; + + public BaseTwinCCSMap(AllocatorHandlerBuilder builder, INodeComparator comparator, + ISerde keyvalueSerde, Class keyClass) { + super(builder, comparator, keyClass, SchemaEnum.same); + + this.keyvalueSerde = keyvalueSerde; + } + + public BaseTwinCCSMap(ICCSList subCCSL, ISerde keyvalueSerde, Class keyClass) { + super(subCCSL, keyClass); + this.keyvalueSerde = keyvalueSerde; + } + + @Override + public K put(K key1, K key2) { + try { + if (key1 == null) { + throw new NullPointerException("can't put null key"); + } + if (key1 != key2) { + throw new IllegalStateException("TwinCCSMap is for same key value."); + } + //1.flip a coin to get the level + int level = this.ccsl.generateRandomIndexLevel(); + int dataLen = this.keyvalueSerde.getSerializedSize(key1); + int metaLen = NodeUtil.getNodeMetaLenByLevel(level); + + //2. allocate the chunk. + long newNodeId = this.ccsl.getNodeAndWriteMeta(level, metaLen, dataLen); + IChunk chunk = allocatorHandler.getChunkById(NodeUtil.getChunkIdByNodeId(newNodeId)); + //3.copy the data to chunk. + int startOffset = (int) NodeUtil.getChunkOffsetByNodeId(newNodeId) + metaLen; + this.keyvalueSerde.serialize(key1, chunk.getByteBuffer(), startOffset, dataLen); + long node = this.ccsl.put(newNodeId, level); + + return node == NIL_NODE_ID ? null : doGetValue(node); + } catch (CCSMapException e) { + throw new IllegalStateException(e); + } + } + + @Override + Entry doGetEntry(long nodeId) throws SerdeException { + K obj = this.doGetValue(nodeId); + return new AbstractMap.SimpleImmutableEntry<>(obj, obj); + } + + @Override + K doGetKey(long nodeId) throws SerdeException { + return doGetValue(nodeId); + } + + @Override + K doGetValue(long nodeId) throws SerdeException { + IChunk chunk = this.ccsl.getChunkById(NodeUtil.getChunkIdByNodeId(nodeId)); + int baseOffset = (int) NodeUtil.getChunkOffsetByNodeId(nodeId); + int dataOffset = ChunkUtil.getNodeDataOffset(chunk, baseOffset); + int dataLen = ChunkUtil.getNodeDataLen(chunk, baseOffset); + return this.keyvalueSerde.deserialize(chunk.getByteBuffer(), dataOffset, dataLen); + } + + @Override + public NavigableMap subMap(K fromKey, boolean fromInclusive, K toKey, boolean toInclusive) { + ICCSList subCCSL = + new SubCompactedConcurrentSkipList<>(this.ccsl, fromKey, toKey, fromInclusive, toInclusive); + return new BaseTwinCCSMap<>(subCCSL, this.keyvalueSerde, this.keyClass); + } + + @Override + public NavigableMap headMap(K toKey, boolean inclusive) { + ICCSList subCCSL = + new SubCompactedConcurrentSkipList<>(this.ccsl, null, toKey, false, inclusive); + return new BaseTwinCCSMap<>(subCCSL, this.keyvalueSerde, this.keyClass); + } + + @Override + public NavigableMap tailMap(K fromKey, boolean inclusive) { + ICCSList subCCSL = + new SubCompactedConcurrentSkipList<>(this.ccsl, fromKey, null, inclusive, false); + return new BaseTwinCCSMap<>(subCCSL, this.keyvalueSerde, this.keyClass); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/ByteBufferUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/ByteBufferUtils.java new file mode 100644 index 0000000000..ea56d71231 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/ByteBufferUtils.java @@ -0,0 +1,92 @@ +/** + * 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.hadoop.hbase.regionserver.skiplist.core; + +import java.nio.ByteBuffer; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * some method directly use the HBase's ByteBufferUtils. + * If we need totally decouple the CCSMap with HBase, we need implement it. + * All method don't change the ByteBuffer's position. + */ +@InterfaceAudience.Private +public final class ByteBufferUtils { + + private ByteBufferUtils() { + } + + public static int toInt(ByteBuffer buffer, int offset) { + return org.apache.hadoop.hbase.util.ByteBufferUtils.toInt(buffer, offset); + } + + public static long toLong(ByteBuffer buffer, int offset) { + return org.apache.hadoop.hbase.util.ByteBufferUtils.toLong(buffer, offset); + } + + public static int toShort(ByteBuffer buffer, int offset) { + return org.apache.hadoop.hbase.util.ByteBufferUtils.toShort(buffer, offset); + } + + public static int toByte(ByteBuffer buffer, int offset) { + return org.apache.hadoop.hbase.util.ByteBufferUtils.toByte(buffer, offset); + } + + public static void putInt(ByteBuffer buffer, int offset, int val) { + org.apache.hadoop.hbase.util.ByteBufferUtils.putInt(buffer, offset, val); + } + + public static void putLong(ByteBuffer buffer, int offset, long val) { + org.apache.hadoop.hbase.util.ByteBufferUtils.putLong(buffer, offset, val); + } + + /** + * Copy from one buffer to another from given offset. This will be absolute positional copying and + * won't affect the position of any of the buffers. + * + * @param in source buffer + * @param out destination buffer + * @param sourceOffset offset in the source buffer + * @param destinationOffset offset in the destination buffer + * @param length how many bytes to copy + */ + public static void copyFromBufferToBuffer(ByteBuffer in, ByteBuffer out, int sourceOffset, + int destinationOffset, int length) { + org.apache.hadoop.hbase.util.ByteBufferUtils + .copyFromBufferToBuffer(in, out, sourceOffset, destinationOffset, length); + } + + public static boolean compareAndSetLong(ByteBuffer byteBuffer, int offset, long expected, + long update) { + return UnsafeHelp.compareAndSetLong(byteBuffer, offset, expected, update); + } + + public static boolean compareAndSetInt(ByteBuffer byteBuffer, int offset, int expected, + int update) { + return UnsafeHelp.compareAndSetInt(byteBuffer, offset, expected, update); + } + + public static int compareTo(ByteBuffer buf1, int o1, int l1, ByteBuffer buf2, int o2, int l2) { + return org.apache.hadoop.hbase.util.ByteBufferUtils.compareTo(buf1, o1, l1, buf2, o2, l2); + } + + public static int compareTo(ByteBuffer buf1, int o1, int l1, byte[] buf2, int o2, int l2) { + return org.apache.hadoop.hbase.util.ByteBufferUtils.compareTo(buf1, o1, l1, buf2, o2, l2); + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/CCSMapChunkPool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/CCSMapChunkPool.java new file mode 100644 index 0000000000..16ea48a2ed --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/CCSMapChunkPool.java @@ -0,0 +1,256 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import java.util.Map; +import java.util.Queue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.regionserver.skiplist.core.AllocatorHandlerRegister.AllocatorHandlerBuilder; +import org.apache.hadoop.hbase.regionserver.skiplist.exception.HeapFullException; +import org.apache.hadoop.hbase.regionserver.skiplist.hbase.CCSMapMemStore; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + +/** + * strategy pattern for allocate Chunk. + */ +@InterfaceAudience.Private +public class CCSMapChunkPool implements IAllocatorHandler { + private static final Logger LOG = LoggerFactory.getLogger(CCSMapChunkPool.class); + private static volatile CCSMapChunkPool globalInstance; + private static final Object initLock = new Object(); + private final HeapMode heapMode; + private final int maxCount; + private final long extraChunkWarnCapacity; + private final int chunkSize; + private final IChunk[] totalPoolChunkArray; + //extra allocate Chunk's Map + private final Map extraChunkMap = new ConcurrentHashMap<>(); + private final int extraChunkInitialId; + private final Queue chunkQueue; + private final IChunkAllocator poolChunkAllocator; + private final IChunkAllocator hugeChunkAllocator; + private final IChunkAllocator normalExtraChunkAllocator; + private final AtomicLong poolChunkIdGenerator = new AtomicLong(0); + private final AtomicLong extraChunkIdGenerator; + private final AtomicLong chunkCounter = new AtomicLong(0); + private final AtomicLong requestCounter = new AtomicLong(0); + private final AtomicLong extraRequestCounter = new AtomicLong(0); + private final AtomicLong extraChunkCapacityUsed = new AtomicLong(0); + + public CCSMapChunkPool(AllocatorHandlerBuilder allocatorHandlerBuilder) { + this.heapMode = allocatorHandlerBuilder.getHeapMode(); + long capacity = allocatorHandlerBuilder.getCapacity(); + this.chunkSize = allocatorHandlerBuilder.getChunkSize(); + + this.maxCount = (int) (capacity / this.chunkSize); + this.extraChunkWarnCapacity = allocatorHandlerBuilder.getExtraChunkWarnCapacity(); + if (this.maxCount <= 0 || extraChunkWarnCapacity <= 0) { + // Enough for about 32TB of cache! + throw new IllegalArgumentException("capacity or extra capacity not permit negative or zero, " + + "maximum support 32TB now. now maxCount=" + maxCount + + " ,extraChunkWarnCapacity=" + extraChunkWarnCapacity); + } + + IChunkAllocator chunkAllocator; + switch (this.heapMode) { + case onHeap: + chunkAllocator = (len) -> new OnHeapChunk(poolChunkIdGenerator.incrementAndGet(), len); + break; + case offHeap: + chunkAllocator = (len) -> new OffHeapChunk(poolChunkIdGenerator.incrementAndGet(), len); + break; + default: + throw new IllegalArgumentException("heapMode error"); + } + this.poolChunkAllocator = chunkAllocator; + this.extraChunkInitialId = maxCount + 1; + this.extraChunkIdGenerator = new AtomicLong(extraChunkInitialId); + this.hugeChunkAllocator = (len) -> new HugeOnHeapChunk(getExtraChunkId(), len); + //extra chunk is not pooled + this.normalExtraChunkAllocator = (len) -> new OnHeapChunk(getExtraChunkId(), len, false); + this.totalPoolChunkArray = new IChunk[maxCount]; + this.chunkQueue = new ConcurrentLinkedQueue<>(); + int initialCount = allocatorHandlerBuilder.getInitialCount(); + if (initialCount > this.maxCount) { + initialCount = this.maxCount; + } + for (int i = 0; i < initialCount; i++) { + if (!this.chunkQueue.offer(doAllocate())) { + throw new IllegalStateException( + "chunkQueue can't offer. now chunkQueue size=" + chunkQueue.size()); + } + } + } + + private long getExtraChunkId() { + long id = incrementAndGetExtraChunkId(); + while (this.extraChunkMap.containsKey(id)) { + id = incrementAndGetExtraChunkId(); + } + return id; + } + + private long incrementAndGetExtraChunkId() { + long id = this.extraChunkIdGenerator.incrementAndGet(); + if (id >= Integer.MAX_VALUE - 1) { + this.extraChunkIdGenerator.set(this.extraChunkInitialId); + id = this.extraChunkIdGenerator.incrementAndGet(); + LOG.debug("{} will overflow, reset it.", id); + } + return id; + } + + @Override + public IChunk allocate(int len) throws HeapFullException { + this.requestCounter.incrementAndGet(); + IChunk chunk; + if (len > this.chunkSize) { + return allocateExtraChunk(len, true); + } + chunk = this.chunkQueue.poll(); + if (chunk == null) { + synchronized (this.poolChunkAllocator) { + if (this.chunkCounter.get() >= this.maxCount) { + LOG.debug("No more available chunk in this pool, " + + "will use extra chunk on heap before pooled chunks reclaimed."); + // Permit chunk pool to use extra capacity when exhausted. + // TODO Ideally we should throw HeapFullException and let upper layer + // (like HRegion) to handle it (such as triggering flush) + return allocateExtraChunk(this.chunkSize, false); + } + chunk = doAllocate(); + } + } + chunk.setLastAllocateStartTime(); + return chunk; + } + + private IChunk allocateExtraChunk(int len, boolean isHuge) { + //extra Chunk is onHeap + if (this.extraChunkCapacityUsed.get() >= this.extraChunkWarnCapacity) { + LOG.warn("Too many extra chunk allocated: {}, Check {} or {}", + this.extraChunkCapacityUsed.get(), CCSMapMemStore.EXTRA_CHUNK_WARN_CAPACITY, + CCSMapMemStore.CHUNK_CAPACITY_KEY); + } + IChunk chunk; + if (isHuge) { + chunk = this.hugeChunkAllocator.allocate(len); + } else { + chunk = this.normalExtraChunkAllocator.allocate(len); + } + this.extraRequestCounter.incrementAndGet(); + long curExtraSize = this.extraChunkCapacityUsed.addAndGet(len); + this.extraChunkMap.put(chunk.getChunkId(), chunk); + LOG.debug("allocate extra chunk. total extra chunk size={} ; chunkId={}, size={}, isHuge=", + curExtraSize, chunk.getChunkId(), len, isHuge); + return chunk; + } + + @Override + public void putbackChunk(IChunk chunk) { + //not support putback duplicate. + if (chunk.getHeapMode() == this.heapMode && chunk.isPooledChunk()) { + chunk.getByteBuffer().rewind(); + chunk.setLastAllocateEndTime(); + if (!this.chunkQueue.offer(chunk)) { + throw new IllegalStateException("chunkQueue can't offer. now chunkQueue size=" + chunkQueue + .size()); + } + } else { + this.extraChunkMap.remove(chunk.getChunkId()); + this.extraChunkCapacityUsed.addAndGet(-chunk.getLimit()); + } + } + + private IChunk doAllocate() { + IChunk chunk = this.poolChunkAllocator.allocate(this.chunkSize); + this.totalPoolChunkArray[(int) (chunk.getChunkId() - 1)] = chunk; + this.chunkCounter.incrementAndGet(); + return chunk; + } + + @Override + public IChunk getChunkById(long chunkId) { + if (chunkId <= maxCount) { + return this.totalPoolChunkArray[(int) (chunkId - 1)]; + } else { + return this.extraChunkMap.get(chunkId); + } + } + + public long getChunkCounter() { + return chunkCounter.get(); + } + + public long getRequestCounter() { + return requestCounter.get(); + } + + public long getExtraRequestCounter() { + return extraRequestCounter.get(); + } + + public AtomicLong getExtraChunkCapacityUsed() { + return extraChunkCapacityUsed; + } + + @VisibleForTesting + public Queue getChunkQueue() { + return chunkQueue; + } + + @VisibleForTesting + public IChunk[] getChunkMap() { + return totalPoolChunkArray; + } + + @VisibleForTesting + public Map getExtraChunkMap() { + return extraChunkMap; + } + + @VisibleForTesting + public int getMaxCount() { + return maxCount; + } + + public static CCSMapChunkPool initialize(Configuration conf, long confCapacity, int confChunkSize, + int initialCount, boolean offheap) { + if (globalInstance != null) { + return globalInstance; + } + synchronized (initLock) { + if (globalInstance == null) { + AllocatorHandlerBuilder builder = AllocatorHandlerBuilder + .initialize(conf, confCapacity, confChunkSize, initialCount, offheap); + CCSMapChunkPool allocatorHandler = new CCSMapChunkPool(builder); + AllocatorHandlerRegister.register(allocatorHandler); + globalInstance = allocatorHandler; + LOG.debug("CCSMapChunkPool initialize"); + } + } + return globalInstance; + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/CCSMapMetrics.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/CCSMapMetrics.java new file mode 100644 index 0000000000..4d8e7d8425 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/CCSMapMetrics.java @@ -0,0 +1,60 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import org.apache.yetus.audience.InterfaceAudience; + +/** + * TODO Metrics + */ +@InterfaceAudience.Private +public final class CCSMapMetrics { + /** + * 大于Chunk size的分配请求数 + */ + + /** + * 碎片率 + */ + + /** + * 同一个Key的写入擦除率 + * 后写入的value大于之前的,重新分配了Chunk来写入,造成该数据被擦除 + */ + + /** + * 同一个Key的写入覆盖成功率 + * 后写入的value小于之前的,直接覆盖写了,但可能留下了碎片(浪费) + */ + + /** + * Level/Node数 + */ + + /** + * 写入histogram + */ + + /** + * 读取histogram + */ + + /** + * Allocator的相关Metrics + */ +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/ChunkUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/ChunkUtil.java new file mode 100644 index 0000000000..f3b2650069 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/ChunkUtil.java @@ -0,0 +1,148 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.BYTE_MARK; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.FOUR_BYTES_MARK; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.INDEX_NOT_INITIAL; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.KEYVALUE_INFRASTRUCTURE_SIZE; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.NEXT_NODE_START_OFFSET; +import org.apache.hadoop.hbase.regionserver.skiplist.exception.SerdeException; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; + +/** + * The data schema: + * - meta: + * - int level && lock: + * -- byte1 level(Level is not more than 127) + * -- byte2 NodeStat + * -- byte3 preserve (for NodeLock) + * -- byte4 preserve + * - int dataLen :total data len + * - long getNextNode: the pointer to next Node. + * - long[] levelIndex; The pointer to the corresponding level layer next Node. + * - data: + * -- int keyLen + * -- int valueLen + * -- byte[] key + * -- byte[] value + */ +@InterfaceAudience.Private +public final class ChunkUtil { + + private ChunkUtil() { + } + /** + * dataLen : + * int keyLen + * int valueLen + * byte[] key + * byte[] value + */ + public static long newNodeOnChunk(IChunk chunk, long offset, int level, int dataLen) { + Preconditions.checkNotNull(chunk, "chunk must be not null."); + Preconditions.checkArgument(offset >= 0, "offset can't be negative."); + long nodeId = ((chunk.getChunkId() & FOUR_BYTES_MARK) << 32) | (offset & FOUR_BYTES_MARK); + Preconditions.checkArgument(dataLen > 0, "dataLen error."); + initNodeMeta(chunk, offset, level, dataLen); + return nodeId; + } + + public static void initNodeMeta(IChunk chunk, long offset, int level, int dataLen) { + ByteBufferUtils.putInt(chunk.getByteBuffer(), (int) offset, level & BYTE_MARK); + ByteBufferUtils.putInt(chunk.getByteBuffer(), (int) offset + Integer.BYTES, dataLen); + + for (int i = level; i > 0; i--) { + ChunkUtil.setNextNodeIdForLevel(chunk, offset, i, INDEX_NOT_INITIAL); + } + + } + + public static int getNodeLevel(IChunk chunk, long offset) { + int meta = ByteBufferUtils.toInt(chunk.getByteBuffer(), (int) offset); + return meta & BYTE_MARK; + } + + public static int getNodeDataLen(IChunk chunk, long offset) { + return ByteBufferUtils.toInt(chunk.getByteBuffer(), (int) offset + Integer.BYTES); + } + + public static int getNodeDataOffset(IChunk chunk, long offset) { + int level = getNodeLevel(chunk, offset); + return (int) offset + NodeUtil.getNodeMetaLenByLevel(level); + } + + public static int getKeyOffset(IChunk chunk, long offset, SchemaEnum schema) { + switch (schema) { + case diff: + return getNodeDataOffset(chunk, offset) + KEYVALUE_INFRASTRUCTURE_SIZE; + case same: + // the node total data's offset + return getNodeDataOffset(chunk, offset); + default: + throw new IllegalArgumentException("error schema:" + schema); + } + + } + + public static int getKeyLen(IChunk chunk, long offset, long keyOffset, SchemaEnum schema) { + switch (schema) { + case diff: + return ByteBufferUtils + .toInt(chunk.getByteBuffer(), (int) keyOffset - KEYVALUE_INFRASTRUCTURE_SIZE); + case same: + // the node total data's len + return getNodeDataLen(chunk, offset); + default: + throw new IllegalArgumentException("error schema:" + schema); + } + } + + /** + * write data + */ + public static void writeNodeData(IChunk chunk, long offset, int metaLen, K key, V value, + ISerde keySerde, ISerde valueSerde, int keyLen, int valueLen) throws SerdeException { + + int startOffset = (int) offset + metaLen; + ByteBufferUtils.putInt(chunk.getByteBuffer(), startOffset, keyLen); + ByteBufferUtils.putInt(chunk.getByteBuffer(), startOffset + Integer.BYTES, valueLen); + keySerde + .serialize(key, chunk.getByteBuffer(), startOffset + KEYVALUE_INFRASTRUCTURE_SIZE, keyLen); + valueSerde.serialize(value, chunk.getByteBuffer(), + startOffset + KEYVALUE_INFRASTRUCTURE_SIZE + keyLen, valueLen); + } + + public static void setNextNodeIdForLevel(IChunk chunk, long offset, int level, long nextNode) { + //no check level + int startOffset = (int) offset + NEXT_NODE_START_OFFSET + level * Long.BYTES; + ByteBufferUtils.putLong(chunk.getByteBuffer(), startOffset, nextNode); + } + + public static long getNextNodeIdForLevel(IChunk chunk, long offset, int level) { + //no check level + int startOffset = (int) offset + NEXT_NODE_START_OFFSET + level * Long.BYTES; + return ByteBufferUtils.toLong(chunk.getByteBuffer(), startOffset); + } + + @VisibleForTesting + public static int testGetNodeKeyLen(IChunk chunk, long offset) { + int startOffset = (int) offset + NodeUtil.getNodeMetaLenByLevel(getNodeLevel(chunk, offset)); + return ByteBufferUtils.toInt(chunk.getByteBuffer(), startOffset); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/CompactedConcurrentSkipList.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/CompactedConcurrentSkipList.java new file mode 100644 index 0000000000..f235ba1898 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/CompactedConcurrentSkipList.java @@ -0,0 +1,921 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import static org.apache.hadoop.hbase.regionserver.skiplist.core.AbstractCCSMap.EQ; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.AbstractCCSMap.LT; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.INDEX_NOT_INITIAL; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.NIL_NODE_ID; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.NO_INDEX; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.SKIP_LIST_HEAD_NODE; +import java.nio.ByteBuffer; +import java.util.Comparator; +import java.util.NoSuchElementException; +import java.util.Random; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.PriorityBlockingQueue; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.hadoop.hbase.regionserver.skiplist.exception.CCSMapException; +import org.apache.hadoop.hbase.regionserver.skiplist.exception.ChunkMissingException; +import org.apache.hadoop.hbase.regionserver.skiplist.exception.HeapFullException; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; + +/** + * A concurrent skip list implementation with more compacted memory usage + * It's the underlying implementation for CompactedConcurrentSkipListMap(CCSMap for short). + *

+ * About concurrency: + * 1. write Data to chunk is concurrently. + * 2. use CAS to update the Node's meta. + *

+ * About writing duplicate key : + * 1. remove first,then put again. + *

+ * About remove: + * 1. remove will make more fragments,CCSList do not do compaction. + * 2. User can do compaction if needed, for example copy to new CCSList. + *

+ * when the data size is bigger than chunk's : + * 1. create the OnHeap chunk, and not pool it. + *

+ * thread-safe + */ +@InterfaceAudience.Private +final class CompactedConcurrentSkipList implements ICCSList { + private static final Logger LOG = LoggerFactory.getLogger(CompactedConcurrentSkipList.class); + private static final Double COMPACTION_REMOVED_RATIO = 0.1; + + //one LevelIndexHeader per CCSL + private final ILevelIndexHeader levelIndexHeader; + + //levelUpdateInd + private final AtomicBoolean levelUpdateInd = new AtomicBoolean(false); + private final INodeComparator comparator; + private final int useOldChunkThreshold; + + private final AtomicInteger size = new AtomicInteger(0); + private final AtomicReference curChunk = new AtomicReference<>(); + + private final IAllocatorHandler allocatorHandler; + private final SchemaEnum schema; + private final int chunkSize; + + //stat + private final Stat ccslStat = new Stat(); + + // use PriorityBlockingQueue to keep the used chunk sorting by the unused the space. + private final BlockingQueue usedChunkQueue = + new PriorityBlockingQueue<>(256, new Comparator() { + @Override + public int compare(IChunk o1, IChunk o2) { + return Long + .compare((o2.getLimit() - o2.getPosition()), (o1.getLimit() - o1.getPosition())); + } + }); + + private volatile boolean closed = false; + + private static final Random seedGenerator = new Random(); + private transient int randomSeed = seedGenerator.nextInt() | 0x0100; // ensure nonzero + + //it's not use Iterator, IIterCCSList can help to reduce the autoBoxing + class Iter implements IIterCCSList { + /** + * the last node returned by next() + */ + long lastReturned = NIL_NODE_ID; + /** + * the next node to return from next(); + */ + long next = NIL_NODE_ID; + + /** + * Initializes ascending iterator for entire range. + */ + Iter() throws ChunkMissingException { + next = findFirst(); + } + + @Override + public final boolean hasNext() { + return next != NIL_NODE_ID; + } + + @Override + public long next() { + long n = next; + try { + advance(); + } catch (ChunkMissingException e) { + throw new IllegalStateException(e); + } + return n; + } + + /** + * Advances next to higher entry. + */ + final void advance() throws ChunkMissingException { + if (next == NIL_NODE_ID) { + throw new NoSuchElementException(); + } + lastReturned = next; + long nextId = getNextNodeById(next); + while (nextId != NIL_NODE_ID && NodeUtil + .isNodeRemoving(CompactedConcurrentSkipList.this, nextId)) { + nextId = getNextNodeById(nextId); + } + next = nextId; + } + + @Override + public void remove() { + long l = lastReturned; + if (l == NIL_NODE_ID) { + throw new NoSuchElementException(); + } + // It would not be worth all of the overhead to directly + // unlink from here. Using remove is fast enough. + + try { + CompactedConcurrentSkipList.this.remove(l); + } catch (CCSMapException e) { + throw new IllegalStateException(e); + } + lastReturned = NIL_NODE_ID; + } + + } + + protected CompactedConcurrentSkipList(AllocatorHandlerRegister.AllocatorHandlerBuilder builder, + INodeComparator comparator, SchemaEnum schema) { + this.allocatorHandler = AllocatorHandlerRegister.getAllocatorHandler(); + this.comparator = comparator; + this.levelIndexHeader = new OnHeapIndexHeader(); + this.schema = schema; + this.useOldChunkThreshold = builder.getUseOldChunkThreshold(); + this.chunkSize = builder.getChunkSize(); + } + + @Override + public long getNextNodeById(long nodeId) throws ChunkMissingException { + if (nodeId == SKIP_LIST_HEAD_NODE) { + return this.levelIndexHeader.getLevelNextNodeId(0); + } + return NodeUtil.getNextNodeIdForLevel(this, nodeId, 0); + } + + @Override + public long getNextNodeByIdAndLevel(long nodeId, int level) throws CCSMapException { + if (nodeId == SKIP_LIST_HEAD_NODE) { + return this.levelIndexHeader.getLevelNextNodeId(level); + } + long result = NodeUtil.getNextNodeIdForLevel(this, nodeId, level); + while (result == INDEX_NOT_INITIAL) { + result = NodeUtil.getNextNodeIdForLevel(this, nodeId, level); + checkState(); + } + return result; + } + + boolean helpCasUpdateNextNode(int level, long b, long expected, long update) + throws ChunkMissingException { + if (b == SKIP_LIST_HEAD_NODE) { + return this.levelIndexHeader.casUpdateLevelNextNodeId(level, expected, update); + } + return NodeUtil.casUpdateNextNodeForLevel(this, level, b, expected, update); + } + + boolean helpCheckNodeRemoving(long nodeId) throws ChunkMissingException { + if (nodeId == SKIP_LIST_HEAD_NODE) { + return false; + } + return NodeUtil.isNodeRemoving(this, nodeId); + } + + @Override + public long put(long newNodeId, int level) throws CCSMapException { + IChunk chunk = getChunkById(NodeUtil.getChunkIdByNodeId(newNodeId)); + long offset = NodeUtil.getChunkOffsetByNodeId(newNodeId); + checkState(); + //1. update the link + long bid, nid, fid; + int c; + boolean replaced; + long result = NIL_NODE_ID; + for (bid = findPredecessor(chunk, offset, 0), nid = getNextNodeById(bid); ; ) { + replaced = false; + if (nid != NIL_NODE_ID) { + fid = getNextNodeById(nid); + if (nid != getNextNodeById(bid)) { + nid = getNextNodeById(bid); + continue; // inconsistent read + } + c = compareNodes(chunk, (int) offset, nid); + if (c > 0) { + bid = nid; + nid = fid; + continue; + } + if (c == 0) { + if (newNodeId == nid) { + throw new CCSMapException("can't put existed node"); + } + //new node is before the old one which was removed + if (!NodeUtil.isNodeRemoving(this, nid)) { + replaced = true; + } + } + } + // c < 0 || n == null; insert to before n + ChunkUtil.setNextNodeIdForLevel(chunk, offset, 0, nid); + if (!helpCasUpdateNextNode(0, bid, nid, newNodeId)) { + nid = getNextNodeById(bid); + continue; // restart if lost race to append to b + } + if (replaced) { + if (NodeUtil.casSetNodeStat(this, nid, NodeStat.REMOVE)) { + this.ccslStat.removedNodeCount.incrementAndGet(); + } + this.ccslStat.replaceRequestCount.incrementAndGet(); + result = nid; + //4. reclaim removed node + //TODO reclaimNode(n); + } else { + size.incrementAndGet(); + } + + break; + } + //5. update HeadIndex if needed + for (; ; ) { + if (level <= levelIndexHeader.getLevel()) { + break; + } + if (!levelUpdateInd.compareAndSet(false, true)) { + continue; + } + try { + levelIndexHeader.updateLevel(level); + } finally { + levelUpdateInd.compareAndSet(true, false); + } + } + + //6. update index + if (level != NO_INDEX) { + putIndex(level, newNodeId, chunk, offset); + } + + if (replaced) { + deleteNode(); + } + return result; + } + + void putIndex(int level, long newNodeId, IChunk chunk, long offset) throws CCSMapException { + + int currLevel = level; + long b = findPredecessor(chunk, offset, level); + long r = getNextNodeByIdAndLevel(b, level); + + for (; ; ) { + if (r != NIL_NODE_ID) { + // compare before deletion check avoids needing recheck + int c = compareNodes(chunk, (int) offset, r); + if (c > 0) { + b = r; + r = getNextNodeByIdAndLevel(b, currLevel); + if (b == r) { + throw new RuntimeException("b=r =" + r); + } + continue; + } + } + + ChunkUtil.setNextNodeIdForLevel(chunk, offset, currLevel, r); + + if (!helpCasUpdateNextNode(currLevel, b, r, newNodeId)) { + r = getNextNodeByIdAndLevel(b, currLevel); + continue; + } + currLevel--; + if (currLevel == 0) { + break; + } + + r = getNextNodeByIdAndLevel(b, currLevel); + } + } + + @Override + public long remove(long nodeId) throws CCSMapException { + IChunk chunk = getChunkById(NodeUtil.getChunkIdByNodeId(nodeId)); + long offset = NodeUtil.getChunkOffsetByNodeId(nodeId); + int keyOffset = ChunkUtil.getKeyOffset(chunk, offset, this.schema); + int keyLen = ChunkUtil.getKeyLen(chunk, offset, keyOffset, this.schema); + return remove(chunk.getByteBuffer(), keyOffset, keyLen); + } + + public long remove(ByteBuffer keyBB, int keyOffset, int keyLen) throws CCSMapException { + checkState(); + long b, n, f; + int c; + outer: + for (; ; ) { + for (b = findPredecessor(keyBB, keyOffset, keyLen, 0), n = getNextNodeById(b); ; ) { + if (n == NIL_NODE_ID) { + break outer; + } + f = getNextNodeById(n); + if (n != getNextNodeById(b)) { // inconsistent read + break; + } + if (helpCheckNodeRemoving(n)) {// n is remove + b = n; + n = f; + continue; + } + + c = compareNodes(keyBB, keyOffset, keyLen, n); + if (c < 0) { + break outer; + } + if (c > 0) { + b = n; + n = f; + continue; + } + + if (!NodeUtil.casSetNodeStat(this, n, NodeStat.REMOVE)) { + b = n; + n = f; + continue; + } + this.ccslStat.removedNodeCount.incrementAndGet(); + deleteNode();//b, n, f); + this.size.decrementAndGet(); + //TODO reclaimNode(n); + return n; + } + } + return NIL_NODE_ID; + } + + @Override + public long remove(K key) throws CCSMapException { + checkState(); + long b, n, f; + int c; + outer: + for (; ; ) { + for (b = findPredecessor(key, 0), n = getNextNodeById(b); ; ) { + if (n == NIL_NODE_ID) { + break outer; + } + f = getNextNodeById(n); + if (n != getNextNodeById(b)) { // inconsistent read + break; + } + if (helpCheckNodeRemoving(n)) {// n is remove + b = n; + n = f; + continue; + } + + c = compareNodes(key, n); + if (c < 0) { + break outer; + } + if (c > 0) { + b = n; + n = f; + continue; + } + + if (!NodeUtil.casSetNodeStat(this, n, NodeStat.REMOVE)) { + b = n; + n = f; + continue; + } + this.ccslStat.removedNodeCount.incrementAndGet(); + deleteNode(); + this.size.decrementAndGet(); + //TODO reclaimNode(n); + return n; + } + } + return NIL_NODE_ID; + } + + @Override + public long get(K key) throws CCSMapException { + checkState(); + long b, n, f; + int c; + outer: + for (; ; ) { + for (b = findPredecessor(key, 0), n = getNextNodeById(b); ; ) { + if (n == NIL_NODE_ID) { + break outer; + } + f = getNextNodeById(n); + + if (helpCheckNodeRemoving(n)) {// n is remove + b = n; + n = f; + continue; + } + if (n != (getNextNodeById(b))) { // inconsistent read + break; + } + + c = compareNodes(key, n); + if (c == 0) { + return n; + } + + if (c < 0) { + break outer; + } + b = n; + n = f; + } + } + return NIL_NODE_ID; + } + + private int compareNodes(IChunk leftChunk, int offset, long right) + throws ChunkMissingException { + + int leftKeyOffset = ChunkUtil.getKeyOffset(leftChunk, offset, this.schema); + int leftKeyLen = ChunkUtil.getKeyLen(leftChunk, offset, leftKeyOffset, this.schema); + return compareNodes(leftChunk.getByteBuffer(), leftKeyOffset, leftKeyLen, right); + } + + @Override + public int compareNodes(long left, ByteBuffer right, int rightKeyOffset, int rightKeyLen) + throws ChunkMissingException { + + IChunk leftChunk = this.getChunkById(NodeUtil.getChunkIdByNodeId(left)); + long baseOffset = (int) NodeUtil.getChunkOffsetByNodeId(left); + int leftKeyOffset = ChunkUtil.getKeyOffset(leftChunk, baseOffset, this.schema); + int leftKeyLen = ChunkUtil.getKeyLen(leftChunk, baseOffset, leftKeyOffset, this.schema); + return compareNodes(leftChunk.getByteBuffer(), leftKeyOffset, leftKeyLen, right, + rightKeyOffset, rightKeyLen); + } + + @Override + public int compareNodes(ByteBuffer left, int leftKeyOffset, int leftKeyLen, long right) + throws ChunkMissingException { + + IChunk rightChunk = this.getChunkById(NodeUtil.getChunkIdByNodeId(right)); + long baseOffset = (int) NodeUtil.getChunkOffsetByNodeId(right); + int rightKeyOffset = ChunkUtil.getKeyOffset(rightChunk, baseOffset, this.schema); + int rightKeyLen = ChunkUtil.getKeyLen(rightChunk, baseOffset, rightKeyOffset, this.schema); + return compareNodes(left, leftKeyOffset, leftKeyLen, rightChunk.getByteBuffer(), + rightKeyOffset, rightKeyLen); + } + + @Override + public int compareNodes(K key, long right) throws ChunkMissingException { + IChunk rightChunk = this.getChunkById(NodeUtil.getChunkIdByNodeId(right)); + long baseOffset = (int) NodeUtil.getChunkOffsetByNodeId(right); + int rightKeyOffset = ChunkUtil.getKeyOffset(rightChunk, baseOffset, this.schema); + int rightKeyLen = ChunkUtil.getKeyLen(rightChunk, baseOffset, rightKeyOffset, this.schema); + return compareNodes(key, rightChunk.getByteBuffer(), rightKeyOffset, rightKeyLen); + } + + @Override + public int compareNodes(long left, K key) throws ChunkMissingException { + IChunk leftChunk = this.getChunkById(NodeUtil.getChunkIdByNodeId(left)); + long baseOffset = (int) NodeUtil.getChunkOffsetByNodeId(left); + int leftKeyOffset = ChunkUtil.getKeyOffset(leftChunk, baseOffset, this.schema); + int leftKeyLen = ChunkUtil.getKeyLen(leftChunk, baseOffset, leftKeyOffset, this.schema); + return compareNodes(leftChunk.getByteBuffer(), leftKeyOffset, leftKeyLen, key); + } + + @Override + public int compareNodes(ByteBuffer left, int leftKeyOffset, int leftKeyLen, ByteBuffer right, + int rightKeyOffset, int rightKeyLen) throws ChunkMissingException { + return this.comparator + .compareTo(left, leftKeyOffset, leftKeyLen, right, rightKeyOffset, rightKeyLen); + } + + public int compareNodes(K left, ByteBuffer right, int rightKeyOffset, int rightKeyLen) { + return this.comparator.compareTo(left, right, rightKeyOffset, rightKeyLen); + } + + @Override + public int compareNodes(ByteBuffer left, int leftKeyOffset, int leftKeyLen, K right) + throws ChunkMissingException { + return this.comparator.compareTo(left, leftKeyOffset, leftKeyLen, right); + } + + @Override + public int compareNodes(K left, K right) throws ChunkMissingException { + return this.comparator.compareTo(left, right); + } + + @Override + public int getCurMaxLevel() { + return this.levelIndexHeader.getLevel(); + } + + private long findPredecessor(IChunk chunk, long offset, int level) throws CCSMapException { + int leftKeyOffset = ChunkUtil.getKeyOffset(chunk, offset, this.schema); + int leftKeyLen = ChunkUtil.getKeyLen(chunk, offset, leftKeyOffset, this.schema); + return findPredecessor(chunk.getByteBuffer(), leftKeyOffset, leftKeyLen, level); + } + + private long findPredecessor(ByteBuffer chunkBB, int keyOffset, int keyLen, int level) + throws CCSMapException { + int curLevel = this.levelIndexHeader.getLevel(); + long bid = SKIP_LIST_HEAD_NODE; + long rid = this.levelIndexHeader.getLevelNextNodeId(curLevel); + for (; ; ) { + if (rid != NIL_NODE_ID) { + if (compareNodes(chunkBB, keyOffset, keyLen, rid) > 0) { + bid = rid; + rid = getNextNodeByIdAndLevel(bid, curLevel); + continue; + } + } + curLevel--; + if (curLevel <= level) { + return bid; + } + rid = getNextNodeByIdAndLevel(bid, curLevel); + } + } + + private long findPredecessor(K key, int level) throws CCSMapException { + int curLevel = this.levelIndexHeader.getLevel(); + long bid = SKIP_LIST_HEAD_NODE; + long rid = this.levelIndexHeader.getLevelNextNodeId(curLevel); + for (; ; ) { + if (rid != NIL_NODE_ID) { + if (compareNodes(key, rid) > 0) { + bid = rid; + rid = getNextNodeByIdAndLevel(bid, curLevel); + continue; + } + } + curLevel--; + if (curLevel <= level) { + return bid; + } + rid = getNextNodeByIdAndLevel(bid, curLevel); + } + } + + @Override + public final long findNear(K key, int rel) throws CCSMapException { + int curLevel = this.levelIndexHeader.getLevel(); + long bid = SKIP_LIST_HEAD_NODE; + long rid = this.levelIndexHeader.getLevelNextNodeId(curLevel); + long nid, fid; + int c; + //1. find findPredecessor but not removed + for (; ; ) { + if (rid != NIL_NODE_ID) { + if (NodeUtil.isNodeRemoving(this, rid)) { + rid = getNextNodeByIdAndLevel(rid, curLevel); + continue; + } + if (compareNodes(key, rid) > 0) { + bid = rid; + rid = getNextNodeByIdAndLevel(bid, curLevel); + continue; + } + } + curLevel--; + if (curLevel <= 0) { + break; + } + rid = getNextNodeByIdAndLevel(bid, curLevel); + } + + //2. find near node + + for (nid = getNextNodeById(bid); ; ) { + if (nid == NIL_NODE_ID) { + return (rel & LT) == 0 ? NIL_NODE_ID : (bid == SKIP_LIST_HEAD_NODE ? NIL_NODE_ID : bid); + } + if (NodeUtil.isNodeRemoving(this, nid)) { + nid = getNextNodeById(nid); + continue; + } + fid = getNextNodeById(nid); + + c = compareNodes(key, nid); + + if ((c == 0 && (rel & EQ) != 0) || (c < 0 && (rel & LT) == 0)) { + return nid; + } + if (c <= 0 && (rel & LT) != 0) { + return (bid == SKIP_LIST_HEAD_NODE ? NIL_NODE_ID : bid); + } + bid = nid; + nid = fid; + } + } + + @Override + public final long findFirst() throws ChunkMissingException { + long firstNodeId = this.levelIndexHeader.getLevelNextNodeId(0); + while (firstNodeId != NIL_NODE_ID) { + if (!NodeUtil.isNodeRemoving(this, firstNodeId)) { + return firstNodeId; + } + firstNodeId = getNextNodeById(firstNodeId); + } + return NIL_NODE_ID; + } + + @Override + public final long findLast() throws CCSMapException { + int curLevel = this.levelIndexHeader.getLevel(); + long bid = SKIP_LIST_HEAD_NODE; + long rid = this.levelIndexHeader.getLevelNextNodeId(curLevel); + long nid; + for (; ; ) { + if (rid != NIL_NODE_ID) { + if (!helpCheckNodeRemoving(rid)) { + bid = rid; + } + rid = getNextNodeByIdAndLevel(rid, curLevel); + } else if (curLevel > 1) { + curLevel--; + rid = getNextNodeByIdAndLevel(bid, curLevel); + } else { + for (nid = getNextNodeById(bid); ; ) { + if (nid == NIL_NODE_ID) { + return bid == SKIP_LIST_HEAD_NODE ? NIL_NODE_ID : bid; + } + if (NodeUtil.isNodeRemoving(this, nid)) { + nid = getNextNodeById(nid); + continue; + } + + bid = nid; + nid = getNextNodeById(nid); + } + } + } + } + + @Override + public long getNodeAndWriteMeta(int level, int metaLen, int dataLen) throws CCSMapException { + int nodeLen = metaLen + dataLen; + checkState(); + if (nodeLen > chunkSize) { + return doAllocateForHuge(level, dataLen, nodeLen); + } else { + return doAllocateForNormal(level, dataLen, nodeLen); + } + } + + /** + * HugeChunk only use once, so no multi thread contention + */ + private long doAllocateForHuge(int level, int dataLen, int nodeLen) throws HeapFullException { + IChunk chunk = allocatorHandler.allocate(nodeLen); + Preconditions.checkArgument(chunk instanceof HugeOnHeapChunk, "It's not a HugeChunk"); + long allocOffset = chunk.allocate(nodeLen); + Preconditions.checkArgument(allocOffset == 0, "HugeChunk only can be use once"); + this.usedChunkQueue.offer(chunk); + return ChunkUtil.newNodeOnChunk(chunk, allocOffset, level, dataLen); + } + + private long doAllocateForNormal(int level, int dataLen, int nodeLen) throws HeapFullException { + IChunk chunk; + long allocOffset; + //useOldChunk control only try old chunk once time. + boolean useOldChunk = false; + while (true) { + chunk = curChunk.get(); + if (chunk == null) { + if (!useOldChunk && this.ccslStat.chunkCount.get() > this.useOldChunkThreshold) { + // The chunks in use may still have tail spaces, so we will check the one + // with most un-used space (best-effort since chunk's tail space will change + // after new allocation but re-ordering will only happen when new chunk + // added to queue) and try to use it for best memory utilization. + // Notes: + // 1. We will try only one chunk even if it's not the best one with enough space + // 2. The huge chunks won't be chosen since no more tail space there + chunk = usedChunkQueue.peek(); + //chunk is not null + useOldChunk = true; + } else { + chunk = allocatorHandler.allocate(nodeLen); + if (!curChunk.compareAndSet(null, chunk)) { + allocatorHandler.putbackChunk(chunk); + continue; + } else { + // new chunk get success + this.ccslStat.chunkCount.incrementAndGet(); + } + } + } else { + useOldChunk = false; + } + // Try to allocate from this chunk + allocOffset = chunk.allocate(nodeLen); + if (allocOffset != -1) { + // We succeeded - this is the common case - small alloc + // from a big buffer + break; + } + // not enough space! + // if this chunk is not old chunk,try to retire it, and add it to usedChunkQueue + if (!useOldChunk && curChunk.compareAndSet(chunk, null)) { + this.usedChunkQueue.offer(chunk); + } + } + return ChunkUtil.newNodeOnChunk(chunk, allocOffset, level, dataLen); + } + + @Override + public IChunk getChunkById(long chunkId) { + return this.allocatorHandler.getChunkById(chunkId); + } + + private void reclaimNode(INode node) { + //TODO + //need implement the reference logic first. + // only all the reference to this node was removed, this node can be reused. #deleteNode + } + + /** + * when removed node access threshold, log it. + */ + private boolean deleteNode() { + //currently only log it. + //if there have more remove, user maybe need do compact himself. + if (this.ccslStat.removedNodeCount.get() >= this.size.get() * COMPACTION_REMOVED_RATIO) { + if (LOG.isTraceEnabled()) { + LOG.trace("WARN! remove Node size={}; totalSize={}", this.ccslStat.removedNodeCount.get(), + this.size.get()); + } + } + return true; + } + + @Override + public int generateRandomIndexLevel() { + int x = randomSeed; + x ^= x << 13; + x ^= x >>> 17; + x ^= x << 5; + randomSeed = x; + // test highest and lowest bits + if ((x & 0x8001) != 0) { + return NO_INDEX; + } + int level = 1; + int curMax = levelIndexHeader.getLevel(); + x >>>= 1; + while ((x & 1) != 0) { + ++level; + x >>>= 1; + //the level only be increased by step + if (level > curMax) { + break; + } + } + return level; + } + + private void checkState() throws CCSMapException { + if (this.closed) { + throw new CCSMapException("CCSL has closed."); + } + } + + @Override + public void close() { + if (this.closed) { + return; + } + + synchronized (this) { + if (this.closed) { + return; + } + IChunk chunk; + while ((chunk = usedChunkQueue.poll()) != null) { + allocatorHandler.putbackChunk(chunk); + } + chunk = this.curChunk.get(); + if (chunk != null) { + allocatorHandler.putbackChunk(chunk); + this.curChunk.set(null); + } + this.closed = true; + } + } + + @Override + public int getSize() { + return size.get(); + } + + @VisibleForTesting + ILevelIndexHeader getLevelIndexHeader() { + return levelIndexHeader; + } + + @VisibleForTesting + BlockingQueue getUsedChunkQueue() { + return usedChunkQueue; + } + + @VisibleForTesting + IChunk getCurChunk() { + return curChunk.get(); + } + + @Override + public IIterCCSList nodeIdIter() throws ChunkMissingException { + return new Iter(); + } + + @Override + public Stat getCcslStat() { + return ccslStat; + } + + @Override + public INodeComparator getComparator() { + return comparator; + } + + public static class Stat { + private final AtomicInteger chunkCount = new AtomicInteger(0); + private final AtomicLong cprRequestCount = new AtomicLong(0); + private final AtomicLong putRequestCount = new AtomicLong(0); + private final AtomicLong replaceRequestCount = new AtomicLong(0); + private final AtomicLong removedNodeCount = new AtomicLong(0); + private final AtomicLong removeRequestRaceFailCount = new AtomicLong(0); + private final AtomicLong debugTimer = new AtomicLong(0); + private final AtomicLong debugCounter = new AtomicLong(0); + + public AtomicInteger getChunkCount() { + return this.chunkCount; + } + + public long getCprRequestCount() { + return cprRequestCount.get(); + } + + public long getPutRequestCount() { + return putRequestCount.get(); + } + + public long getRemovedNodeCount() { + return removedNodeCount.get(); + } + + public long getRemoveRequestRaceFailCount() { + return removeRequestRaceFailCount.get(); + } + + public long getReplaceRequestCount() { + return replaceRequestCount.get(); + } + + public long getDebugTimer() { + return debugTimer.get(); + } + + public long getDebugCounter() { + return debugCounter.get(); + } + + @Override + public String toString() { + return "TODO"; + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/Constant.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/Constant.java new file mode 100644 index 0000000000..37241fc672 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/Constant.java @@ -0,0 +1,38 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public final class Constant { + private Constant() { + } + + public static final int NO_INDEX = 0; + public static final long NIL_NODE_ID = 0; + public static final long INDEX_NOT_INITIAL = -2; + public static final long SKIP_LIST_HEAD_NODE = -1; + public final static int MAX_LEVEL = 255; + public final static int DEFAULT_LEVEL = 32; + public final static long FOUR_BYTES_MARK = 0xFFFFFFFFL; + public final static int BYTE_MARK = 0xFF; + public final static int NEXT_NODE_START_OFFSET = Integer.BYTES * 2; + public final static int KEYVALUE_INFRASTRUCTURE_SIZE = Integer.BYTES * 2; + public final static int LEVEL_INDEX_START_OFFSET = NEXT_NODE_START_OFFSET + Long.BYTES; +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/HeapMode.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/HeapMode.java new file mode 100644 index 0000000000..38f1752a8b --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/HeapMode.java @@ -0,0 +1,25 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public enum HeapMode { + onHeap, offHeap +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/HugeOnHeapChunk.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/HugeOnHeapChunk.java new file mode 100644 index 0000000000..64417fae66 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/HugeOnHeapChunk.java @@ -0,0 +1,48 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; + +@InterfaceAudience.Private +public class HugeOnHeapChunk extends OnHeapChunk { + + public HugeOnHeapChunk(long chunkId, int capacity) { + //HugeChunk is not pooled + super(chunkId, capacity, false); + } + + /** + * HugeChunk only allocate once. + */ + @Override + public long allocate(int len) { + int oldOffset = nextFreeOffset.get(); + Preconditions + .checkArgument(len == getLimit(), "HugeChunk's capacity not fit the nodeLen(" + len + ")."); + Preconditions.checkArgument(oldOffset == 0, "HugeChunk only use once, cur offset=" + oldOffset); + + // Try to atomically claim this chunk + if (nextFreeOffset.compareAndSet(oldOffset, oldOffset + len)) { + return oldOffset; + } else { + return -1; // alloc doesn't fit + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/IAllocatorHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/IAllocatorHandler.java new file mode 100644 index 0000000000..5ef0582cc8 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/IAllocatorHandler.java @@ -0,0 +1,54 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import org.apache.hadoop.hbase.regionserver.skiplist.exception.HeapFullException; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * strategy pattern for allocate Chunk. + */ +@InterfaceAudience.Private +public interface IAllocatorHandler extends IChunkVisible { + /** + * the allocation is done by an implementation of IAllocatorHandler + * + * @param len the Node data len + * @return a Chunk + * @throws HeapFullException when no any Chunk can be allocated. + */ + IChunk allocate(int len) throws HeapFullException; + + /** + * put back a Chunk which be allocated before. + * + * @param chunk Which Chunk be put back. + */ + void putbackChunk(IChunk chunk); + + interface IChunkAllocator { + /** + * allocate a Chunk by IChunkAllocator + * + * @param len the Node data len + * @return a Chunk + * @throws HeapFullException when no any Chunk can be allocated. + */ + IChunk allocate(int len); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/ICCSList.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/ICCSList.java new file mode 100644 index 0000000000..f08a24a6a9 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/ICCSList.java @@ -0,0 +1,249 @@ +/** + * 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.hadoop.hbase.regionserver.skiplist.core; + +import java.nio.ByteBuffer; +import org.apache.hadoop.hbase.regionserver.skiplist.exception.CCSMapException; +import org.apache.hadoop.hbase.regionserver.skiplist.exception.ChunkMissingException; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public interface ICCSList extends IChunkVisible { + + /** + * Returns a nodeId associated with the least + * key in this list, or {@code NIL_NODE_ID} if the list is empty. + * + * @return a nodeId with the least key, or {@code NIL_NODE_ID} if this list is empty + */ + long findFirst() throws CCSMapException; + + /** + * Returns a nodeId associated with the greatest + * key in this list, or {@code NIL_NODE_ID} if the list is empty. + * + * @return a nodeId with the greatest key, or {@code NIL_NODE_ID} if this list is empty + */ + long findLast() throws CCSMapException; + + /** + * @return the number of item in this list. + */ + int getSize(); + + /** + * Closes this list and releases any memory associated + * with it. If the list is already closed then invoking this + * method has no effect. + */ + void close(); + + /** + * @return stat of this List + */ + CompactedConcurrentSkipList.Stat getCcslStat(); + + /** + * it's not use Jdk's Iterator, IIterCCSList can help to reduce the autoBoxing + * + * @return An iterator of this List + * @throws CCSMapException if a CCSMap error occurs. + */ + IIterCCSList nodeIdIter() throws CCSMapException; + + /** + * Returns the nodeId to which the specified key, + * or {@code NIL_NODE_ID} if this List contains no node for the key. + */ + long get(K key) throws CCSMapException; + + /** + * Generate the random level for new node which will be put to this List. + * + * @return a random level + */ + int generateRandomIndexLevel(); + + /** + * allocate a Chunk and offset for putting new node, then write node's meta. + * + * @param level the putting node's level + * @param metaLen the putting node's meta len + * @param dataLen the putting node's data len. + * @return nodeId for the putting node. + * @throws CCSMapException if a CCSMap error occurs. + */ + long getNodeAndWriteMeta(int level, int metaLen, int dataLen) throws CCSMapException; + + /** + * Appends the specified node to this list. + * + * @param newNodeId the putting node's nodeId + * @param level the putting node's level + * @return the previous nodeId associated with key, or + * NIL_NODE_ID if there was no mapping for key. + * @throws CCSMapException CCSMapException + */ + long put(long newNodeId, int level) throws CCSMapException; + + /** + * Removes the node for a key from this List if it is present. + */ + long remove(K key) throws CCSMapException; + + /** + * Removes the node for a nodeId from this List if it is present. + */ + long remove(long nodeId) throws CCSMapException; + + /** + * @return INodeComparator for this List + */ + INodeComparator getComparator(); + + /** + * Get the next node of the specified node on the corresponding layer,or {@code NIL_NODE_ID} + * + * @param currentNodeId the specified node + * @param level skip list layer + * @return nodeId, or {@code NIL_NODE_ID} + * @throws CCSMapException if a CCSMap error occurs. + */ + long getNextNodeByIdAndLevel(long currentNodeId, int level) throws CCSMapException; + + /** + * Get the next node of the specified node on the linked list which contains all nodes, + * or {@code NIL_NODE_ID} + * + * @param currentNodeId the specified node + * @return nodeId, or {@code NIL_NODE_ID} + * @throws CCSMapException if a CCSMap error occurs. + */ + long getNextNodeById(long currentNodeId) throws ChunkMissingException; + + /** + * Utility for ceiling, floor, lower, higher methods. + * + * @param key the key + * @param rel the relation -- OR'ed combination of EQ, LT, GT + * @return nearest node fitting relation, or {@code NIL_NODE_ID} if no such + */ + long findNear(K key, int rel) throws CCSMapException; + + /** + * Compares for order. Returns a negative integer, + * zero, or a positive integer as the first node is less than, equal + * to, or greater than the second.

+ * + * @param leftNodeId the node data's node ID (chunkId,offset) + * @param byteBuffer Key's ByteBuffer + * @param keyOffset Key's ByteBuffer's offset + * @param keyLen Key's ByteBuffer's len + * @return An integer result of the comparison. + * @throws ChunkMissingException when can't find Chunk for leftNodeId + */ + int compareNodes(long leftNodeId, ByteBuffer byteBuffer, int keyOffset, int keyLen) + throws ChunkMissingException; + + /** + * Compares for order. Returns a negative integer, + * zero, or a positive integer as the first node is less than, equal + * to, or greater than the second.

+ * + * @param left Key's ByteBuffer + * @param leftKeyOffset Key's ByteBuffer's offset + * @param leftKeyLen Key's ByteBuffer's len + * @param rightNodeId the node data's node ID (chunkId,offset) + * @return An integer result of the comparison. + * @throws ChunkMissingException when can't find Chunk for leftNodeId + */ + int compareNodes(ByteBuffer left, int leftKeyOffset, int leftKeyLen, long rightNodeId) + throws ChunkMissingException; + + /** + * Compares for order. Returns a negative integer, + * zero, or a positive integer as the first node is less than, equal + * to, or greater than the second.

+ * + * @param left left Key's ByteBuffer + * @param leftKeyOffset left Key's ByteBuffer's offset + * @param leftKeyLen left Key's ByteBuffer's len + * @param right right Key's ByteBuffer + * @param rightKeyOffset right Key's ByteBuffer's offset + * @param rightKeyLen right Key's ByteBuffer's len + * @return An integer result of the comparison. + * @throws ChunkMissingException when can't find Chunk for leftNodeId + */ + int compareNodes(ByteBuffer left, int leftKeyOffset, int leftKeyLen, ByteBuffer right, + int rightKeyOffset, int rightKeyLen) throws ChunkMissingException; + + /** + * Compares for order. Returns a negative integer, + * zero, or a positive integer as the first node is less than, equal + * to, or greater than the second.

+ * + * @param left left Key's ByteBuffer + * @param leftKeyOffset left Key's ByteBuffer's offset + * @param leftKeyLen left Key's ByteBuffer's len + * @param right right Key's Object + * @return An integer result of the comparison. + * @throws ChunkMissingException when can't find Chunk for leftNodeId + */ + int compareNodes(ByteBuffer left, int leftKeyOffset, int leftKeyLen, K right) + throws ChunkMissingException; + + /** + * Compares for order. Returns a negative integer, + * zero, or a positive integer as the first node is less than, equal + * to, or greater than the second.

+ * + * @param leftNodeId the node data's node ID (chunkId,offset) + * @param right right Key's Object + * @return An integer result of the comparison. + * @throws ChunkMissingException when can't find Chunk for leftNodeId + */ + int compareNodes(long leftNodeId, K right) throws ChunkMissingException; + + /** + * Compares for order. Returns a negative integer, + * zero, or a positive integer as the first node is less than, equal + * to, or greater than the second.

+ * + * @param left left Key's Object + * @param rightNodeId the node data's node ID (chunkId,offset) + * @return An integer result of the comparison. + * @throws ChunkMissingException when can't find Chunk for leftNodeId + */ + int compareNodes(K left, long rightNodeId) throws ChunkMissingException; + + /** + * Compares for order. Returns a negative integer, + * zero, or a positive integer as the first Key Object is less than, equal + * to, or greater than the second.

+ * + * @param left left Key's Object + * @param right right Key's Object + * @return An integer result of the comparison. + * @throws ChunkMissingException when can't find Chunk for leftNodeId + */ + int compareNodes(K left, K right) throws ChunkMissingException; + + /** + * @return the cur max level of this List + */ + int getCurMaxLevel(); + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/ICCSMap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/ICCSMap.java new file mode 100644 index 0000000000..19d539306a --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/ICCSMap.java @@ -0,0 +1,36 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import java.util.NavigableMap; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public interface ICCSMap extends NavigableMap { + + /** + * @return Stat of this Map + */ + CompactedConcurrentSkipList.Stat getCcslStat(); + + /** + * Closes this Map and releases any memory associated + * with it. If the map is already closed then invoking this + * method has no effect. + */ + void close(); + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/IChunk.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/IChunk.java new file mode 100644 index 0000000000..52d659632a --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/IChunk.java @@ -0,0 +1,105 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import java.nio.ByteBuffer; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * A Chunk of memory out of which allocations are sliced. + */ +@InterfaceAudience.Private +public interface IChunk { + /** + * @return this Chunk's HeapMode(onHeap or offHeap) + */ + HeapMode getHeapMode(); + + /** + * @return Id of this Chunk + */ + long getChunkId(); + + /** + * @return offset of this Chunk + */ + long getPosition(); + + /** + * Try to allocate len bytes from the chunk. + * + * @return the offset of the successful allocation, or -1 to indicate not-enough-space + */ + long allocate(int len); + + /** + * reclaim a node's space which allocated before. + * TODO we need to implement clear logic first. That need reference mechanism to make sure that + * all objects which are references of this node have been GC. + * + * @param offset offset of this chunk. + * @param len how many bytes this node occupy. + */ + void reclaim(long offset, int len); + + /** + * @return the total len of this Chunk. + */ + long getLimit(); + + /** + * @return This chunk's backing ByteBuffer. + */ + ByteBuffer getByteBuffer(); + + /** + * Creates a new byte buffer that shares this buffer's content. + * + * @return ByteBuffer + */ + ByteBuffer asSubByteBuffer(long offset, int len); + + /** + * free space percent of this Chunk. It includes as following: + * 1. the tail of this Chunk have not been allocated. + * 2. reclaim a node's space which allocated before. + * + * @return N=0~100. N% + */ + int occupancy(); + + /** + * the last allocate time of this Chunk. + */ + void setLastAllocateStartTime(); + + /** + * the last putback time of this Chunk. + */ + void setLastAllocateEndTime(); + + /** + * the frequency of use this chunk since it allocated. + */ + long getUsedPercent(); + + /** + * @return if this Chunk is a pooled + */ + boolean isPooledChunk(); +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/IChunkVisible.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/IChunkVisible.java new file mode 100644 index 0000000000..c1115a9b09 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/IChunkVisible.java @@ -0,0 +1,31 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public interface IChunkVisible { + /** + * get Chunk by ChunkId. + * + * @param chunkId chunkId + * @return A chunk whose Id eq the specified chunkId + */ + IChunk getChunkById(long chunkId); +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/IIterCCSList.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/IIterCCSList.java new file mode 100644 index 0000000000..a972258249 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/IIterCCSList.java @@ -0,0 +1,62 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import java.util.NoSuchElementException; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public interface IIterCCSList { + + /** + * Returns {@code true} if the iteration has more elements. + * (In other words, returns {@code true} if {@link #next} would + * return an element rather than throwing an exception.) + * + * @return {@code true} if the iteration has more elements + */ + boolean hasNext(); + + /** + * Returns the next element's nodeId in the iteration. + * + * @return the next element's nodeId in the iteration + * @throws NoSuchElementException if the iteration has no more elements + */ + long next(); + + /** + * Removes from the underlying collection the last element returned + * by this iterator (optional operation). This method can be called + * only once per call to {@link #next}. The behavior of an iterator + * is unspecified if the underlying collection is modified while the + * iteration is in progress in any way other than by calling this + * method. + * + * The default implementation throws an instance of {@link UnsupportedOperationException} and + * performs no other action. + * + * @throws UnsupportedOperationException if the {@code remove} + * operation is not supported by this iterator + * @throws IllegalStateException if the {@code next} method has not + * yet been called, or the {@code remove} method has already + * been called after the last call to the {@code next} + * method + * + */ + void remove(); + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/ILevelIndexHeader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/ILevelIndexHeader.java new file mode 100644 index 0000000000..60e59586a4 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/ILevelIndexHeader.java @@ -0,0 +1,57 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import org.apache.yetus.audience.InterfaceAudience; + +/** + * the LevelIndexHeader is small(normally 2K),no much GC because objects are very few. so we only + * provide OnHeapIndexHeader first. + */ +@InterfaceAudience.Private +public interface ILevelIndexHeader { + /** + * @return cur max level + */ + int getLevel(); + + /** + * update cur max level + * + * @param level the dst level + */ + void updateLevel(int level); + + /** + * cas update the next Node for level layer on this levelIndex + * + * @param level which layer + * @param expectedNode the expected nodeId + * @param newNode the new nodeId + * @return true if update success, otherwise false. + */ + boolean casUpdateLevelNextNodeId(int level, long expectedNode, long newNode); + + /** + * Get the next node of the specified node on the corresponding layer,or {@code NIL_NODE_ID} + * + * @param level skip list layer + * @return nodeId, or {@code NIL_NODE_ID} + */ + long getLevelNextNodeId(int level); +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/INode.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/INode.java new file mode 100644 index 0000000000..b199b36690 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/INode.java @@ -0,0 +1,42 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import java.nio.ByteBuffer; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public interface INode { + + /** + * the Object(for Key) serialize to ByteBuffer. + * + * @return Key's ByteBuffer + */ + ByteBuffer getByteBuffer(); + + /** + * @return Key's ByteBuffer's offset + */ + int getKeyOffset(); + + /** + * @return Key's len + */ + int getKeyLen(); +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/INodeComparator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/INodeComparator.java new file mode 100644 index 0000000000..d5925d1985 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/INodeComparator.java @@ -0,0 +1,83 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import java.nio.ByteBuffer; +import org.apache.hadoop.hbase.regionserver.skiplist.exception.ChunkMissingException; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public interface INodeComparator { + + /** + * Compares for order. Returns a negative integer, + * zero, or a positive integer as the first node is less than, equal + * to, or greater than the second.

+ * + * @param left left Key's ByteBuffer + * @param leftOffset left Key's ByteBuffer's offset + * @param leftLen left Key's ByteBuffer's len + * @param right right Key's ByteBuffer + * @param rightOffset right Key's ByteBuffer's offset + * @param rightLen right Key's ByteBuffer's len + * @return An integer result of the comparison. + * @throws ChunkMissingException when can't find Chunk for leftNodeId + **/ + int compareTo(ByteBuffer left, int leftOffset, int leftLen, ByteBuffer right, int rightOffset, + int rightLen); + + /** + * Compares for order. Returns a negative integer, + * zero, or a positive integer as the first node is less than, equal + * to, or greater than the second.

+ * + * @param left left Key's Object + * @param right right Key's ByteBuffer + * @param rightKeyOffset right Key's ByteBuffer's offset + * @param rightKeyLen right Key's ByteBuffer's len + * @return An integer result of the comparison. + * @throws ChunkMissingException when can't find Chunk for leftNodeId + */ + int compareTo(K left, ByteBuffer right, int rightKeyOffset, int rightKeyLen); + + /** + * Compares for order. Returns a negative integer, + * zero, or a positive integer as the first node is less than, equal + * to, or greater than the second.

+ * + * @param left left Key's ByteBuffer + * @param leftKeyOffset left Key's ByteBuffer's offset + * @param leftKeyLen left Key's ByteBuffer's len + * @param right right Key's Object + * @return An integer result of the comparison. + * @throws ChunkMissingException when can't find Chunk for leftNodeId + */ + int compareTo(ByteBuffer left, int leftKeyOffset, int leftKeyLen, K right); + + /** + * Compares for order. Returns a negative integer, + * zero, or a positive integer as the first Key Object is less than, equal + * to, or greater than the second.

+ * + * @param left left Key's Object + * @param right right Key's Object + * @return An integer result of the comparison. + * @throws ChunkMissingException when can't find Chunk for leftNodeId + */ + int compareTo(K left, K right); +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/ISerde.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/ISerde.java new file mode 100644 index 0000000000..16b5a378c1 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/ISerde.java @@ -0,0 +1,60 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import java.nio.ByteBuffer; +import org.apache.hadoop.hbase.regionserver.skiplist.exception.SerdeException; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * serializer and Deserializer. + * Implementation class needs thread-safety + */ +@InterfaceAudience.Private +public interface ISerde { + + /** + * write to ByteBuffer + * + * @param obj key/value + * @param destination the destination ByteBuffer of Chunk + * @param offset the destination ByteBuffer's offset + * @param len data len + * @throws SerdeException SerdeException + */ + void serialize(T obj, ByteBuffer destination, int offset, int len) throws SerdeException; + + /** + * the len of key or value which will be writing to Chunk. + * + * @param obj key/value + * @return the total len + * @throws SerdeException SerdeException + */ + int getSerializedSize(T obj) throws SerdeException; + + /** + * read ByteBuffer + * + * @param bb the ByteBuffer of Chunk + * @return the obj of key or value. + * @throws SerdeException SerdeException + */ + T deserialize(ByteBuffer bb, int offset, int len) throws SerdeException; + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/NodeForKey.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/NodeForKey.java new file mode 100644 index 0000000000..28213c7825 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/NodeForKey.java @@ -0,0 +1,51 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import java.nio.ByteBuffer; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +final class NodeForKey implements INode { + + private final ByteBuffer keyBB; + private final int keyOffset; + private final int keyLen; + + public NodeForKey(ByteBuffer keyBB, int keyOffset, int keyLen) { + this.keyBB = keyBB; + this.keyOffset = keyOffset; + this.keyLen = keyLen; + } + + @Override + public int getKeyOffset() { + return this.keyOffset; + } + + @Override + public int getKeyLen() { + return this.keyLen; + } + + @Override + public ByteBuffer getByteBuffer() { + return this.keyBB; + } + +} + + diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/NodeStat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/NodeStat.java new file mode 100644 index 0000000000..8bc2136608 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/NodeStat.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.hadoop.hbase.regionserver.skiplist.core; + +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public enum NodeStat { + PUT((byte) 7), REMOVE((byte) 15); + + private final byte value; + + NodeStat(byte value) { + this.value = value; + } + + public byte getValue() { + return value; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/NodeUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/NodeUtil.java new file mode 100644 index 0000000000..bc0729b57e --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/NodeUtil.java @@ -0,0 +1,173 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.BYTE_MARK; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.FOUR_BYTES_MARK; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.KEYVALUE_INFRASTRUCTURE_SIZE; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.LEVEL_INDEX_START_OFFSET; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.NEXT_NODE_START_OFFSET; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.SKIP_LIST_HEAD_NODE; +import java.nio.ByteBuffer; +import org.apache.hadoop.hbase.regionserver.skiplist.exception.ChunkMissingException; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * The data schema: + * - meta: + * - int level && lock: + * -- byte1 level(Level is not more than 127) + * -- byte2 NodeStat + * -- byte3 preserve (for NodeLock) + * -- byte4 preserve + * - int dataLen :total data len + * - long getNextNode: the pointer to next Node. + * - long[] levelIndex; The pointer to the corresponding level layer next Node. + * - data: + * -- int keyLen + * -- int valueLen + * -- byte[] key + * -- byte[] value + */ +@InterfaceAudience.Private +public final class NodeUtil { + + //max level is 255. + //reduce repeated meta length calculation + private final static int[] metaLenByLevelArray = new int[255]; + + static { + for (int i = 0; i < metaLenByLevelArray.length; i++) { + metaLenByLevelArray[i] = LEVEL_INDEX_START_OFFSET + i * Long.BYTES; + } + } + + private NodeUtil() { + } + + public static int getNodeMetaLenByLevel(int level) { + return metaLenByLevelArray[level]; + } + + public static int getNodeLen(int level, int keyLen, int totalValueLen) { + return getNodeMetaLenByLevel(level) + getDataLen(keyLen, totalValueLen); + } + + public static int getDataLen(int keyLen, int totalValueLen) { + return keyLen + totalValueLen + KEYVALUE_INFRASTRUCTURE_SIZE; + } + + public static long getChunkIdByNodeId(long nodeId) { + return (nodeId >>> 32) & FOUR_BYTES_MARK; + } + + public static long getChunkOffsetByNodeId(long nodeId) { + return nodeId & FOUR_BYTES_MARK; + } + + public static boolean casSetNodeStat(IChunkVisible chunkVisible, long nodeId, NodeStat nodeStat) { + + if (nodeId == SKIP_LIST_HEAD_NODE) { + return true; + } + + long chunkId = NodeUtil.getChunkIdByNodeId(nodeId); + long offset = NodeUtil.getChunkOffsetByNodeId(nodeId); + ByteBuffer nodeTrunk = chunkVisible.getChunkById(chunkId).getByteBuffer(); + int meta = ByteBufferUtils.toInt(nodeTrunk, (int) offset); + int level = meta & BYTE_MARK; + int expected = level & BYTE_MARK; + int update = (((int) nodeStat.getValue()) & BYTE_MARK) << 8 | expected; + return ByteBufferUtils.compareAndSetInt(nodeTrunk, (int) offset, expected, update); + } + + public static boolean casNodeLock(IChunkVisible chunkVisible, long nodeId, NodeStat nodeStat) { + return doCasNodeLock(chunkVisible, nodeId, nodeStat, true); + } + + public static boolean casNodeUnLock(IChunkVisible chunkVisible, long nodeId, NodeStat nodeStat) { + + return doCasNodeLock(chunkVisible, nodeId, nodeStat, false); + } + + private static boolean doCasNodeLock(IChunkVisible chunkVisible, long nodeId, NodeStat nodeStat, + boolean lock) { + if (nodeId == SKIP_LIST_HEAD_NODE) { + return true; + } + long chunkId = NodeUtil.getChunkIdByNodeId(nodeId); + long offset = NodeUtil.getChunkOffsetByNodeId(nodeId); + ByteBuffer nodeTrunk = chunkVisible.getChunkById(chunkId).getByteBuffer(); + int meta = ByteBufferUtils.toInt(nodeTrunk, (int) offset); + int level = meta & BYTE_MARK; + int expected = level & BYTE_MARK; + int update = ((int) nodeStat.getValue()) << 16 | expected; + if (lock) { + return ByteBufferUtils.compareAndSetInt(nodeTrunk, (int) offset, expected, update); + } else { + return ByteBufferUtils.compareAndSetInt(nodeTrunk, (int) offset, update, expected); + } + } + + public static ByteBuffer duplicateNodeDataByteBuffer(IChunkVisible chunkVisible, long nodeId) { + long chunkId = NodeUtil.getChunkIdByNodeId(nodeId); + long offset = NodeUtil.getChunkOffsetByNodeId(nodeId); + IChunk chunk = chunkVisible.getChunkById(chunkId); + return chunk.asSubByteBuffer(ChunkUtil.getNodeDataOffset(chunk, offset), + ChunkUtil.getNodeDataLen(chunk, offset)); + } + + public static long getNextNodeIdForLevel(IChunkVisible chunkVisible, long nodeId, int curLevel) + throws ChunkMissingException { + long chunkId = NodeUtil.getChunkIdByNodeId(nodeId); + long offset = NodeUtil.getChunkOffsetByNodeId(nodeId); + IChunk chunk = chunkVisible.getChunkById(chunkId); + if (chunk == null) { + throw new ChunkMissingException( + "nodeId=" + nodeId + "chunkid=" + chunkId + "offset=" + offset); + } + return ChunkUtil.getNextNodeIdForLevel(chunk, offset, curLevel); + } + + public static boolean casUpdateNextNodeForLevel(IChunkVisible chunkVisible, int level, + long nodeId, long expected, long update) + throws ChunkMissingException { + long chunkId = (nodeId >>> 32) & FOUR_BYTES_MARK; + long offset = nodeId & FOUR_BYTES_MARK; + IChunk chunk = chunkVisible.getChunkById(chunkId); + if (chunk == null) { + throw new ChunkMissingException("no chunk found. " + chunkId + "=chunkId" + " offset=" + + offset); + } + int offsetS = (int) offset + NEXT_NODE_START_OFFSET + level * Long.BYTES; + return ByteBufferUtils.compareAndSetLong(chunk.getByteBuffer(), offsetS, expected, update); + } + + public static boolean isNodeRemoving(IChunkVisible chunkVisible, long nodeId) + throws ChunkMissingException { + long chunkId = NodeUtil.getChunkIdByNodeId(nodeId); + long offset = NodeUtil.getChunkOffsetByNodeId(nodeId); + IChunk chunk = chunkVisible.getChunkById(chunkId); + if (chunk == null) { + throw new ChunkMissingException("no chunk found. " + chunkId + "=chunkId" + " offset=" + + offset); + } + int meta = ByteBufferUtils.toInt(chunk.getByteBuffer(), (int) offset); + byte lock = (byte) (meta >>> 8 & BYTE_MARK); + return lock == NodeStat.REMOVE.getValue(); + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/OffHeapChunk.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/OffHeapChunk.java new file mode 100644 index 0000000000..621161dbf0 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/OffHeapChunk.java @@ -0,0 +1,41 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import static org.apache.hadoop.hbase.regionserver.skiplist.core.HeapMode.offHeap; +import java.nio.ByteBuffer; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public class OffHeapChunk extends AbstractChunk { + + public OffHeapChunk(long countId, int capacity) { + this(countId, capacity, true); + } + + public OffHeapChunk(long countId, int capacity, boolean isPooled) { + super(countId, capacity, isPooled); + data = ByteBuffer.allocateDirect(capacity); + } + + @Override + public HeapMode getHeapMode() { + return offHeap; + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/OnHeapChunk.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/OnHeapChunk.java new file mode 100644 index 0000000000..b0fdfee65d --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/OnHeapChunk.java @@ -0,0 +1,41 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import static org.apache.hadoop.hbase.regionserver.skiplist.core.HeapMode.onHeap; +import java.nio.ByteBuffer; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public class OnHeapChunk extends AbstractChunk { + + public OnHeapChunk(long chunkId, int capacity) { + this(chunkId, capacity, true); + } + + public OnHeapChunk(long chunkId, int capacity, boolean isPooled) { + super(chunkId, capacity, isPooled); + data = ByteBuffer.allocate(capacity); + } + + @Override + public HeapMode getHeapMode() { + return onHeap; + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/OnHeapIndexHeader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/OnHeapIndexHeader.java new file mode 100644 index 0000000000..c7e753a80b --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/OnHeapIndexHeader.java @@ -0,0 +1,106 @@ +/** + * 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.hadoop.hbase.regionserver.skiplist.core; + +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.BYTE_MARK; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.DEFAULT_LEVEL; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.MAX_LEVEL; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.NIL_NODE_ID; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; + +/** + * Index include the level skip index and link index + * not thread safe + */ +@InterfaceAudience.Private +public class OnHeapIndexHeader implements ILevelIndexHeader { + + //max level is 255. + private volatile int level; + //level Index Head + private volatile AtomicLong[] levelNodeNext = new AtomicLong[DEFAULT_LEVEL]; + //Link Node Head + private final AtomicLong nodeNext = new AtomicLong(NIL_NODE_ID); + + public OnHeapIndexHeader() { + this.level = 1; + initLevelIndex(this.levelNodeNext); + } + + private void initLevelIndex(AtomicLong[] levelIndex) { + for (int i = 0; i < levelIndex.length; i++) { + levelIndex[i] = new AtomicLong(NIL_NODE_ID); + } + } + + @Override + public int getLevel() { + return level & BYTE_MARK; + } + + @VisibleForTesting + public List getLevelNodeNext() { + return Arrays.stream(levelNodeNext).map(AtomicLong::get).collect(Collectors.toList()); + } + + /** + * not thread safe + */ + @Override + public void updateLevel(int dstLevel) { + Preconditions + .checkArgument(dstLevel <= MAX_LEVEL, "dst level>" + MAX_LEVEL + ", It's unbelievable."); + Preconditions.checkArgument(dstLevel >= 1, "dst level less than 1."); + Preconditions.checkArgument(dstLevel <= this.level + 1, + "updateLevel can't skip. now level=" + this.level + " ,dst level=" + dstLevel); + + //default Level is 32, support 4G number of objects,it's rare to expand + if (levelNodeNext.length < dstLevel) { + AtomicLong[] tmpLevelIndex = new AtomicLong[this.levelNodeNext.length * 2]; + initLevelIndex(tmpLevelIndex); + System.arraycopy(this.levelNodeNext, 0, tmpLevelIndex, 0, this.levelNodeNext.length); + this.levelNodeNext = tmpLevelIndex; + } + if (this.level < dstLevel) { + this.level = dstLevel & BYTE_MARK; + } + } + + @Override + public long getLevelNextNodeId(int level) { + Preconditions.checkArgument(level < this.levelNodeNext.length, "invalid overflow level"); + if (level == 0) { + return this.nodeNext.get(); + } + return this.levelNodeNext[level - 1].get(); + } + + @Override + public boolean casUpdateLevelNextNodeId(int level, long expectedNode, long newNode) { + Preconditions.checkArgument(level < this.levelNodeNext.length, "invalid overflow level"); + if (level == 0) { + return this.nodeNext.compareAndSet(expectedNode, newNode); + } + AtomicLong levelNext = levelNodeNext[level - 1]; + return levelNext.compareAndSet(expectedNode, newNode); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/SchemaEnum.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/SchemaEnum.java new file mode 100644 index 0000000000..57cca45c44 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/SchemaEnum.java @@ -0,0 +1,28 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public enum SchemaEnum { + //the key value of Map is same one. such as Map,map.put + same, + + //normal Map,such as Map + diff + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/SubCompactedConcurrentSkipList.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/SubCompactedConcurrentSkipList.java new file mode 100644 index 0000000000..05d31941ab --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/SubCompactedConcurrentSkipList.java @@ -0,0 +1,402 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import static org.apache.hadoop.hbase.regionserver.skiplist.core.AbstractCCSMap.EQ; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.AbstractCCSMap.GT; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.AbstractCCSMap.LT; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.NIL_NODE_ID; +import java.nio.ByteBuffer; +import java.util.NoSuchElementException; +import org.apache.hadoop.hbase.regionserver.skiplist.exception.CCSMapException; +import org.apache.hadoop.hbase.regionserver.skiplist.exception.ChunkMissingException; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Sub view of CompactedConcurrentSkipList + * not support Descending + */ +@InterfaceAudience.Private +final class SubCompactedConcurrentSkipList implements ICCSList { + private final ICCSList ccsl; + private final K lo; + /** + * upper bound key, or null if to end + */ + private final K hi; + /** + * inclusion flag for lo + */ + private final boolean loInclusive; + /** + * inclusion flag for hi + */ + private final boolean hiInclusive; + + class SubIter implements IIterCCSList { + /** + * the last node returned by next() + */ + long lastReturned = NIL_NODE_ID; + /** + * the next node to return from next(); + */ + long next = NIL_NODE_ID; + + /** + * Initializes ascending iterator for entire range. + */ + SubIter() throws CCSMapException { + next = findFirst(); + } + + public final boolean hasNext() { + return next != NIL_NODE_ID; + } + + public long next() { + long n = next; + try { + advance(); + } catch (ChunkMissingException e) { + throw new IllegalStateException(e); + } + return n; + } + + /** + * Advances next to higher entry. + */ + final void advance() throws ChunkMissingException { + if (next == NIL_NODE_ID) { + throw new NoSuchElementException(); + } + lastReturned = next; + long nextId = ccsl.getNextNodeById(next); + while (nextId != NIL_NODE_ID && NodeUtil + .isNodeRemoving(SubCompactedConcurrentSkipList.this, nextId)) { + nextId = getNextNodeById(nextId); + } + next = nextId; + + if (tooHigh(next)) { + next = NIL_NODE_ID; + } + } + + public void remove() { + long l = lastReturned; + if (l == NIL_NODE_ID) { + throw new NoSuchElementException(); + } + // It would not be worth all of the overhead to directly + // unlink from here. Using remove is fast enough. + + try { + ccsl.remove(l); + } catch (CCSMapException e) { + throw new IllegalStateException(e); + } + lastReturned = NIL_NODE_ID; + } + + } + + public SubCompactedConcurrentSkipList(ICCSList ccsl, K lo, K hi, boolean loInclusive, + boolean hiInclusive) { + this.ccsl = ccsl; + this.lo = lo; + this.hi = hi; + this.loInclusive = loInclusive; + this.hiInclusive = hiInclusive; + } + + @Override + public IChunk getChunkById(long chunkId) { + return this.ccsl.getChunkById(chunkId); + } + + @Override + public long findFirst() throws CCSMapException { + return loNode(); + } + + @Override + public long findLast() throws CCSMapException { + return hiNode(); + } + + @Override + public int getSize() { + //it's too slow. + try { + long count = 0; + long n = loNode(); + for (; ; ) { + if (n == NIL_NODE_ID) { + break; + } + + if (tooHigh(n)) { + break; + } + if (!NodeUtil.isNodeRemoving(this, n)) { + ++count; + } + + n = this.ccsl.getNextNodeById(n); + } + return (int) count; + } catch (CCSMapException e) { + throw new IllegalStateException(e); + } + } + + @Override + public void close() { + this.ccsl.close(); + } + + @Override + public CompactedConcurrentSkipList.Stat getCcslStat() { + return this.ccsl.getCcslStat(); + } + + @Override + public IIterCCSList nodeIdIter() throws CCSMapException { + return new SubIter(); + } + + @Override + public long get(K key) throws CCSMapException { + return inBounds(key) ? this.ccsl.get(key) : NIL_NODE_ID; + } + + @Override + public int generateRandomIndexLevel() { + return this.ccsl.generateRandomIndexLevel(); + } + + @Override + public long getNodeAndWriteMeta(int level, int metaLen, int dataLen) throws CCSMapException { + return this.ccsl.getNodeAndWriteMeta(level, metaLen, dataLen); + } + + @Override + public long put(long newNodeId, int level) throws CCSMapException { + checkKeyBounds(newNodeId); + return this.ccsl.put(newNodeId, level); + } + + @Override + public long remove(K key) throws CCSMapException { + return inBounds(key) ? this.ccsl.remove(key) : NIL_NODE_ID; + } + + public long remove(long nodeId) throws CCSMapException { + return inBounds(nodeId) ? this.ccsl.remove(nodeId) : NIL_NODE_ID; + } + + @Override + public INodeComparator getComparator() { + return this.ccsl.getComparator(); + } + + @Override + public long getNextNodeByIdAndLevel(long next, int level) throws CCSMapException { + return this.ccsl.getNextNodeByIdAndLevel(next, level); + } + + @Override + public long getNextNodeById(long next) throws ChunkMissingException { + return this.ccsl.getNextNodeById(next); + } + + @Override + public long findNear(K key, int rel) throws CCSMapException { + + if (tooLow(key)) { + if ((rel & LT) == 0) { + long n = loNode(); + if (isBeforeEnd(n)) { + return n; + } + } + return NIL_NODE_ID; + } + if (tooHigh(key)) { + if ((rel & LT) != 0) { + long n = hiNode(); + if (n != NIL_NODE_ID) { + if (inBounds(n)) { + return n; + } + } + } + return NIL_NODE_ID; + } + + long n = this.ccsl.findNear(key, rel); + if (n == NIL_NODE_ID || !inBounds(n)) { + return NIL_NODE_ID; + } + + return n; + } + + long hiNode() throws CCSMapException { + long result; + if (hi == null) { + result = this.ccsl.findLast(); + } else if (hiInclusive) { + result = this.ccsl.findNear(hi, LT | EQ); + } else { + result = this.ccsl.findNear(hi, LT); + } + + if (result == NIL_NODE_ID) { + return result; + } + + if (tooLow(result)) { + return NIL_NODE_ID; + } + return result; + } + + long loNode() throws CCSMapException { + long result; + if (lo == null) { + result = this.ccsl.findFirst(); + } else if (loInclusive) { + result = this.ccsl.findNear(lo, GT | EQ); + } else { + result = this.ccsl.findNear(lo, GT); + } + + if (result == NIL_NODE_ID) { + return result; + } + + if (tooHigh(result)) { + return NIL_NODE_ID; + } + return result; + } + + private boolean isBeforeEnd(long nodeId) throws ChunkMissingException { + if (nodeId == NIL_NODE_ID) { + return false; + } + if (hi == null) { + return true; + } + int c = compareNodes(nodeId, hi); + return !(c > 0 || (c == 0 && !hiInclusive)); + } + + @Override + public int compareNodes(long leftNodeId, ByteBuffer right, int rightKeyOffset, int rightKeyLen) + throws ChunkMissingException { + return this.ccsl.compareNodes(leftNodeId, right, rightKeyOffset, rightKeyLen); + } + + @Override + public int compareNodes(ByteBuffer left, int leftKeyOffset, int leftKeyLen, long rightNodeId) + throws ChunkMissingException { + return this.ccsl.compareNodes(left, leftKeyOffset, leftKeyLen, rightNodeId); + } + + @Override + public int compareNodes(ByteBuffer left, int leftKeyOffset, int leftKeyLen, ByteBuffer right, + int rightKeyOffset, int rightKeyLen) throws ChunkMissingException { + return this.ccsl + .compareNodes(left, leftKeyOffset, leftKeyLen, right, rightKeyOffset, rightKeyLen); + } + + @Override + public int compareNodes(ByteBuffer left, int leftKeyOffset, int leftKeyLen, K right) + throws ChunkMissingException { + return this.ccsl.compareNodes(left, leftKeyOffset, leftKeyLen, right); + } + + @Override + public int compareNodes(long leftNodeId, K right) throws ChunkMissingException { + return this.ccsl.compareNodes(leftNodeId, right); + } + + @Override + public int compareNodes(K left, long rightNodeId) throws ChunkMissingException { + return this.ccsl.compareNodes(left, rightNodeId); + } + + @Override + public int compareNodes(K left, K right) throws ChunkMissingException { + return this.ccsl.compareNodes(left, right); + } + + @Override + public int getCurMaxLevel() { + return this.ccsl.getCurMaxLevel(); + } + + private boolean inBounds(long nodeId) throws ChunkMissingException { + return !tooLow(nodeId) && !tooHigh(nodeId); + } + + private boolean inBounds(K key) throws ChunkMissingException { + return !tooLow(key) && !tooHigh(key); + } + + private void checkKeyBounds(long nodeId) throws ChunkMissingException { + if (!inBounds(nodeId)) { + throw new IllegalArgumentException("key out of range"); + } + } + + private boolean tooHigh(K key) throws ChunkMissingException { + if (hi == null) { + return false; + } + int c = ccsl.compareNodes(key, hi); + return c > 0 || (c == 0 && !hiInclusive); + } + + private boolean tooHigh(long nodeId) throws ChunkMissingException { + if (hi == null) { + return false; + } + int c = ccsl.compareNodes(nodeId, hi); + return c > 0 || (c == 0 && !hiInclusive); + } + + private boolean tooLow(K key) throws ChunkMissingException { + if (lo == null) { + return false; + } + int c = ccsl.compareNodes(key, lo); + return c < 0 || (c == 0 && !loInclusive); + } + + private boolean tooLow(long nodeId) throws ChunkMissingException { + if (lo == null) { + return false; + } + int c = ccsl.compareNodes(nodeId, lo); + return c < 0 || (c == 0 && !loInclusive); + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/UnsafeHelp.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/UnsafeHelp.java new file mode 100644 index 0000000000..18e6e1491b --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/core/UnsafeHelp.java @@ -0,0 +1,93 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import java.lang.reflect.Field; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.security.AccessController; +import java.security.PrivilegedAction; +import org.apache.yetus.audience.InterfaceAudience; +import sun.misc.Unsafe; +import sun.nio.ch.DirectBuffer; + +@InterfaceAudience.Private +public final class UnsafeHelp { + + // Unsafe mechanics + static final Unsafe UNSAFE; + + /** + * The offset to the first element in a byte array. + */ + public static final long BYTE_ARRAY_BASE_OFFSET; + + static final boolean littleEndian = ByteOrder.nativeOrder().equals(ByteOrder.LITTLE_ENDIAN); + + static { + try { + UNSAFE = (Unsafe) AccessController.doPrivileged(new PrivilegedAction() { + @Override + public Object run() { + try { + Field f = Unsafe.class.getDeclaredField("theUnsafe"); + f.setAccessible(true); + return f.get(null); + } catch (Throwable e) { + throw new Error(e); + } + } + }); + BYTE_ARRAY_BASE_OFFSET = UNSAFE.arrayBaseOffset(byte[].class); + } catch (Exception e) { + throw new Error(e); + } + } + + private UnsafeHelp() { + } + + public static boolean compareAndSetLong(ByteBuffer buf, int offset, long expected, long update) { + if (littleEndian) { + expected = Long.reverseBytes(expected); + update = Long.reverseBytes(update); + } + if (buf.isDirect()) { + return UNSAFE + .compareAndSwapLong(null, ((DirectBuffer) buf).address() + offset, expected, update); + } + return UNSAFE + .compareAndSwapLong(buf.array(), BYTE_ARRAY_BASE_OFFSET + buf.arrayOffset() + offset, + expected, update); + } + + public static boolean compareAndSetInt(ByteBuffer buf, int offset, int expected, int update) { + if (littleEndian) { + expected = Integer.reverseBytes(expected); + update = Integer.reverseBytes(update); + } + if (buf.isDirect()) { + return UNSAFE + .compareAndSwapInt(null, ((DirectBuffer) buf).address() + offset, expected, update); + } + return UNSAFE + .compareAndSwapInt(buf.array(), BYTE_ARRAY_BASE_OFFSET + buf.arrayOffset() + offset, + expected, update); + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/exception/CCSMapException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/exception/CCSMapException.java new file mode 100644 index 0000000000..75a7b715c2 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/exception/CCSMapException.java @@ -0,0 +1,42 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.exception; + +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public class CCSMapException extends Exception { + + private static final long serialVersionUID = 3710550267352306257L; + + public CCSMapException() { + super(); + } + + public CCSMapException(String message) { + super(message); + } + + public CCSMapException(String message, Throwable cause) { + super(message, cause); + } + + public CCSMapException(Throwable cause) { + super(cause); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/exception/ChunkMissingException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/exception/ChunkMissingException.java new file mode 100644 index 0000000000..f3159abaf5 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/exception/ChunkMissingException.java @@ -0,0 +1,42 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.exception; + +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public class ChunkMissingException extends CCSMapException { + + private static final long serialVersionUID = -1842797678596726118L; + + public ChunkMissingException() { + super(); + } + + public ChunkMissingException(String message) { + super(message); + } + + public ChunkMissingException(String message, Throwable cause) { + super(message, cause); + } + + public ChunkMissingException(Throwable cause) { + super(cause); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/exception/HeapFullException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/exception/HeapFullException.java new file mode 100644 index 0000000000..78f4366970 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/exception/HeapFullException.java @@ -0,0 +1,42 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.exception; + +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public class HeapFullException extends CCSMapException { + + private static final long serialVersionUID = -2473741608268169616L; + + public HeapFullException() { + super(); + } + + public HeapFullException(String message) { + super(message); + } + + public HeapFullException(String message, Throwable cause) { + super(message, cause); + } + + public HeapFullException(Throwable cause) { + super(cause); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/exception/SerdeException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/exception/SerdeException.java new file mode 100644 index 0000000000..7e91419e9a --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/exception/SerdeException.java @@ -0,0 +1,42 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.exception; + +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public class SerdeException extends CCSMapException { + + private static final long serialVersionUID = 4936582923968747781L; + + public SerdeException() { + super(); + } + + public SerdeException(String message) { + super(message); + } + + public SerdeException(String message, Throwable cause) { + super(message, cause); + } + + public SerdeException(Throwable cause) { + super(cause); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/CCSMapCellComparatorDefault.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/CCSMapCellComparatorDefault.java new file mode 100644 index 0000000000..0de7f6cf74 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/CCSMapCellComparatorDefault.java @@ -0,0 +1,81 @@ +/** + * 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.hadoop.hbase.regionserver.skiplist.hbase; + +import java.nio.ByteBuffer; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.CellComparatorImpl; +import org.apache.hadoop.hbase.regionserver.skiplist.core.INodeComparator; +import org.apache.hadoop.hbase.regionserver.skiplist.exception.SerdeException; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * CCSMapCellComparatorDefault will autoBoxing the Cell. + * it's slower than CCSMapCellComparatorDirectly + */ +@InterfaceAudience.Private +public class CCSMapCellComparatorDefault implements INodeComparator { + + public static final CCSMapCellComparatorDefault INSTNANCE = + new CCSMapCellComparatorDefault(CellComparatorImpl.COMPARATOR); + + private final CellComparator cellComparator; + + public CCSMapCellComparatorDefault(CellComparator cellComparator) { + this.cellComparator = cellComparator; + } + + @Override + public int compareTo(ByteBuffer left, int leftOffset, int leftLen, ByteBuffer right, + int rightOffset, int rightLen) { + try { + Cell leftCell = CellSerde.INSTANCE.deserialize(left, leftOffset, leftLen); + Cell rightCell = CellSerde.INSTANCE.deserialize(right, rightOffset, rightLen); + return this.compareTo(leftCell, rightCell); + } catch (SerdeException e) { + throw new IllegalStateException(e); + } + } + + @Override + public int compareTo(Cell left, ByteBuffer right, int rightOffset, int rightLen) { + try { + Cell rightCell = CellSerde.INSTANCE.deserialize(right, rightOffset, rightLen); + return this.compareTo(left, rightCell); + } catch (SerdeException e) { + throw new IllegalStateException(e); + } + } + + @Override + public int compareTo(ByteBuffer left, int leftOffset, int leftLen, Cell right) { + try { + Cell leftCell = CellSerde.INSTANCE.deserialize(left, leftOffset, leftLen); + return this.compareTo(leftCell, right); + } catch (SerdeException e) { + throw new IllegalStateException(e); + } + } + + @Override + public int compareTo(Cell left, Cell right) { + return this.cellComparator.compare(left, right); + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/CCSMapCellComparatorDirectly.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/CCSMapCellComparatorDirectly.java new file mode 100644 index 0000000000..09e4fee43c --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/CCSMapCellComparatorDirectly.java @@ -0,0 +1,258 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.hbase; + +import static org.apache.hadoop.hbase.CellComparatorImpl.COMPARATOR; +import java.nio.ByteBuffer; +import org.apache.hadoop.hbase.ByteBufferExtendedCell; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.regionserver.skiplist.core.ByteBufferUtils; +import org.apache.hadoop.hbase.regionserver.skiplist.core.INodeComparator; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hbase.thirdparty.com.google.common.primitives.Longs; + +/** + * CCSMapCellComparatorDirectly compare the ByteBuffer directly, avoid autoboxing the Cell + */ +@InterfaceAudience.Private +public class CCSMapCellComparatorDirectly implements INodeComparator { + + public static final CCSMapCellComparatorDirectly INSTNANCE = + new CCSMapCellComparatorDirectly(COMPARATOR); + + private final CellComparator cellComparator; + + public CCSMapCellComparatorDirectly(CellComparator cellComparator) { + this.cellComparator = cellComparator; + } + + @Override + public int compareTo(ByteBuffer left, int leftOffset, int leftLen, ByteBuffer right, + int rightOffset, int rightLen) { + + int rowOffsetBase = KeyValue.ROW_OFFSET + Long.BYTES; + int rowDataOffsetBase = KeyValue.ROW_LENGTH_SIZE + rowOffsetBase; + int leftRowDataOffset = leftOffset + rowDataOffsetBase; + int rightRowdataOffset = rightOffset + rowDataOffsetBase; + int leftRowDataLeght = ByteBufferUtils.toShort(left, leftOffset + rowOffsetBase); + int rightRowDataLeght = ByteBufferUtils.toShort(right, rightOffset + rowOffsetBase); + int c; + //compare row + c = ByteBufferUtils + .compareTo(left, leftRowDataOffset, leftRowDataLeght, right, rightRowdataOffset, + rightRowDataLeght); + if (c != 0) { + return c; + } + + //compare without row + + int lFamOffset = leftRowDataOffset + leftRowDataLeght; + int rFamOffset = rightRowdataOffset + rightRowDataLeght; + + int lFamLength = ByteBufferUtils.toByte(left, lFamOffset); + int rFamLength = ByteBufferUtils.toByte(right, rFamOffset); + + int lKeyLen = ByteBufferUtils.toInt(left, leftOffset + Long.BYTES); + int rKeyLen = ByteBufferUtils.toInt(right, rightOffset + Long.BYTES); + + int lQualLength = lKeyLen - KeyValue.KEY_INFRASTRUCTURE_SIZE - leftRowDataLeght - lFamLength; + int rQualLength = rKeyLen - KeyValue.KEY_INFRASTRUCTURE_SIZE - rightRowDataLeght - rFamLength; + + int lTypeByte = -999; + if (lFamLength + lQualLength == 0) { + lTypeByte = ByteBufferUtils.toByte(left, leftOffset + lKeyLen + rowOffsetBase - 1); + if (lTypeByte == KeyValue.Type.Minimum.getCode()) { + // left is "bigger", i.e. it appears later in the sorted order + return 1; + } + } + int rTypeByte = -999; + if (rFamLength + rQualLength == 0) { + rTypeByte = ByteBufferUtils.toByte(right, rightOffset + rKeyLen + rowOffsetBase - 1); + if (rTypeByte == KeyValue.Type.Minimum.getCode()) { + return -1; + } + } + + // comparing column family + c = ByteBufferUtils + .compareTo(left, lFamOffset + Byte.BYTES, lFamLength, right, rFamOffset + Byte.BYTES, + rFamLength); + if (c != 0) { + return c; + } + // Compare qualifier + c = ByteBufferUtils.compareTo(left, lFamOffset + Byte.BYTES + lFamLength, lQualLength, right, + rFamOffset + Byte.BYTES + rFamLength, rQualLength); + if (c != 0) { + return c; + } + + //compare TimeStamp + long leftTS = ByteBufferUtils + .toLong(left, leftOffset + rowOffsetBase + lKeyLen - KeyValue.TIMESTAMP_TYPE_SIZE); + long rightTS = ByteBufferUtils + .toLong(right, rightOffset + rowOffsetBase + rKeyLen - KeyValue.TIMESTAMP_TYPE_SIZE); + + c = Longs.compare(rightTS, leftTS); + if (c != 0) { + return c; + } + + //compare type + if (lTypeByte == -999) { + lTypeByte = ByteBufferUtils.toByte(left, leftOffset + lKeyLen + rowOffsetBase - 1); + } + + if (rTypeByte == -999) { + rTypeByte = ByteBufferUtils.toByte(right, rightOffset + rKeyLen + rowOffsetBase - 1); + } + + c = (0xff & rTypeByte) - (0xff & lTypeByte); + if (c != 0) { + return c; + } + + //compare sequenceId + //always compare sequenceID + long leftSequenceId = ByteBufferUtils.toLong(left, leftOffset); + long rightSequenceId = ByteBufferUtils.toLong(right, rightOffset); + return Longs.compare(rightSequenceId, leftSequenceId); + + } + + @Override + public int compareTo(Cell left, ByteBuffer right, int rightOffset, int rightLen) { + int result = this.compareTo(right, rightOffset, rightLen, left); + return result < 0 ? 1 : result == 0 ? 0 : -1; + } + + @Override + public int compareTo(ByteBuffer left, int leftOffset, int leftLen, Cell right) { + int rowOffsetBase = KeyValue.ROW_OFFSET + Long.BYTES; + int rowDataOffsetBase = KeyValue.ROW_LENGTH_SIZE + rowOffsetBase; + int leftRowDataOffset = leftOffset + rowDataOffsetBase; + int leftRowDataLeght = ByteBufferUtils.toShort(left, leftOffset + rowOffsetBase); + int c; + //compare row + + if (right instanceof ByteBufferExtendedCell) { + c = ByteBufferUtils.compareTo(left, leftRowDataOffset, leftRowDataLeght, + ((ByteBufferExtendedCell) right).getRowByteBuffer(), + ((ByteBufferExtendedCell) right).getRowPosition(), right.getRowLength()); + } else { + c = ByteBufferUtils.compareTo(left, leftRowDataOffset, leftRowDataLeght, right.getRowArray(), + right.getRowOffset(), right.getRowLength()); + } + + if (c != 0) { + return c; + } + + //compare without row + + int lFamOffset = leftRowDataOffset + leftRowDataLeght; + + int lFamLength = ByteBufferUtils.toByte(left, lFamOffset); + int rFamLength = right.getFamilyLength(); + + int lKeyLen = ByteBufferUtils.toInt(left, leftOffset + Long.BYTES); + + int lQualLength = lKeyLen - KeyValue.KEY_INFRASTRUCTURE_SIZE - leftRowDataLeght - lFamLength; + int rQualLength = right.getQualifierLength(); + + int lTypeByte = -999; + if (lFamLength + lQualLength == 0) { + lTypeByte = ByteBufferUtils.toByte(left, leftOffset + lKeyLen + rowOffsetBase - 1); + if (lTypeByte == KeyValue.Type.Minimum.getCode()) { + // left is "bigger", i.e. it appears later in the sorted order + return 1; + } + } + int rTypeByte = right.getTypeByte(); + if (rFamLength + rQualLength == 0) { + if (rTypeByte == KeyValue.Type.Minimum.getCode()) { + return -1; + } + } + + // comparing column family + + if (right instanceof ByteBufferExtendedCell) { + c = ByteBufferUtils.compareTo(left, lFamOffset + Byte.BYTES, lFamLength, + ((ByteBufferExtendedCell) right).getFamilyByteBuffer(), + ((ByteBufferExtendedCell) right).getFamilyPosition(), right.getFamilyLength()); + } else { + c = ByteBufferUtils + .compareTo(left, lFamOffset + Byte.BYTES, lFamLength, right.getFamilyArray(), + right.getFamilyOffset(), right.getFamilyLength()); + } + + if (c != 0) { + return c; + } + // Compare qualifier + if (right instanceof ByteBufferExtendedCell) { + c = ByteBufferUtils.compareTo(left, lFamOffset + Byte.BYTES + lFamLength, lQualLength, + ((ByteBufferExtendedCell) right).getQualifierByteBuffer(), + ((ByteBufferExtendedCell) right).getQualifierPosition(), right.getQualifierLength()); + } else { + c = ByteBufferUtils.compareTo(left, lFamOffset + Byte.BYTES + lFamLength, lQualLength, + right.getQualifierArray(), right.getQualifierOffset(), right.getQualifierLength()); + } + + if (c != 0) { + return c; + } + + //compare TimeStamp + long leftTS = ByteBufferUtils + .toLong(left, leftOffset + rowOffsetBase + lKeyLen - KeyValue.TIMESTAMP_TYPE_SIZE); + long rightTS = right.getTimestamp(); + + c = Longs.compare(rightTS, leftTS); + if (c != 0) { + return c; + } + + //compare type + if (lTypeByte == -999) { + lTypeByte = ByteBufferUtils.toByte(left, leftOffset + lKeyLen + rowOffsetBase - 1); + } + + c = (0xff & rTypeByte) - (0xff & lTypeByte); + if (c != 0) { + return c; + } + + //compare sequenceId + //always compare sequenceID + long leftSequenceId = ByteBufferUtils.toLong(left, leftOffset); + long rightSequenceId = right.getSequenceId(); + return Longs.compare(rightSequenceId, leftSequenceId); + + } + + @Override + public int compareTo(Cell left, Cell right) { + return this.cellComparator.compare(left, right); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/CCSMapMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/CCSMapMemStore.java new file mode 100644 index 0000000000..d509fc38c7 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/CCSMapMemStore.java @@ -0,0 +1,112 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.hbase; + +import static org.apache.hadoop.hbase.regionserver.HStore.MEMSTORE_CLASS_NAME; +import static org.apache.hadoop.hbase.regionserver.skiplist.hbase.CellCCSMap.CCSMAP_COMPARATOR_DEFAULT; +import static org.apache.hadoop.hbase.regionserver.skiplist.hbase.CellCCSMap.CCSMAP_COMPARATOR_KEY; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.MemoryCompactionPolicy; +import org.apache.hadoop.hbase.regionserver.CompactingMemStore; +import org.apache.hadoop.hbase.regionserver.DefaultMemStore; +import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.regionserver.MemStoreSizing; +import org.apache.hadoop.hbase.regionserver.MemStoreSnapshot; +import org.apache.hadoop.hbase.regionserver.SegmentFactory; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@InterfaceAudience.Private +public class CCSMapMemStore extends DefaultMemStore { + private static final Logger LOG = LoggerFactory.getLogger(CCSMapMemStore.class); + public static final String CHUNK_CAPACITY_KEY = "hbase.hregion.memstore.ccsmap.chunkcapacity"; + public static final String EXTRA_CHUNK_WARN_CAPACITY = + "hbase.hregion.memstore.ccsmap.extra.chunk.warn.capacity"; + public static final String CHUNK_SIZE_KEY = "hbase.hregion.memstore.ccsmap.chunksize"; + public static final String INITIAL_CHUNK_COUNT_KEY = + "hbase.hregion.memstore.ccsmap.chunk.initial.count"; + public static final int CHUNK_SIZE_DEFAULT = 8 * 1024 * 1024; + public static final String CCSMAP_USE_OFFHEAP = "hbase.hregion.memstore.ccsmap.use.offheap"; + + public CCSMapMemStore(final Configuration conf, final CellComparator c) { + super(conf, c); + LOG.debug("CCSMap memStore, comparator type={}; isMeta={}", + conf.get(CCSMAP_COMPARATOR_KEY, CCSMAP_COMPARATOR_DEFAULT), isMeta); + } + + protected void resetActive() { + // Reset heap to not include any keys + this.active = SegmentFactory.instance().createMutableSegmentCCSMap(conf, comparator, isMeta); + this.timeOfOldestEdit = Long.MAX_VALUE; + } + + protected void initSnapshot(final CellComparator c) { + this.snapshot = SegmentFactory.instance().createImmutableSegment(c); + } + + @Override + public MemStoreSnapshot snapshot() { + // If snapshot currently has entries, then flusher failed or didn't call + // cleanup. Log a warning. + if (!this.snapshot.isEmpty()) { + LOG.warn("Snapshot called again without clearing previous. " + + "Doing nothing. Another ongoing flush or did we fail last attempt?"); + } else { + this.snapshotId = EnvironmentEdgeManager.currentTime(); + if (!this.active.isEmpty()) { + this.snapshot = SegmentFactory.instance(). + createImmutableSegmentCCSMap(this.active); + resetActive(); + } + } + return new MemStoreSnapshot(this.snapshotId, this.snapshot); + + } + + @Override + public void add(Cell cell, MemStoreSizing memstoreSize) { + internalAdd(cell, memstoreSize); + } + + private void internalAdd(final Cell toAdd, MemStoreSizing memstoreSize) { + active.add(toAdd, true, memstoreSize); + setOldestEditTimeToNow(); + checkActiveSize(); + } + + public void upsert(Cell cell, long readpoint, MemStoreSizing memstoreSize) { + + this.active.upsert(cell, readpoint, memstoreSize); + setOldestEditTimeToNow(); + checkActiveSize(); + } + + public static boolean isEnabled(Configuration conf) { + String className = conf.get(MEMSTORE_CLASS_NAME, HStore.DEFAULT_MEMSTORE_CLASS_NAME); + MemoryCompactionPolicy inMemoryCompaction = MemoryCompactionPolicy.valueOf( + conf.get(CompactingMemStore.COMPACTING_MEMSTORE_TYPE_KEY, + CompactingMemStore.COMPACTING_MEMSTORE_TYPE_DEFAULT)); + return inMemoryCompaction == MemoryCompactionPolicy.NONE && className + .equals(CCSMapMemStore.class.getName()); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/CCSMapMetaCellComparatorDefault.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/CCSMapMetaCellComparatorDefault.java new file mode 100644 index 0000000000..3f3f1fee57 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/CCSMapMetaCellComparatorDefault.java @@ -0,0 +1,81 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.hbase; + +import static org.apache.hadoop.hbase.CellComparatorImpl.META_COMPARATOR; +import java.nio.ByteBuffer; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.regionserver.skiplist.core.INodeComparator; +import org.apache.hadoop.hbase.regionserver.skiplist.exception.SerdeException; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * For HBase meta. + * CCSMapMetaCellComparatorDefault will autoBoxing the Cell. + */ +@InterfaceAudience.Private +public class CCSMapMetaCellComparatorDefault implements INodeComparator { + + public static final CCSMapMetaCellComparatorDefault INSTNANCE = + new CCSMapMetaCellComparatorDefault(META_COMPARATOR); + + private final CellComparator metaCellComparator; + + public CCSMapMetaCellComparatorDefault(CellComparator metaCellComparator) { + this.metaCellComparator = metaCellComparator; + } + + @Override + public int compareTo(ByteBuffer left, int leftOffset, int leftLen, ByteBuffer right, + int rightOffset, int rightLen) { + try { + Cell leftCell = CellSerde.INSTANCE.deserialize(left, leftOffset, leftLen); + Cell rightCell = CellSerde.INSTANCE.deserialize(right, rightOffset, rightLen); + return this.compareTo(leftCell, rightCell); + } catch (SerdeException e) { + throw new IllegalStateException(e); + } + } + + @Override + public int compareTo(Cell left, ByteBuffer right, int rightOffset, int rightLen) { + try { + Cell rightCell = CellSerde.INSTANCE.deserialize(right, rightOffset, rightLen); + return this.compareTo(left, rightCell); + } catch (SerdeException e) { + throw new IllegalStateException(e); + } + } + + @Override + public int compareTo(ByteBuffer left, int leftOffset, int leftLen, Cell right) { + try { + Cell leftCell = CellSerde.INSTANCE.deserialize(left, leftOffset, leftLen); + return this.compareTo(leftCell, right); + } catch (SerdeException e) { + throw new IllegalStateException(e); + } + } + + @Override + public int compareTo(Cell left, Cell right) { + return this.metaCellComparator.compare(left, right); + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/CellCCSMap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/CellCCSMap.java new file mode 100644 index 0000000000..8ac9edeb70 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/CellCCSMap.java @@ -0,0 +1,72 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.hbase; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.regionserver.skiplist.core.AllocatorHandlerRegister; +import org.apache.hadoop.hbase.regionserver.skiplist.core.AllocatorHandlerRegister.AllocatorHandlerBuilder; +import org.apache.hadoop.hbase.regionserver.skiplist.core.BaseTwinCCSMap; +import org.apache.hadoop.hbase.regionserver.skiplist.core.CCSMapChunkPool; +import org.apache.hadoop.hbase.regionserver.skiplist.core.INodeComparator; +import org.apache.hadoop.hbase.regionserver.skiplist.core.ISerde; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * for HBase use NavigableMap<\Cell,Cell\> simplify. + */ +@InterfaceAudience.Private +public class CellCCSMap extends BaseTwinCCSMap { + public static final String CCSMAP_COMPARATOR_KEY = "hbase.hregion.memstore.ccsmap.comparator"; + public static final String CCSMAP_COMPARATOR_DEFAULT = "directly"; + + public static final String TEST_CHUNK_CAPACITY_KEY = + "test.hbase.hregion.memstore.ccsmap.chunkcapacity"; + public static final String TEST_CHUNK_SIZE_KEY = "test.hbase.hregion.memstore.ccsmap.chunksize"; + public static final String TEST_CCSMAP_USE_OFFHEAP = + "test.hbase.hregion.memstore.ccsmap.use.offheap"; + + public CellCCSMap(AllocatorHandlerRegister.AllocatorHandlerBuilder builder, + INodeComparator cellComparator, ISerde keyvalueSerde) { + super(builder, cellComparator, keyvalueSerde, Cell.class); + } + + public static CellCCSMap buildCellCCSMap(final Configuration conf, CellComparator comparator, + boolean isMeta) { + ISerde cellSerde = CellSerde.INSTANCE; + if (AllocatorHandlerBuilder.getINSTANCE() == null) { + //only for test + long capacity = conf.getLong(TEST_CHUNK_CAPACITY_KEY, 1024 * 1024 * 1024L); + int chunkSize = conf.getInt(TEST_CHUNK_SIZE_KEY, 4 * 1024); + boolean offheap = conf.getBoolean(TEST_CCSMAP_USE_OFFHEAP, false); + CCSMapChunkPool.initialize(conf, capacity, chunkSize, 10, offheap); + } + + INodeComparator nodeComparator = CCSMapCellComparatorDirectly.INSTNANCE; + if ("useDefault".equalsIgnoreCase(conf.get(CCSMAP_COMPARATOR_KEY, CCSMAP_COMPARATOR_DEFAULT))) { + nodeComparator = CCSMapCellComparatorDefault.INSTNANCE; + } + if (isMeta) { + nodeComparator = CCSMapMetaCellComparatorDefault.INSTNANCE; + } + return new CellCCSMap(AllocatorHandlerRegister.AllocatorHandlerBuilder.getINSTANCE(), + nodeComparator, cellSerde); + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/CellSerde.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/CellSerde.java new file mode 100644 index 0000000000..338d982191 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/CellSerde.java @@ -0,0 +1,70 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.hbase; + +import java.nio.ByteBuffer; +import org.apache.hadoop.hbase.ByteBufferKeyValue; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.ExtendedCell; +import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.regionserver.skiplist.core.ByteBufferUtils; +import org.apache.hadoop.hbase.regionserver.skiplist.core.ISerde; +import org.apache.hadoop.hbase.regionserver.skiplist.exception.SerdeException; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * HBase Cell Serde + */ +@InterfaceAudience.Private +public class CellSerde implements ISerde { + public static final CellSerde INSTANCE = new CellSerde(); + + @Override + public int getSerializedSize(Cell cell) throws SerdeException { + // Normally all Cell impls within Server will be of type ExtendedCell. Just considering the + // other case also. The data fragments within Cell is copied into buf as in KeyValue + // serialization format only. + + // Long is for seqId,We need save the Seq on the CCSMap. + // we need true for total all data len. + return KeyValueUtil.getSerializedSize(cell, true) + Long.BYTES; + } + + @Override + public void serialize(Cell cell, ByteBuffer destination, int offset, int len) + throws SerdeException { + //seqId is the first. + ByteBufferUtils.putLong(destination, offset, cell.getSequenceId()); + + if (cell instanceof ExtendedCell) { + ((ExtendedCell) cell).write(destination, offset + Long.BYTES); + } else { + // Normally all Cell impls within Server will be of type ExtendedCell. Just considering the + // other case also. The data fragments within Cell is copied into buf as in KeyValue + // serialization format only. + KeyValueUtil.appendTo(cell, destination, offset + Long.BYTES, true); + } + } + + @Override + public Cell deserialize(ByteBuffer bb, int offset, int len) throws SerdeException { + long seqId = ByteBufferUtils.toLong(bb, offset); + return new ByteBufferKeyValue(bb, offset + Long.BYTES, len - Long.BYTES, seqId); + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/ImmutableSegmentOnCCSMap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/ImmutableSegmentOnCCSMap.java new file mode 100644 index 0000000000..d8a8e592a5 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/ImmutableSegmentOnCCSMap.java @@ -0,0 +1,39 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.hbase; + +import org.apache.hadoop.hbase.regionserver.ImmutableSegment; +import org.apache.hadoop.hbase.regionserver.Segment; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public class ImmutableSegmentOnCCSMap extends ImmutableSegment { + + public ImmutableSegmentOnCCSMap(Segment segment) { + super(segment); + } + + @Override + protected long indexEntrySize() { + return 0; + } + + @Override protected boolean canBeFlattened() { + return false; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/MemstoreLABProxyForCCSMap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/MemstoreLABProxyForCCSMap.java new file mode 100644 index 0000000000..364e85a1b6 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/MemstoreLABProxyForCCSMap.java @@ -0,0 +1,102 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.hbase; + +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.regionserver.Chunk; +import org.apache.hadoop.hbase.regionserver.ChunkCreator; +import org.apache.hadoop.hbase.regionserver.MemStoreLAB; +import org.apache.hadoop.hbase.regionserver.skiplist.core.ICCSMap; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Actually, CCSMap not need MemStoreLAB, but need interface as incScannerCount/decScannerCount + * TODO introduce the new interface ScannerCountable, and MemStoreLAB implement ScannerCountable + */ +@InterfaceAudience.Private +public class MemstoreLABProxyForCCSMap implements MemStoreLAB { + private static final Logger LOG = LoggerFactory.getLogger(MemstoreLABProxyForCCSMap.class); + // Current count of open scanners which reading data from this MemStoreLAB + private final AtomicInteger openScannerCount = new AtomicInteger(); + private volatile boolean closed = false; + + private final ICCSMap ccsmap; + + public MemstoreLABProxyForCCSMap(ICCSMap ccsmap) { + this.ccsmap = ccsmap; + } + + @Override + public Cell copyCellInto(Cell cell) { + return cell; + } + + @Override + public Cell forceCopyOfBigCellInto(Cell cell) { + return null; + } + + @Override + public void close() { + this.closed = true; + // We could put back the chunks to pool for reusing only when there is no + // opening scanner which will read their data + int count = openScannerCount.get(); + if (count == 0) { + LOG.debug("will close CCSMap: {}", ccsmap.getCcslStat().toString()); + ccsmap.close(); + } + } + + @Override + public void incScannerCount() { + this.openScannerCount.incrementAndGet(); + } + + @Override + public void decScannerCount() { + int count = this.openScannerCount.decrementAndGet(); + if (this.closed && count == 0) { + LOG.debug("will close CCSMap on decScanner: {}", ccsmap.getCcslStat().toString()); + ccsmap.close(); + } + } + + @Override + public Chunk getNewExternalChunk(ChunkCreator.ChunkType chunkType) { + throw new UnsupportedOperationException("ccsmap not support"); + } + + @Override + public Chunk getNewExternalChunk(int size) { + throw new UnsupportedOperationException("ccsmap not support"); + } + + @Override + public boolean isOnHeap() { + throw new UnsupportedOperationException("ccsmap not support"); + } + + @Override + public boolean isOffHeap() { + throw new UnsupportedOperationException("ccsmap not support"); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/MutableSegmentOnCCSMap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/MutableSegmentOnCCSMap.java new file mode 100644 index 0000000000..3836df4729 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/MutableSegmentOnCCSMap.java @@ -0,0 +1,71 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.hbase; + +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.regionserver.CellSet; +import org.apache.hadoop.hbase.regionserver.MemStoreLAB; +import org.apache.hadoop.hbase.regionserver.MemStoreSizing; +import org.apache.hadoop.hbase.regionserver.MutableSegment; +import org.apache.hadoop.hbase.regionserver.skiplist.core.Constant; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * A mutable segment in memstore, specifically the active segment. + */ +@InterfaceAudience.Private +public class MutableSegmentOnCCSMap extends MutableSegment { + + public MutableSegmentOnCCSMap(CellSet cellSet, CellComparator comparator, + MemStoreLAB memStoreLAB) { + super(cellSet, comparator, memStoreLAB); + } + + protected void updateMetaInfo(Cell cellToAdd, boolean succ, boolean mslabUsed, + MemStoreSizing memstoreSize) { + //We can't get the really size from CellSet API. + //8 is mean 8B, this value come from the more test. it's a conservative estimate value. + //Feature, we can expand the CellSet API to support return the real size. + long cellSize = getDataSizeByCell(cellToAdd); + long heapSize = getHeapSizeByCell(cellToAdd); + incSize(cellSize, heapSize, cellSize); + if (memstoreSize != null) { + memstoreSize.incMemStoreSize(cellSize, heapSize, cellSize); + } + getTimeRangeTracker().includeTimestamp(cellToAdd); + minSequenceId = Math.min(minSequenceId, cellToAdd.getSequenceId()); + // In no tags case this NoTagsKeyValue.getTagsLength() is a cheap call. + // When we use ACL CP or Visibility CP which deals with Tags during + // mutation, the TagRewriteCell.getTagsLength() is a cheaper call. We do not + // parse the byte[] to identify the tags length. + if (cellToAdd.getTagsLength() > 0) { + tagsPresent = true; + } + } + + protected int getDataSizeByCell(Cell cell) { + return getCellLength(cell) + Constant.LEVEL_INDEX_START_OFFSET + Long.BYTES //seq + + 8; + } + + protected long getHeapSizeByCell(Cell cell) { + return 0; + } + +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java index 06d617aad4..a263978f17 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.io.hfile; +import static org.apache.hadoop.hbase.regionserver.HStore.MEMSTORE_CLASS_NAME; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -39,6 +40,7 @@ import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.regionserver.ChunkCreator; +import org.apache.hadoop.hbase.regionserver.DefaultMemStore; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.InternalScanner; @@ -87,6 +89,7 @@ public class TestScannerFromBucketCache { conf.setFloat("hfile.block.cache.size", 0.2f); conf.setFloat("hbase.regionserver.global.memstore.size", 0.1f); } + conf.set(MEMSTORE_CLASS_NAME, DefaultMemStore.class.getName()); tableName = TableName.valueOf(name.getMethodName()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java index 6da5ec0a74..66b008294c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java @@ -1102,5 +1102,6 @@ public class TestDefaultMemStore { for (int i = 0 ; i < 50 ; i++) doScan(ms, i); } + } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java index 8dadd9ba02..f686abfd49 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java @@ -1470,7 +1470,8 @@ public class TestHStore { public void testRunDoubleMemStoreCompactors() throws IOException, InterruptedException { int flushSize = 500; Configuration conf = HBaseConfiguration.create(); - conf.set(HStore.MEMSTORE_CLASS_NAME, MyCompactingMemStoreWithCustomCompactor.class.getName()); + conf.set(HStore.COMPACTING_MEMSTORE_CLASS_NAME, + MyCompactingMemStoreWithCustomCompactor.class.getName()); conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.25); MyCompactingMemStoreWithCustomCompactor.RUNNER_COUNT.set(0); conf.set(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, String.valueOf(flushSize)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java index 4f3de36a33..fe257f306b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.regionserver; +import static org.apache.hadoop.hbase.regionserver.HStore.MEMSTORE_CLASS_NAME; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -27,6 +28,7 @@ import java.util.List; import java.util.Random; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ByteBufferKeyValue; +import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; @@ -60,6 +62,7 @@ public class TestMemStoreChunkPool { public static void setUpBeforeClass() throws Exception { conf.setBoolean(MemStoreLAB.USEMSLAB_KEY, true); conf.setFloat(MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY, 0.2f); + conf.set(MEMSTORE_CLASS_NAME, DefaultMemStore.class.getName()); chunkPoolDisabledBeforeTest = ChunkCreator.chunkPoolDisabled; ChunkCreator.chunkPoolDisabled = false; long globalMemStoreLimit = (long) (ManagementFactory.getMemoryMXBean().getHeapMemoryUsage() @@ -126,7 +129,7 @@ public class TestMemStoreChunkPool { byte[] qf5 = Bytes.toBytes("testqualifier5"); byte[] val = Bytes.toBytes("testval"); - DefaultMemStore memstore = new DefaultMemStore(); + DefaultMemStore memstore = new DefaultMemStore(conf, CellComparatorImpl.COMPARATOR); // Setting up memstore memstore.add(new KeyValue(row, fam, qf1, val), null); @@ -167,7 +170,7 @@ public class TestMemStoreChunkPool { byte[] qf7 = Bytes.toBytes("testqualifier7"); byte[] val = Bytes.toBytes("testval"); - DefaultMemStore memstore = new DefaultMemStore(); + DefaultMemStore memstore = new DefaultMemStore(conf, CellComparatorImpl.COMPARATOR); // Setting up memstore memstore.add(new KeyValue(row, fam, qf1, val), null); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAccounting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAccounting.java index 7bd9e163a0..87590bde36 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAccounting.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAccounting.java @@ -17,8 +17,8 @@ */ package org.apache.hadoop.hbase.regionserver; +import static org.apache.hadoop.hbase.regionserver.HStore.MEMSTORE_CLASS_NAME; import static org.junit.Assert.assertEquals; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; @@ -39,6 +39,7 @@ public class TestRegionServerAccounting { public void testOnheapMemstoreHigherWaterMarkLimits() { Configuration conf = HBaseConfiguration.create(); conf.setFloat(MemorySizeUtil.MEMSTORE_SIZE_KEY, 0.2f); + conf.set(MEMSTORE_CLASS_NAME, DefaultMemStore.class.getName()); // try for default cases RegionServerAccounting regionServerAccounting = new RegionServerAccounting(conf); MemStoreSize memstoreSize = @@ -52,6 +53,7 @@ public class TestRegionServerAccounting { public void testOnheapMemstoreLowerWaterMarkLimits() { Configuration conf = HBaseConfiguration.create(); conf.setFloat(MemorySizeUtil.MEMSTORE_SIZE_KEY, 0.2f); + conf.set(MEMSTORE_CLASS_NAME, DefaultMemStore.class.getName()); // try for default cases RegionServerAccounting regionServerAccounting = new RegionServerAccounting(conf); MemStoreSize memstoreSize = @@ -61,6 +63,34 @@ public class TestRegionServerAccounting { regionServerAccounting.isAboveLowWaterMark()); } + @Test + public void testOnheapMemstoreLowerWaterMarkLimitsForCCSMap() { + Configuration conf = HBaseConfiguration.create(); + conf.set(CompactingMemStore.COMPACTING_MEMSTORE_TYPE_KEY, "NONE"); + conf.setFloat(MemorySizeUtil.MEMSTORE_SIZE_KEY, 0.2f); + // try for default cases + RegionServerAccounting regionServerAccounting = new RegionServerAccounting(conf); + MemStoreSize memstoreSize = + new MemStoreSize((3L * 1024L * 1024L * 1024L), (1L * 1024L * 1024L * 1024L), + (3L * 1024L * 1024L * 1024L)); + regionServerAccounting.incGlobalMemStoreSize(memstoreSize); + assertEquals(FlushType.ABOVE_OFFHEAP_LOWER_MARK, regionServerAccounting.isAboveLowWaterMark()); + } + + @Test + public void testOnheapMemstoreHigherWaterMarkLimitsForCCSMap() { + Configuration conf = HBaseConfiguration.create(); + conf.set(CompactingMemStore.COMPACTING_MEMSTORE_TYPE_KEY, "NONE"); + conf.setFloat(MemorySizeUtil.MEMSTORE_SIZE_KEY, 0.2f); + // try for default cases + RegionServerAccounting regionServerAccounting = new RegionServerAccounting(conf); + MemStoreSize memstoreSize = + new MemStoreSize((3L * 1024L * 1024L * 1024L), (1L * 1024L * 1024L * 1024L), + (3L * 1024L * 1024L * 1024L)); + regionServerAccounting.incGlobalMemStoreSize(memstoreSize); + assertEquals(FlushType.ABOVE_OFFHEAP_HIGHER_MARK, + regionServerAccounting.isAboveHighWaterMark()); + } @Test public void testOffheapMemstoreHigherWaterMarkLimitsDueToDataSize() { Configuration conf = HBaseConfiguration.create(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/MockBaseCCSMap.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/MockBaseCCSMap.java new file mode 100644 index 0000000000..927fe5774f --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/MockBaseCCSMap.java @@ -0,0 +1,101 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import java.nio.ByteBuffer; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.regionserver.skiplist.exception.SerdeException; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; +import org.junit.experimental.categories.Category; + +@Category(SmallTests.class) +public final class MockBaseCCSMap extends BaseCCSMap { + + @ClassRule public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(MockBaseCCSMap.class); + + public static final INodeComparator CCSMAP_TEST_BYTE_COMPARTOR = + new INodeComparator() { + @Override + public int compareTo(ByteBuffer left, int leftOffset, int leftLen, ByteBuffer right, + int rightOffset, int rightLen) { + return ByteBufferUtils.compareTo(left, leftOffset, leftLen, right, rightOffset, rightLen); + } + + @Override + public int compareTo(byte[] left, ByteBuffer right, int rightKeyOffset, int rightKeyLen) { + ByteBuffer leftBB = ByteBuffer.wrap(left); + return ByteBufferUtils + .compareTo(leftBB, 0, left.length, right, rightKeyOffset, rightKeyLen); + } + + @Override + public int compareTo(ByteBuffer left, int leftKeyOffset, int leftKeyLen, byte[] right) { + ByteBuffer rightBB = ByteBuffer.wrap(right); + return ByteBufferUtils.compareTo(left, leftKeyOffset, leftKeyLen, rightBB, 0, right.length); + } + + @Override + public int compareTo(byte[] left, byte[] right) { + return Bytes.compareTo(left, right); + } + }; + + private MockBaseCCSMap(AllocatorHandlerRegister.AllocatorHandlerBuilder builder, + INodeComparator comparator, ISerde keySerde, + ISerde valueSerde, Class keyClass) { + + super(builder, comparator, keySerde, valueSerde, keyClass); + } + + public static MockBaseCCSMap newMockBaseCCSMap(HeapMode heapMode) { + + AllocatorHandlerRegister.AllocatorHandlerBuilder builder = + new AllocatorHandlerRegister.AllocatorHandlerBuilder(); + builder.setCapacity(512 * 1024 * 1024); + builder.setChunkSize(4 * 1024 * 1024); + builder.setInitialCount(Integer.MAX_VALUE); + builder.setHeapMode(heapMode); + CCSMapChunkPool allocatorHandler = new CCSMapChunkPool(builder); + AllocatorHandlerRegister.register(allocatorHandler); + ISerde keySerde = new ISerde() { + @Override + public void serialize(byte[] obj, ByteBuffer destination, int offset, int len) + throws SerdeException { + org.apache.hadoop.hbase.util.ByteBufferUtils + .copyFromArrayToBuffer(destination, offset, obj, 0, obj.length); + } + + @Override + public int getSerializedSize(byte[] obj) throws SerdeException { + return obj.length; + } + + @Override + public byte[] deserialize(ByteBuffer bb, int offset, int len) throws SerdeException { + byte[] result = new byte[len]; + org.apache.hadoop.hbase.util.ByteBufferUtils + .copyFromBufferToArray(result, bb, offset, 0, len); + return result; + } + + }; + return new MockBaseCCSMap(builder, CCSMAP_TEST_BYTE_COMPARTOR, keySerde, keySerde, + byte[].class); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestAbstractChunk.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestAbstractChunk.java new file mode 100644 index 0000000000..1c1ae4f5f2 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestAbstractChunk.java @@ -0,0 +1,303 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import java.nio.ByteBuffer; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(SmallTests.class) +public class TestAbstractChunk { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAbstractChunk.class); + @Test + public void testNormal() throws Exception { + int len = 4 * 1024 * 1024; + OnHeapChunk chunk = new OnHeapChunk(1234, len); + Assert.assertEquals(1234, chunk.getChunkId()); + Assert.assertEquals(0, chunk.getPosition()); + Assert.assertEquals(len, chunk.getLimit()); + Assert.assertEquals(0, chunk.getUsedPercent()); + Assert.assertTrue(chunk.isPooledChunk()); + + Assert.assertEquals(len, chunk.occupancy()); + // + long position1 = chunk.allocate(1023); + Assert.assertEquals(0, position1); + long position2 = chunk.allocate(1025); + Assert.assertEquals(1024, position2); + Assert.assertEquals(1024 + 1032, chunk.getPosition()); + chunk.reclaim(position1, 1025); + Assert.assertEquals(len - 1024 - 1032 + 1025, chunk.occupancy()); + + //test usedPercent + chunk.setLastAllocateStartTime(); + Thread.sleep(1000); + chunk.setLastAllocateEndTime(); + Thread.sleep(1100); + Assert.assertTrue(chunk.getUsedPercent() > 0); + Assert.assertTrue(chunk.getUsedPercent() < 50); + + String expectedException = "WANT_EXCEPTION"; + try { + chunk.setLastAllocateEndTime(); + } catch (Exception e) { + expectedException = e.getMessage(); + } + Assert.assertEquals("chunk stat error. no allocateTime, cur=-1", expectedException); + + chunk.setLastAllocateStartTime(); + expectedException = "WANT_EXCEPTION"; + try { + chunk.setLastAllocateStartTime(); + } catch (Exception e) { + expectedException = e.getMessage(); + } + Assert.assertTrue(expectedException.contains("chunk stat error. existed allocateTime")); + + ByteBuffer bb = chunk.getByteBuffer(); + Assert.assertEquals(len, bb.limit()); + Assert.assertEquals(len, bb.capacity()); + Assert.assertEquals(0, bb.position()); + + int len2 = 4096; + ByteBuffer bb2 = chunk.asSubByteBuffer(100, len2); + Assert.assertEquals(len2, bb2.limit()); + Assert.assertEquals(len2, bb2.capacity()); + Assert.assertEquals(0, bb2.position()); + + Assert.assertEquals(1234, chunk.hashCode()); + + OnHeapChunk chunk2 = new OnHeapChunk(1234, len); + //As long as chunkId is same, Chunk is the same + Assert.assertEquals(chunk, chunk2); + Assert.assertEquals(len, chunk.getByteBuffer().limit()); + + OnHeapChunk chunk3 = new OnHeapChunk(1235, len, false); + Assert.assertFalse(chunk3.isPooledChunk()); + } + + @Test + public void testHugeChunk() throws Exception { + int len = 1023; + HugeOnHeapChunk chunk = new HugeOnHeapChunk(1234, len); + Assert.assertFalse(chunk.isPooledChunk()); + + String expectException = "WANT_EXCEPTION"; + try { + chunk.allocate(1233); + } catch (Exception e) { + expectException = e.getMessage(); + } + Assert.assertEquals("HugeChunk's capacity not fit the nodeLen(1233).", expectException); + + chunk.allocate(1023); + Assert.assertEquals(len, chunk.getPosition()); + + expectException = "WANT_EXCEPTION"; + try { + chunk.allocate(1023); + } catch (Exception e) { + expectException = e.getMessage(); + } + Assert.assertEquals("HugeChunk only use once, cur offset=1023", expectException); + } + + @SuppressWarnings("AssertionFailureIgnored") + @Test + public void testConcurrentWriteOffHeap() throws Exception { + int len = 4 * 1024 * 1024; + OffHeapChunk chunk = new OffHeapChunk(1234, len); + + int concurrent = 50; + ByteBuffer[] bbArray = new ByteBuffer[concurrent]; + + for (int i = 0; i < concurrent; i++) { + bbArray[i] = chunk.asSubByteBuffer(i * 2049, 1023); + } + + AtomicBoolean hasError = new AtomicBoolean(false); + Thread[] ths = new Thread[concurrent]; + + for (int i = 0; i < concurrent; i++) { + final int thid = i; + ths[i] = new Thread(new Runnable() { + @Override + public void run() { + ByteBuffer bb = ByteBuffer.allocate(13); + bb.put((byte) thid); + bb.putInt(thid); + bb.putLong(thid); + bb.flip(); + try { + Assert.assertEquals(0, bbArray[thid].position()); + Thread.sleep(1000); + bbArray[thid].put((byte) thid); + Assert.assertEquals(1, bbArray[thid].position()); + Thread.sleep(1000); + bbArray[thid].putInt(thid); + Assert.assertEquals(1 + 4, bbArray[thid].position()); + Thread.sleep(1000); + bbArray[thid].putLong((long) thid); + Assert.assertEquals(1 + 4 + 8, bbArray[thid].position()); + Thread.sleep(1000); + bbArray[thid].put(bb); + Assert.assertEquals(1 + 4 + 8 + 13, bbArray[thid].position()); + } catch (Throwable e) { + e.printStackTrace(); + hasError.set(true); + } + } + }); + } + + for (int j = 0; j < concurrent; j++) { + ths[j].start(); + } + + for (int j = 0; j < concurrent; j++) { + ths[j].join(); + } + + Assert.assertTrue(!hasError.get()); + + for (int j = 0; j < concurrent; j++) { + bbArray[j].rewind(); + Assert.assertEquals(0, bbArray[j].position()); + Assert.assertEquals(j, bbArray[j].get()); + Assert.assertEquals(1, bbArray[j].position()); + Assert.assertEquals(j, bbArray[j].getInt()); + Assert.assertEquals(1 + 4, bbArray[j].position()); + Assert.assertEquals(j, bbArray[j].getLong()); + Assert.assertEquals(1 + 4 + 8, bbArray[j].position()); + Assert.assertEquals(j, bbArray[j].get()); + Assert.assertEquals(1 + 4 + 8 + 1, bbArray[j].position()); + Assert.assertEquals(j, bbArray[j].getInt()); + Assert.assertEquals(1 + 4 + 8 + 1 + 4, bbArray[j].position()); + Assert.assertEquals(j, bbArray[j].getLong()); + Assert.assertEquals(1 + 4 + 8 + 1 + 4 + 8, bbArray[j].position()); + } + + ByteBuffer bb = chunk.getByteBuffer(); + bb.rewind(); + for (int j = 0; j < concurrent; j++) { + bb.position(j * 2049); + Assert.assertEquals(j, bb.get()); + Assert.assertEquals(j, bb.getInt()); + Assert.assertEquals(j, bb.getLong()); + Assert.assertEquals(j, bb.get()); + Assert.assertEquals(j, bb.getInt()); + Assert.assertEquals(j, bb.getLong()); + } + } + + @SuppressWarnings("AssertionFailureIgnored") + @Test + public void testConcurrentWriteOnHeap() throws Exception { + int len = 4 * 1024 * 1024; + OnHeapChunk chunk = new OnHeapChunk(1234, len); + + int concurrent = 50; + ByteBuffer[] bbArray = new ByteBuffer[concurrent]; + + for (int i = 0; i < concurrent; i++) { + bbArray[i] = chunk.asSubByteBuffer(i * 2049, 1023); + } + + AtomicBoolean hasError = new AtomicBoolean(false); + Thread[] ths = new Thread[concurrent]; + + for (int i = 0; i < concurrent; i++) { + final int thid = i; + ths[i] = new Thread(new Runnable() { + @Override + public void run() { + ByteBuffer bb = ByteBuffer.allocate(13); + bb.put((byte) thid); + bb.putInt(thid); + bb.putLong(thid); + bb.flip(); + try { + Assert.assertEquals(0, bbArray[thid].position()); + Thread.sleep(100); + bbArray[thid].put((byte) thid); + Assert.assertEquals(1, bbArray[thid].position()); + Thread.sleep(100); + bbArray[thid].putInt(thid); + Assert.assertEquals(1 + 4, bbArray[thid].position()); + Thread.sleep(100); + bbArray[thid].putLong((long) thid); + Assert.assertEquals(1 + 4 + 8, bbArray[thid].position()); + Thread.sleep(100); + bbArray[thid].put(bb); + Assert.assertEquals(1 + 4 + 8 + 13, bbArray[thid].position()); + } catch (Throwable e) { + e.printStackTrace(); + hasError.set(true); + } + } + }); + } + + for (int j = 0; j < concurrent; j++) { + ths[j].start(); + } + + for (int j = 0; j < concurrent; j++) { + ths[j].join(); + } + + Assert.assertTrue(!hasError.get()); + + for (int j = 0; j < concurrent; j++) { + bbArray[j].rewind(); + Assert.assertEquals(0, bbArray[j].position()); + Assert.assertEquals(j, bbArray[j].get()); + Assert.assertEquals(1, bbArray[j].position()); + Assert.assertEquals(j, bbArray[j].getInt()); + Assert.assertEquals(1 + 4, bbArray[j].position()); + Assert.assertEquals(j, bbArray[j].getLong()); + Assert.assertEquals(1 + 4 + 8, bbArray[j].position()); + Assert.assertEquals(j, bbArray[j].get()); + Assert.assertEquals(1 + 4 + 8 + 1, bbArray[j].position()); + Assert.assertEquals(j, bbArray[j].getInt()); + Assert.assertEquals(1 + 4 + 8 + 1 + 4, bbArray[j].position()); + Assert.assertEquals(j, bbArray[j].getLong()); + Assert.assertEquals(1 + 4 + 8 + 1 + 4 + 8, bbArray[j].position()); + } + + ByteBuffer bb = chunk.getByteBuffer(); + bb.rewind(); + for (int j = 0; j < concurrent; j++) { + bb.position(j * 2049); + Assert.assertEquals(j, bb.get()); + Assert.assertEquals(j, bb.getInt()); + Assert.assertEquals(j, bb.getLong()); + Assert.assertEquals(j, bb.get()); + Assert.assertEquals(j, bb.getInt()); + Assert.assertEquals(j, bb.getLong()); + } + } +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestBaseTwinCCSMap.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestBaseTwinCCSMap.java new file mode 100644 index 0000000000..450a67ca46 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestBaseTwinCCSMap.java @@ -0,0 +1,126 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import static org.apache.hadoop.hbase.regionserver.skiplist.core.MockBaseCCSMap.CCSMAP_TEST_BYTE_COMPARTOR; +import java.nio.ByteBuffer; +import java.util.Random; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.regionserver.skiplist.exception.SerdeException; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(SmallTests.class) +public class TestBaseTwinCCSMap { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBaseTwinCCSMap.class); + private final Random random = new Random(7); + + private static final ISerde sameSerde = new ISerde() { + @Override + public void serialize(byte[] obj, ByteBuffer destination, int offset, int len) + throws SerdeException { + org.apache.hadoop.hbase.util.ByteBufferUtils + .copyFromArrayToBuffer(destination, offset, obj, 0, obj.length); + } + + @Override + public int getSerializedSize(byte[] obj) throws SerdeException { + return obj.length; + } + + @Override + public byte[] deserialize(ByteBuffer bb, int offset, int len) throws SerdeException { + byte[] result = new byte[len]; + org.apache.hadoop.hbase.util.ByteBufferUtils + .copyFromBufferToArray(result, bb, offset, 0, result.length); + return result; + } + + }; + + private static final ISerde errorSerde = new ISerde() { + @Override + public void serialize(byte[] obj, ByteBuffer destination, int offset, int len) + throws SerdeException { + org.apache.hadoop.hbase.util.ByteBufferUtils + .copyFromArrayToBuffer(destination, offset, obj, 0, obj.length); + } + + @Override + public int getSerializedSize(byte[] obj) throws SerdeException { + return obj.length; + } + + @Override + public byte[] deserialize(ByteBuffer bb, int offset, int len) throws SerdeException { + byte[] result = new byte[len]; + org.apache.hadoop.hbase.util.ByteBufferUtils + .copyFromBufferToArray(result, bb, offset, 0, result.length); + return result; + } + + }; + + @Test + public void testNormal() throws Exception { + doNormal(sameSerde); + doNormal(errorSerde); + } + + private void doNormal(ISerde serde){ + BaseTwinCCSMap testCCSMap = newBaseTwinCCSMap(HeapMode.onHeap, serde); + + int keyLenght = 50; + byte[] bytes = new byte[keyLenght]; + random.nextBytes(bytes); + testCCSMap.put(bytes, bytes); + + Assert.assertEquals(1, testCCSMap.size()); + + byte[] dst = testCCSMap.get(bytes); + Assert.assertArrayEquals(bytes, dst); + + byte[] rm = testCCSMap.remove(bytes); + Assert.assertArrayEquals(bytes, rm); + + Assert.assertEquals(null, testCCSMap.get(bytes)); + } + + public static BaseTwinCCSMap newBaseTwinCCSMap(HeapMode heapMode) { + return newBaseTwinCCSMap(heapMode, sameSerde); + } + + public static BaseTwinCCSMap newBaseTwinCCSMap(HeapMode heapMode, ISerde serde) { + + AllocatorHandlerRegister.AllocatorHandlerBuilder builder = + new AllocatorHandlerRegister.AllocatorHandlerBuilder(); + builder.setCapacity(512 * 1024 * 1024); + builder.setChunkSize(4 * 1024 * 1024); + builder.setInitialCount(Integer.MAX_VALUE); + builder.setHeapMode(heapMode); + CCSMapChunkPool allocatorHandler = new CCSMapChunkPool(builder); + AllocatorHandlerRegister.register(allocatorHandler); + + return new BaseTwinCCSMap<>(builder, CCSMAP_TEST_BYTE_COMPARTOR, serde, byte[].class); + } +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestByteBufferUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestByteBufferUtils.java new file mode 100644 index 0000000000..60f21ececc --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestByteBufferUtils.java @@ -0,0 +1,153 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import java.nio.ByteBuffer; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(SmallTests.class) +public class TestByteBufferUtils { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestByteBufferUtils.class); + @Test + public void testBBWriteAndRead() throws Exception { + ByteBuffer bb = ByteBuffer.allocateDirect(4096); + doTest(bb, 0); + doTest(bb, 1); + doTest(bb, 2); + doTest(bb, 3); + doTest(bb, 4); + doTest(bb, 5); + doTest(bb, 6); + doTest(bb, 7); + + bb = ByteBuffer.allocate(4096); + doTest(bb, 0); + doTest(bb, 1); + doTest(bb, 2); + doTest(bb, 3); + doTest(bb, 4); + doTest(bb, 5); + doTest(bb, 6); + doTest(bb, 7); + } + + private void doTest(ByteBuffer bb, int offset) { + int positionOri = bb.position(); + ByteBufferUtils.putInt(bb, offset, 123); + Assert.assertEquals(positionOri, bb.position()); + Assert.assertEquals(123, ByteBufferUtils.toInt(bb, offset)); + Assert.assertEquals(positionOri, bb.position()); + + ByteBufferUtils.putLong(bb, offset + 4, 1234); + Assert.assertEquals(positionOri, bb.position()); + Assert.assertEquals(1234, ByteBufferUtils.toLong(bb, offset + 4)); + Assert.assertEquals(positionOri, bb.position()); + + Assert.assertEquals(123, ByteBufferUtils.toInt(bb, offset)); + Assert.assertEquals(positionOri, bb.position()); + } + + @Test + public void testBBCAS() throws Exception { + ByteBuffer bb = ByteBuffer.allocate(4096); + doTestBBCAS(bb, 0); + doTestBBCAS(bb, 12); + doTestBBCAS(bb, 25); + + bb = ByteBuffer.allocateDirect(4096); + ByteBufferUtils.toLong(bb, 0); + Assert.assertEquals(0, ByteBufferUtils.toLong(bb, 0)); + + doTestBBCAS(bb, 0); + doTestBBCAS(bb, 12); + doTestBBCAS(bb, 25); + } + + private void doTestBBCAS(ByteBuffer bb, int offset) { + + int positionOri = bb.position(); + Assert.assertTrue(!ByteBufferUtils.compareAndSetInt(bb, offset, 11, 101)); + + Assert.assertEquals(positionOri, bb.position()); + int v0 = ByteBufferUtils.toInt(bb, offset); + Assert.assertEquals(0, v0); + Assert.assertEquals(positionOri, bb.position()); + + Assert.assertTrue(ByteBufferUtils.compareAndSetInt(bb, offset, 0, 101)); + Assert.assertEquals(positionOri, bb.position()); + + int v1 = ByteBufferUtils.toInt(bb, offset); + Assert.assertEquals(101, v1); + Assert.assertEquals(positionOri, bb.position()); + + Assert.assertTrue(ByteBufferUtils.compareAndSetInt(bb, offset, v1, 102)); + Assert.assertEquals(positionOri, bb.position()); + + int v2 = ByteBufferUtils.toInt(bb, offset); + Assert.assertEquals(102, v2); + Assert.assertEquals(positionOri, bb.position()); + + Assert.assertTrue(!ByteBufferUtils.compareAndSetInt(bb, offset, 11, 103)); + + int v3 = ByteBufferUtils.toInt(bb, offset); + Assert.assertEquals(v3, 102); + Assert.assertEquals(positionOri, bb.position()); + + Assert.assertTrue(!ByteBufferUtils.compareAndSetLong(bb, offset + 4, 11, 101)); + + long value0 = ByteBufferUtils.toLong(bb, offset + 4); + Assert.assertEquals(0, value0); + Assert.assertEquals(positionOri, bb.position()); + + Assert.assertTrue(ByteBufferUtils.compareAndSetLong(bb, offset + 4, 0, 101)); + Assert.assertEquals(positionOri, bb.position()); + + long value1 = ByteBufferUtils.toLong(bb, offset + 4); + Assert.assertEquals(101, value1); + Assert.assertEquals(positionOri, bb.position()); + + Assert.assertTrue(ByteBufferUtils.compareAndSetLong(bb, offset + 4, value1, 102)); + Assert.assertEquals(positionOri, bb.position()); + + long value2 = ByteBufferUtils.toLong(bb, offset + 4); + Assert.assertEquals(102, value2); + Assert.assertEquals(positionOri, bb.position()); + + Assert.assertTrue(!ByteBufferUtils.compareAndSetLong(bb, offset + 4, 11, 103)); + + long value3 = ByteBufferUtils.toLong(bb, offset + 4); + Assert.assertEquals(102, value3); + Assert.assertEquals(positionOri, bb.position()); + + ByteBufferUtils.compareTo(bb, 0, 4, bb, 0, 4); + Assert.assertEquals(positionOri, bb.position()); + + ByteBuffer bb2 = ByteBuffer.allocate(4096); + ByteBufferUtils.copyFromBufferToBuffer(bb, bb2, 0, 0, 4); + Assert.assertEquals(positionOri, bb.position()); + } + +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestCCSMap.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestCCSMap.java new file mode 100644 index 0000000000..91d2d3ba50 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestCCSMap.java @@ -0,0 +1,387 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import static org.apache.hadoop.hbase.regionserver.skiplist.core.TestBaseTwinCCSMap.newBaseTwinCCSMap; +import java.util.Collection; +import java.util.Iterator; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Random; +import java.util.Set; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.Assert; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(SmallTests.class) +public class TestCCSMap { + private final Random random = new Random(7); + private byte[] startKey = new byte[4]; + private byte[] endKey = new byte[4]; + + @ClassRule public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCCSMap.class); + + @Before + public void setUp() throws Exception { + Bytes.putInt(startKey, 0, 8); + Bytes.putInt(endKey, 0, 28); + } + + @Test + public void testCCSMapBasic() throws Exception { + NavigableMap testCCSMap = MockBaseCCSMap.newMockBaseCCSMap(HeapMode.onHeap); + + Assert.assertArrayEquals(null, testCCSMap.firstKey()); + Assert.assertArrayEquals(null, testCCSMap.lastKey()); + + int keyLenght = 50; + byte[] bytes = new byte[keyLenght]; + random.nextBytes(bytes); + testCCSMap.put(bytes, bytes); + + Assert.assertEquals(1, testCCSMap.size()); + + byte[] dst = testCCSMap.get(bytes); + Assert.assertArrayEquals(bytes, dst); + + Set> entrySet = testCCSMap.entrySet(); + Assert.assertEquals(1, entrySet.size()); + Map.Entry entry = entrySet.iterator().next(); + Assert.assertArrayEquals(bytes, entry.getKey()); + Assert.assertArrayEquals(bytes, entry.getValue()); + + Assert.assertArrayEquals(bytes, testCCSMap.firstKey()); + Assert.assertArrayEquals(bytes, testCCSMap.lastKey()); + + byte[] rm = testCCSMap.remove(bytes); + Assert.assertArrayEquals(bytes, rm); + + Assert.assertEquals(null, testCCSMap.get(bytes)); + + Assert.assertArrayEquals(null, testCCSMap.firstKey()); + Assert.assertArrayEquals(null, testCCSMap.lastKey()); + } + + @Test + public void testBaseTwinCCSMap() throws Exception { + int keyLenght = 50; + final NavigableMap testMap = newBaseTwinCCSMap(HeapMode.onHeap); + + Assert.assertArrayEquals(null, testMap.firstKey()); + Assert.assertArrayEquals(null, testMap.lastKey()); + + byte[] bytes = new byte[keyLenght]; + random.nextBytes(bytes); + testMap.put(bytes, bytes); + + Assert.assertEquals(1, testMap.size()); + + byte[] dst = testMap.get(bytes); + Assert.assertArrayEquals(bytes, dst); + + Assert.assertArrayEquals(bytes, testMap.firstKey()); + Assert.assertArrayEquals(bytes, testMap.lastKey()); + + byte[] rm = testMap.remove(bytes); + Assert.assertArrayEquals(bytes, rm); + + Assert.assertEquals(null, testMap.get(bytes)); + Assert.assertArrayEquals(null, testMap.firstKey()); + Assert.assertArrayEquals(null, testMap.lastKey()); + } + + @Test + public void testCCSLBasicMapSubMapWhenMapIsEmpty() throws Exception { + final NavigableMap testMap = MockBaseCCSMap.newMockBaseCCSMap(HeapMode.onHeap); + //map is empty + Assert.assertEquals(null, testMap.firstKey()); + Assert.assertEquals(null, testMap.lastKey()); + Assert.assertEquals(null, testMap.lowerKey(startKey)); + Assert.assertEquals(null, testMap.floorKey(startKey)); + Assert.assertEquals(null, testMap.ceilingKey(startKey)); + Assert.assertEquals(null, testMap.higherKey(startKey)); + + Assert.assertEquals(null, testMap.firstEntry()); + Assert.assertEquals(null, testMap.lastEntry()); + Assert.assertEquals(null, testMap.lowerEntry(startKey)); + Assert.assertEquals(null, testMap.floorEntry(startKey)); + Assert.assertEquals(null, testMap.ceilingEntry(startKey)); + Assert.assertEquals(null, testMap.higherEntry(startKey)); + } + + @Test + public void testCCSLBasicMapSubMapWhenMapIsNotEmpty() throws Exception { + final NavigableMap testMap = MockBaseCCSMap.newMockBaseCCSMap(HeapMode.onHeap); + + //put some data + for (int i = 0; i < 50; i++) { + byte[] bb = new byte[4]; + Bytes.putInt(bb, 0, (i + 1) * 2); + testMap.put(bb, bb); + } + + Assert.assertEquals(50, testMap.size()); + Assert.assertEquals(2, Bytes.toInt(testMap.firstKey())); + Assert.assertEquals(100, Bytes.toInt(testMap.lastKey())); + Assert.assertEquals(6, Bytes.toInt(testMap.lowerKey(startKey))); + Assert.assertEquals(8, Bytes.toInt(testMap.floorKey(startKey))); + Assert.assertEquals(8, Bytes.toInt(testMap.ceilingKey(startKey))); + Assert.assertEquals(28, Bytes.toInt(testMap.ceilingKey(endKey))); + Assert.assertEquals(30, Bytes.toInt(testMap.higherKey(endKey))); + + Assert.assertEquals(2, Bytes.toInt(testMap.firstEntry().getKey())); + Assert.assertEquals(100, Bytes.toInt(testMap.lastEntry().getKey())); + Assert.assertEquals(6, Bytes.toInt(testMap.lowerEntry(startKey).getKey())); + Assert.assertEquals(8, Bytes.toInt(testMap.floorEntry(startKey).getKey())); + Assert.assertEquals(8, Bytes.toInt(testMap.ceilingEntry(startKey).getKey())); + Assert.assertEquals(28, Bytes.toInt(testMap.ceilingEntry(endKey).getKey())); + Assert.assertEquals(30, Bytes.toInt(testMap.higherEntry(endKey).getKey())); + + final NavigableMap headMap1 = testMap.headMap(endKey, false); + Assert.assertEquals(13, headMap1.size()); + Assert.assertEquals(2, Bytes.toInt(headMap1.firstKey())); + Assert.assertEquals(26, Bytes.toInt(headMap1.lastKey())); + + final NavigableMap headMap2 = testMap.headMap(endKey, true); + Assert.assertEquals(14, headMap2.size()); + Assert.assertEquals(2, Bytes.toInt(headMap2.firstKey())); + Assert.assertEquals(28, Bytes.toInt(headMap2.lastKey())); + + final NavigableMap tailMap1 = testMap.tailMap(startKey, false); + Assert.assertEquals(46, tailMap1.size()); + Assert.assertEquals(10, Bytes.toInt(tailMap1.firstKey())); + Assert.assertEquals(100, Bytes.toInt(tailMap1.lastKey())); + + final NavigableMap tailMap2 = testMap.tailMap(startKey, true); + Assert.assertEquals(47, tailMap2.size()); + Assert.assertEquals(8, Bytes.toInt(tailMap2.firstKey())); + Assert.assertEquals(100, Bytes.toInt(tailMap2.lastKey())); + + Set> allEntrySet = testMap.entrySet(); + Collection allValues = testMap.values(); + Set allKeys = testMap.keySet(); + + Assert.assertEquals(50, allEntrySet.size()); + Assert.assertEquals(50, allValues.size()); + Assert.assertEquals(50, allKeys.size()); + + int i = 0; + for (Map.Entry entry : testMap.entrySet()) { + Assert.assertEquals((i + 1) * 2, Bytes.toInt(entry.getKey())); + i++; + } + + i = 0; + for (byte[] value : testMap.values()) { + Assert.assertEquals((i + 1) * 2, Bytes.toInt(value)); + i++; + } + + i = 0; + for (byte[] value : testMap.keySet()) { + Assert.assertEquals((i + 1) * 2, Bytes.toInt(value)); + i++; + } + + Assert.assertEquals(13, headMap1.entrySet().size()); + Assert.assertEquals(13, headMap1.values().size()); + Assert.assertEquals(13, headMap1.keySet().size()); + + i = 0; + for (Map.Entry entry : headMap1.entrySet()) { + Assert.assertEquals((i + 1) * 2, Bytes.toInt(entry.getKey())); + i++; + } + + i = 0; + for (byte[] value : headMap1.values()) { + Assert.assertEquals((i + 1) * 2, Bytes.toInt(value)); + i++; + } + + i = 0; + for (byte[] value : headMap1.keySet()) { + Assert.assertEquals((i + 1) * 2, Bytes.toInt(value)); + i++; + } + doTestAfterRemoveHead(testMap, headMap1); + doTestAfterRemoveTail(testMap, headMap1, tailMap1); + } + + private void doTestAfterRemoveTail(NavigableMap testMap, + NavigableMap headMap1, + NavigableMap tailMap1) { + int i;//remove the tail + + Iterator tailIterator = tailMap1.keySet().iterator(); + i = 0; + while (tailIterator.hasNext()) { + Assert.assertEquals((i + 5) * 2, Bytes.toInt(tailIterator.next())); + if (i == 45) { + //remove the tail + tailIterator.remove(); + } + i++; + } + + Assert.assertEquals(12, headMap1.entrySet().size()); + Assert.assertEquals(12, headMap1.values().size()); + Assert.assertEquals(12, headMap1.keySet().size()); + + Assert.assertEquals(48, testMap.entrySet().size()); + Assert.assertEquals(48, testMap.values().size()); + Assert.assertEquals(48, testMap.keySet().size()); + + Assert.assertEquals(45, tailMap1.entrySet().size()); + Assert.assertEquals(45, tailMap1.values().size()); + Assert.assertEquals(45, tailMap1.keySet().size()); + + Assert.assertEquals(4, Bytes.toInt(headMap1.firstKey())); + Assert.assertEquals(26, Bytes.toInt(headMap1.lastKey())); + + Assert.assertEquals(4, Bytes.toInt(testMap.firstKey())); + Assert.assertEquals(98, Bytes.toInt(testMap.lastKey())); + + Assert.assertEquals(45, tailMap1.size()); + Assert.assertEquals(10, Bytes.toInt(tailMap1.firstKey())); + Assert.assertEquals(98, Bytes.toInt(tailMap1.lastKey())); + } + + private void doTestAfterRemoveHead(NavigableMap testMap, + NavigableMap headMap1) { + int i;//remove Head + Iterator headIterable = headMap1.keySet().iterator(); + i = 0; + while (headIterable.hasNext()) { + Assert.assertEquals((i + 1) * 2, Bytes.toInt(headIterable.next())); + if (i == 0) { + //remove Head + headIterable.remove(); + } + i++; + } + + Assert.assertEquals(12, headMap1.entrySet().size()); + Assert.assertEquals(12, headMap1.values().size()); + Assert.assertEquals(12, headMap1.keySet().size()); + + Assert.assertEquals(49, testMap.entrySet().size()); + Assert.assertEquals(49, testMap.values().size()); + Assert.assertEquals(49, testMap.keySet().size()); + + Assert.assertEquals(4, Bytes.toInt(headMap1.firstKey())); + Assert.assertEquals(26, Bytes.toInt(headMap1.lastKey())); + + Assert.assertEquals(4, Bytes.toInt(testMap.firstKey())); + Assert.assertEquals(100, Bytes.toInt(testMap.lastKey())); + + i = 0; + for (Map.Entry entry : headMap1.entrySet()) { + Assert.assertEquals((i + 2) * 2, Bytes.toInt(entry.getKey())); + i++; + } + + i = 0; + for (byte[] value : headMap1.values()) { + Assert.assertEquals((i + 2) * 2, Bytes.toInt(value)); + i++; + } + + i = 0; + for (byte[] value : headMap1.keySet()) { + Assert.assertEquals((i + 2) * 2, Bytes.toInt(value)); + i++; + } + } + + @Test + public void testCCSMapTwinSubMap() throws Exception { + byte[] startKey = new byte[4]; + byte[] endKey = new byte[4]; + + Bytes.putInt(startKey, 0, 8); + Bytes.putInt(endKey, 0, 28); + + final NavigableMap testMap = newBaseTwinCCSMap(HeapMode.onHeap); + + Assert.assertEquals(null, testMap.firstKey()); + Assert.assertEquals(null, testMap.lastKey()); + Assert.assertEquals(null, testMap.lowerKey(startKey)); + Assert.assertEquals(null, testMap.floorKey(startKey)); + Assert.assertEquals(null, testMap.ceilingKey(startKey)); + Assert.assertEquals(null, testMap.higherKey(startKey)); + + Assert.assertEquals(null, testMap.firstEntry()); + Assert.assertEquals(null, testMap.lastEntry()); + Assert.assertEquals(null, testMap.lowerEntry(startKey)); + Assert.assertEquals(null, testMap.floorEntry(startKey)); + Assert.assertEquals(null, testMap.ceilingEntry(startKey)); + Assert.assertEquals(null, testMap.higherEntry(startKey)); + + for (int i = 0; i < 50; i++) { + byte[] bb = new byte[4]; + Bytes.putInt(bb, 0, (i + 1) * 2); + testMap.put(bb, bb); + } + + Assert.assertEquals(50, testMap.size()); + Assert.assertEquals(2, Bytes.toInt(testMap.firstKey())); + Assert.assertEquals(100, Bytes.toInt(testMap.lastKey())); + Assert.assertEquals(6, Bytes.toInt(testMap.lowerKey(startKey))); + Assert.assertEquals(8, Bytes.toInt(testMap.floorKey(startKey))); + Assert.assertEquals(8, Bytes.toInt(testMap.ceilingKey(startKey))); + Assert.assertEquals(28, Bytes.toInt(testMap.ceilingKey(endKey))); + Assert.assertEquals(30, Bytes.toInt(testMap.higherKey(endKey))); + + Assert.assertEquals(2, Bytes.toInt(testMap.firstEntry().getKey())); + Assert.assertEquals(100, Bytes.toInt(testMap.lastEntry().getKey())); + Assert.assertEquals(6, Bytes.toInt(testMap.lowerEntry(startKey).getKey())); + Assert.assertEquals(8, Bytes.toInt(testMap.floorEntry(startKey).getKey())); + Assert.assertEquals(8, Bytes.toInt(testMap.ceilingEntry(startKey).getKey())); + Assert.assertEquals(28, Bytes.toInt(testMap.ceilingEntry(endKey).getKey())); + Assert.assertEquals(30, Bytes.toInt(testMap.higherEntry(endKey).getKey())); + + final NavigableMap headMap1 = testMap.headMap(endKey, false); + Assert.assertEquals(13, headMap1.size()); + Assert.assertEquals(2, Bytes.toInt(headMap1.firstKey())); + Assert.assertEquals(26, Bytes.toInt(headMap1.lastKey())); + + final NavigableMap headMap2 = testMap.headMap(endKey, true); + Assert.assertEquals(14, headMap2.size()); + Assert.assertEquals(2, Bytes.toInt(headMap2.firstKey())); + Assert.assertEquals(28, Bytes.toInt(headMap2.lastKey())); + + final NavigableMap tailMap1 = testMap.tailMap(startKey, false); + Assert.assertEquals(46, tailMap1.size()); + Assert.assertEquals(10, Bytes.toInt(tailMap1.firstKey())); + Assert.assertEquals(100, Bytes.toInt(tailMap1.lastKey())); + + final NavigableMap tailMap2 = testMap.tailMap(startKey, true); + Assert.assertEquals(47, tailMap2.size()); + Assert.assertEquals(8, Bytes.toInt(tailMap2.firstKey())); + Assert.assertEquals(100, Bytes.toInt(tailMap2.lastKey())); + + } + +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestCCSMapChunkPool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestCCSMapChunkPool.java new file mode 100644 index 0000000000..1b5d9143fb --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestCCSMapChunkPool.java @@ -0,0 +1,255 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import static org.mockito.Mockito.mock; +import java.lang.reflect.Field; +import java.lang.reflect.Method; +import java.util.Map; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(SmallTests.class) +public class TestCCSMapChunkPool { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCCSMapChunkPool.class); + @Test + public void testNormal() throws Exception { + AllocatorHandlerRegister.AllocatorHandlerBuilder builder = + new AllocatorHandlerRegister.AllocatorHandlerBuilder(); + builder.setCapacity(8 * 1024 * 1024); + builder.setChunkSize(4 * 1024); + builder.setInitialCount(Integer.MAX_VALUE); + builder.setHeapMode(HeapMode.offHeap); + CCSMapChunkPool allocatorHandler = new CCSMapChunkPool(builder); + + Field chunkQueueField = CCSMapChunkPool.class.getDeclaredField("chunkQueue"); + chunkQueueField.setAccessible(true); + @SuppressWarnings("unchecked") ConcurrentLinkedQueue chunkQueue = + (ConcurrentLinkedQueue) chunkQueueField.get(allocatorHandler); + Assert.assertEquals(2 * 1024, chunkQueue.size()); + + Field chunkMapField = CCSMapChunkPool.class.getDeclaredField("totalPoolChunkArray"); + chunkMapField.setAccessible(true); + IChunk[] totalPoolChunkArray = (IChunk[]) chunkMapField.get(allocatorHandler); + + Map hugeChunkMap = allocatorHandler.getExtraChunkMap(); + + Assert.assertEquals(totalPoolChunkArray.length, chunkQueue.size()); + + Assert.assertEquals(2 * 1024, allocatorHandler.getChunkCounter()); + Assert.assertEquals(2 * 1024, allocatorHandler.getMaxCount()); + Assert.assertEquals(0, allocatorHandler.getRequestCounter()); + Assert.assertEquals(0, allocatorHandler.getExtraRequestCounter()); + Assert.assertEquals(0, hugeChunkMap.size()); + + IChunk chunk = allocatorHandler.allocate(4 * 1024 - 1); + Assert.assertTrue(chunk.isPooledChunk()); + Assert.assertEquals(HeapMode.offHeap, chunk.getHeapMode()); + Assert.assertEquals(4 * 1024, chunk.getLimit()); + Assert.assertEquals(1, chunk.getChunkId()); + Assert.assertEquals(0, chunk.getPosition()); + + Assert.assertEquals(2 * 1024 - 1, chunkQueue.size()); + Assert.assertEquals(2 * 1024, totalPoolChunkArray.length); + + for (int i = 0; i < totalPoolChunkArray.length; i++) { + Assert.assertEquals(i + 1, totalPoolChunkArray[i].getChunkId()); + } + + Assert.assertEquals(2 * 1024, allocatorHandler.getChunkCounter()); + Assert.assertEquals(1, allocatorHandler.getRequestCounter()); + Assert.assertEquals(0, allocatorHandler.getExtraRequestCounter()); + Assert.assertEquals(0, allocatorHandler.getExtraChunkCapacityUsed().get()); + + IChunk hugeChunk = allocatorHandler.allocate(4 * 1024 + 1); + Assert.assertTrue(hugeChunk instanceof HugeOnHeapChunk); + Assert.assertFalse(hugeChunk.isPooledChunk()); + Assert.assertEquals(HeapMode.onHeap, hugeChunk.getHeapMode()); + Assert.assertEquals(4 * 1024 + 1, hugeChunk.getLimit()); + Assert.assertEquals(2 * 1024 + 2, hugeChunk.getChunkId()); + + Assert.assertEquals(2 * 1024 - 1, chunkQueue.size()); + Assert.assertEquals(2 * 1024, totalPoolChunkArray.length); + + Assert.assertEquals(2 * 1024, allocatorHandler.getChunkCounter()); + Assert.assertEquals(2, allocatorHandler.getRequestCounter()); + Assert.assertEquals(1, allocatorHandler.getExtraRequestCounter()); + Assert.assertEquals(4097, allocatorHandler.getExtraChunkCapacityUsed().get()); + Assert.assertEquals(1, hugeChunkMap.size()); + Assert.assertEquals(hugeChunk, hugeChunkMap.get(hugeChunk.getChunkId())); + + System.out.println("chunk position=" + chunk.getPosition()); + System.out.println("chunk limit=" + chunk.getLimit()); + System.out.println("chunk BB position=" + chunk.getByteBuffer().position()); + System.out.println("chunk BB limit=" + chunk.getByteBuffer().limit()); + System.out.println("chunk BB capacity=" + chunk.getByteBuffer().capacity()); + allocatorHandler.putbackChunk(chunk); + Assert.assertEquals(2 * 1024, chunkQueue.size()); + Assert.assertEquals(2 * 1024, totalPoolChunkArray.length); + Assert.assertEquals(2 * 1024, allocatorHandler.getChunkCounter()); + Assert.assertEquals(2, allocatorHandler.getRequestCounter()); + Assert.assertEquals(1, allocatorHandler.getExtraRequestCounter()); + + Assert.assertEquals(0, chunk.getPosition()); + Assert.assertEquals(0, chunk.getByteBuffer().position()); + Assert.assertEquals(4 * 1024, chunk.getByteBuffer().limit()); + Assert.assertEquals(4 * 1024, chunk.getLimit()); + Assert.assertEquals(1, chunk.getChunkId()); + System.out.println("chunk position=" + chunk.getPosition()); + System.out.println("chunk limit=" + chunk.getLimit()); + System.out.println("chunk BB position=" + chunk.getByteBuffer().position()); + System.out.println("chunk BB limit=" + chunk.getByteBuffer().limit()); + System.out.println("chunk BB capacity=" + chunk.getByteBuffer().capacity()); + + allocatorHandler.putbackChunk(hugeChunk); + Assert.assertEquals(2 * 1024, chunkQueue.size()); + Assert.assertEquals(2 * 1024, totalPoolChunkArray.length); + Assert.assertEquals(2 * 1024, allocatorHandler.getChunkCounter()); + Assert.assertEquals(2, allocatorHandler.getRequestCounter()); + Assert.assertEquals(1, allocatorHandler.getExtraRequestCounter()); + Assert.assertEquals(0, hugeChunkMap.size()); + Assert.assertEquals(0, allocatorHandler.getExtraChunkCapacityUsed().get()); + + AllocatorHandlerRegister.register(allocatorHandler); + Assert.assertEquals(allocatorHandler, AllocatorHandlerRegister.getAllocatorHandler()); + + Method methodgetExtraChunkId = CCSMapChunkPool.class.getDeclaredMethod("getExtraChunkId"); + methodgetExtraChunkId.setAccessible(true); + long id = (long) methodgetExtraChunkId.invoke(allocatorHandler); + Assert.assertEquals(2051, id); + hugeChunkMap.put(2052L, mock(IChunk.class)); + hugeChunkMap.put(2053L, mock(IChunk.class)); + id = (long) methodgetExtraChunkId.invoke(allocatorHandler); + Assert.assertEquals(2054, id); + + Field fieldExtraChunkIdGenerator = + CCSMapChunkPool.class.getDeclaredField("extraChunkIdGenerator"); + fieldExtraChunkIdGenerator.setAccessible(true); + AtomicLong hugeIdGen = (AtomicLong) fieldExtraChunkIdGenerator.get(allocatorHandler); + hugeIdGen.set(Integer.MAX_VALUE - 2); + id = (long) methodgetExtraChunkId.invoke(allocatorHandler); + Assert.assertEquals(2050, id); + } + + @Test + public void testExaustedNormalChunk() throws Exception { + AllocatorHandlerRegister.AllocatorHandlerBuilder builder = + new AllocatorHandlerRegister.AllocatorHandlerBuilder(); + builder.setCapacity(8 * 1024); + builder.setExtraChunkWarnCapacity(8 * 1024); + builder.setChunkSize(4 * 1024); + builder.setInitialCount(Integer.MAX_VALUE); + builder.setHeapMode(HeapMode.offHeap); + CCSMapChunkPool allocatorHandler = new CCSMapChunkPool(builder); + + Field chunkQueueField = CCSMapChunkPool.class.getDeclaredField("chunkQueue"); + chunkQueueField.setAccessible(true); + @SuppressWarnings("unchecked") ConcurrentLinkedQueue chunkQueue = + (ConcurrentLinkedQueue) chunkQueueField.get(allocatorHandler); + Assert.assertEquals(2, chunkQueue.size()); + + Field chunkMapField = CCSMapChunkPool.class.getDeclaredField("totalPoolChunkArray"); + chunkMapField.setAccessible(true); + IChunk[] totalPoolChunkArray = (IChunk[]) chunkMapField.get(allocatorHandler); + + Map hugeChunkMap = allocatorHandler.getExtraChunkMap(); + + Assert.assertEquals(totalPoolChunkArray.length, chunkQueue.size()); + + Assert.assertEquals(2, allocatorHandler.getChunkCounter()); + Assert.assertEquals(2, allocatorHandler.getMaxCount()); + Assert.assertEquals(0, allocatorHandler.getRequestCounter()); + Assert.assertEquals(0, allocatorHandler.getExtraRequestCounter()); + Assert.assertEquals(0, hugeChunkMap.size()); + + IChunk chunk = allocatorHandler.allocate(4 * 1024 - 1); + Assert.assertTrue(chunk.isPooledChunk()); + Assert.assertEquals(HeapMode.offHeap, chunk.getHeapMode()); + Assert.assertEquals(4 * 1024, chunk.getLimit()); + Assert.assertEquals(1, chunk.getChunkId()); + Assert.assertEquals(0, chunk.getPosition()); + + Assert.assertEquals(0, hugeChunkMap.size()); + Assert.assertEquals(1, chunkQueue.size()); + Assert.assertEquals(2, totalPoolChunkArray.length); + + IChunk chunk2 = allocatorHandler.allocate(4 * 1024 - 2); + Assert.assertTrue(chunk2.isPooledChunk()); + Assert.assertEquals(HeapMode.offHeap, chunk2.getHeapMode()); + Assert.assertEquals(4 * 1024, chunk2.getLimit()); + Assert.assertEquals(2, chunk2.getChunkId()); + Assert.assertEquals(0, chunk2.getPosition()); + + Assert.assertEquals(0, hugeChunkMap.size()); + Assert.assertEquals(0, chunkQueue.size()); + Assert.assertEquals(2, totalPoolChunkArray.length); + + //exhausted + + IChunk chunk3 = allocatorHandler.allocate(4 * 1024 - 3); + Assert.assertEquals("OnHeapChunk", chunk3.getClass().getSimpleName()); + Assert.assertFalse(chunk3.isPooledChunk()); + Assert.assertEquals(HeapMode.onHeap, chunk3.getHeapMode()); + Assert.assertEquals(4 * 1024, chunk3.getLimit()); + Assert.assertEquals(4, chunk3.getChunkId()); + + Assert.assertEquals(1, hugeChunkMap.size()); + Assert.assertEquals(0, chunkQueue.size()); + Assert.assertEquals(2, totalPoolChunkArray.length); + + IChunk chunk4 = allocatorHandler.allocate(4 * 1024 - 4); + Assert.assertEquals("OnHeapChunk", chunk4.getClass().getSimpleName()); + Assert.assertFalse(chunk4.isPooledChunk()); + Assert.assertEquals(HeapMode.onHeap, chunk4.getHeapMode()); + Assert.assertEquals(4 * 1024, chunk4.getLimit()); + Assert.assertEquals(5, chunk4.getChunkId()); + + Assert.assertEquals(2, hugeChunkMap.size()); + Assert.assertEquals(0, chunkQueue.size()); + Assert.assertEquals(2, totalPoolChunkArray.length); + + allocatorHandler.putbackChunk(chunk4); + Assert.assertEquals(1, hugeChunkMap.size()); + Assert.assertEquals(0, chunkQueue.size()); + Assert.assertEquals(2, totalPoolChunkArray.length); + + chunk4 = allocatorHandler.allocate(4 * 1024 - 4); + Assert.assertEquals("OnHeapChunk", chunk4.getClass().getSimpleName()); + Assert.assertTrue(chunk4 instanceof OnHeapChunk); + Assert.assertFalse(chunk4.isPooledChunk()); + Assert.assertEquals(HeapMode.onHeap, chunk4.getHeapMode()); + Assert.assertEquals(4 * 1024, chunk4.getLimit()); + Assert.assertEquals(6, chunk4.getChunkId()); + + Assert.assertEquals(2, hugeChunkMap.size()); + Assert.assertEquals(0, chunkQueue.size()); + Assert.assertEquals(2, totalPoolChunkArray.length); + + //Huge exhausted + allocatorHandler.allocate(4 * 1024 - 4); + } +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestChunkUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestChunkUtil.java new file mode 100644 index 0000000000..8d7088e6ee --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestChunkUtil.java @@ -0,0 +1,142 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.INDEX_NOT_INITIAL; +import java.nio.ByteBuffer; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.regionserver.skiplist.exception.SerdeException; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(SmallTests.class) +public class TestChunkUtil { + + @ClassRule public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestChunkUtil.class); + + @Test + public void testNormal() throws Exception { + int offset = 1; + IChunk chunk = new OnHeapChunk(offset, 130); + dotest(chunk, offset, 0, SchemaEnum.diff); + dotest(chunk, offset, 3, SchemaEnum.diff); + dotest(chunk, offset, 0, SchemaEnum.same); + dotest(chunk, offset, 3, SchemaEnum.same); + + offset = 11; + chunk = new OffHeapChunk(offset, 130); + dotest(chunk, offset, 0, SchemaEnum.diff); + dotest(chunk, offset, 3, SchemaEnum.diff); + dotest(chunk, offset, 0, SchemaEnum.same); + dotest(chunk, offset, 3, SchemaEnum.same); + } + + private void dotest(IChunk chunk, int offset, int level, SchemaEnum schemaEnum) throws Exception { + int keylen = 12; + int valueLen = 23; + + int dataLen = NodeUtil.getDataLen(keylen, valueLen); + int metaLen = NodeUtil.getNodeMetaLenByLevel(level); + + ChunkUtil.newNodeOnChunk(chunk, offset, level, dataLen); + for (int i = 1; i < level; i++) { + Assert.assertEquals(INDEX_NOT_INITIAL, ChunkUtil.getNextNodeIdForLevel(chunk, offset, i)); + } + Assert.assertEquals(level, ChunkUtil.getNodeLevel(chunk, offset)); + Assert.assertEquals(ChunkUtil.getKeyOffset(chunk, offset, schemaEnum), + schemaEnum == SchemaEnum.diff ? offset + metaLen + 2 * Integer.BYTES : offset + metaLen); + Assert.assertEquals(dataLen, ChunkUtil.getNodeDataLen(chunk, offset)); + Assert.assertEquals(offset + metaLen, ChunkUtil.getNodeDataOffset(chunk, offset)); + if (schemaEnum == SchemaEnum.same) { + Assert.assertEquals(dataLen, ChunkUtil.getKeyLen(chunk, offset, + ChunkUtil.getKeyOffset(chunk, offset, schemaEnum), schemaEnum)); + //if SchemaEnum is diff, the Keylen can be read only after written. + } + + ChunkUtil + .writeNodeData(chunk, offset, metaLen, new Object(), new Object(), serde, serde, keylen, + valueLen); + + //put data + int startOffset = + offset + NodeUtil.getNodeMetaLenByLevel(ChunkUtil.getNodeLevel(chunk, offset)); + for (int i = 0; i < keylen; i++) { + Assert.assertEquals('N', chunk.getByteBuffer().get(startOffset + 8 + i)); + } + + for (int i = 0; i < valueLen; i++) { + Assert.assertEquals('N', chunk.getByteBuffer().get(startOffset + 8 + keylen + i)); + } + + Assert.assertEquals(valueLen, chunk.getByteBuffer().getInt(startOffset + 4)); + Assert.assertEquals(keylen, ChunkUtil.testGetNodeKeyLen(chunk, offset)); + + ChunkUtil.setNextNodeIdForLevel(chunk, offset, 0, 1234); + Assert.assertEquals(1234, ChunkUtil.getNextNodeIdForLevel(chunk, offset, 0)); + + String expectedException = "WANT_EXCEPTION"; + try { + ChunkUtil.newNodeOnChunk(null, offset, level, dataLen); + } catch (Exception e) { + expectedException = e.getMessage(); + } + Assert.assertEquals("chunk must be not null.", expectedException); + + expectedException = "WANT_EXCEPTION"; + try { + ChunkUtil.newNodeOnChunk(chunk, -1, level, dataLen); + } catch (Exception e) { + expectedException = e.getMessage(); + } + Assert.assertEquals("offset can't be negative.", expectedException); + + expectedException = "WANT_EXCEPTION"; + try { + ChunkUtil.newNodeOnChunk(chunk, 0, level, -1); + } catch (Exception e) { + expectedException = e.getMessage(); + } + Assert.assertEquals("dataLen error.", expectedException); + } + + private final ISerde serde = new ISerde() { + + @Override + public void serialize(Object obj, ByteBuffer destination, int offset, int len) + throws SerdeException { + for (int i = 0; i < len; i++) { + destination.put(offset + i, (byte) 'N'); + } + } + + @Override + public int getSerializedSize(Object obj) throws SerdeException { + return 0; + } + + @Override + public Object deserialize(ByteBuffer bb, int offset, int len) throws SerdeException { + return null; + } + + }; +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestCompactedConcurrentSkipList.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestCompactedConcurrentSkipList.java new file mode 100644 index 0000000000..26d41bcee9 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestCompactedConcurrentSkipList.java @@ -0,0 +1,1062 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import static org.apache.hadoop.hbase.regionserver.skiplist.core.AbstractCCSMap.EQ; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.AbstractCCSMap.GT; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.AbstractCCSMap.LT; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.NIL_NODE_ID; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.NO_INDEX; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.SKIP_LIST_HEAD_NODE; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.MockBaseCCSMap.CCSMAP_TEST_BYTE_COMPARTOR; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiPredicate; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.regionserver.skiplist.exception.CCSMapException; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@SuppressWarnings("PointlessBitwiseExpression") +@Category(SmallTests.class) +public class TestCompactedConcurrentSkipList { + + @ClassRule public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCompactedConcurrentSkipList.class); + + @Test + public void testClose() throws Exception { + CompactedConcurrentSkipList ccsl = newCCSList(HeapMode.offHeap, 128 * 1024, 100); + + ccsl.getNodeAndWriteMeta(0, NodeUtil.getNodeMetaLenByLevel(0), NodeUtil.getDataLen(10, 40)); + //curChunk has value + Assert.assertEquals(1, ccsl.getCurChunk().getChunkId()); + + Thread[] ts = new Thread[10]; + + AtomicBoolean hasCloseException = new AtomicBoolean(false); + + for (int i = 0; i < ts.length; i++) { + ts[i] = new Thread(new Runnable() { + @Override + public void run() { + try { + ccsl.close(); + } catch (Exception e) { + hasCloseException.set(true); + } + } + }); + } + + for (Thread t : ts) { + t.start(); + } + + for (Thread t : ts) { + t.join(); + } + + Assert.assertEquals(null, ccsl.getCurChunk()); + Assert.assertFalse(hasCloseException.get()); + } + + @Test + public void testCCSLAllocate() throws Exception { + + CompactedConcurrentSkipList ccsl = newCCSList(HeapMode.offHeap, 128 * 1024, 100); + BlockingQueue usedChunk = ccsl.getUsedChunkQueue(); + + //Huge Chunk + long node0 = + ccsl.getNodeAndWriteMeta(0, NodeUtil.getNodeMetaLenByLevel(0), NodeUtil.getDataLen(13, 100)); + Assert.assertEquals(1, usedChunk.size()); + Assert.assertEquals(null, ccsl.getCurChunk()); + + long node1 = + ccsl.getNodeAndWriteMeta(0, NodeUtil.getNodeMetaLenByLevel(0), NodeUtil.getDataLen(10, 40)); + Assert.assertEquals(1, usedChunk.size()); + Assert.assertEquals(1, ccsl.getCcslStat().getChunkCount().get()); + Assert.assertEquals(1, ccsl.getCurChunk().getChunkId()); + + long node2 = + ccsl.getNodeAndWriteMeta(0, NodeUtil.getNodeMetaLenByLevel(0), NodeUtil.getDataLen(11, 28)); + Assert.assertEquals(2, usedChunk.size()); + Assert.assertEquals(1, usedChunk.peek().getChunkId()); + Assert.assertEquals(2, ccsl.getCcslStat().getChunkCount().get()); + Assert.assertEquals(2, ccsl.getCurChunk().getChunkId()); + + long node3 = + ccsl.getNodeAndWriteMeta(0, NodeUtil.getNodeMetaLenByLevel(0), NodeUtil.getDataLen(12, 47)); + Assert.assertEquals(3, usedChunk.size()); + Assert.assertEquals(3, ccsl.getCurChunk().getChunkId()); + + long node4 = + ccsl.getNodeAndWriteMeta(0, NodeUtil.getNodeMetaLenByLevel(0), NodeUtil.getDataLen(13, 43)); + Assert.assertEquals(4, usedChunk.size()); + Assert.assertEquals(4, ccsl.getCurChunk().getChunkId()); + + Assert.assertEquals(1312, NodeUtil.getChunkIdByNodeId(node0)); + Assert.assertEquals(1, NodeUtil.getChunkIdByNodeId(node1)); + Assert.assertEquals(2, NodeUtil.getChunkIdByNodeId(node2)); + Assert.assertEquals(3, NodeUtil.getChunkIdByNodeId(node3)); + Assert.assertEquals(4, NodeUtil.getChunkIdByNodeId(node4)); + + Assert.assertEquals(4, usedChunk.size()); + IChunk chunk = usedChunk.poll(); + long last = -1; + while (chunk != null) { + long nowLen = (chunk.getLimit() - chunk.getPosition()); + System.out.println("id=" + chunk.getChunkId() + " ;" + nowLen); + if (last != -1) { + Assert.assertTrue(nowLen <= last); + } + + if (!chunk.isPooledChunk()) { + Assert.assertEquals(0, nowLen); + } + last = nowLen; + + chunk = usedChunk.poll(); + } + } + + @Test + public void testReplace() throws Exception { + long testValue = 123; + byte[] readBB = new byte[8]; + Bytes.putLong(readBB, 0, testValue); + + byte[] lessNode = new byte[8]; + Bytes.putLong(lessNode, 0, testValue - 1); + + byte[] bigNode = new byte[8]; + Bytes.putLong(bigNode, 0, testValue + 1); + + CompactedConcurrentSkipList ccsl = newCCSList(HeapMode.onHeap, 128 * 1024, 100); + //ccsl is empty + Assert.assertEquals(NIL_NODE_ID, ccsl.findFirst()); + Assert.assertEquals(NIL_NODE_ID, ccsl.findLast()); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(readBB, EQ)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(readBB, LT)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(readBB, LT | EQ)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(readBB, GT)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(readBB, GT | EQ)); + + long node1 = + ccsl.getNodeAndWriteMeta(0, NodeUtil.getNodeMetaLenByLevel(0), NodeUtil.getDataLen(8, 8)); + ByteBuffer keyvalueBB = NodeUtil.duplicateNodeDataByteBuffer(ccsl, node1); + ByteBufferUtils.putInt(keyvalueBB, 0, 8); + ByteBufferUtils.putInt(keyvalueBB, Integer.BYTES, 8); + ByteBufferUtils.putLong(keyvalueBB, 2 * Integer.BYTES, testValue); + ByteBufferUtils.putLong(keyvalueBB, 2 * Integer.BYTES + 8, testValue); + + //only put one data to ccsl + ccsl.put(node1, 0); + Assert.assertEquals(1, ccsl.getSize()); + Assert.assertEquals(ccsl.findFirst(), node1); + Assert.assertEquals(ccsl.findLast(), node1); + Assert.assertEquals(ccsl.findNear(readBB, EQ), node1); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(readBB, LT)); + Assert.assertEquals(ccsl.findNear(readBB, LT | EQ), node1); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(readBB, GT)); + Assert.assertEquals(ccsl.findNear(readBB, GT | EQ), node1); + + Assert.assertEquals(ccsl.findNear(lessNode, EQ), node1); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(lessNode, LT)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(lessNode, LT | EQ)); + Assert.assertEquals(ccsl.findNear(lessNode, GT), node1); + Assert.assertEquals(ccsl.findNear(lessNode, GT | EQ), node1); + + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(bigNode, EQ)); + Assert.assertEquals(ccsl.findNear(bigNode, LT), node1); + Assert.assertEquals(ccsl.findNear(bigNode, LT | EQ), node1); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(bigNode, GT)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(bigNode, GT | EQ)); + + printStat("replace test", ccsl); + + long getNode = ccsl.get(readBB); + Assert.assertEquals(getNode, node1); + + doTestAfterPutManySameKeyDataToCCSL(testValue, readBB, lessNode, bigNode, ccsl); + + //test after remove the key + getNode = ccsl.get(readBB); + long removeNOde = ccsl.remove(readBB); + + Assert.assertEquals(0, ccsl.getSize()); + Assert.assertEquals(removeNOde, getNode); + + Assert.assertEquals(NIL_NODE_ID, ccsl.findFirst()); + Assert.assertEquals(NIL_NODE_ID, ccsl.findLast()); + + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(readBB, EQ)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(readBB, LT)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(readBB, LT | EQ)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(readBB, GT)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(readBB, GT | EQ)); + + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(lessNode, EQ)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(lessNode, LT)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(lessNode, LT | EQ)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(lessNode, GT)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(lessNode, GT | EQ)); + + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(bigNode, EQ)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(bigNode, LT)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(bigNode, LT | EQ)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(bigNode, GT)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(bigNode, GT | EQ)); + + removeNOde = ccsl.remove(readBB); + Assert.assertEquals(NIL_NODE_ID, removeNOde); + printStat("replace test", ccsl); + } + + private void doTestAfterPutManySameKeyDataToCCSL(final long testValue, byte[] readBB, + byte[] lessNode, byte[] bigNode, + final CompactedConcurrentSkipList ccsl) + throws InterruptedException, CCSMapException { + + final List totalExpectedValue = doPutSameKeyToCCSLReturnAllValueList(testValue, ccsl); + + ILevelIndexHeader index = ccsl.getLevelIndexHeader(); + long scanNode = index.getLevelNextNodeId(0); + + boolean first1 = true; + long firstNodeId = -999; + List lastTotalValue1 = new ArrayList<>(); + List lastTotalNodeid = new ArrayList<>(); + IIterCCSList iter = ccsl.nodeIdIter(); + while (iter.hasNext()) { + long iterId = iter.next(); + ByteBuffer bb = NodeUtil.duplicateNodeDataByteBuffer(ccsl, iterId); + long currentKey = ByteBufferUtils.toLong(bb, 8); + long currentValue = ByteBufferUtils.toLong(bb, 16); + lastTotalValue1.add(currentValue); + System.out.println( + iterId + " : " + NodeUtil.isNodeRemoving(ccsl, iterId) + " ;key=" + currentKey + " ,value=" + + currentValue); + //The same key data, the first one is valid + + if (first1) { + Assert.assertFalse(NodeUtil.isNodeRemoving(ccsl, iterId)); + first1 = false; + firstNodeId = iterId; + } else { + Assert.assertTrue(NodeUtil.isNodeRemoving(ccsl, iterId)); + } + lastTotalNodeid.add(iterId); + } + + Assert.assertEquals(1, ccsl.getSize()); + Assert.assertEquals(100, ccsl.getCcslStat().getRemovedNodeCount()); + Assert.assertEquals(100, ccsl.getCcslStat().getReplaceRequestCount()); + Assert.assertEquals(firstNodeId, ccsl.findFirst()); + Assert.assertEquals(firstNodeId, ccsl.findLast()); + + Assert.assertEquals(firstNodeId, ccsl.findNear(readBB, EQ)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(readBB, LT)); + Assert.assertEquals(firstNodeId, ccsl.findNear(readBB, LT | EQ)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(readBB, GT)); + Assert.assertEquals(firstNodeId, ccsl.findNear(readBB, GT | EQ)); + + Assert.assertEquals(firstNodeId, ccsl.findNear(lessNode, EQ)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(lessNode, LT)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(lessNode, LT | EQ)); + Assert.assertEquals(firstNodeId, ccsl.findNear(lessNode, GT)); + Assert.assertEquals(firstNodeId, ccsl.findNear(lessNode, GT | EQ)); + + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(bigNode, EQ)); + Assert.assertEquals(firstNodeId, ccsl.findNear(bigNode, LT)); + Assert.assertEquals(firstNodeId, ccsl.findNear(bigNode, LT | EQ)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(bigNode, GT)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(bigNode, GT | EQ)); + + Assert.assertEquals(1, lastTotalValue1.size()); + Assert.assertEquals(1, lastTotalNodeid.size()); + Assert.assertFalse(NodeUtil.isNodeRemoving(ccsl, lastTotalNodeid.get(0))); + + boolean first = true; + firstNodeId = -999; + List totalValue = new ArrayList<>(); + while (scanNode != NIL_NODE_ID) { + ByteBuffer bb = NodeUtil.duplicateNodeDataByteBuffer(ccsl, scanNode); + long currentKey = ByteBufferUtils.toLong(bb, 8); + long currentValue = ByteBufferUtils.toLong(bb, 16); + totalValue.add(currentValue); + System.out.println( + scanNode + " : " + NodeUtil.isNodeRemoving(ccsl, scanNode) + " ;key=" + currentKey + + " ,value=" + currentValue); + + if (first) { + Assert.assertFalse(NodeUtil.isNodeRemoving(ccsl, scanNode)); + first = false; + firstNodeId = scanNode; + } else { + Assert.assertTrue(NodeUtil.isNodeRemoving(ccsl, scanNode)); + } + + scanNode = NodeUtil.getNextNodeIdForLevel(ccsl, scanNode, 0); + + } + + Assert.assertEquals(1, ccsl.getSize()); + Assert.assertEquals(100, ccsl.getCcslStat().getRemovedNodeCount()); + Assert.assertEquals(100, ccsl.getCcslStat().getReplaceRequestCount()); + Assert.assertEquals(firstNodeId, ccsl.findFirst()); + Assert.assertEquals(firstNodeId, ccsl.findLast()); + + Assert.assertEquals(firstNodeId, ccsl.findNear(readBB, EQ)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(readBB, LT)); + Assert.assertEquals(firstNodeId, ccsl.findNear(readBB, LT | EQ)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(readBB, GT)); + Assert.assertEquals(firstNodeId, ccsl.findNear(readBB, GT | EQ)); + + Assert.assertEquals(firstNodeId, ccsl.findNear(lessNode, EQ)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(lessNode, LT)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(lessNode, LT | EQ)); + Assert.assertEquals(firstNodeId, ccsl.findNear(lessNode, GT)); + Assert.assertEquals(firstNodeId, ccsl.findNear(lessNode, GT | EQ)); + + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(bigNode, EQ)); + Assert.assertEquals(firstNodeId, ccsl.findNear(bigNode, LT)); + Assert.assertEquals(firstNodeId, ccsl.findNear(bigNode, LT | EQ)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(bigNode, GT)); + Assert.assertEquals(NIL_NODE_ID, ccsl.findNear(bigNode, GT | EQ)); + + Collections.sort(totalExpectedValue); + Collections.sort(totalValue); + Assert.assertArrayEquals(totalExpectedValue.toArray(), totalValue.toArray()); + + printStat("replace test", ccsl); + } + + private List doPutSameKeyToCCSLReturnAllValueList( + final long testValue, final CompactedConcurrentSkipList ccsl) + throws InterruptedException { + // replica put same key , but value is different + Thread[] threads = new Thread[100]; + final List totalExpectedValue = new ArrayList<>(); + totalExpectedValue.add(testValue); + AtomicBoolean hasException = new AtomicBoolean(false); + for (int i = 0; i < threads.length; i++) { + final int id = i + 1; + threads[i] = new Thread(new Runnable() { + @Override + public void run() { + try { + long node = ccsl.getNodeAndWriteMeta(0, NodeUtil.getNodeMetaLenByLevel(0), + NodeUtil.getDataLen(8, 8)); + ByteBuffer keyvalueBB = NodeUtil.duplicateNodeDataByteBuffer(ccsl, node); + ByteBufferUtils.putInt(keyvalueBB, 0, 8); + ByteBufferUtils.putInt(keyvalueBB, Integer.BYTES, 8); + ByteBufferUtils.putLong(keyvalueBB, 2 * Integer.BYTES, testValue); + ByteBufferUtils.putLong(keyvalueBB, 2 * Integer.BYTES + 8, testValue + id); + + ccsl.put(node, 0); + synchronized (totalExpectedValue) { + totalExpectedValue.add(testValue + id); + } + } catch (Exception e) { + e.printStackTrace(); + hasException.set(true); + } + } + }); + } + + for (Thread th : threads) { + th.start(); + } + + for (Thread th : threads) { + th.join(); + } + + Assert.assertFalse(hasException.get()); + Assert.assertEquals(1, ccsl.getSize()); + return totalExpectedValue; + } + + @Test + public void testFindNear() throws Exception { + CompactedConcurrentSkipList ccsl = newCCSList(HeapMode.onHeap, 128 * 1024, 100); + + boolean first = true; + long firstNodeId = -999; + long lastNodeId = -999; + for (int i = 0; i < 100; i++) { + int level = ccsl.generateRandomIndexLevel(); + long node = ccsl.getNodeAndWriteMeta(level, NodeUtil.getNodeMetaLenByLevel(level), + NodeUtil.getDataLen(8, 8)); + ByteBuffer keyvalueBB = NodeUtil.duplicateNodeDataByteBuffer(ccsl, node); + ByteBufferUtils.putInt(keyvalueBB, 0, 8); + ByteBufferUtils.putInt(keyvalueBB, Integer.BYTES, 8); + ByteBufferUtils.putLong(keyvalueBB, 2 * Integer.BYTES, i * 2); + ByteBufferUtils.putLong(keyvalueBB, 2 * Integer.BYTES + 8, i); + + ccsl.put(node, level); + if (first) { + firstNodeId = node; + first = false; + } else { + lastNodeId = node; + } + } + + Assert.assertEquals(100, ccsl.getSize()); + Assert.assertEquals(firstNodeId, ccsl.findFirst()); + Assert.assertEquals(lastNodeId, ccsl.findLast()); + + for (int i = 99; i >= 0; i--) { + byte[] existedBB = new byte[8]; + Bytes.putLong(existedBB, 0, i * 2); + + Assert.assertEquals(i * 2, readKey(ccsl.findNear(existedBB, EQ), ccsl)); + if (i == 0) { + Assert.assertEquals(NIL_NODE_ID, readKey(ccsl.findNear(existedBB, LT), ccsl)); + } else { + Assert.assertEquals(i * 2 - 2, readKey(ccsl.findNear(existedBB, LT), ccsl)); + } + Assert.assertEquals(i * 2, readKey(ccsl.findNear(existedBB, LT | EQ), ccsl)); + if (i == 99) { + Assert.assertEquals(NIL_NODE_ID, readKey(ccsl.findNear(existedBB, GT), ccsl)); + } else { + Assert.assertEquals(i * 2 + 2, readKey(ccsl.findNear(existedBB, GT), ccsl)); + } + Assert.assertEquals(i * 2, readKey(ccsl.findNear(existedBB, GT | EQ), ccsl)); + + byte[] noExistedBB = new byte[8]; + Bytes.putLong(noExistedBB, 0, i * 2 + 1); + + if (i == 99) { + Assert.assertEquals(NIL_NODE_ID, readKey(ccsl.findNear(noExistedBB, EQ), ccsl)); + } else { + Assert.assertEquals(i * 2 + 2, readKey(ccsl.findNear(noExistedBB, EQ), ccsl)); + } + + Assert.assertEquals(i * 2, readKey(ccsl.findNear(noExistedBB, LT), ccsl)); + Assert.assertEquals(i * 2, readKey(ccsl.findNear(noExistedBB, LT | EQ), ccsl)); + + if (i == 99) { + Assert.assertEquals(NIL_NODE_ID, readKey(ccsl.findNear(noExistedBB, GT), ccsl)); + Assert.assertEquals(NIL_NODE_ID, readKey(ccsl.findNear(noExistedBB, GT | EQ), ccsl)); + } else { + Assert.assertEquals(i * 2 + 2, readKey(ccsl.findNear(noExistedBB, GT), ccsl)); + Assert.assertEquals(i * 2 + 2, readKey(ccsl.findNear(noExistedBB, GT | EQ), ccsl)); + } + } + } + + public static long readKey(long nodeid, IChunkVisible ccsl) { + if (nodeid == NIL_NODE_ID) { + return NIL_NODE_ID; + } + ByteBuffer keyvaluebb = NodeUtil.duplicateNodeDataByteBuffer(ccsl, nodeid); + return ByteBufferUtils.toLong(keyvaluebb, 8); + } + + @Test + public void testLevel() throws Exception { + + AtomicInteger idGenerator = new AtomicInteger(0); + CompactedConcurrentSkipList ccsl = newCCSList(HeapMode.onHeap, 128 * 1024, 100); + Assert.assertEquals(NIL_NODE_ID, ccsl.findFirst()); + Assert.assertEquals(NIL_NODE_ID, ccsl.findLast()); + + long testValue = 123; + List totalNodeId = new ArrayList<>(); + Map> totalLevelNodeIdMaps = new HashMap<>(); + + for (int m = 1; m <= 10; m++) { + totalLevelNodeIdMaps.put(m, new ArrayList<>()); + } + + long node1 = + ccsl.getNodeAndWriteMeta(1, NodeUtil.getNodeMetaLenByLevel(1), NodeUtil.getDataLen(8, 8)); + totalNodeId.add(node1); + copyDataToNode(ccsl, node1, testValue, idGenerator.incrementAndGet()); + totalLevelNodeIdMaps.get(1).add(node1); + ccsl.put(node1, 1); + Assert.assertEquals(node1, ccsl.findFirst()); + Assert.assertEquals(node1, ccsl.findLast()); + + long headNode = SKIP_LIST_HEAD_NODE; + Assert.assertEquals(node1, ccsl.getNextNodeByIdAndLevel(headNode, 1)); + Assert.assertEquals(NIL_NODE_ID, + ccsl.getNextNodeByIdAndLevel(ccsl.getNextNodeByIdAndLevel(headNode, 1), 1)); + + Assert.assertEquals(node1, ccsl.getNextNodeByIdAndLevel(headNode, 0)); + Assert.assertEquals(NIL_NODE_ID, + ccsl.getNextNodeByIdAndLevel(ccsl.getNextNodeByIdAndLevel(headNode, 0), 0)); + + for (int i = 0; i < 10; i++) { + long node = ccsl.getNodeAndWriteMeta(i + 1, NodeUtil.getNodeMetaLenByLevel(i + 1), + NodeUtil.getDataLen(8, 8)); + + totalNodeId.add(node); + + for (int d = i + 1; d >= 1; d--) { + totalLevelNodeIdMaps.get(d).add(node); + } + copyDataToNode(ccsl, node, testValue, idGenerator.incrementAndGet()); + System.out.println("cur nodeId=" + node); + ccsl.put(node, i + 1); + } + + System.out.println("total nodeids=" + totalNodeId); + long node = ccsl.getNextNodeByIdAndLevel(headNode, 0); + List nodeIdList = new ArrayList<>(); + while (node != NIL_NODE_ID) { + ByteBuffer keyvaluebb = NodeUtil.duplicateNodeDataByteBuffer(ccsl, node); + long currentKey = ByteBufferUtils.toLong(keyvaluebb, 8); + + int level = ChunkUtil.getNodeLevel(ccsl.getChunkById(NodeUtil.getChunkIdByNodeId(node)), + NodeUtil.getChunkOffsetByNodeId(node)); + for (int i = 1; i <= level; i++) { + System.out.println( + "nodeId=" + node + " ,level=" + level + ",key=" + currentKey + " ;level" + i + " next=>" + + ccsl.getNextNodeByIdAndLevel(node, i)); + } + nodeIdList.add(node); + + node = ccsl.getNextNodeByIdAndLevel(node, 0); + + } + + Assert.assertArrayEquals(totalNodeId.toArray(), nodeIdList.toArray()); + + Assert.assertEquals(ccsl.findFirst(), totalNodeId.get(0).longValue()); + Assert.assertEquals(ccsl.findLast(), totalNodeId.get(totalNodeId.size() - 1).longValue()); + + Assert.assertEquals(10, ccsl.getLevelIndexHeader().getLevel()); + Map> levelNodeMaps = new HashMap<>(); + for (int j = 10; j >= 1; j--) { + long nodeLevel = ccsl.getNextNodeByIdAndLevel(SKIP_LIST_HEAD_NODE, j); + List levelNodeList = new ArrayList<>(); + while (nodeLevel != NIL_NODE_ID) { + levelNodeList.add(nodeLevel); + nodeLevel = ccsl.getNextNodeByIdAndLevel(nodeLevel, j); + } + levelNodeMaps.put(j, levelNodeList); + } + for (int j = 10; j >= 1; j--) { + if (j == 1) { + Assert.assertEquals(11, levelNodeMaps.get(j).size()); + } else { + Assert.assertEquals(10 - j + 1, levelNodeMaps.get(j).size()); + } + Assert + .assertArrayEquals(totalLevelNodeIdMaps.get(j).toArray(), levelNodeMaps.get(j).toArray()); + } + + for (int m = 1; m <= 10; m++) { + + byte[] readBB = new byte[8]; + Bytes.putLong(readBB, 0, testValue + m); + long getNode = ccsl.get(readBB); + + long currentKey = + ByteBufferUtils.toLong(NodeUtil.duplicateNodeDataByteBuffer(ccsl, getNode), 8); + Assert.assertEquals(currentKey, testValue + m); + } + + } + + @Test + public void testForMultiTimes() throws Exception { + //Correctness verification of large concurrency, and 5 repeated runs + for (int i = 0; i < 5; i++) { + int size = 111; + int thread = 301; + + System.out.println( + "============================ " + i + " time test" + "============================ "); + doMultiPutGet(HeapMode.offHeap, size, thread); + doMultiPutGet(HeapMode.onHeap, size, thread); + } + + } + + private void doMultiPutGet(HeapMode heapmode, int size, int thread) throws Exception { + CompactedConcurrentSkipList ccsl = newCCSList(heapmode, 128 * 1024 * 1024, 4 * 1024); + + Assert.assertEquals(0, ccsl.getCcslStat().getCprRequestCount()); + Assert.assertEquals(0, ccsl.getCcslStat().getPutRequestCount()); + Assert.assertEquals(0, ccsl.getCcslStat().getRemovedNodeCount()); + Assert.assertEquals(0, ccsl.getCcslStat().getRemoveRequestRaceFailCount()); + Assert.assertEquals(0, ccsl.getCcslStat().getReplaceRequestCount()); + + final ConcurrentLinkedQueue keyQueueOri = new ConcurrentLinkedQueue<>(); + final ConcurrentLinkedQueue keyQueueForPut = new ConcurrentLinkedQueue<>(); + final ConcurrentLinkedQueue nodeIdQueueForPut = new ConcurrentLinkedQueue<>(); + AtomicBoolean hasException = new AtomicBoolean(false); + + //put by multi threas + Thread[] threads = new Thread[thread]; + for (int th = 0; th < thread; th++) { + threads[th] = new Thread(new Runnable() { + @Override + public void run() { + try { + for (int i = 0; i < size; i++) { + long node1 = generateNode(ccsl, keyQueueOri); + nodeIdQueueForPut.add(node1); + keyQueueForPut + .add(ByteBufferUtils.toLong(NodeUtil.duplicateNodeDataByteBuffer(ccsl, node1), 8)); + ccsl.put(node1, ChunkUtil + .getNodeLevel(ccsl.getChunkById(NodeUtil.getChunkIdByNodeId(node1)), + NodeUtil.getChunkOffsetByNodeId(node1))); + } + } catch (Exception e) { + e.printStackTrace(); + hasException.set(true); + } + } + }); + } + + for (Thread t : threads) { + t.start(); + } + + for (Thread t : threads) { + t.join(); + } + + Assert.assertFalse(hasException.get()); + + Assert.assertEquals(size * thread, ccsl.getSize()); + Assert.assertEquals(0, ccsl.getCcslStat().getReplaceRequestCount()); + + List keyListForScan = new ArrayList<>(); + List nodeIdListForScan = new ArrayList<>(); + + doCheckScanForHead(ccsl, keyListForScan, nodeIdListForScan, + (Long t, Long u) -> t.longValue() == u); + + Long[] keyArrayOri = keyQueueOri.toArray(new Long[0]); + Long[] keyArrayForPut = keyQueueForPut.toArray(new Long[0]); + Long[] nodeIdArrayForPut = nodeIdQueueForPut.toArray(new Long[0]); + Arrays.sort(keyArrayOri); + Arrays.sort(keyArrayForPut); + Arrays.sort(nodeIdArrayForPut); + + Assert.assertArrayEquals(keyArrayOri, keyListForScan.toArray()); + Assert.assertArrayEquals(keyArrayForPut, keyListForScan.toArray()); + + Collections.sort(nodeIdListForScan); + Assert.assertArrayEquals(nodeIdArrayForPut, nodeIdListForScan.toArray()); + + //check level + System.out.println(" total key => " + keyListForScan); + checkLevel(ccsl); + + //check mem + BlockingQueue chunks = ccsl.getUsedChunkQueue(); + long totalSize = 0; + int countNum = 0; + long maxOccupancy = Long.MIN_VALUE; + int totalOccupancy = 0; + int totalAlignedOccupancy = 0; + for (IChunk chunk : chunks) { + totalSize += chunk.getPosition(); + countNum++; + totalOccupancy += chunk.occupancy(); + totalAlignedOccupancy += ((AbstractChunk) chunk).alignOccupancy(); + if (maxOccupancy < chunk.occupancy()) { + maxOccupancy = chunk.occupancy(); + } + } + System.out.println( + "cal totalSize=" + totalSize + " ;countNum=" + countNum + " ; maxOcc=" + maxOccupancy + + " ;totalOccupancy=" + totalOccupancy + " ;alignOccupancy=" + totalAlignedOccupancy); + System.out.println( + "ccsl totalNum=" + ccsl.getSize() + " ; count=" + ccsl.getCcslStat().getChunkCount() + + " ; replace=" + ccsl.getCcslStat().getReplaceRequestCount()); + + // test get + + List keyListForGet = new ArrayList<>(); + List nodeIdListForGet = new ArrayList<>(); + + for (Long id : keyArrayOri) { + byte[] readBB = new byte[8]; + Bytes.putLong(readBB, 0, id); + long getNode = ccsl.get(readBB); + keyListForGet + .add(ByteBufferUtils.toLong(NodeUtil.duplicateNodeDataByteBuffer(ccsl, getNode), 8)); + nodeIdListForGet.add(getNode); + } + + Collections.sort(keyListForGet); + Collections.sort(nodeIdListForGet); + + Assert.assertArrayEquals(keyListForGet.toArray(), keyListForScan.toArray()); + Assert.assertArrayEquals(nodeIdListForGet.toArray(), nodeIdListForScan.toArray()); + + // test get no existed key + long noexisted = (long) random.nextInt(Integer.MAX_VALUE); + for (int i = 0; i < 100; i++) { + while (keyListForGet.contains(noexisted)) { + noexisted = (long) random.nextInt(Integer.MAX_VALUE); + } + + byte[] readBB = new byte[8]; + Bytes.putLong(readBB, 0, noexisted); + long noExistedNode = ccsl.get(readBB); + Assert.assertEquals(NIL_NODE_ID, noExistedNode); + } + + //test put same key by multi threads + doCheckPutSameKey(size, thread, ccsl, keyQueueOri, hasException, threads, keyArrayOri, + keyListForGet, nodeIdListForGet); + + } + + private void doCheckPutSameKey(int size, int thread, + final CompactedConcurrentSkipList ccsl, + final ConcurrentLinkedQueue keyQueueOri, + final AtomicBoolean hasException, Thread[] threads, + Long[] keyArrayOri, List keyListForGet, + List nodeIdListForGet) + throws InterruptedException, CCSMapException { + List keyListForScan; + List nodeIdListForScan; + Long[] nodeIdArrayForPut;//repeated write + final ConcurrentLinkedQueue nodeIdForRepeatedList = new ConcurrentLinkedQueue<>(); + for (int th = 0; th < thread; th++) { + threads[th] = new Thread(new Runnable() { + @Override + public void run() { + try { + Long id; + while ((id = keyQueueOri.poll()) != null) { + int level = ccsl.generateRandomIndexLevel(); + long newNode = ccsl.getNodeAndWriteMeta(level, NodeUtil.getNodeMetaLenByLevel(level), + NodeUtil.getDataLen(8, 8)); + ByteBuffer keyvalueBB = NodeUtil.duplicateNodeDataByteBuffer(ccsl, newNode); + ByteBufferUtils.putInt(keyvalueBB, 0, 8); + ByteBufferUtils.putInt(keyvalueBB, Integer.BYTES, 8); + ByteBufferUtils.putLong(keyvalueBB, 2 * Integer.BYTES, id); + ByteBufferUtils.putLong(keyvalueBB, 2 * Integer.BYTES + 8, id + 1010); + nodeIdForRepeatedList.add(newNode); + + ccsl.put(newNode, level); + } + } catch (Exception e) { + e.printStackTrace(); + hasException.set(true); + } + } + }); + } + + for (Thread t : threads) { + t.start(); + } + + for (Thread t : threads) { + t.join(); + } + + Assert.assertFalse(hasException.get()); + Assert.assertEquals(size * thread, ccsl.getSize()); + + // printStat("put replica"); + keyListForScan = new ArrayList<>(); + nodeIdListForScan = new ArrayList<>(); + + doCheckScanForHead(ccsl, keyListForScan, nodeIdListForScan, (Long t, Long u) -> t + 1010 == u); + + Collections.sort(nodeIdListForScan); + + nodeIdArrayForPut = nodeIdForRepeatedList.toArray(new Long[0]); + Arrays.sort(nodeIdArrayForPut); + + Assert.assertArrayEquals(keyArrayOri, keyListForScan.toArray()); + + Assert.assertArrayEquals(nodeIdArrayForPut, nodeIdListForScan.toArray()); + + keyListForGet.clear(); + nodeIdListForGet.clear(); + for (Long id : keyArrayOri) { + + byte[] readBB = new byte[8]; + Bytes.putLong(readBB, 0, id); + long getNode = ccsl.get(readBB); + ByteBuffer bb = NodeUtil.duplicateNodeDataByteBuffer(ccsl, getNode); + long key = ByteBufferUtils.toLong(bb, 8); + long value = ByteBufferUtils.toLong(bb, 16); + //After rewrite, the new get is a new value + if (value != key + 1010) { + System.out.println( + "read error nodeId=" + getNode + "key=" + key + " ;" + nodeIdForRepeatedList + .contains(getNode)); + } + Assert.assertEquals(value, key + 1010); + keyListForGet.add(key); + nodeIdListForGet.add(getNode); + } + + Collections.sort(keyListForGet); + Collections.sort(nodeIdListForGet); + + Assert.assertArrayEquals(keyListForGet.toArray(), keyListForScan.toArray()); + Assert.assertArrayEquals(nodeIdListForGet.toArray(), nodeIdListForScan.toArray()); + + printStat("get replica", ccsl); + + //check index + checkLevel(ccsl); + + ccsl.close(); + } + + private void checkLevel(CompactedConcurrentSkipList ccsl) throws CCSMapException { + long node = ccsl.getNextNodeByIdAndLevel(SKIP_LIST_HEAD_NODE, 0); + Map levelCount = new HashMap<>(); + + while (node != NIL_NODE_ID) { + int level = ChunkUtil.getNodeLevel(ccsl.getChunkById(NodeUtil.getChunkIdByNodeId(node)), + NodeUtil.getChunkOffsetByNodeId(node)); + if (NodeUtil.isNodeRemoving(ccsl, node) || level == 0) { + node = ccsl.getNextNodeByIdAndLevel(node, 0); + continue; + } + levelCount.putIfAbsent(level, new AtomicInteger(0)); + levelCount.get(level).incrementAndGet(); + node = ccsl.getNextNodeByIdAndLevel(node, 0); + } + + System.out.println(" level 1 => " + levelCount.get(1)); + System.out.println(" level 2 => " + levelCount.get(2)); + System.out.println(" level 3 => " + levelCount.get(3)); + + Map> levelKeys = new HashMap<>(); + for (int level = 1; level <= ccsl.getLevelIndexHeader().getLevel(); level++) { + long nodeLevel = ccsl.getNextNodeByIdAndLevel(SKIP_LIST_HEAD_NODE, level); + levelKeys.put(level, new ArrayList<>()); + while (nodeLevel != NIL_NODE_ID) { + if (NodeUtil.isNodeRemoving(ccsl, nodeLevel)) { + nodeLevel = ccsl.getNextNodeByIdAndLevel(nodeLevel, level); + continue; + } + levelKeys.get(level) + .add(ByteBufferUtils.toLong(NodeUtil.duplicateNodeDataByteBuffer(ccsl, nodeLevel), 8)); + nodeLevel = ccsl.getNextNodeByIdAndLevel(nodeLevel, level); + } + } + int totalSize = 0; + int totalLevel = 0; + for (int le = 1; le <= ccsl.getLevelIndexHeader().getLevel(); le++) { + System.out.println("le => " + levelKeys.get(le)); + totalSize += levelKeys.get(le).size() * 8; + totalLevel += levelKeys.get(le).size(); + } + System.out.println("total Index size = " + + (totalSize + levelCount.get(1).get() * NodeUtil.getNodeMetaLenByLevel(1)) + + " ; totalLevel = " + totalLevel); + } + + private void doCheckScanForHead(CompactedConcurrentSkipList ccsl, List keyListForScan, + List nodeIdListForScan, BiPredicate predicate) + throws CCSMapException { + long fistNodeId = -999; + long lastNodeId = -999; + IIterCCSList iter = ccsl.nodeIdIter(); + boolean first = true; + long lastKey = -1; + int totalSize = 0; + int totalIndexSize = 0; + int totalDataSize = 0; + while (iter.hasNext()) { + long node = iter.next(); + if (NodeUtil.isNodeRemoving(ccsl, node)) { + continue; + } + IChunk chunk = ccsl.getChunkById(NodeUtil.getChunkIdByNodeId(node)); + long offset = NodeUtil.getChunkOffsetByNodeId(node); + ByteBuffer bb = NodeUtil.duplicateNodeDataByteBuffer(ccsl, node); + long currentKey = ByteBufferUtils.toLong(bb, 8); + long currentValue = ByteBufferUtils.toLong(bb, 16); + if (!predicate.test(currentKey, currentValue)) { + System.out.println("error nodeId=" + node + ", key=" + currentKey); + throw new RuntimeException("error nodeId=" + node + ", key=" + currentKey + " ,value=" + + currentValue); + } + + if (first) { + fistNodeId = node; + lastKey = currentKey; + first = false; + } else { + lastNodeId = node; + Assert.assertTrue(currentKey >= lastKey); + } + keyListForScan.add(currentKey); + nodeIdListForScan.add(node); + totalSize += 24 + NodeUtil.getNodeMetaLenByLevel(ChunkUtil.getNodeLevel(chunk, offset)); + totalIndexSize += NodeUtil.getNodeMetaLenByLevel(ChunkUtil.getNodeLevel(chunk, offset)); + totalDataSize += 24; + } + //it looks like 40% memory savings can be achieved + System.out.println( + "dddddddd= " + totalSize + " ; totalLevel=" + totalIndexSize + " totalDataSize=" + + totalDataSize); + Assert.assertEquals(fistNodeId, ccsl.findFirst()); + Assert.assertEquals(lastNodeId, ccsl.findLast()); + } + + private long generateNode(CompactedConcurrentSkipList ccsl, ConcurrentLinkedQueue keyList) + throws CCSMapException { + int level = ccsl.generateRandomIndexLevel(); + long newNode = ccsl.getNodeAndWriteMeta(level, NodeUtil.getNodeMetaLenByLevel(level), + NodeUtil.getDataLen(8, 8)); + copyDataToNode(ccsl, newNode, keyList); + return newNode; + } + + private final Random random = new Random(7); + + private void copyDataToNode(CompactedConcurrentSkipList ccsl, long node, + ConcurrentLinkedQueue keyList) { + ByteBuffer keyvalueBB = NodeUtil.duplicateNodeDataByteBuffer(ccsl, node); + ByteBufferUtils.putInt(keyvalueBB, 0, 8); + ByteBufferUtils.putInt(keyvalueBB, Integer.BYTES, 8); + int keyInt = random.nextInt(Integer.MAX_VALUE); + long key = (long) keyInt; + while (keyList.contains(key)) { + key = random.nextInt(Integer.MAX_VALUE); + } + keyList.add(key); + ByteBufferUtils.putLong(keyvalueBB, 2 * Integer.BYTES, key); + ByteBufferUtils.putLong(keyvalueBB, 2 * Integer.BYTES + 8, key); + } + + private void copyDataToNode(CompactedConcurrentSkipList ccsl, long node, long testValue, int id) { + ByteBuffer keyvalueBB = NodeUtil.duplicateNodeDataByteBuffer(ccsl, node); + ByteBufferUtils.putInt(keyvalueBB, 0, 8); + ByteBufferUtils.putInt(keyvalueBB, Integer.BYTES, 8); + ByteBufferUtils.putLong(keyvalueBB, 2 * Integer.BYTES, testValue + id); + ByteBufferUtils.putLong(keyvalueBB, 2 * Integer.BYTES + 8, testValue + id); + } + + public static CompactedConcurrentSkipList newCCSList(HeapMode heapmode, long capacity, + int chunkSize) { + AllocatorHandlerRegister.AllocatorHandlerBuilder builder = + new AllocatorHandlerRegister.AllocatorHandlerBuilder(); + builder.setCapacity(capacity); + builder.setChunkSize(chunkSize); + builder.setInitialCount(Integer.MAX_VALUE); + builder.setHeapMode(heapmode); + builder.setUseOldChunkThreshold(-1); + CCSMapChunkPool allocatorHandler = new CCSMapChunkPool(builder); + AllocatorHandlerRegister.register(allocatorHandler); + return new CompactedConcurrentSkipList<>(builder, CCSMAP_TEST_BYTE_COMPARTOR, SchemaEnum.diff); + } + + private void printStat(String msg, CompactedConcurrentSkipList ccsl) { + + System.out.println(msg + " :cprRequest count = " + ccsl.getCcslStat().getCprRequestCount()); + System.out.println(msg + " : putReq count = " + ccsl.getCcslStat().getPutRequestCount()); + System.out.println(msg + " : removed count = " + ccsl.getCcslStat().getRemovedNodeCount()); + System.out.println( + msg + " : removeReqRace count = " + ccsl.getCcslStat().getRemoveRequestRaceFailCount()); + System.out.println(msg + " : replace count = " + ccsl.getCcslStat().getReplaceRequestCount()); + + System.out.println("ccsl queue size=" + ccsl.getUsedChunkQueue().size()); + System.out.println( + "global chunk count = " + ((CCSMapChunkPool) AllocatorHandlerRegister.getAllocatorHandler()) + .getChunkCounter()); + System.out.println( + "global map size=" + ((CCSMapChunkPool) AllocatorHandlerRegister.getAllocatorHandler()) + .getChunkMap().length); + System.out.println( + "global queue size=" + ((CCSMapChunkPool) AllocatorHandlerRegister.getAllocatorHandler()) + .getChunkQueue().size()); + } + + @Test + public void testLevelRandom() throws Exception { + CompactedConcurrentSkipList ccsl = newCCSList(HeapMode.offHeap, 128 * 1024, 100); + int level; + Map allLevels = new ConcurrentHashMap<>(); + int loop = 5000000; + for (int i = 0; i < loop; i++) { + + level = ccsl.generateRandomIndexLevel(); + + if (level == NO_INDEX) { + continue; + } + + if (!allLevels.containsKey(level)) { + allLevels.putIfAbsent(level, new AtomicInteger(0)); + } + allLevels.get(level).incrementAndGet(); + + if (level > ccsl.getLevelIndexHeader().getLevel()) { + ccsl.getLevelIndexHeader().updateLevel(level); + } + } + + List allLevelList = new ArrayList<>(allLevels.keySet()); + Collections.sort(allLevelList); + + int totalInd = 0; + int totalLevel = 0; + int totalSize = 0; + for (Integer id : allLevelList) { + int c = allLevels.get(id).get(); + totalInd += c; + totalLevel += id * c; + System.out.println(id + " ===> " + c); + totalSize += c * 8; + } + System.out.println(" totalInd ===> " + totalInd); + System.out.println(" totalLevel ===> " + totalLevel); + System.out.println(" loop* 26/100 ===> " + (loop / 100 * 26)); + System.out.println(" loop* 24/100 ===> " + (loop / 100 * 24)); + + Assert.assertTrue(totalInd >= loop / 100 * 24); + Assert.assertTrue(totalInd <= loop / 100 * 26); + + System.out.println("jdk index mem: " + totalLevel * 40); + System.out.println("ccsmap index mem: " + totalSize); + + } +} + diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestNodeUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestNodeUtil.java new file mode 100644 index 0000000000..01d8817445 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestNodeUtil.java @@ -0,0 +1,175 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import static org.apache.hadoop.hbase.regionserver.skiplist.core.ChunkUtil.getNodeLevel; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.INDEX_NOT_INITIAL; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.NIL_NODE_ID; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.MockBaseCCSMap.CCSMAP_TEST_BYTE_COMPARTOR; +import java.nio.ByteBuffer; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.regionserver.skiplist.exception.CCSMapException; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(SmallTests.class) +public class TestNodeUtil { + + @ClassRule public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestNodeUtil.class); + + @Test + public void testNormal() throws Exception { + testNormal(HeapMode.offHeap, SchemaEnum.diff); + testNormal(HeapMode.onHeap, SchemaEnum.diff); + testNormal(HeapMode.offHeap, SchemaEnum.same); + testNormal(HeapMode.onHeap, SchemaEnum.same); + } + + private void testNormal(HeapMode heapMode, SchemaEnum schemaEnum) throws CCSMapException { + AllocatorHandlerRegister.AllocatorHandlerBuilder builder = + new AllocatorHandlerRegister.AllocatorHandlerBuilder(); + builder.setCapacity(8 * 1024 * 1024); + builder.setChunkSize(4 * 1024); + builder.setInitialCount(Integer.MAX_VALUE); + builder.setHeapMode(heapMode); + CCSMapChunkPool allocatorHandler = new CCSMapChunkPool(builder); + AllocatorHandlerRegister.register(allocatorHandler); + CompactedConcurrentSkipList ccsl = + new CompactedConcurrentSkipList<>(builder, CCSMAP_TEST_BYTE_COMPARTOR, schemaEnum); + int keyLen = 6; + int valueLen = 21; + long node1 = ccsl.getNodeAndWriteMeta(0, NodeUtil.getNodeMetaLenByLevel(0), + NodeUtil.getDataLen(keyLen, valueLen)); + long node2 = ccsl.getNodeAndWriteMeta(1, NodeUtil.getNodeMetaLenByLevel(1), + NodeUtil.getDataLen(keyLen + 1, valueLen + 1)); + + IChunk chunk1 = allocatorHandler.getChunkById(NodeUtil.getChunkIdByNodeId(node1)); + IChunk chunk2 = allocatorHandler.getChunkById(NodeUtil.getChunkIdByNodeId(node2)); + + int offset1 = (int) NodeUtil.getChunkOffsetByNodeId(node1); + + int startOffset1 = offset1 + NodeUtil.getNodeMetaLenByLevel(getNodeLevel(chunk1, offset1)); + //write data + ByteBufferUtils.putInt(chunk1.getByteBuffer(), startOffset1, keyLen); + ByteBufferUtils.putInt(chunk1.getByteBuffer(), startOffset1 + Integer.BYTES, valueLen); + + int offset2 = (int) NodeUtil.getChunkOffsetByNodeId(node2); + + int startOffset2 = offset2 + NodeUtil.getNodeMetaLenByLevel(getNodeLevel(chunk2, offset2)); + ByteBufferUtils.putInt(chunk2.getByteBuffer(), startOffset2, keyLen + 1); + ByteBufferUtils.putInt(chunk2.getByteBuffer(), startOffset2 + Integer.BYTES, valueLen + 1); + + Assert.assertEquals(16, NodeUtil.getNodeMetaLenByLevel(0)); + Assert.assertEquals(16 + 8, NodeUtil.getNodeMetaLenByLevel(1)); + Assert.assertEquals(16 + 2 * 8, NodeUtil.getNodeMetaLenByLevel(2)); + + Assert.assertEquals(51, NodeUtil.getNodeLen(0, keyLen, valueLen)); + Assert.assertEquals(59, NodeUtil.getNodeLen(1, keyLen, valueLen)); + + Assert.assertEquals(1, NodeUtil.getChunkIdByNodeId(node1)); + Assert.assertEquals(0, NodeUtil.getChunkOffsetByNodeId(node1)); + Assert.assertEquals(1, NodeUtil.getChunkIdByNodeId(node2)); + Assert.assertEquals(56, NodeUtil.getChunkOffsetByNodeId(node2)); + + //keyOffset is right, when do lock or unlock + Assert.assertEquals(schemaEnum == SchemaEnum.diff ? 24 : 16, + ChunkUtil.getKeyOffset(chunk1, NodeUtil.getChunkOffsetByNodeId(node1), schemaEnum)); + Assert.assertEquals(schemaEnum == SchemaEnum.diff ? 88 : 80, + ChunkUtil.getKeyOffset(chunk1, NodeUtil.getChunkOffsetByNodeId(node2), schemaEnum)); + + Assert.assertFalse(NodeUtil.isNodeRemoving(allocatorHandler, node1)); + Assert.assertTrue(NodeUtil.casSetNodeStat(allocatorHandler, node1, NodeStat.REMOVE)); + Assert.assertTrue(NodeUtil.isNodeRemoving(allocatorHandler, node1)); + Assert.assertFalse(NodeUtil.casSetNodeStat(allocatorHandler, node1, NodeStat.REMOVE)); + Assert.assertTrue(NodeUtil.isNodeRemoving(allocatorHandler, node1)); + + Assert.assertTrue(NodeUtil.casNodeLock(allocatorHandler, node2, NodeStat.PUT)); + Assert.assertFalse(NodeUtil.isNodeRemoving(allocatorHandler, node2)); + Assert.assertFalse(NodeUtil.casNodeLock(allocatorHandler, node2, NodeStat.PUT)); + Assert.assertFalse(NodeUtil.casNodeLock(allocatorHandler, node2, NodeStat.REMOVE)); + Assert.assertFalse(NodeUtil.casNodeUnLock(allocatorHandler, node2, NodeStat.REMOVE)); + Assert.assertFalse(NodeUtil.isNodeRemoving(allocatorHandler, node2)); + Assert.assertTrue(NodeUtil.casNodeUnLock(allocatorHandler, node2, NodeStat.PUT)); + Assert.assertTrue(NodeUtil.casNodeLock(allocatorHandler, node2, NodeStat.REMOVE)); + + Assert.assertFalse(NodeUtil.isNodeRemoving(allocatorHandler, node2)); + Assert.assertFalse(NodeUtil.casSetNodeStat(allocatorHandler, node2, NodeStat.REMOVE)); + + Assert.assertTrue(NodeUtil.isNodeRemoving(allocatorHandler, node1)); + + Assert.assertEquals(NIL_NODE_ID, NodeUtil.getNextNodeIdForLevel(allocatorHandler, node1, 0)); + Assert.assertEquals(NIL_NODE_ID, NodeUtil.getNextNodeIdForLevel(allocatorHandler, node2, 0)); + Assert + .assertEquals(INDEX_NOT_INITIAL, NodeUtil.getNextNodeIdForLevel(allocatorHandler, node2, 1)); + + //keyOffset is right, when do lock or unlock + Assert.assertEquals(schemaEnum == SchemaEnum.diff ? 24 : 16, + ChunkUtil.getKeyOffset(chunk1, NodeUtil.getChunkOffsetByNodeId(node1), schemaEnum)); + Assert.assertEquals(schemaEnum == SchemaEnum.diff ? 88 : 80, + ChunkUtil.getKeyOffset(chunk1, NodeUtil.getChunkOffsetByNodeId(node2), schemaEnum)); + + Assert + .assertTrue(NodeUtil.casUpdateNextNodeForLevel(allocatorHandler, 0, node1, NIL_NODE_ID, 123)); + Assert.assertEquals(123, NodeUtil.getNextNodeIdForLevel(allocatorHandler, node1, 0)); + + Assert.assertFalse(NodeUtil.casUpdateNextNodeForLevel(allocatorHandler, 0, node2, 1, 123)); + Assert.assertEquals(NIL_NODE_ID, NodeUtil.getNextNodeIdForLevel(allocatorHandler, node2, 0)); + + Assert.assertTrue(NodeUtil.casUpdateNextNodeForLevel(allocatorHandler, 0, node1, 123, 124)); + Assert.assertEquals(124, NodeUtil.getNextNodeIdForLevel(allocatorHandler, node1, 0)); + + Assert.assertEquals(keyLen, ChunkUtil.testGetNodeKeyLen(chunk1, offset1)); + Assert + .assertEquals(schemaEnum == SchemaEnum.diff ? keyLen : NodeUtil.getDataLen(keyLen, valueLen), + ChunkUtil.getKeyLen(chunk1, NodeUtil.getChunkOffsetByNodeId(node1), + ChunkUtil.getKeyOffset(chunk1, NodeUtil.getChunkOffsetByNodeId(node1), schemaEnum), + schemaEnum)); + + ByteBuffer keyvalueBB = NodeUtil.duplicateNodeDataByteBuffer(allocatorHandler, node1); + Assert.assertEquals(0, keyvalueBB.position()); + Assert.assertEquals(keyLen + valueLen + 8, keyvalueBB.limit()); + Assert.assertEquals(keyLen + valueLen + 8, keyvalueBB.capacity()); + + Assert.assertEquals(keyLen, ByteBufferUtils.toInt(keyvalueBB, 0)); + Assert.assertEquals(keyLen, keyvalueBB.getInt(0)); + Assert.assertEquals(valueLen, keyvalueBB.getInt(4)); + + Assert.assertEquals(keyvalueBB.getInt(0), chunk1.getByteBuffer().getInt(startOffset1)); + Assert.assertEquals(keyvalueBB.getInt(Integer.BYTES), + chunk1.getByteBuffer().getInt(startOffset1 + Integer.BYTES)); + + ByteBuffer keyvalueBB2 = NodeUtil.duplicateNodeDataByteBuffer(allocatorHandler, node2); + Assert.assertEquals(0, keyvalueBB2.position()); + Assert.assertEquals(keyLen + 1 + valueLen + 1 + 8, keyvalueBB2.limit()); + Assert.assertEquals(keyLen + 1 + valueLen + 1 + 8, keyvalueBB2.capacity()); + + Assert.assertEquals(keyLen + 1, ByteBufferUtils.toInt(keyvalueBB2, 0)); + Assert.assertEquals(keyLen + 1, keyvalueBB2.getInt(0)); + Assert.assertEquals(valueLen + 1, keyvalueBB2.getInt(4)); + + Assert.assertEquals(keyvalueBB2.getInt(0), chunk1.getByteBuffer().getInt(startOffset2)); + Assert.assertEquals(keyvalueBB2.getInt(Integer.BYTES), + chunk1.getByteBuffer().getInt(startOffset2 + Integer.BYTES)); + + } + +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestOffHeapChunk.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestOffHeapChunk.java new file mode 100644 index 0000000000..f248a244d7 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestOffHeapChunk.java @@ -0,0 +1,39 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(SmallTests.class) +public class TestOffHeapChunk { + + @ClassRule public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestOffHeapChunk.class); + + @Test + public void testNormal() throws Exception { + OffHeapChunk offHeapChunk = new OffHeapChunk(2, 3 * 1024 * 1024); + Assert.assertEquals(HeapMode.offHeap, offHeapChunk.getHeapMode()); + Assert.assertEquals(2, offHeapChunk.getChunkId()); + } +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestOnHeapChunk.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestOnHeapChunk.java new file mode 100644 index 0000000000..6fbfb8710d --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestOnHeapChunk.java @@ -0,0 +1,40 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(SmallTests.class) +public class TestOnHeapChunk { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestOnHeapChunk.class); + @Test + public void testNormal() throws Exception { + OnHeapChunk onHeapChunk = new OnHeapChunk(1, 4 * 1024 * 1024); + Assert.assertEquals(HeapMode.onHeap, onHeapChunk.getHeapMode()); + Assert.assertEquals(1, onHeapChunk.getChunkId()); + + } +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestOnHeapHeadIndex.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestOnHeapHeadIndex.java new file mode 100644 index 0000000000..dfcbfca297 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestOnHeapHeadIndex.java @@ -0,0 +1,200 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.DEFAULT_LEVEL; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.FOUR_BYTES_MARK; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.NIL_NODE_ID; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(SmallTests.class) +public class TestOnHeapHeadIndex { + + @ClassRule public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestOnHeapHeadIndex.class); + + @Test + public void testNormal() throws Exception { + + OnHeapIndexHeader heapHeadIndex = new OnHeapIndexHeader(); + Assert.assertEquals(1, heapHeadIndex.getLevel()); + Assert.assertEquals(DEFAULT_LEVEL, heapHeadIndex.getLevelNodeNext().size()); + int l = 0; + for (long index : heapHeadIndex.getLevelNodeNext()) { + Assert.assertEquals(NIL_NODE_ID, index); + if (l + 1 <= heapHeadIndex.getLevel()) { + Assert.assertEquals(NIL_NODE_ID, heapHeadIndex.getLevelNextNodeId(l + 1)); + } + l++; + } + + //update level 1, unsigned test + long offsetLevel1 = 0xFFFFFFFFL; + long chunkIdLevel1 = 0xFFFFFFFFL; + heapHeadIndex.updateLevel(1); + + Assert.assertEquals(1, heapHeadIndex.getLevel()); + Assert.assertEquals(DEFAULT_LEVEL, heapHeadIndex.getLevelNodeNext().size()); + + long node1 = getNodeId(chunkIdLevel1, offsetLevel1); + Assert.assertTrue(heapHeadIndex.casUpdateLevelNextNodeId(1, NIL_NODE_ID, node1)); + Assert.assertFalse(heapHeadIndex.casUpdateLevelNextNodeId(1, NIL_NODE_ID, node1)); + + for (int i = 0; i < heapHeadIndex.getLevelNodeNext().size(); i++) { + if (i == 0) { + long chunkIdOffset = heapHeadIndex.getLevelNodeNext().get(0); + Assert.assertEquals(chunkIdLevel1, ((chunkIdOffset >>> 32) & FOUR_BYTES_MARK)); + Assert.assertEquals(offsetLevel1, (chunkIdOffset & FOUR_BYTES_MARK)); + } else { + Assert.assertEquals(NIL_NODE_ID, heapHeadIndex.getLevelNodeNext().get(i).longValue()); + } + } + + offsetLevel1 = 1; + chunkIdLevel1 = 0xFFFFFFFEL; + long node2 = getNodeId(chunkIdLevel1, offsetLevel1); + Assert.assertTrue(heapHeadIndex.casUpdateLevelNextNodeId(1, node1, node2)); + + Assert.assertEquals(1, heapHeadIndex.getLevel()); + Assert.assertEquals(DEFAULT_LEVEL, heapHeadIndex.getLevelNodeNext().size()); + for (int i = 0; i < heapHeadIndex.getLevelNodeNext().size(); i++) { + if (i == 0) { + long chunkIdOffset = heapHeadIndex.getLevelNodeNext().get(0); + Assert.assertEquals(chunkIdLevel1, ((chunkIdOffset >>> 32) & FOUR_BYTES_MARK)); + Assert.assertEquals(offsetLevel1, (chunkIdOffset & FOUR_BYTES_MARK)); + } else { + Assert.assertEquals(NIL_NODE_ID, heapHeadIndex.getLevelNodeNext().get(i).longValue()); + } + } + + offsetLevel1 = 0xFFFFFFF1L; + chunkIdLevel1 = 1; + long node3 = getNodeId(chunkIdLevel1, offsetLevel1); + Assert.assertTrue(heapHeadIndex.casUpdateLevelNextNodeId(1, node2, node3)); + + Assert.assertEquals(1, heapHeadIndex.getLevel()); + Assert.assertEquals(DEFAULT_LEVEL, heapHeadIndex.getLevelNodeNext().size()); + for (int i = 0; i < heapHeadIndex.getLevelNodeNext().size(); i++) { + if (i == 0) { + long chunkIdOffset = heapHeadIndex.getLevelNodeNext().get(0); + Assert.assertEquals(chunkIdLevel1, ((chunkIdOffset >>> 32) & FOUR_BYTES_MARK)); + Assert.assertEquals(offsetLevel1, (chunkIdOffset & FOUR_BYTES_MARK)); + } else { + Assert.assertEquals(NIL_NODE_ID, heapHeadIndex.getLevelNodeNext().get(i).longValue()); + } + } + + // try to grow by one level + long offsetLevel2 = 23456; + long chunkIdLevel2 = 12345; + long node4 = getNodeId(chunkIdLevel2, offsetLevel2); + heapHeadIndex.updateLevel(2); + Assert.assertTrue(heapHeadIndex.casUpdateLevelNextNodeId(2, NIL_NODE_ID, node4)); + + Assert.assertEquals(2, heapHeadIndex.getLevel()); + Assert.assertEquals(DEFAULT_LEVEL, heapHeadIndex.getLevelNodeNext().size()); + for (int i = 0; i < heapHeadIndex.getLevelNodeNext().size(); i++) { + if (i == 0) { + long chunkIdOffset = heapHeadIndex.getLevelNodeNext().get(0); + Assert.assertEquals(chunkIdLevel1, ((chunkIdOffset >>> 32) & FOUR_BYTES_MARK)); + Assert.assertEquals(offsetLevel1, (chunkIdOffset & FOUR_BYTES_MARK)); + } else if (i == 1) { + long chunkIdOffset = heapHeadIndex.getLevelNodeNext().get(1); + Assert.assertEquals(chunkIdLevel2, ((chunkIdOffset >>> 32) & FOUR_BYTES_MARK)); + Assert.assertEquals(offsetLevel2, (chunkIdOffset & FOUR_BYTES_MARK)); + } else { + Assert.assertEquals(NIL_NODE_ID, heapHeadIndex.getLevelNodeNext().get(i).longValue()); + } + } + + //error level + String expectExceptionMsg = "WANT_EXCEPTION"; + try { + heapHeadIndex.updateLevel(4); + } catch (Exception e) { + expectExceptionMsg = e.getMessage(); + } + + Assert.assertEquals("updateLevel can't skip. now level=2 ,dst level=4", expectExceptionMsg); + + //test access DEFAULT_LEVEL, heapHeadIndex expand the indexArray + + for (int i = 1; i < 34; i++) { + heapHeadIndex.updateLevel(i); + + } + + Assert.assertEquals(33, heapHeadIndex.getLevel()); + Assert.assertEquals(DEFAULT_LEVEL * 2, heapHeadIndex.getLevelNodeNext().size()); + + for (int i = 0; i < heapHeadIndex.getLevelNodeNext().size(); i++) { + if (i == 0) { + long chunkIdOffset = heapHeadIndex.getLevelNodeNext().get(0); + Assert.assertEquals(chunkIdLevel1, ((chunkIdOffset >>> 32) & FOUR_BYTES_MARK)); + Assert.assertEquals(offsetLevel1, (chunkIdOffset & FOUR_BYTES_MARK)); + } else if (i == 1) { + long chunkIdOffset = heapHeadIndex.getLevelNodeNext().get(1); + Assert.assertEquals(chunkIdLevel2, ((chunkIdOffset >>> 32) & FOUR_BYTES_MARK)); + Assert.assertEquals(offsetLevel2, (chunkIdOffset & FOUR_BYTES_MARK)); + } else { + Assert.assertEquals(NIL_NODE_ID, heapHeadIndex.getLevelNodeNext().get(i).longValue()); + } + } + } + + @Test + public void testException() throws Exception { + OnHeapIndexHeader heapHeadIndex = new OnHeapIndexHeader(); + //error level + String expectExceptionMsg = "WANT_EXCEPTION"; + try { + heapHeadIndex.updateLevel(3); + } catch (Exception e) { + expectExceptionMsg = e.getMessage(); + } + + Assert.assertEquals("updateLevel can't skip. now level=1 ,dst level=3", expectExceptionMsg); + + expectExceptionMsg = "WANT_EXCEPTION"; + try { + heapHeadIndex.updateLevel(0); + } catch (Exception e) { + expectExceptionMsg = e.getMessage(); + } + + Assert.assertEquals("dst level less than 1.", expectExceptionMsg); + + expectExceptionMsg = "WANT_EXCEPTION"; + try { + heapHeadIndex.updateLevel(256); + } catch (Exception e) { + expectExceptionMsg = e.getMessage(); + } + + Assert.assertEquals("dst level>255, It's unbelievable.", expectExceptionMsg); + } + + private long getNodeId(long chunkId, long offset) { + return ((chunkId & FOUR_BYTES_MARK) << 32) | (offset & FOUR_BYTES_MARK); + } +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestRandomHelp.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestRandomHelp.java new file mode 100644 index 0000000000..637ea9e173 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestRandomHelp.java @@ -0,0 +1,96 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(SmallTests.class) +public class TestRandomHelp { + + @ClassRule public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRandomHelp.class); + + @Test + public void testNormal() throws Exception { + Random seedGenerator = new Random(); + int randomSeed = seedGenerator.nextInt() | 0x0100; + + int level; + Map allLevels = new ConcurrentHashMap<>(); + int loop = 5000000; + for (int i = 0; i < loop; i++) { + + int x = randomSeed; + x ^= x << 13; + x ^= x >>> 17; + x ^= x << 5; + randomSeed = x; + + if ((x & 0x8001) == 0) { // test highest and lowest bits + level = 1; + x >>>= 1; + while ((x & 1) != 0) { + ++level; + x >>>= 1; + } + + if (!allLevels.containsKey(level)) { + allLevels.putIfAbsent(level, new AtomicInteger(0)); + } + allLevels.get(level).incrementAndGet(); + + } + } + + List allLevelList = new ArrayList<>(allLevels.keySet()); + Collections.sort(allLevelList); + + int totalInd = 0; + int totalLevel = 0; + int totalSize = 0; + for (Integer id : allLevelList) { + int c = allLevels.get(id).get(); + totalInd += c; + totalLevel += id * c; + System.out.println(id + " ===> " + c); + totalSize += c * 8; + } + System.out.println(" totalInd ===> " + totalInd); + System.out.println(" totalLevel ===> " + totalLevel); + System.out.println(" loop* 26/100 ===> " + (loop / 100 * 26)); + System.out.println(" loop* 24/100 ===> " + (loop / 100 * 24)); + + Assert.assertTrue(totalInd >= loop / 100 * 24); + Assert.assertTrue(totalInd <= loop / 100 * 26); + + System.out.println("jdk index mem: " + totalLevel * 40); + System.out.println("ccsmap index mem: " + totalSize); + } +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestSubCompactedConcurrentSkipList.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestSubCompactedConcurrentSkipList.java new file mode 100644 index 0000000000..135966c40d --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/core/TestSubCompactedConcurrentSkipList.java @@ -0,0 +1,403 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.core; + +import static org.apache.hadoop.hbase.regionserver.skiplist.core.AbstractCCSMap.EQ; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.AbstractCCSMap.GT; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.AbstractCCSMap.LT; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.Constant.NIL_NODE_ID; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.TestCompactedConcurrentSkipList.newCCSList; +import static org.apache.hadoop.hbase.regionserver.skiplist.core.TestCompactedConcurrentSkipList.readKey; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@SuppressWarnings("PointlessBitwiseExpression") +@Category(SmallTests.class) +public class TestSubCompactedConcurrentSkipList { + + @ClassRule public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSubCompactedConcurrentSkipList.class); + + @Test + public void testNormal() throws Exception { + CompactedConcurrentSkipList ccslOri = newCCSList(HeapMode.onHeap, 128 * 1024, 100); + + //This subMap have all item. + SubCompactedConcurrentSkipList subCcslUnbounded = + new SubCompactedConcurrentSkipList<>(ccslOri, null, null, false, false); + + List nodeList = doTestUnboundedSubMap(ccslOri, subCcslUnbounded); + IIterCCSList subIter; + List nodeIter; + + //test Head sub + byte[] headBB = new byte[8]; + Bytes.putLong(headBB, 0, 2); + + SubCompactedConcurrentSkipList subCcslOnlyHiBound = + new SubCompactedConcurrentSkipList<>(ccslOri, null, headBB, false, false); + Assert.assertEquals(0, subCcslOnlyHiBound.getSize()); + + long loNodeId2_1 = subCcslOnlyHiBound.loNode(); + long hiNodeId2_1 = subCcslOnlyHiBound.hiNode(); + System.out.println("loNodeId2_1:" + loNodeId2_1 + " => " + readKey(loNodeId2_1, ccslOri)); + System.out.println("hiNodeId2_1:" + hiNodeId2_1 + " => " + readKey(hiNodeId2_1, ccslOri)); + Assert.assertEquals(NIL_NODE_ID, readKey(loNodeId2_1, ccslOri)); + Assert.assertEquals(NIL_NODE_ID, readKey(hiNodeId2_1, ccslOri)); + + subIter = subCcslOnlyHiBound.nodeIdIter(); + nodeIter = new ArrayList<>(); + while (subIter.hasNext()) { + long iterNode = subIter.next(); + nodeIter.add(iterNode); + } + + Assert.assertEquals(0, nodeIter.size()); + + //test tail sub + byte[] tailBB = new byte[8]; + Bytes.putLong(tailBB, 0, 200); + + SubCompactedConcurrentSkipList subCcsl2_2 = + new SubCompactedConcurrentSkipList<>(ccslOri, tailBB, null, false, false); + Assert.assertEquals(0, subCcsl2_2.getSize()); + + long loNodeId2_2 = subCcsl2_2.loNode(); + long hiNodeId2_2 = subCcsl2_2.hiNode(); + System.out.println("loNodeId2_2:" + loNodeId2_2 + " => " + readKey(loNodeId2_2, ccslOri)); + System.out.println("hiNodeId2_2:" + hiNodeId2_2 + " => " + readKey(hiNodeId2_2, ccslOri)); + Assert.assertEquals(NIL_NODE_ID, readKey(loNodeId2_2, ccslOri)); + Assert.assertEquals(NIL_NODE_ID, readKey(hiNodeId2_2, ccslOri)); + + subIter = subCcsl2_2.nodeIdIter(); + nodeIter = new ArrayList<>(); + while (subIter.hasNext()) { + long iterNode = subIter.next(); + nodeIter.add(iterNode); + } + + Assert.assertEquals(0, nodeIter.size()); + + //Test open interval subset + byte[] startBB = new byte[8]; + Bytes.putLong(startBB, 0, 4); + + byte[] endBB = new byte[8]; + Bytes.putLong(endBB, 0, 14); + + SubCompactedConcurrentSkipList subCcslHiLoExclusive = + new SubCompactedConcurrentSkipList<>(ccslOri, startBB, endBB, false, false); + + long loNodeId = subCcslHiLoExclusive.loNode(); + long hiNodeId = subCcslHiLoExclusive.hiNode(); + System.out.println("loNodeId:" + loNodeId + " => " + readKey(loNodeId, ccslOri)); + System.out.println("hiNodeId:" + hiNodeId + " => " + readKey(hiNodeId, ccslOri)); + Assert.assertEquals(6, readKey(loNodeId, ccslOri)); + Assert.assertEquals(12, readKey(hiNodeId, ccslOri)); + Assert.assertEquals(4, subCcslHiLoExclusive.getSize()); + + subIter = subCcslHiLoExclusive.nodeIdIter(); + nodeIter = new ArrayList<>(); + while (subIter.hasNext()) { + long iterNode = subIter.next(); + nodeIter.add(iterNode); + } + + Assert.assertEquals(nodeIter.size(), 4); + for (int i = 0; i < nodeIter.size(); i++) { + Assert.assertEquals(6 + i * 2, readKey(nodeIter.get(i), subCcslHiLoExclusive)); + } + + Assert.assertEquals(NIL_NODE_ID, subCcslHiLoExclusive.get(startBB)); + Assert.assertEquals(NIL_NODE_ID, subCcslHiLoExclusive.get(endBB)); + + Assert.assertEquals(NIL_NODE_ID, subCcslHiLoExclusive.remove(startBB)); + Assert.assertEquals(NIL_NODE_ID, subCcslHiLoExclusive.remove(endBB)); + + String expectException = "WANT_EXCEPTION"; + try { + subCcslHiLoExclusive.put(nodeList.get(1), 1); + } catch (Exception e) { + expectException = e.getMessage(); + } + + Assert.assertEquals("key out of range", expectException); + + //Test closed interval subset + + SubCompactedConcurrentSkipList subCcslHiLoInclusive = + new SubCompactedConcurrentSkipList<>(ccslOri, startBB, endBB, true, true); + + doTestHiLoInclusiveSubMap(ccslOri, nodeList, startBB, endBB, subCcslHiLoInclusive); + + doRewriteDeletedNode(ccslOri, subCcslUnbounded, subCcslHiLoInclusive); + } + + private void doTestHiLoInclusiveSubMap(CompactedConcurrentSkipList ccslOri, + List nodeList, byte[] startBB, byte[] endBB, + SubCompactedConcurrentSkipList subCcslHiLoInclusive) + throws org.apache.hadoop.hbase.regionserver.skiplist.exception.CCSMapException { + long loNodeId; + long hiNodeId; + IIterCCSList subIter; + List nodeIter; + String expectException; + loNodeId = subCcslHiLoInclusive.loNode(); + hiNodeId = subCcslHiLoInclusive.hiNode(); + System.out.println("loNodeId:" + loNodeId + " => " + readKey(loNodeId, ccslOri)); + System.out.println("hiNodeId:" + hiNodeId + " => " + readKey(hiNodeId, ccslOri)); + Assert.assertEquals(4, readKey(loNodeId, ccslOri)); + Assert.assertEquals(14, readKey(hiNodeId, ccslOri)); + Assert.assertEquals(6, subCcslHiLoInclusive.getSize()); + + subIter = subCcslHiLoInclusive.nodeIdIter(); + nodeIter = new ArrayList<>(); + while (subIter.hasNext()) { + long iterNode = subIter.next(); + nodeIter.add(iterNode); + } + + Assert.assertEquals(6, nodeIter.size()); + for (int i = 0; i < nodeIter.size(); i++) { + Assert.assertEquals(4 + i * 2, readKey(nodeIter.get(i), subCcslHiLoInclusive)); + } + + Assert.assertEquals(subCcslHiLoInclusive.get(startBB), nodeList.get(1).longValue()); + Assert.assertEquals(subCcslHiLoInclusive.get(endBB), nodeList.get(6).longValue()); + + Assert.assertEquals(subCcslHiLoInclusive.remove(startBB), nodeList.get(1).longValue()); + Assert.assertEquals(subCcslHiLoInclusive.remove(endBB), nodeList.get(6).longValue()); + + expectException = "WANT_EXCEPTION"; + try { + subCcslHiLoInclusive.put(nodeList.get(0), 1); + } catch (Exception e) { + expectException = e.getMessage(); + } + + Assert.assertEquals("key out of range", expectException); + + loNodeId = subCcslHiLoInclusive.loNode(); + hiNodeId = subCcslHiLoInclusive.hiNode(); + + System.out.println("loNodeId:" + loNodeId + " => " + readKey(loNodeId, ccslOri)); + System.out.println("hiNodeId:" + hiNodeId + " => " + readKey(hiNodeId, ccslOri)); + + Assert.assertEquals(6, readKey(loNodeId, ccslOri)); + Assert.assertEquals(12, readKey(hiNodeId, ccslOri)); + + Assert.assertEquals(4, subCcslHiLoInclusive.getSize()); + Assert.assertEquals(98, ccslOri.getSize()); + + subIter = subCcslHiLoInclusive.nodeIdIter(); + nodeIter = new ArrayList<>(); + while (subIter.hasNext()) { + long iterNode = subIter.next(); + nodeIter.add(iterNode); + } + + IIterCCSList oriIter = ccslOri.nodeIdIter(); + List allNodeIterList = new ArrayList<>(); + while (oriIter.hasNext()) { + long iterNode = oriIter.next(); + allNodeIterList.add(iterNode); + } + + Assert.assertEquals(98, allNodeIterList.size()); + Assert.assertEquals(4, nodeIter.size()); + for (int i = 0; i < nodeIter.size(); i++) { + Assert.assertEquals(6 + i * 2, readKey(nodeIter.get(i), subCcslHiLoInclusive)); + } + + expectException = "WANT_EXCEPTION"; + try { + subCcslHiLoInclusive.put(nodeList.get(1), 1); + } catch (Exception e) { + expectException = e.getMessage(); + } + + Assert.assertEquals("can't put existed node", expectException); + } + + private List doTestUnboundedSubMap(CompactedConcurrentSkipList ccslOri, + SubCompactedConcurrentSkipList subCcslUnbounded) + throws org.apache.hadoop.hbase.regionserver.skiplist.exception.CCSMapException { + IIterCCSList subIter = subCcslUnbounded.nodeIdIter(); + Assert.assertFalse(subIter.hasNext()); + + boolean first = true; + long firstNodeId = -999; + long lastNodeId = -999; + List nodeList = new ArrayList<>(); + //put 100 data + for (int i = 1; i <= 100; i++) { + int level = subCcslUnbounded.generateRandomIndexLevel(); + long node = subCcslUnbounded.getNodeAndWriteMeta(level, NodeUtil.getNodeMetaLenByLevel(level), + NodeUtil.getDataLen(8, 8)); + ByteBuffer keyvalueBB = NodeUtil.duplicateNodeDataByteBuffer(subCcslUnbounded, node); + ByteBufferUtils.putInt(keyvalueBB, 0, 8); + ByteBufferUtils.putInt(keyvalueBB, Integer.BYTES, 8); + ByteBufferUtils.putLong(keyvalueBB, 2 * Integer.BYTES, i * 2); + ByteBufferUtils.putLong(keyvalueBB, 2 * Integer.BYTES + 8, i); + + subCcslUnbounded.put(node, level); + if (first) { + firstNodeId = node; + first = false; + } else { + lastNodeId = node; + } + + nodeList.add(i - 1, node); + } + + Assert.assertEquals(100, subCcslUnbounded.getSize()); + Assert.assertEquals(100, ccslOri.getSize()); + Assert.assertEquals(firstNodeId, ccslOri.findFirst()); + Assert.assertEquals(firstNodeId, subCcslUnbounded.findFirst()); + Assert.assertEquals(lastNodeId, ccslOri.findLast()); + Assert.assertEquals(lastNodeId, subCcslUnbounded.findLast()); + + subIter = subCcslUnbounded.nodeIdIter(); + List nodeIter = new ArrayList<>(); + while (subIter.hasNext()) { + long iterNode = subIter.next(); + nodeIter.add(iterNode); + } + + Assert.assertArrayEquals(nodeList.toArray(), nodeIter.toArray()); + + for (int i = 100; i >= 1; i--) { + byte[] existedBB = new byte[8]; + Bytes.putLong(existedBB, 0, i * 2); + + Assert + .assertEquals(i * 2, readKey(subCcslUnbounded.findNear(existedBB, EQ), subCcslUnbounded)); + if (i == 0) { + Assert.assertEquals(NIL_NODE_ID, + readKey(subCcslUnbounded.findNear(existedBB, LT), subCcslUnbounded)); + } else { + Assert.assertEquals(i * 2 - 2, + readKey(subCcslUnbounded.findNear(existedBB, LT), subCcslUnbounded)); + } + Assert.assertEquals(i * 2, + readKey(subCcslUnbounded.findNear(existedBB, LT | EQ), subCcslUnbounded)); + if (i == 100) { + Assert.assertEquals(NIL_NODE_ID, + readKey(subCcslUnbounded.findNear(existedBB, GT), subCcslUnbounded)); + } else { + Assert.assertEquals(i * 2 + 2, + readKey(subCcslUnbounded.findNear(existedBB, GT), subCcslUnbounded)); + } + Assert.assertEquals(i * 2, + readKey(subCcslUnbounded.findNear(existedBB, GT | EQ), subCcslUnbounded)); + + byte[] noExistedBB = new byte[8]; + Bytes.putLong(noExistedBB, 0, i * 2 + 1); + + if (i == 100) { + Assert.assertEquals(NIL_NODE_ID, + readKey(subCcslUnbounded.findNear(noExistedBB, EQ), subCcslUnbounded)); + } else { + Assert.assertEquals(i * 2 + 2, + readKey(subCcslUnbounded.findNear(noExistedBB, EQ), subCcslUnbounded)); + } + + Assert + .assertEquals(i * 2, readKey(subCcslUnbounded.findNear(noExistedBB, LT), subCcslUnbounded)); + Assert.assertEquals(i * 2, + readKey(subCcslUnbounded.findNear(noExistedBB, LT | EQ), subCcslUnbounded)); + + if (i == 100) { + Assert.assertEquals(NIL_NODE_ID, + readKey(subCcslUnbounded.findNear(noExistedBB, GT), subCcslUnbounded)); + Assert.assertEquals(NIL_NODE_ID, + readKey(subCcslUnbounded.findNear(noExistedBB, GT | EQ), subCcslUnbounded)); + } else { + Assert.assertEquals(i * 2 + 2, + readKey(subCcslUnbounded.findNear(noExistedBB, GT), subCcslUnbounded)); + Assert.assertEquals(i * 2 + 2, + readKey(subCcslUnbounded.findNear(noExistedBB, GT | EQ), subCcslUnbounded)); + } + } + return nodeList; + } + + private void doRewriteDeletedNode(CompactedConcurrentSkipList ccslOri, + SubCompactedConcurrentSkipList subCcsl1, + SubCompactedConcurrentSkipList subCcsl3) + throws org.apache.hadoop.hbase.regionserver.skiplist.exception.CCSMapException { + long loNodeId; + long hiNodeId; + IIterCCSList subIter; + List nodeIter;//rewrite the data which are deleted. + int level = subCcsl1.generateRandomIndexLevel(); + long node1_2 = subCcsl1 + .getNodeAndWriteMeta(level, NodeUtil.getNodeMetaLenByLevel(level), NodeUtil.getDataLen(8, 8)); + ByteBuffer keyvalueBB = NodeUtil.duplicateNodeDataByteBuffer(subCcsl1, node1_2); + ByteBufferUtils.putInt(keyvalueBB, 0, 8); + ByteBufferUtils.putInt(keyvalueBB, Integer.BYTES, 8); + ByteBufferUtils.putLong(keyvalueBB, 2 * Integer.BYTES, 4); + ByteBufferUtils.putLong(keyvalueBB, 2 * Integer.BYTES + 8, 4); + + subCcsl1.put(node1_2, level); + + long node6_2 = subCcsl1 + .getNodeAndWriteMeta(level, NodeUtil.getNodeMetaLenByLevel(level), NodeUtil.getDataLen(8, 8)); + keyvalueBB = NodeUtil.duplicateNodeDataByteBuffer(subCcsl1, node6_2); + ByteBufferUtils.putInt(keyvalueBB, 0, 8); + ByteBufferUtils.putInt(keyvalueBB, Integer.BYTES, 8); + ByteBufferUtils.putLong(keyvalueBB, 2 * Integer.BYTES, 14); + ByteBufferUtils.putLong(keyvalueBB, 2 * Integer.BYTES + 8, 14); + + subCcsl1.put(node6_2, level); + + Assert.assertEquals(6, subCcsl3.getSize()); + Assert.assertEquals(100, ccslOri.getSize()); + + loNodeId = subCcsl3.loNode(); + hiNodeId = subCcsl3.hiNode(); + System.out.println("loNodeId:" + loNodeId + " => " + readKey(loNodeId, ccslOri)); + System.out.println("hiNodeId:" + hiNodeId + " => " + readKey(hiNodeId, ccslOri)); + Assert.assertEquals(4, readKey(loNodeId, ccslOri)); + Assert.assertEquals(14, readKey(hiNodeId, ccslOri)); + + subIter = subCcsl3.nodeIdIter(); + nodeIter = new ArrayList<>(); + while (subIter.hasNext()) { + long iterNode = subIter.next(); + nodeIter.add(iterNode); + } + + Assert.assertEquals(6, nodeIter.size()); + for (int i = 0; i < nodeIter.size(); i++) { + Assert.assertEquals(4 + i * 2, readKey(nodeIter.get(i), subCcsl3)); + } + + Assert.assertEquals(node1_2, nodeIter.get(0).longValue()); + Assert.assertEquals(node6_2, nodeIter.get(5).longValue()); + } +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/TestCCSMapMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/TestCCSMapMemStore.java new file mode 100644 index 0000000000..334d24e8ba --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/TestCCSMapMemStore.java @@ -0,0 +1,76 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.hbase; + +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparatorImpl; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.regionserver.KeyValueScanner; +import org.apache.hadoop.hbase.regionserver.skiplist.core.AllocatorHandlerRegister; +import org.apache.hadoop.hbase.regionserver.skiplist.core.CCSMapChunkPool; +import org.apache.hadoop.hbase.regionserver.skiplist.core.HeapMode; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(SmallTests.class) +public class TestCCSMapMemStore { + + @ClassRule public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCCSMapMemStore.class); + + private static CCSMapMemStore buildCCSMapMemStore() { + AllocatorHandlerRegister.AllocatorHandlerBuilder builder = + new AllocatorHandlerRegister.AllocatorHandlerBuilder(); + builder.setCapacity(1024 * 1024 * 1024); + builder.setChunkSize(8 * 1024 * 1024); + builder.setInitialCount(Integer.MAX_VALUE); + builder.setHeapMode(HeapMode.offHeap); + CCSMapChunkPool allocatorHandler = new CCSMapChunkPool(builder); + AllocatorHandlerRegister.register(allocatorHandler); + AllocatorHandlerRegister.AllocatorHandlerBuilder.setINSTANCE(builder); + + Configuration conf = HBaseConfiguration.create(); + return new CCSMapMemStore(conf, CellComparatorImpl.COMPARATOR); + } + + @Test + public void testNormal() throws Exception { + CCSMapMemStore ms = buildCCSMapMemStore(); + Cell cell1 = TestCellCCSMap.buildCell(1234, 8, 2345, 1); + ms.add(cell1, null); + List xx = ms.getScanners(2); + int count = 0; + for (KeyValueScanner scanner : xx) { + + Cell cell2 = scanner.next(); + if (cell2 != null) { + Assert.assertEquals(0, CellComparatorImpl.COMPARATOR.compare(cell1, cell2)); + count++; + } + } + Assert.assertEquals(1, count); + + System.out.println(ms.getFlushableSize()); + } +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/TestCellCCSMap.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/TestCellCCSMap.java new file mode 100644 index 0000000000..0b11021960 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/skiplist/hbase/TestCellCCSMap.java @@ -0,0 +1,460 @@ +/* + * 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.hadoop.hbase.regionserver.skiplist.hbase; + +import static org.apache.hadoop.hbase.regionserver.skiplist.hbase.CellCCSMap.CCSMAP_COMPARATOR_KEY; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.SortedMap; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ByteBufferKeyValue; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparatorImpl; +import org.apache.hadoop.hbase.ExtendedCell; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.NoTagsByteBufferKeyValue; +import org.apache.hadoop.hbase.regionserver.skiplist.core.AllocatorHandlerRegister; +import org.apache.hadoop.hbase.regionserver.skiplist.core.CCSMapChunkPool; +import org.apache.hadoop.hbase.regionserver.skiplist.core.HeapMode; +import org.apache.hadoop.hbase.regionserver.skiplist.core.INodeComparator; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.Assert; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(SmallTests.class) +public class TestCellCCSMap { + + @ClassRule public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCellCCSMap.class); + + private Cell keyValue1; + private Cell bbCell1; + private Cell cellBigRow; + private Cell cellSmallRow; + private Cell cellBigF; + private Cell cellSmallF; + private Cell cellBigTs; + private Cell cellSmallTs; + private Cell cellBigSeqId; + private Cell cellSmallSeqId; + private Cell noExistedBigCell; + private Cell noExistedSmallCell; + private Map orderMaps; + private final ByteBuffer oriBB = ByteBuffer.allocate(1024); + private final long seqId = 123456L; + + @Before + public void setUp() throws Exception { + int offset = 3; + keyValue1 = buildCell(1234, 8, 2345, seqId); + int len = ((KeyValue) keyValue1).getSerializedSize(true); + bbCell1 = copyCellTo(keyValue1, oriBB, offset, len); + + } + + private void init(boolean returnByteBufferCell) { + cellBigRow = buildCell(1235, 8, 2345, seqId, returnByteBufferCell); + cellSmallRow = buildCell(1233, 8, 2345, seqId, returnByteBufferCell); + + cellBigF = buildCell(1234, 9, 2345, seqId, returnByteBufferCell); + cellSmallF = buildCell(1234, 7, 2345, seqId, returnByteBufferCell); + + cellBigTs = buildCell(1234, 8, 2346, seqId, returnByteBufferCell); + cellSmallTs = buildCell(1234, 8, 2344, seqId, returnByteBufferCell); + + cellBigSeqId = buildCell(1234, 8, 2345, seqId + 1, returnByteBufferCell); + cellSmallSeqId = buildCell(1234, 8, 2345, seqId - 1, returnByteBufferCell); + + noExistedBigCell = buildCell(2000, 8, 2345, seqId, returnByteBufferCell); + noExistedSmallCell = buildCell(1000, 8, 2345, seqId, returnByteBufferCell); + + //cellSmallRow < cellSmallF < cellBigTs < cellBigSeqId < keyvalue1 < cellSmallSeqId + // < cellSmallTs < cellBigF < cellBigRow + orderMaps = new HashMap<>(); + orderMaps.put(0, cellSmallRow); + orderMaps.put(1, cellSmallF); + orderMaps.put(2, cellBigTs); + orderMaps.put(3, cellBigSeqId); + orderMaps.put(4, keyValue1); + orderMaps.put(5, cellSmallSeqId); + orderMaps.put(6, cellSmallTs); + orderMaps.put(7, cellBigF); + orderMaps.put(8, cellBigRow); + } + + static Cell buildCell(long rowValue, int id, long ts, long seqId) { + return buildCell(rowValue, id, ts, seqId, false); + } + + private static Cell buildCell(long rowValue, int id, long ts, long seqId, + boolean returnByteBufferCell) { + byte[] row = new byte[8]; + Bytes.putLong(row, 0, rowValue); + byte[] fam = Bytes.toBytes("testfamily" + id); + byte[] qf = Bytes.toBytes("testqualifier" + id); + byte[] val = Bytes.toBytes("testval" + id); + Cell keyValue = new KeyValue(row, fam, qf, ts, val); + ((KeyValue) keyValue).setSequenceId(seqId); + + if (returnByteBufferCell) { + ByteBuffer oriBB = ByteBuffer.allocate(1024); + int offset = 1; + int len = ((KeyValue) keyValue).getSerializedSize(true); + return copyCellTo(keyValue, oriBB, offset, len); + } + return keyValue; + } + + @Test + public void testUtilsNotChangePosition() throws Exception { + + ByteBuffer buf = ByteBuffer.allocate(1024); + Assert.assertEquals(0, buf.position()); + Assert.assertEquals(1024, buf.limit()); + Assert.assertEquals(1024, buf.capacity()); + int offset = 2; + System.out.println("==>" + ((KeyValue) keyValue1).getSerializedSize(true)); + System.out.println("==>" + ((KeyValue) keyValue1).getSerializedSize(false)); + System.out.println("==>" + KeyValueUtil.appendTo(keyValue1, buf, offset, true)); + Assert.assertEquals(0, buf.position()); + Assert.assertEquals(1024, buf.limit()); + Assert.assertEquals(1024, buf.capacity()); + + ByteBuffer buf2 = ByteBuffer.allocate(1024); + Assert.assertEquals(0, buf2.position()); + Assert.assertEquals(1024, buf2.limit()); + Assert.assertEquals(1024, buf2.capacity()); + offset = 0; + System.out.println("==>" + ((KeyValue) keyValue1).getSerializedSize(true)); + System.out.println("==>" + ((KeyValue) keyValue1).getSerializedSize(false)); + ((ExtendedCell) keyValue1).write(buf2, offset); + Assert.assertEquals(0, buf2.position()); + Assert.assertEquals(1024, buf2.limit()); + Assert.assertEquals(1024, buf2.capacity()); + + } + + @Test + public void testSerde() throws Exception { + init(false); + doTestSerdeAndComparator(keyValue1, CCSMapCellComparatorDefault.INSTNANCE); + doTestSerdeAndComparator(bbCell1, CCSMapCellComparatorDefault.INSTNANCE); + + doTestSerdeAndComparator(keyValue1, CCSMapCellComparatorDirectly.INSTNANCE); + doTestSerdeAndComparator(bbCell1, CCSMapCellComparatorDirectly.INSTNANCE); + + init(true); + doTestSerdeAndComparator(keyValue1, CCSMapCellComparatorDefault.INSTNANCE); + doTestSerdeAndComparator(bbCell1, CCSMapCellComparatorDefault.INSTNANCE); + + doTestSerdeAndComparator(keyValue1, CCSMapCellComparatorDirectly.INSTNANCE); + doTestSerdeAndComparator(bbCell1, CCSMapCellComparatorDirectly.INSTNANCE); + } + + private void doTestSerdeAndComparator(Cell cell, INodeComparator comparator) + throws Exception { + CellSerde cellSerde = CellSerde.INSTANCE; + + Assert.assertEquals(69, cellSerde.getSerializedSize(cell)); + + ByteBuffer dstBB = ByteBuffer.allocate(2048); + int offset = 123; + + cellSerde.serialize(keyValue1, dstBB, offset, cellSerde.getSerializedSize(cell)); + + Cell deserializeCell = cellSerde.deserialize(dstBB, offset, cellSerde.getSerializedSize(cell)); + + Assert.assertEquals(seqId, deserializeCell.getSequenceId()); + Assert.assertEquals(seqId, cell.getSequenceId()); + Assert.assertEquals(61, ((ExtendedCell) cell).getSerializedSize(true)); + Assert.assertEquals(61, ((ExtendedCell) deserializeCell).getSerializedSize(true)); + Assert.assertEquals(0, CellComparatorImpl.COMPARATOR.compare(cell, deserializeCell)); + + Assert + .assertEquals(comparator.compareTo(dstBB, offset, cellSerde.getSerializedSize(cell), cell), + 0); + + Assert + .assertEquals(comparator.compareTo(cell, dstBB, offset, cellSerde.getSerializedSize(cell)), + 0); + + //Comare + Assert.assertTrue( + comparator.compareTo(cellBigRow, dstBB, offset, cellSerde.getSerializedSize(cell)) > 0); + Assert.assertTrue( + comparator.compareTo(dstBB, offset, cellSerde.getSerializedSize(cell), cellBigRow) < 0); + + Assert.assertTrue( + comparator.compareTo(cellSmallRow, dstBB, offset, cellSerde.getSerializedSize(cell)) < 0); + Assert.assertTrue( + comparator.compareTo(dstBB, offset, cellSerde.getSerializedSize(cell), cellSmallRow) > 0); + + Assert.assertTrue( + comparator.compareTo(cellBigF, dstBB, offset, cellSerde.getSerializedSize(cell)) > 0); + Assert.assertTrue( + comparator.compareTo(dstBB, offset, cellSerde.getSerializedSize(cell), cellBigF) < 0); + + Assert.assertTrue( + comparator.compareTo(cellSmallF, dstBB, offset, cellSerde.getSerializedSize(cell)) < 0); + Assert.assertTrue( + comparator.compareTo(dstBB, offset, cellSerde.getSerializedSize(cell), cellSmallF) > 0); + + Assert.assertTrue(comparator.compareTo(cellBigTs, cell) < 0); + + Assert.assertTrue( + comparator.compareTo(cellBigTs, dstBB, offset, cellSerde.getSerializedSize(cell)) < 0); + Assert.assertTrue( + comparator.compareTo(dstBB, offset, cellSerde.getSerializedSize(cell), cellBigTs) > 0); + + Assert.assertTrue(comparator.compareTo(cellSmallTs, cell) > 0); + + Assert.assertTrue( + comparator.compareTo(cellSmallTs, dstBB, offset, cellSerde.getSerializedSize(cell)) > 0); + Assert.assertTrue( + comparator.compareTo(dstBB, offset, cellSerde.getSerializedSize(cell), cellSmallTs) < 0); + + Assert.assertTrue(comparator.compareTo(cellBigSeqId, cell) < 0); + + Assert.assertTrue( + comparator.compareTo(cellBigSeqId, dstBB, offset, cellSerde.getSerializedSize(cell)) < 0); + Assert.assertTrue( + comparator.compareTo(dstBB, offset, cellSerde.getSerializedSize(cell), cellBigSeqId) > 0); + + Assert.assertTrue(comparator.compareTo(cellSmallSeqId, cell) > 0); + + Assert.assertTrue( + comparator.compareTo(cellSmallSeqId, dstBB, offset, cellSerde.getSerializedSize(cell)) > 0); + Assert.assertTrue( + comparator.compareTo(dstBB, offset, cellSerde.getSerializedSize(cell), cellSmallSeqId) < 0); + } + + @Test + public void testCellCCSMap() throws Exception { + init(false); + Configuration conf = HBaseConfiguration.create(); + conf.set(CCSMAP_COMPARATOR_KEY, "useDefault"); + doTestCellCCSMap(conf); + + init(true); + conf = HBaseConfiguration.create(); + conf.set(CCSMAP_COMPARATOR_KEY, "useDefault"); + doTestCellCCSMap(conf); + + init(true); + conf = HBaseConfiguration.create(); + conf.set(CCSMAP_COMPARATOR_KEY, "directly"); + doTestCellCCSMap(conf); + + init(false); + conf = HBaseConfiguration.create(); + conf.set(CCSMAP_COMPARATOR_KEY, "directly"); + doTestCellCCSMap(conf); + } + + private void doTestCellCCSMap(Configuration conf) { + AllocatorHandlerRegister.AllocatorHandlerBuilder builder = + new AllocatorHandlerRegister.AllocatorHandlerBuilder(); + builder.setCapacity(8 * 1024 * 1024); + builder.setChunkSize(4 * 1024); + builder.setInitialCount(Integer.MAX_VALUE); + builder.setHeapMode(HeapMode.offHeap); + CCSMapChunkPool allocatorHandler = new CCSMapChunkPool(builder); + AllocatorHandlerRegister.register(allocatorHandler); + AllocatorHandlerRegister.AllocatorHandlerBuilder.setINSTANCE(builder); + + CellCCSMap cellCCSMap = CellCCSMap.buildCellCCSMap(conf, CellComparatorImpl.COMPARATOR, false); + + cellCCSMap.put(keyValue1, keyValue1); + Assert.assertEquals(1, cellCCSMap.size()); + Cell getCell = cellCCSMap.get(keyValue1); + Assert.assertEquals(0, CellComparatorImpl.COMPARATOR.compare(getCell, keyValue1)); + + //rewrite the Cell + cellCCSMap.put(bbCell1, bbCell1); + Assert.assertEquals(1, cellCCSMap.size()); + getCell = cellCCSMap.get(keyValue1); + Assert.assertEquals(0, CellComparatorImpl.COMPARATOR.compare(getCell, keyValue1)); + + //put the cellSmallRow + doPutNoRepicaCellPutAndGet(cellCCSMap, cellSmallRow, 1); + doPutNoRepicaCellPutAndGet(cellCCSMap, cellBigRow, 2); + doPutNoRepicaCellPutAndGet(cellCCSMap, cellSmallF, 3); + doPutNoRepicaCellPutAndGet(cellCCSMap, cellBigF, 4); + doPutNoRepicaCellPutAndGet(cellCCSMap, cellSmallTs, 5); + doPutNoRepicaCellPutAndGet(cellCCSMap, cellBigTs, 6); + doPutNoRepicaCellPutAndGet(cellCCSMap, cellSmallSeqId, 7); + doPutNoRepicaCellPutAndGet(cellCCSMap, cellBigSeqId, 8); + + Cell firstCell = cellCCSMap.firstKey(); + Assert.assertEquals(0, CellComparatorImpl.COMPARATOR.compare(firstCell, cellSmallRow)); + + Cell lastCell = cellCCSMap.lastKey(); + Assert.assertEquals(0, CellComparatorImpl.COMPARATOR.compare(lastCell, cellBigRow)); + + List sortList = new ArrayList<>(); + sortList.add(cellSmallRow); + sortList.add(cellBigRow); + sortList.add(cellSmallF); + sortList.add(cellBigF); + sortList.add(cellSmallTs); + sortList.add(cellBigTs); + sortList.add(cellSmallSeqId); + sortList.add(cellBigSeqId); + sortList.add(keyValue1); + Collections.sort(sortList, CellComparatorImpl.COMPARATOR); + + //full scan the map + Iterator> entryIterator = cellCCSMap.entrySet().iterator(); + int i = 0; + while (entryIterator.hasNext()) { + Cell cell = entryIterator.next().getKey(); + Assert.assertEquals(0, CellComparatorImpl.COMPARATOR.compare(cell, sortList.get(i))); + Assert.assertEquals(0, CellComparatorImpl.COMPARATOR.compare(cell, orderMaps.get(i))); + + i++; + } + Assert.assertEquals(9, i); + + //null tail + SortedMap nullTailMap = cellCCSMap.tailMap(noExistedBigCell); + Assert.assertEquals(0, nullTailMap.size()); + Assert.assertEquals(null, nullTailMap.firstKey()); + Assert.assertEquals(null, nullTailMap.lastKey()); + + Iterator> nullTailMapEntries = nullTailMap.entrySet().iterator(); + int zz = 0; + while (nullTailMapEntries.hasNext()) { + nullTailMapEntries.next().getKey(); + zz++; + } + Assert.assertEquals(0, zz); + + //null head + SortedMap nullHeadMap = cellCCSMap.headMap(noExistedSmallCell); + Assert.assertEquals(0, nullHeadMap.size()); + Assert.assertEquals(null, nullHeadMap.firstKey()); + Assert.assertEquals(null, nullHeadMap.lastKey()); + + Iterator> nullHeadMapEntries = nullHeadMap.entrySet().iterator(); + int zzz = 0; + while (nullHeadMapEntries.hasNext()) { + nullHeadMapEntries.next().getKey(); + zzz++; + } + Assert.assertEquals(0, zzz); + + //Full scan to verify the tailMap + for (int j = 0; j < sortList.size(); j++) { + //headMap inclusive is false + Map tailMap = cellCCSMap.tailMap(sortList.get(j)); + Assert.assertEquals(tailMap.size(), sortList.size() - j); + + for (int m = 0; m < sortList.size(); m++) { + Cell getCellTailMap = tailMap.get(sortList.get(m)); + if (m < (sortList.size() - tailMap.size())) { + Assert.assertEquals(null, getCellTailMap); + } else { + Assert.assertEquals(0, + CellComparatorImpl.COMPARATOR.compare(getCellTailMap, sortList.get(m))); + } + } + + Iterator> entrys = tailMap.entrySet().iterator(); + int z = 0; + while (entrys.hasNext()) { + Cell cell = entrys.next().getKey(); + Assert.assertEquals(0, CellComparatorImpl.COMPARATOR.compare(cell, sortList.get(z + j))); + Assert.assertEquals(0, CellComparatorImpl.COMPARATOR.compare(cell, orderMaps.get(z + j))); + z++; + } + Assert.assertEquals(z, sortList.size() - j); + } + + //Full scan to verify the heapMap + for (int j = 0; j < sortList.size(); j++) { + //headMap inclusive is false + Map headMap = cellCCSMap.headMap(sortList.get(j)); + Assert.assertEquals(j, headMap.size()); + + for (int m = 0; m < sortList.size(); m++) { + Cell getCellHeadMap = headMap.get(sortList.get(m)); + if (m < headMap.size()) { + Assert.assertEquals(0, + CellComparatorImpl.COMPARATOR.compare(getCellHeadMap, sortList.get(m))); + } else { + Assert.assertEquals(null, getCellHeadMap); + } + } + + Iterator> entrys = headMap.entrySet().iterator(); + int z = 0; + while (entrys.hasNext()) { + Cell cell = entrys.next().getKey(); + Assert.assertEquals(0, CellComparatorImpl.COMPARATOR.compare(cell, sortList.get(z))); + Assert.assertEquals(0, CellComparatorImpl.COMPARATOR.compare(cell, orderMaps.get(z))); + z++; + } + Assert.assertEquals(z, j); + } + + } + + private void doPutNoRepicaCellPutAndGet(CellCCSMap cellCCSMap, Cell cell, int lastSize) { + cellCCSMap.put(cell, cell); + Assert.assertEquals(lastSize + 1, cellCCSMap.size()); + + Cell getCell = cellCCSMap.get(cell); + Assert.assertEquals(0, CellComparatorImpl.COMPARATOR.compare(getCell, cell)); + } + + /** + * Clone the passed cell by copying its data into the passed buf. + */ + private static Cell copyCellTo(Cell cell, ByteBuffer buf, int offset, int len) { + int tagsLen = cell.getTagsLength(); + if (cell instanceof ExtendedCell) { + ((ExtendedCell) cell).write(buf, offset); + } else { + // Normally all Cell impls within Server will be of type ExtendedCell. Just considering the + // other case also. The data fragments within Cell is copied into buf as in KeyValue + // serialization format only. + KeyValueUtil.appendTo(cell, buf, offset, true); + } + if (tagsLen == 0) { + // When tagsLen is 0, make a NoTagsByteBufferKeyValue version. This is an optimized class + // which directly return tagsLen as 0. So we avoid parsing many length components in + // reading the tagLength stored in the backing buffer. The Memstore addition of every Cell + // call getTagsLength(). + return new NoTagsByteBufferKeyValue(buf, offset, len, cell.getSequenceId()); + } else { + return new ByteBufferKeyValue(buf, offset, len, cell.getSequenceId()); + } + } +} \ No newline at end of file -- 2.15.1 (Apple Git-101)