From 8b4bfa5060589a6dda70bed3cd245fb0e686135f Mon Sep 17 00:00:00 2001 From: eshcar Date: Tue, 3 May 2016 16:35:39 +0300 Subject: [PATCH] HBASE-14920: Compacting memstore --- .../org/apache/hadoop/hbase/HColumnDescriptor.java | 49 +- .../hbase/test/IntegrationTestBigLinkedList.java | 4 +- .../hbase/regionserver/AbstractMemStore.java | 31 +- .../hbase/regionserver/CompactingMemStore.java | 409 +++++++++++ .../hbase/regionserver/CompactionPipeline.java | 190 +++++ .../hadoop/hbase/regionserver/DefaultMemStore.java | 22 +- .../regionserver/FlushAllLargeStoresPolicy.java | 75 ++ .../hbase/regionserver/FlushLargeStoresPolicy.java | 54 +- .../FlushNonSloppyStoresFirstPolicy.java | 66 ++ .../hbase/regionserver/FlushPolicyFactory.java | 2 +- .../hadoop/hbase/regionserver/HMobStore.java | 4 + .../apache/hadoop/hbase/regionserver/HRegion.java | 35 +- .../apache/hadoop/hbase/regionserver/HStore.java | 17 +- .../hbase/regionserver/ImmutableSegment.java | 4 - .../apache/hadoop/hbase/regionserver/MemStore.java | 14 +- .../hbase/regionserver/MemStoreCompactor.java | 197 ++++++ .../regionserver/RegionServicesForStores.java | 49 +- .../apache/hadoop/hbase/regionserver/Segment.java | 44 +- .../hadoop/hbase/regionserver/SegmentFactory.java | 6 +- .../apache/hadoop/hbase/regionserver/Store.java | 1 + .../hbase/regionserver/VersionedSegmentsList.java | 54 ++ .../hbase/regionserver/wal/AbstractFSWAL.java | 19 +- .../regionserver/wal/SequenceIdAccounting.java | 35 +- .../hadoop/hbase/wal/DisabledWALProvider.java | 8 +- .../main/java/org/apache/hadoop/hbase/wal/WAL.java | 15 +- .../apache/hadoop/hbase/HBaseTestingUtility.java | 51 +- .../org/apache/hadoop/hbase/TestIOFencing.java | 5 + .../org/apache/hadoop/hbase/io/TestHeapSize.java | 12 +- .../hbase/regionserver/TestCompactingMemStore.java | 729 ++++++++++++++++++++ .../hbase/regionserver/TestDefaultMemStore.java | 159 +++-- .../hadoop/hbase/regionserver/TestHRegion.java | 60 +- .../regionserver/TestHRegionWithInMemoryFlush.java | 61 ++ .../regionserver/TestPerColumnFamilyFlush.java | 29 +- .../TestWalAndCompactingMemStoreFlush.java | 566 +++++++++++++++ hbase-shell/src/main/ruby/hbase.rb | 1 + hbase-shell/src/main/ruby/hbase/admin.rb | 1 + 36 files changed, 2826 insertions(+), 252 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionPipeline.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushAllLargeStoresPolicy.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushNonSloppyStoresFirstPolicy.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactor.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/VersionedSegmentsList.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionWithInMemoryFlush.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java index 3c16f4e..1e7df5a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java @@ -18,6 +18,8 @@ */ package org.apache.hadoop.hbase; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; import java.io.IOException; import java.util.Collections; import java.util.HashMap; @@ -38,7 +40,6 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.PrettyPrinter; import org.apache.hadoop.hbase.util.PrettyPrinter.Unit; -import com.google.common.base.Preconditions; /** * An HColumnDescriptor contains information about a column family such as the @@ -62,6 +63,8 @@ public class HColumnDescriptor implements Comparable { // Version 11 -- add column family level configuration. private static final byte COLUMN_DESCRIPTOR_VERSION = (byte) 11; + private static final String IN_MEMORY_COMPACTION = "IN_MEMORY_COMPACTION"; + // These constants are used as FileInfo keys public static final String COMPRESSION = "COMPRESSION"; public static final String COMPRESSION_COMPACT = "COMPRESSION_COMPACT"; @@ -151,7 +154,7 @@ public class HColumnDescriptor implements Comparable { * Default number of versions of a record to keep. */ public static final int DEFAULT_VERSIONS = HBaseConfiguration.create().getInt( - "hbase.column.max.version", 1); + "hbase.column.max.version", 1); /** * Default is not to keep a minimum of versions. @@ -170,6 +173,11 @@ public class HColumnDescriptor implements Comparable { public static final boolean DEFAULT_IN_MEMORY = false; /** + * Default setting for whether to set the memstore of this column family as compacting or not. + */ + public static final boolean DEFAULT_IN_MEMORY_COMPACTION = false; + + /** * Default setting for preventing deleted from being collected immediately. */ public static final KeepDeletedCells DEFAULT_KEEP_DELETED = KeepDeletedCells.FALSE; @@ -254,6 +262,8 @@ public class HColumnDescriptor implements Comparable { DEFAULT_VALUES.put(TTL, String.valueOf(DEFAULT_TTL)); DEFAULT_VALUES.put(BLOCKSIZE, String.valueOf(DEFAULT_BLOCKSIZE)); DEFAULT_VALUES.put(HConstants.IN_MEMORY, String.valueOf(DEFAULT_IN_MEMORY)); + DEFAULT_VALUES.put(IN_MEMORY_COMPACTION, String.valueOf( + DEFAULT_IN_MEMORY_COMPACTION)); DEFAULT_VALUES.put(BLOCKCACHE, String.valueOf(DEFAULT_BLOCKCACHE)); DEFAULT_VALUES.put(KEEP_DELETED_CELLS, String.valueOf(DEFAULT_KEEP_DELETED)); DEFAULT_VALUES.put(DATA_BLOCK_ENCODING, String.valueOf(DEFAULT_DATA_BLOCK_ENCODING)); @@ -319,6 +329,7 @@ public class HColumnDescriptor implements Comparable { setMinVersions(DEFAULT_MIN_VERSIONS); setKeepDeletedCells(DEFAULT_KEEP_DELETED); setInMemory(DEFAULT_IN_MEMORY); + setInMemoryCompaction(DEFAULT_IN_MEMORY_COMPACTION); setBlockCacheEnabled(DEFAULT_BLOCKCACHE); setTimeToLive(DEFAULT_TTL); setCompressionType(Compression.Algorithm.valueOf(DEFAULT_COMPRESSION.toUpperCase())); @@ -676,6 +687,40 @@ public class HColumnDescriptor implements Comparable { return setValue(HConstants.IN_MEMORY, Boolean.toString(inMemory)); } + /** + * @return True if we prefer to keep the in-memory data compacted + * for this column family + */ + public boolean isInMemoryCompaction() { + String value = getValue(IN_MEMORY_COMPACTION); + if (value != null) { + return Boolean.parseBoolean(value); + } + return DEFAULT_IN_MEMORY_COMPACTION; + } + + /** + * @param inMemoryCompaction True if we prefer to keep the in-memory data compacted + * for this column family + * @return this (for chained invocation) + */ + public HColumnDescriptor setInMemoryCompaction(boolean inMemoryCompaction) { + return setValue(IN_MEMORY_COMPACTION, Boolean.toString(inMemoryCompaction)); + } + + /** + * @param className the name of the class to be used as a memstore + * @return this (for chained invocation) + */ + @VisibleForTesting + public HColumnDescriptor setMemStoreClass(String className) { + + if (className.equalsIgnoreCase("org.apache.hadoop.hbase.regionserver.CompactingMemStore")) { + return setValue(IN_MEMORY_COMPACTION, Boolean.toString(true)); + } + else return setValue(IN_MEMORY_COMPACTION, Boolean.toString(false)); + } + public KeepDeletedCells getKeepDeletedCells() { String value = getValue(KEEP_DELETED_CELLS); if (value != null) { diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java index c864580..430c8a6 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java @@ -80,7 +80,7 @@ import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil; import org.apache.hadoop.hbase.mapreduce.TableMapper; import org.apache.hadoop.hbase.mapreduce.TableRecordReaderImpl; import org.apache.hadoop.hbase.mapreduce.WALPlayer; -import org.apache.hadoop.hbase.regionserver.FlushLargeStoresPolicy; +import org.apache.hadoop.hbase.regionserver.FlushAllLargeStoresPolicy; import org.apache.hadoop.hbase.regionserver.FlushPolicyFactory; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.testclassification.IntegrationTests; @@ -1586,7 +1586,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { Configuration conf = getTestingUtil(getConf()).getConfiguration(); if (isMultiUnevenColumnFamilies(getConf())) { // make sure per CF flush is on - conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushLargeStoresPolicy.class.getName()); + conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushAllLargeStoresPolicy.class.getName()); } int ret = ToolRunner.run(conf, new Loop(), new String[] { "1", "1", "2000000", 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 c3724fc..60bd751 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 @@ -64,7 +64,7 @@ public abstract class AbstractMemStore implements MemStore { (2 * Bytes.SIZEOF_LONG)); public final static long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD + - 2 * (ClassSize.ATOMIC_LONG + ClassSize.TIMERANGE_TRACKER + + (ClassSize.ATOMIC_LONG + ClassSize.TIMERANGE_TRACKER + ClassSize.CELL_SKIPLIST_SET + ClassSize.CONCURRENT_SKIPLISTMAP)); @@ -162,17 +162,9 @@ public abstract class AbstractMemStore implements MemStore { } /** - * An override on snapshot so the no arg version of the method implies zero seq num, - * like for cases without wal - */ - public MemStoreSnapshot snapshot() { - return snapshot(0); - } - - /** * The passed snapshot was successfully persisted; it can be let go. * @param id Id of the snapshot to clean out. - * @see MemStore#snapshot(long) + * @see MemStore#snapshot() */ @Override public void clearSnapshot(long id) throws UnexpectedStateException { @@ -201,18 +193,6 @@ public abstract class AbstractMemStore implements MemStore { } /** - * On flush, how much memory we will clear from the active cell set. - * - * @return size of data that is going to be flushed from active set - */ - @Override - public long getFlushableSize() { - long snapshotSize = getSnapshot().getSize(); - return snapshotSize > 0 ? snapshotSize : keySize(); - } - - - /** * @return a list containing a single memstore scanner. */ @Override @@ -230,7 +210,7 @@ public abstract class AbstractMemStore implements MemStore { StringBuffer buf = new StringBuffer(); int i = 1; try { - for (Segment segment : getListOfSegments()) { + for (Segment segment : getSegments()) { buf.append("Segment (" + i + ") " + segment.toString() + "; "); i++; } @@ -471,9 +451,6 @@ public abstract class AbstractMemStore implements MemStore { * Returns an ordered list of segments from most recent to oldest in memstore * @return an ordered list of segments from most recent to oldest in memstore */ - protected abstract List getListOfSegments() throws IOException; + protected abstract List getSegments() throws IOException; - public long getActiveSize() { - return getActive().getSize(); - } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java new file mode 100644 index 0000000..183ad71 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java @@ -0,0 +1,409 @@ +/** + * + * 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; + +import com.google.common.annotations.VisibleForTesting; +import java.io.IOException; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.ClassSize; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.wal.WAL; + +/** + * A memstore implementation which supports in-memory compaction. + * A compaction pipeline is added between the active set and the snapshot data structures; + * it consists of a list of kv-sets that are subject to compaction. + * Like the snapshot, all pipeline components are read-only; updates only affect the active set. + * To ensure this property we take advantage of the existing blocking mechanism -- the active set + * is pushed to the pipeline while holding the region's updatesLock in exclusive mode. + * Periodically, a compaction is applied in the background to all pipeline components resulting + * in a single read-only component. The ``old'' components are discarded when no scanner is reading + * them. + */ +@InterfaceAudience.Private +public class CompactingMemStore extends AbstractMemStore { + public final static long DEEP_OVERHEAD_PER_PIPELINE_ITEM = ClassSize.align( + ClassSize.TIMERANGE_TRACKER + ClassSize.TIMERANGE + + ClassSize.CELL_SKIPLIST_SET + ClassSize.CONCURRENT_SKIPLISTMAP); + // Default fraction of in-memory-flush size w.r.t. flush-to-disk size + public static final String IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY = + "hbase.memestore.inmemoryflush.threshold.factor"; + private static final double IN_MEMORY_FLUSH_THRESHOLD_FACTOR_DEFAULT = 0.25; + + private static final Log LOG = LogFactory.getLog(CompactingMemStore.class); + private Store store; + private RegionServicesForStores regionServices; + private CompactionPipeline pipeline; + private MemStoreCompactor compactor; + // the threshold on active size for in-memory flush + private long inmemoryFlushSize; + private final AtomicBoolean inMemoryFlushInProgress = new AtomicBoolean(false); + @VisibleForTesting + private final AtomicBoolean allowCompaction = new AtomicBoolean(true); + + public CompactingMemStore(Configuration conf, CellComparator c, + HStore store, RegionServicesForStores regionServices) throws IOException { + super(conf, c); + this.store = store; + this.regionServices = regionServices; + this.pipeline = new CompactionPipeline(getRegionServices()); + this.compactor = new MemStoreCompactor(this); + initInmemoryFlushSize(conf); + } + + private void initInmemoryFlushSize(Configuration conf) { + long memstoreFlushSize = getRegionServices().getMemstoreFlushSize(); + int numStores = getRegionServices().getNumStores(); + if (numStores <= 1) { + // Family number might also be zero in some of our unit test case + numStores = 1; + } + inmemoryFlushSize = memstoreFlushSize / numStores; + // multiply by a factor + double factor = conf.getDouble(IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, + IN_MEMORY_FLUSH_THRESHOLD_FACTOR_DEFAULT); + inmemoryFlushSize *= factor; + LOG.debug("Setting in-memory flush size threshold to " + inmemoryFlushSize); + } + + public static long getSegmentSize(Segment segment) { + return segment.getSize() - DEEP_OVERHEAD_PER_PIPELINE_ITEM; + } + + public static long getSegmentsSize(List list) { + long res = 0; + for (Segment segment : list) { + res += getSegmentSize(segment); + } + return res; + } + + /** + * @return Total memory occupied by this MemStore. + * This is not thread safe and the memstore may be changed while computing its size. + * It is the responsibility of the caller to make sure this doesn't happen. + */ + @Override + public long size() { + long res = 0; + for (Segment item : getSegments()) { + res += item.getSize(); + } + return res; + } + + /** + * This method is called when it is clear that the flush to disk is completed. + * The store may do any post-flush actions at this point. + * One example is to update the wal with sequence number that is known only at the store level. + */ + @Override public void finalizeFlush() { + updateLowestUnflushedSequenceIdInWal(false); + } + + @Override public boolean isSloppy() { + return true; + } + + /** + * Push the current active memstore segment into the pipeline + * and create a snapshot of the tail of current compaction pipeline + * Snapshot must be cleared by call to {@link #clearSnapshot}. + * {@link #clearSnapshot(long)}. + * @return {@link MemStoreSnapshot} + */ + @Override + public MemStoreSnapshot snapshot() { + MutableSegment active = getActive(); + // If snapshot currently has entries, then flusher failed or didn't call + // cleanup. Log a warning. + if (!getSnapshot().isEmpty()) { + LOG.warn("Snapshot called again without clearing previous. " + + "Doing nothing. Another ongoing flush or did we fail last attempt?"); + } else { + LOG.info("FLUSHING TO DISK: region "+ getRegionServices().getRegionInfo() + .getRegionNameAsString() + "store: "+ getFamilyName()); + stopCompaction(); + pushActiveToPipeline(active); + snapshotId = EnvironmentEdgeManager.currentTime(); + pushTailToSnapshot(); + } + return new MemStoreSnapshot(snapshotId, getSnapshot()); + } + + /** + * On flush, how much memory we will clear. + * @return size of data that is going to be flushed + */ + @Override public long getFlushableSize() { + long snapshotSize = getSnapshot().getSize(); + if(snapshotSize == 0) { + //if snapshot is empty the tail of the pipeline is flushed + snapshotSize = pipeline.getTailSize(); + } + return snapshotSize > 0 ? snapshotSize : keySize(); + } + + @Override + public void updateLowestUnflushedSequenceIdInWal(boolean onlyIfGreater) { + long minSequenceId = pipeline.getMinSequenceId(); + if(minSequenceId != Long.MAX_VALUE) { + byte[] encodedRegionName = getRegionServices().getRegionInfo().getEncodedNameAsBytes(); + byte[] familyName = getFamilyNameInByte(); + WAL wal = getRegionServices().getWAL(); + if (wal != null) { + wal.updateStore(encodedRegionName, familyName, minSequenceId, onlyIfGreater); + } + } + } + + @Override + public List getSegments() { + List pipelineList = pipeline.getSegments(); + List list = new LinkedList(); + list.add(getActive()); + list.addAll(pipelineList); + list.add(getSnapshot()); + return list; + } + + public void setInMemoryFlushInProgress(boolean inMemoryFlushInProgress) { + this.inMemoryFlushInProgress.set(inMemoryFlushInProgress); + } + + public void swapCompactedSegments(VersionedSegmentsList versionedList, ImmutableSegment result) { + pipeline.swap(versionedList, result); + } + + public boolean hasCompactibleSegments() { + return !pipeline.isEmpty(); + } + + public VersionedSegmentsList getCompactibleSegments() { + return pipeline.getVersionedList(); + } + + public long getSmallestReadPoint() { + return store.getSmallestReadPoint(); + } + + public Store getStore() { + return store; + } + + public String getFamilyName() { + return Bytes.toString(getFamilyNameInByte()); + } + + @Override + protected List getListOfScanners(long readPt) throws IOException { + List pipelineList = pipeline.getSegments(); + LinkedList list = new LinkedList(); + list.add(getActive().getSegmentScanner(readPt)); + for (Segment item : pipelineList) { + list.add(item.getSegmentScanner(readPt)); + } + list.add(getSnapshot().getSegmentScanner + (readPt)); + // set sequence ids by decsending order + Iterator iterator = list.descendingIterator(); + int seqId = 0; + while (iterator.hasNext()) { + iterator.next().setSequenceID(seqId); + seqId++; + } + return list; + } + + /** + * Check whether anything need to be done based on the current active set size. + * The method is invoked upon every addition to the active set. + * For CompactingMemStore, flush the active set to the read-only memory if it's + * size is above threshold + */ + @Override + protected void checkActiveSize() { + if (shouldFlushInMemory()) { + /* The thread is dispatched to flush-in-memory. This cannot be done + * on the same thread, because for flush-in-memory we require updatesLock + * in exclusive mode while this method (checkActiveSize) is invoked holding updatesLock + * in the shared mode. */ + InMemoryFlushRunnable runnable = new InMemoryFlushRunnable(); + LOG.info("Dispatching the MemStore in-memory flush for store " + store.getColumnFamilyName()); + getPool().execute(runnable); + // guard against queuing same old compactions over and over again + inMemoryFlushInProgress.set(true); + } + } + + // internally used method, externally visible only for tests + // when invoked directly from tests it must be verified that the caller doesn't hold updatesLock, + // otherwise there is a deadlock + @VisibleForTesting + void flushInMemory() throws IOException { + // Phase I: Update the pipeline + getRegionServices().blockUpdates(); + try { + MutableSegment active = getActive(); + LOG.info("IN-MEMORY FLUSH: Pushing active segment into compaction pipeline, " + + "and initiating compaction."); + pushActiveToPipeline(active); + } finally { + getRegionServices().unblockUpdates(); + } + // Phase II: Compact the pipeline + try { + if (allowCompaction.get()) { + // setting the inMemoryFlushInProgress flag again for the case this method is invoked + // directly (only in tests) in the common path setting from true to true is idempotent + inMemoryFlushInProgress.set(true); + // Speculative compaction execution, may be interrupted if flush is forced while + // compaction is in progress + compactor.startCompaction(); + } + } catch (IOException e) { + LOG.warn("Unable to run memstore compaction. region " + + getRegionServices().getRegionInfo().getRegionNameAsString() + + "store: "+ getFamilyName(), e); + } + } + + private byte[] getFamilyNameInByte() { + return store.getFamily().getName(); + } + + private ThreadPoolExecutor getPool() { + return getRegionServices().getInMemoryCompactionPool(); + } + + private boolean shouldFlushInMemory() { + if(getActive().getSize() > inmemoryFlushSize) { + // size above flush threshold + return (allowCompaction.get() && !inMemoryFlushInProgress.get()); + } + return false; + } + + /** + * The request to cancel the compaction asynchronous task (caused by in-memory flush) + * The compaction may still happen if the request was sent too late + * Non-blocking request + */ + private void stopCompaction() { + if (inMemoryFlushInProgress.get()) { + compactor.stopCompact(); + inMemoryFlushInProgress.set(false); + } + } + + private void pushActiveToPipeline(MutableSegment active) { + if (!active.isEmpty()) { + long delta = DEEP_OVERHEAD_PER_PIPELINE_ITEM - DEEP_OVERHEAD; + active.setSize(active.getSize() + delta); + pipeline.pushHead(active); + resetCellSet(); + } + } + + private void pushTailToSnapshot() { + ImmutableSegment tail = pipeline.pullTail(); + if (!tail.isEmpty()) { + setSnapshot(tail); + long size = getSegmentSize(tail); + setSnapshotSize(size); + } + } + + private RegionServicesForStores getRegionServices() { + return regionServices; + } + + /** + * The in-memory-flusher thread performs the flush asynchronously. + * There is at most one thread per memstore instance. + * It takes the updatesLock exclusively, pushes active into the pipeline, releases updatesLock + * and compacts the pipeline. + */ + private class InMemoryFlushRunnable implements Runnable { + + @Override public void run() { + try { + flushInMemory(); + } catch (IOException e) { + LOG.warn("Unable to run memstore compaction. region " + + getRegionServices().getRegionInfo().getRegionNameAsString() + + "store: "+ getFamilyName(), e); + } + } + } + + //---------------------------------------------------------------------- + //methods for tests + //---------------------------------------------------------------------- + boolean isMemStoreFlushingInMemory() { + return inMemoryFlushInProgress.get(); + } + + void disableCompaction() { + allowCompaction.set(false); + } + + void enableCompaction() { + allowCompaction.set(true); + } + + /** + * @param cell Find the row that comes after this one. If null, we return the + * first. + * @return Next row or null if none found. + */ + Cell getNextRow(final Cell cell) { + Cell lowest = null; + List segments = getSegments(); + for (Segment segment : segments) { + if (lowest == null) { + lowest = getNextRow(cell, segment.getCellSet()); + } else { + lowest = getLowest(lowest, getNextRow(cell, segment.getCellSet())); + } + } + return lowest; + } + + // debug method + private void debug() { + String msg = "active size="+getActive().getSize(); + msg += " threshold="+IN_MEMORY_FLUSH_THRESHOLD_FACTOR_DEFAULT* inmemoryFlushSize; + msg += " allow compaction is "+ (allowCompaction.get() ? "true" : "false"); + msg += " inMemoryFlushInProgress is "+ (inMemoryFlushInProgress.get() ? "true" : "false"); + LOG.debug(msg); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionPipeline.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionPipeline.java new file mode 100644 index 0000000..e33ceae --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionPipeline.java @@ -0,0 +1,190 @@ +/** + * + * 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; + +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * The compaction pipeline of a {@link CompactingMemStore}, is a FIFO queue of segments. + * It supports pushing a segment at the head of the pipeline and pulling a segment from the + * tail to flush to disk. + * It also supports swap operation to allow the compactor swap a subset of the segments with a new + * (compacted) one. This swap succeeds only if the version number passed with the list of segments + * to swap is the same as the current version of the pipeline. + * The pipeline version is updated whenever swapping segments or pulling the segment at the tail. + */ +@InterfaceAudience.Private +public class CompactionPipeline { + private static final Log LOG = LogFactory.getLog(CompactionPipeline.class); + + private final RegionServicesForStores region; + private LinkedList pipeline; + private long version; + + private static final ImmutableSegment EMPTY_MEM_STORE_SEGMENT = SegmentFactory.instance() + .createImmutableSegment(null, + CompactingMemStore.DEEP_OVERHEAD_PER_PIPELINE_ITEM); + + public CompactionPipeline(RegionServicesForStores region) { + this.region = region; + this.pipeline = new LinkedList(); + this.version = 0; + } + + public boolean pushHead(MutableSegment segment) { + ImmutableSegment immutableSegment = SegmentFactory.instance(). + createImmutableSegment(segment); + synchronized (pipeline){ + return addFirst(immutableSegment); + } + } + + public ImmutableSegment pullTail() { + synchronized (pipeline){ + if(pipeline.isEmpty()) { + return EMPTY_MEM_STORE_SEGMENT; + } + return removeLast(); + } + } + + public VersionedSegmentsList getVersionedList() { + synchronized (pipeline){ + LinkedList segmentList = new LinkedList(pipeline); + VersionedSegmentsList res = new VersionedSegmentsList(segmentList, version); + return res; + } + } + + /** + * Swaps the versioned list at the tail of the pipeline with the new compacted segment. + * Swapping only if there were no changes to the suffix of the list while it was compacted. + * @param versionedList tail of the pipeline that was compacted + * @param segment new compacted segment + * @return true iff swapped tail with new compacted segment + */ + public boolean swap(VersionedSegmentsList versionedList, ImmutableSegment segment) { + if(versionedList.getVersion() != version) { + return false; + } + LinkedList suffix; + synchronized (pipeline){ + if(versionedList.getVersion() != version) { + return false; + } + suffix = versionedList.getStoreSegments(); + LOG.info("Swapping pipeline suffix with compacted item. " + +"Just before the swap the number of segments in pipeline is:" + +versionedList.getStoreSegments().size() + +", and the number of cells in new segment is:"+segment.getCellsCount()); + swapSuffix(suffix,segment); + } + if(region != null) { + // update the global memstore size counter + long suffixSize = CompactingMemStore.getSegmentsSize(suffix); + long newSize = CompactingMemStore.getSegmentSize(segment); + long delta = suffixSize - newSize; + long globalMemstoreSize = region.addAndGetGlobalMemstoreSize(-delta); + LOG.info("Suffix size: "+ suffixSize+" compacted item size: "+newSize+ + " globalMemstoreSize: "+globalMemstoreSize); + } + return true; + } + + public boolean isEmpty() { + return pipeline.isEmpty(); + } + + public List getSegments() { + synchronized (pipeline){ + List res = new LinkedList(pipeline); + return res; + } + } + + public long size() { + return pipeline.size(); + } + + public long getMinSequenceId() { + long minSequenceId = Long.MAX_VALUE; + if(!isEmpty()) { + minSequenceId = pipeline.getLast().getMinSequenceId(); + } + return minSequenceId; + } + + public long getTailSize() { + if(isEmpty()) return 0; + return CompactingMemStore.getSegmentSize(pipeline.peekLast()); + } + + private void swapSuffix(LinkedList suffix, ImmutableSegment segment) { + version++; + for(Segment itemInSuffix : suffix) { + itemInSuffix.close(); + } + pipeline.removeAll(suffix); + pipeline.addLast(segment); + } + + private ImmutableSegment removeLast() { + version++; + return pipeline.removeLast(); + } + + private boolean addFirst(ImmutableSegment segment) { + pipeline.add(0,segment); + return true; + } + + // debug method + private boolean validateSuffixList(LinkedList suffix) { + if(suffix.isEmpty()) { + // empty suffix is always valid + return true; + } + + Iterator pipelineBackwardIterator = pipeline.descendingIterator(); + Iterator suffixBackwardIterator = suffix.descendingIterator(); + ImmutableSegment suffixCurrent; + ImmutableSegment pipelineCurrent; + for( ; suffixBackwardIterator.hasNext(); ) { + if(!pipelineBackwardIterator.hasNext()) { + // a suffix longer than pipeline is invalid + return false; + } + suffixCurrent = suffixBackwardIterator.next(); + pipelineCurrent = pipelineBackwardIterator.next(); + if(suffixCurrent != pipelineCurrent) { + // non-matching suffix + return false; + } + } + // suffix matches pipeline suffix + return true; + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java index 3d65bca..8f82aea 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java @@ -77,10 +77,9 @@ public class DefaultMemStore extends AbstractMemStore { /** * Creates a snapshot of the current memstore. * Snapshot must be cleared by call to {@link #clearSnapshot(long)} - * @param flushOpSeqId the sequence id that is attached to the flush operation in the wal */ @Override - public MemStoreSnapshot snapshot(long flushOpSeqId) { + public MemStoreSnapshot snapshot() { // If snapshot currently has entries, then flusher failed or didn't call // cleanup. Log a warning. if (!getSnapshot().isEmpty()) { @@ -90,7 +89,7 @@ public class DefaultMemStore extends AbstractMemStore { this.snapshotId = EnvironmentEdgeManager.currentTime(); if (!getActive().isEmpty()) { ImmutableSegment immutableSegment = SegmentFactory.instance(). - createImmutableSegment(getConfiguration(), getActive()); + createImmutableSegment(getActive()); setSnapshot(immutableSegment); setSnapshotSize(keySize()); resetCellSet(); @@ -99,6 +98,17 @@ public class DefaultMemStore extends AbstractMemStore { return new MemStoreSnapshot(this.snapshotId, getSnapshot()); } + /** + * On flush, how much memory we will clear from the active cell set. + * + * @return size of data that is going to be flushed from active set + */ + @Override + public long getFlushableSize() { + long snapshotSize = getSnapshot().getSize(); + return snapshotSize > 0 ? snapshotSize : keySize(); + } + @Override protected List getListOfScanners(long readPt) throws IOException { List list = new ArrayList(2); @@ -108,7 +118,7 @@ public class DefaultMemStore extends AbstractMemStore { } @Override - protected List getListOfSegments() throws IOException { + protected List getSegments() throws IOException { List list = new ArrayList(2); list.add(0, getActive()); list.add(1, getSnapshot()); @@ -150,6 +160,10 @@ public class DefaultMemStore extends AbstractMemStore { public void finalizeFlush() { } + @Override public boolean isSloppy() { + return false; + } + /** * Code to help figure if our approximation of object heap sizes is close * enough. See hbase-900. Fills memstores then waits so user can heap diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushAllLargeStoresPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushAllLargeStoresPolicy.java new file mode 100644 index 0000000..362d0f9 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushAllLargeStoresPolicy.java @@ -0,0 +1,75 @@ +/** + * 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; + +import java.util.Collection; +import java.util.HashSet; +import java.util.Set; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +/** + * A {@link FlushPolicy} that only flushes store larger a given threshold. If no store is large + * enough, then all stores will be flushed. + */ +public class FlushAllLargeStoresPolicy extends FlushLargeStoresPolicy{ + + private static final Log LOG = LogFactory.getLog(FlushAllLargeStoresPolicy.class); + + @Override + protected void configureForRegion(HRegion region) { + super.configureForRegion(region); + int familyNumber = region.getTableDesc().getFamilies().size(); + if (familyNumber <= 1) { + // No need to parse and set flush size lower bound if only one family + // Family number might also be zero in some of our unit test case + return; + } + this.flushSizeLowerBound = getFlushSizeLowerBound(region); + } + + @Override + public Collection selectStoresToFlush() { + // no need to select stores if only one family + if (region.getTableDesc().getFamilies().size() == 1) { + return region.stores.values(); + } + // start selection + Collection stores = region.stores.values(); + Set specificStoresToFlush = new HashSet(); + for (Store store : stores) { + if (shouldFlush(store)) { + specificStoresToFlush.add(store); + } + } + if (!specificStoresToFlush.isEmpty()) return specificStoresToFlush; + + // Didn't find any CFs which were above the threshold for selection. + if (LOG.isDebugEnabled()) { + LOG.debug("Since none of the CFs were above the size, flushing all."); + } + return stores; + } + + @Override + protected boolean shouldFlush(Store store) { + return (super.shouldFlush(store) || region.shouldFlushStore(store)); + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushLargeStoresPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushLargeStoresPolicy.java index b4d47c7..49cb747 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushLargeStoresPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushLargeStoresPolicy.java @@ -17,10 +17,6 @@ */ package org.apache.hadoop.hbase.regionserver; -import java.util.Collection; -import java.util.HashSet; -import java.util.Set; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HBaseInterfaceAudience; @@ -31,7 +27,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; * enough, then all stores will be flushed. */ @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) -public class FlushLargeStoresPolicy extends FlushPolicy { +public abstract class FlushLargeStoresPolicy extends FlushPolicy { private static final Log LOG = LogFactory.getLog(FlushLargeStoresPolicy.class); @@ -41,20 +37,13 @@ public class FlushLargeStoresPolicy extends FlushPolicy { public static final String HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN = "hbase.hregion.percolumnfamilyflush.size.lower.bound.min"; - private static final long DEFAULT_HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN = + public static final long DEFAULT_HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN = 1024 * 1024 * 16L; - private long flushSizeLowerBound = -1; + protected long flushSizeLowerBound = -1; - @Override - protected void configureForRegion(HRegion region) { - super.configureForRegion(region); + protected long getFlushSizeLowerBound(HRegion region) { int familyNumber = region.getTableDesc().getFamilies().size(); - if (familyNumber <= 1) { - // No need to parse and set flush size lower bound if only one family - // Family number might also be zero in some of our unit test case - return; - } // For multiple families, lower bound is the "average flush size" by default // unless setting in configuration is larger. long flushSizeLowerBound = region.getMemstoreFlushSize() / familyNumber; @@ -85,44 +74,19 @@ public class FlushLargeStoresPolicy extends FlushPolicy { } } - this.flushSizeLowerBound = flushSizeLowerBound; + return flushSizeLowerBound; } - private boolean shouldFlush(Store store) { + protected boolean shouldFlush(Store store) { if (store.getMemStoreSize() > this.flushSizeLowerBound) { if (LOG.isDebugEnabled()) { LOG.debug("Flush Column Family " + store.getColumnFamilyName() + " of " + - region.getRegionInfo().getEncodedName() + " because memstoreSize=" + - store.getMemStoreSize() + " > lower bound=" + this.flushSizeLowerBound); + region.getRegionInfo().getEncodedName() + " because memstoreSize=" + + store.getMemStoreSize() + " > lower bound=" + this.flushSizeLowerBound); } return true; } - return region.shouldFlushStore(store); - } - - @Override - public Collection selectStoresToFlush() { - // no need to select stores if only one family - if (region.getTableDesc().getFamilies().size() == 1) { - return region.stores.values(); - } - // start selection - Collection stores = region.stores.values(); - Set specificStoresToFlush = new HashSet(); - for (Store store : stores) { - if (shouldFlush(store)) { - specificStoresToFlush.add(store); - } - } - // Didn't find any CFs which were above the threshold for selection. - if (specificStoresToFlush.isEmpty()) { - if (LOG.isDebugEnabled()) { - LOG.debug("Since none of the CFs were above the size, flushing all."); - } - return stores; - } else { - return specificStoresToFlush; - } + return false; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushNonSloppyStoresFirstPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushNonSloppyStoresFirstPolicy.java new file mode 100644 index 0000000..2921f23 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushNonSloppyStoresFirstPolicy.java @@ -0,0 +1,66 @@ +/** + * 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; + +import java.util.Collection; +import java.util.HashSet; + +/** + * A {@link FlushPolicy} that only flushes store larger than a given threshold. If no store is large + * enough, then all stores will be flushed. + * Gives priority to selecting regular stores first, and only if no other + * option, selects sloppy stores which normaly require more memory. + */ +public class FlushNonSloppyStoresFirstPolicy extends FlushLargeStoresPolicy { + + private Collection regularStores = new HashSet<>(); + private Collection sloppyStores = new HashSet<>(); + + /** + * @return the stores need to be flushed. + */ + @Override public Collection selectStoresToFlush() { + Collection specificStoresToFlush = new HashSet(); + for(Store store : regularStores) { + if(shouldFlush(store) || region.shouldFlushStore(store)) { + specificStoresToFlush.add(store); + } + } + if(!specificStoresToFlush.isEmpty()) return specificStoresToFlush; + for(Store store : sloppyStores) { + if(shouldFlush(store)) { + specificStoresToFlush.add(store); + } + } + if(!specificStoresToFlush.isEmpty()) return specificStoresToFlush; + return region.stores.values(); + } + + @Override + protected void configureForRegion(HRegion region) { + super.configureForRegion(region); + this.flushSizeLowerBound = getFlushSizeLowerBound(region); + for(Store store : region.stores.values()) { + if(store.getMemStore().isSloppy()) { + sloppyStores.add(store); + } else { + regularStores.add(store); + } + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushPolicyFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushPolicyFactory.java index e80b696..b93594e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushPolicyFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushPolicyFactory.java @@ -41,7 +41,7 @@ public class FlushPolicyFactory { public static final String HBASE_FLUSH_POLICY_KEY = "hbase.regionserver.flush.policy"; private static final Class DEFAULT_FLUSH_POLICY_CLASS = - FlushLargeStoresPolicy.class; + FlushAllLargeStoresPolicy.class; /** * Create the FlushPolicy configured for the given table. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java index 54b942d..672cd79 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java @@ -514,6 +514,10 @@ public class HMobStore extends HStore { @Override public void finalizeFlush() { } + @Override public MemStore getMemStore() { + return null; + } + public void updateCellsCountCompactedToMob(long count) { cellsCountCompactedToMob += count; } 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 72238cc..ca36842 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 @@ -19,6 +19,19 @@ package org.apache.hadoop.hbase.regionserver; import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_LOCAL; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.io.Closeables; +import com.google.protobuf.ByteString; +import com.google.protobuf.Descriptors; +import com.google.protobuf.Message; +import com.google.protobuf.RpcCallback; +import com.google.protobuf.RpcController; +import com.google.protobuf.Service; +import com.google.protobuf.TextFormat; import java.io.EOFException; import java.io.FileNotFoundException; import java.io.IOException; @@ -183,19 +196,6 @@ import org.apache.hadoop.util.StringUtils; import org.apache.htrace.Trace; import org.apache.htrace.TraceScope; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Optional; -import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.io.Closeables; -import com.google.protobuf.ByteString; -import com.google.protobuf.Descriptors; -import com.google.protobuf.Message; -import com.google.protobuf.RpcCallback; -import com.google.protobuf.RpcController; -import com.google.protobuf.Service; -import com.google.protobuf.TextFormat; @SuppressWarnings("deprecation") @InterfaceAudience.Private @@ -924,11 +924,16 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi }); } boolean allStoresOpened = false; + boolean hasSloppyStores = false; try { for (int i = 0; i < htableDescriptor.getFamilies().size(); i++) { Future future = completionService.take(); HStore store = future.get(); this.stores.put(store.getFamily().getName(), store); + MemStore memStore = store.getMemStore(); + if(memStore != null && memStore.isSloppy()) { + hasSloppyStores = true; + } long storeMaxSequenceId = store.getMaxSequenceId(); maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), @@ -942,6 +947,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } } allStoresOpened = true; + if(hasSloppyStores) { + htableDescriptor.setFlushPolicyClassName(FlushNonSloppyStoresFirstPolicy.class + .getName()); + } } catch (InterruptedException e) { throw (InterruptedIOException)new InterruptedIOException().initCause(e); } catch (ExecutionException e) { 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 7468be0..e4c72fc 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 @@ -230,8 +230,15 @@ public class HStore implements Store { // to clone it? scanInfo = new ScanInfo(conf, family, ttl, timeToPurgeDeletes, this.comparator); String className = conf.get(MEMSTORE_CLASS_NAME, DefaultMemStore.class.getName()); - this.memstore = ReflectionUtils.instantiateWithCustomCtor(className, new Class[] { - Configuration.class, CellComparator.class }, new Object[] { conf, this.comparator }); + if (family.isInMemoryCompaction()) { + className = CompactingMemStore.class.getName(); + this.memstore = new CompactingMemStore(conf, this.comparator, this, + this.getHRegion().getRegionServicesForStores()); + } else { + this.memstore = ReflectionUtils.instantiateWithCustomCtor(className, new Class[] { + Configuration.class, CellComparator.class }, new Object[] { conf, this.comparator }); + } + LOG.info("Memstore class name is " + className); this.offPeakHours = OffPeakHours.getInstance(conf); // Setting up cache configuration for this family @@ -2148,7 +2155,7 @@ public class HStore implements Store { @Override public void prepare() { // passing the current sequence number of the wal - to allow bookkeeping in the memstore - this.snapshot = memstore.snapshot(cacheFlushSeqNum); + this.snapshot = memstore.snapshot(); this.cacheFlushCount = snapshot.getCellsCount(); this.cacheFlushSize = snapshot.getSize(); committedFiles = new ArrayList(1); @@ -2475,6 +2482,10 @@ public class HStore implements Store { memstore.finalizeFlush(); } + @Override public MemStore getMemStore() { + return memstore; + } + private void clearCompactedfiles(final List filesToRemove) throws IOException { if (LOG.isTraceEnabled()) { LOG.trace("Clearing the compacted file " + filesToRemove + " from this store"); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableSegment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableSegment.java index 70a608d..13d9fbf 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableSegment.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableSegment.java @@ -64,8 +64,4 @@ public class ImmutableSegment extends Segment { return this.timeRange.getMin(); } - @Override - protected void updateMetaInfo(Cell toAdd, long s) { - throw new IllegalAccessError("This is an immutable segment"); - } } \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java index ea72b7f..00d49d1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java @@ -42,19 +42,10 @@ public interface MemStore extends HeapSize { MemStoreSnapshot snapshot(); /** - * Creates a snapshot of the current memstore. Snapshot must be cleared by call to - * {@link #clearSnapshot(long)}. - * @param flushOpSeqId the current sequence number of the wal; to be attached to the flushed - * segment - * @return {@link MemStoreSnapshot} - */ - MemStoreSnapshot snapshot(long flushOpSeqId); - - /** * Clears the current snapshot of the Memstore. * @param id * @throws UnexpectedStateException - * @see #snapshot(long) + * @see #snapshot() */ void clearSnapshot(long id) throws UnexpectedStateException; @@ -144,4 +135,7 @@ public interface MemStore extends HeapSize { * One example is to update the wal with sequence number that is known only at the store level. */ void finalizeFlush(); + + /* Return true if the memstore may need some extra memory space*/ + boolean isSloppy(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactor.java new file mode 100644 index 0000000..0d9ad7a --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactor.java @@ -0,0 +1,197 @@ +/** + * + * 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; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.client.Scan; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * The ongoing MemStore Compaction manager, dispatches a solo running compaction + * and interrupts the compaction if requested. + * The MemStoreScanner is used to traverse the compaction pipeline. The MemStoreScanner + * is included in internal store scanner, where all compaction logic is implemented. + * Threads safety: It is assumed that the compaction pipeline is immutable, + * therefore no special synchronization is required. + */ +class MemStoreCompactor { + + private static final Log LOG = LogFactory.getLog(MemStoreCompactor.class); + private CompactingMemStore compactingMemStore; + private MemStoreScanner scanner; // scanner for pipeline only + // scanner on top of MemStoreScanner that uses ScanQueryMatcher + private StoreScanner compactingScanner; + + // smallest read point for any ongoing MemStore scan + private long smallestReadPoint; + + // a static version of the segment list from the pipeline + private VersionedSegmentsList versionedList; + private final AtomicBoolean isInterrupted = new AtomicBoolean(false); + + public MemStoreCompactor(CompactingMemStore compactingMemStore) { + this.compactingMemStore = compactingMemStore; + } + + /** + * The request to dispatch the compaction asynchronous task. + * The method returns true if compaction was successfully dispatched, or false if there + * is already an ongoing compaction or nothing to compact. + */ + public boolean startCompaction() throws IOException { + if (!compactingMemStore.hasCompactibleSegments()) return false; // no compaction on empty + + List scanners = new ArrayList(); + // get the list of segments from the pipeline + versionedList = compactingMemStore.getCompactibleSegments(); + // the list is marked with specific version + + // create the list of scanners with maximally possible read point, meaning that + // all KVs are going to be returned by the pipeline traversing + for (Segment segment : versionedList.getStoreSegments()) { + scanners.add(segment.getSegmentScanner(Long.MAX_VALUE)); + } + scanner = + new MemStoreScanner(compactingMemStore, scanners, Long.MAX_VALUE, + MemStoreScanner.Type.COMPACT_FORWARD); + + smallestReadPoint = compactingMemStore.getSmallestReadPoint(); + compactingScanner = createScanner(compactingMemStore.getStore()); + + LOG.info("Starting the MemStore in-memory compaction for store " + + compactingMemStore.getStore().getColumnFamilyName()); + + doCompaction(); + return true; + } + + /** + * The request to cancel the compaction asynchronous task + * The compaction may still happen if the request was sent too late + * Non-blocking request + */ + public void stopCompact() { + isInterrupted.compareAndSet(false, true); + } + + + /** + * Close the scanners and clear the pointers in order to allow good + * garbage collection + */ + private void releaseResources() { + isInterrupted.set(false); + scanner.close(); + scanner = null; + compactingScanner.close(); + compactingScanner = null; + versionedList = null; + } + + /** + * The worker thread performs the compaction asynchronously. + * The solo (per compactor) thread only reads the compaction pipeline. + * There is at most one thread per memstore instance. + */ + private void doCompaction() { + + ImmutableSegment result = SegmentFactory.instance() // create the scanner + .createImmutableSegment( + compactingMemStore.getConfiguration(), compactingMemStore.getComparator(), + CompactingMemStore.DEEP_OVERHEAD_PER_PIPELINE_ITEM); + + // the compaction processing + try { + // Phase I: create the compacted MutableCellSetSegment + compactSegments(result); + + // Phase II: swap the old compaction pipeline + if (!isInterrupted.get()) { + compactingMemStore.swapCompactedSegments(versionedList, result); + // update the wal so it can be truncated and not get too long + compactingMemStore.updateLowestUnflushedSequenceIdInWal(true); // only if greater + } + } catch (Exception e) { + LOG.debug("Interrupting the MemStore in-memory compaction for store " + compactingMemStore + .getFamilyName()); + Thread.currentThread().interrupt(); + return; + } finally { + releaseResources(); + compactingMemStore.setInMemoryFlushInProgress(false); + } + + } + + /** + * Creates the scanner for compacting the pipeline. + * + * @return the scanner + */ + private StoreScanner createScanner(Store store) throws IOException { + + Scan scan = new Scan(); + scan.setMaxVersions(); //Get all available versions + + StoreScanner internalScanner = + new StoreScanner(store, store.getScanInfo(), scan, Collections.singletonList(scanner), + ScanType.COMPACT_RETAIN_DELETES, smallestReadPoint, HConstants.OLDEST_TIMESTAMP); + + return internalScanner; + } + + /** + * Updates the given single Segment using the internal store scanner, + * who in turn uses ScanQueryMatcher + */ + private void compactSegments(Segment result) throws IOException { + + List kvs = new ArrayList(); + // get the limit to the size of the groups to be returned by compactingScanner + int compactionKVMax = compactingMemStore.getConfiguration().getInt( + HConstants.COMPACTION_KV_MAX, + HConstants.COMPACTION_KV_MAX_DEFAULT); + + ScannerContext scannerContext = + ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build(); + + boolean hasMore; + do { + hasMore = compactingScanner.next(kvs, scannerContext); + if (!kvs.isEmpty()) { + for (Cell c : kvs) { + // The scanner is doing all the elimination logic + // now we just copy it to the new segment + Cell newKV = result.maybeCloneWithAllocator(c); + result.internalAdd(newKV); + + } + kvs.clear(); + } + } while (hasMore && (!isInterrupted.get())); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java index d3c35b3..72f7bf5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java @@ -18,8 +18,15 @@ */ package org.apache.hadoop.hbase.regionserver; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.util.StealJobQueue; +import org.apache.hadoop.hbase.wal.WAL; /** * Services a Store needs from a Region. @@ -32,6 +39,20 @@ import org.apache.hadoop.hbase.classification.InterfaceStability; @InterfaceStability.Evolving public class RegionServicesForStores { + private static final int POOL_SIZE = 10; + private static final ThreadPoolExecutor INMEMORY_COMPACTION_POOL = + new ThreadPoolExecutor(POOL_SIZE, POOL_SIZE, 60, TimeUnit.SECONDS, + new StealJobQueue().getStealFromQueue(), + new ThreadFactory() { + @Override + public Thread newThread(Runnable r) { + Thread t = new Thread(r); + t.setName(Thread.currentThread().getName() + + "-inmemoryCompactions-" + + System.currentTimeMillis()); + return t; + } + }); private final HRegion region; public RegionServicesForStores(HRegion region) { @@ -39,15 +60,37 @@ public class RegionServicesForStores { } public void blockUpdates() { - this.region.blockUpdates(); + region.blockUpdates(); } public void unblockUpdates() { - this.region.unblockUpdates(); + region.unblockUpdates(); } public long addAndGetGlobalMemstoreSize(long size) { - return this.region.addAndGetGlobalMemstoreSize(size); + return region.addAndGetGlobalMemstoreSize(size); + } + + public HRegionInfo getRegionInfo() { + return region.getRegionInfo(); } + public WAL getWAL() { + return region.getWAL(); + } + + public ThreadPoolExecutor getInMemoryCompactionPool() { return INMEMORY_COMPACTION_POOL; } + + public long getMemstoreFlushSize() { + return region.getMemstoreFlushSize(); + } + + public int getNumStores() { + return region.getStores().size(); + } + + // methods for tests + long getGlobalMemstoreTotalSize() { + return region.getMemstoreSize(); + } } 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 dcad5a0..a90a6ba 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 @@ -23,6 +23,7 @@ import java.util.SortedSet; import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.KeyValue; @@ -40,8 +41,11 @@ import org.apache.hadoop.hbase.util.ByteRange; */ @InterfaceAudience.Private public abstract class Segment { + + private static final Log LOG = LogFactory.getLog(Segment.class); private volatile CellSet cellSet; private final CellComparator comparator; + private long minSequenceId; private volatile MemStoreLAB memStoreLAB; protected final AtomicLong size; protected volatile boolean tagsPresent; @@ -51,6 +55,7 @@ public abstract class Segment { long size) { this.cellSet = cellSet; this.comparator = comparator; + this.minSequenceId = Long.MAX_VALUE; this.memStoreLAB = memStoreLAB; this.size = new AtomicLong(size); this.tagsPresent = false; @@ -60,6 +65,7 @@ public abstract class Segment { protected Segment(Segment segment) { this.cellSet = segment.getCellSet(); this.comparator = segment.getComparator(); + this.minSequenceId = segment.getMinSequenceId(); this.memStoreLAB = segment.getMemStoreLAB(); this.size = new AtomicLong(segment.getSize()); this.tagsPresent = segment.isTagsPresent(); @@ -183,6 +189,10 @@ public abstract class Segment { size.addAndGet(delta); } + public long getMinSequenceId() { + return minSequenceId; + } + public TimeRangeTracker getTimeRangeTracker() { return this.timeRangeTracker; } @@ -231,10 +241,18 @@ public abstract class Segment { return s; } - /** - * Only mutable Segments implement this. - */ - protected abstract void updateMetaInfo(Cell toAdd, long s); + protected void updateMetaInfo(Cell toAdd, long s) { + getTimeRangeTracker().includeTimestamp(toAdd); + size.addAndGet(s); + minSequenceId = Math.min(minSequenceId, toAdd.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(toAdd.getTagsLength() > 0) { + tagsPresent = true; + } + } /** * Returns a subset of the segment cell set, which starts with the given cell @@ -259,6 +277,24 @@ public abstract class Segment { } } + void assertMinSequenceId(Log log) { + for (Cell cell: getCellSet()) { + if (cell.getSequenceId() == getMinSequenceId()) { + log.info("ESHCAR: cell with minimal sequence id is "+cell.toString()+", seqid=" + + cell.getSequenceId()); + return; + } + } + log.error("ESHCAR: no cell with minimal sequence id "+getMinSequenceId()); + } + + void assertCellType(Cell cell, Log log) { + if(KeyValue.Type.codeToType(cell.getTypeByte()) != KeyValue.Type.Put + && KeyValue.Type.codeToType(cell.getTypeByte()) != KeyValue.Type.Delete) { + log.info("Eshcar: cell="+cell.toString()); + } + } + @Override public String toString() { String res = "Store segment of type "+this.getClass().getName()+"; "; 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 394ffa1..7ac80ae 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 @@ -44,16 +44,16 @@ public final class SegmentFactory { final CellComparator comparator, long size) { MemStoreLAB memStoreLAB = getMemStoreLAB(conf); MutableSegment segment = generateMutableSegment(conf, comparator, memStoreLAB, size); - return createImmutableSegment(conf, segment); + return createImmutableSegment(segment); } public ImmutableSegment createImmutableSegment(CellComparator comparator, long size) { MutableSegment segment = generateMutableSegment(null, comparator, null, size); - return createImmutableSegment(null, segment); + return createImmutableSegment(segment); } - public ImmutableSegment createImmutableSegment(final Configuration conf, MutableSegment segment) { + public ImmutableSegment createImmutableSegment(MutableSegment segment) { return new ImmutableSegment(segment); } public MutableSegment createMutableSegment(final Configuration conf, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java index b77a33b..3419937 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java @@ -520,4 +520,5 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf */ void finalizeFlush(); + MemStore getMemStore(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/VersionedSegmentsList.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/VersionedSegmentsList.java new file mode 100644 index 0000000..9d7a723 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/VersionedSegmentsList.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; + +import java.util.LinkedList; + +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * A list of segment managers coupled with the version of the memstore (version at the time it was + * created). + * This structure helps to guarantee that the compaction pipeline updates after the compaction is + * updated in a consistent (atomic) way. + * Specifically, swapping some of the elements in a compaction pipeline with a new compacted + * element is permitted only if the pipeline version is the same as the version attached to the + * elements. + * + */ +@InterfaceAudience.Private +public class VersionedSegmentsList { + + private final LinkedList storeSegments; + private final long version; + + public VersionedSegmentsList( + LinkedList storeSegments, long version) { + this.storeSegments = storeSegments; + this.version = version; + } + + public LinkedList getStoreSegments() { + return storeSegments; + } + + public long getVersion() { + return version; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java index 3736ae3..3a38571 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java @@ -19,6 +19,8 @@ package org.apache.hadoop.hbase.regionserver.wal; import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.WAL_FILE_NAME_DELIMITER; +import com.google.common.annotations.VisibleForTesting; + import java.io.IOException; import java.io.InterruptedIOException; import java.lang.management.ManagementFactory; @@ -41,8 +43,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantLock; -import com.google.common.annotations.VisibleForTesting; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -772,6 +772,21 @@ public abstract class AbstractFSWAL implements WAL { LOG.info("Closed WAL: " + toString()); } + /** + * updates the sequence number of a specific store. + * depending on the flag: replaces current seq number if the given seq id is bigger, + * or even if it is lower than existing one + * @param encodedRegionName + * @param familyName + * @param sequenceid + * @param onlyIfGreater + */ + @Override public void updateStore(byte[] encodedRegionName, byte[] familyName, Long sequenceid, + boolean onlyIfGreater) { + sequenceIdAccounting.updateStore(encodedRegionName,familyName,sequenceid,onlyIfGreater); + } + + protected SyncFuture getSyncFuture(final long sequence, Span span) { SyncFuture syncFuture = this.syncFuturesByHandler.get(Thread.currentThread()); if (syncFuture == null) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceIdAccounting.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceIdAccounting.java index 6e10f3c..53c501f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceIdAccounting.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceIdAccounting.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.regionserver.wal; +import com.google.common.collect.Maps; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -32,8 +33,6 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.util.Bytes; -import com.google.common.collect.Maps; - /** * Accounting of sequence ids per region and then by column family. So we can our accounting * current, call startCacheFlush and then finishedCacheFlush or abortCacheFlush so this instance @@ -163,6 +162,38 @@ class SequenceIdAccounting { } } + /** + * Update the store sequence id, e.g., upon executing in-memory compaction + */ + void updateStore(byte[] encodedRegionName, byte[] familyName, Long sequenceId, + boolean onlyIfGreater) { + if(sequenceId == null) return; + Long highest = this.highestSequenceIds.get(encodedRegionName); + if(highest == null || sequenceId > highest) { + this.highestSequenceIds.put(encodedRegionName,sequenceId); + } + synchronized (this.tieLock) { + ConcurrentMap m = getOrCreateLowestSequenceIds(encodedRegionName); + boolean replaced = false; + while (!replaced) { + Long oldSeqId = m.get(familyName); + if (oldSeqId == null) { + m.put(familyName, sequenceId); + replaced = true; + } else if (onlyIfGreater) { + if (sequenceId > oldSeqId) { + replaced = m.replace(familyName, oldSeqId, sequenceId); + } else { + return; + } + } else { // replace even if sequence id is not greater than oldSeqId + m.put(familyName, sequenceId); + return; + } + } + } + } + ConcurrentMap getOrCreateLowestSequenceIds(byte[] encodedRegionName) { // Intentionally, this access is done outside of this.regionSequenceIdLock. Done per append. ConcurrentMap m = this.lowestUnflushedSequenceIds.get(encodedRegionName); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java index 10fe04c..b5ddd00 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java @@ -25,19 +25,19 @@ import java.util.concurrent.atomic.AtomicBoolean; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.util.FSUtils; // imports for things that haven't moved from regionserver.wal yet. import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; import org.apache.hadoop.hbase.regionserver.wal.WALCoprocessorHost; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; +import org.apache.hadoop.hbase.util.FSUtils; /** * No-op implementation of {@link WALProvider} used when the WAL is disabled. @@ -170,6 +170,10 @@ class DisabledWALProvider implements WALProvider { } @Override + public void updateStore(byte[] encodedRegionName, byte[] familyName, + Long sequenceid, boolean onlyIfGreater) { return; } + + @Override public void sync() { if (!this.listeners.isEmpty()) { for (WALActionsListener listener : this.listeners) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java index 051ce54..af63b0b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.wal; +import com.google.common.annotations.VisibleForTesting; import java.io.Closeable; import java.io.IOException; import java.util.Set; @@ -35,8 +36,6 @@ import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; import org.apache.hadoop.hbase.regionserver.wal.WALCoprocessorHost; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; -import com.google.common.annotations.VisibleForTesting; - /** * A Write Ahead Log (WAL) provides service for reading, writing waledits. This interface provides * APIs for WAL users (such as RegionServer) to use the WAL (do append, sync, etc). @@ -118,6 +117,18 @@ public interface WAL { long append(HRegionInfo info, WALKey key, WALEdit edits, boolean inMemstore) throws IOException; /** + * updates the seuence number of a specific store. + * depending on the flag: replaces current seq number if the given seq id is bigger, + * or even if it is lower than existing one + * @param encodedRegionName + * @param familyName + * @param sequenceid + * @param onlyIfGreater + */ + void updateStore(byte[] encodedRegionName, byte[] familyName, Long sequenceid, + boolean onlyIfGreater); + + /** * Sync what we have in the WAL. * @throws IOException */ diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index 010e184..0d644db 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java @@ -219,7 +219,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { }); /** This is for unit tests parameterized with a single boolean. */ - public static final List MEMSTORETS_TAGS_PARAMETRIZED = memStoreTSAndTagsCombination() ; + public static final List MEMSTORETS_TAGS_PARAMETRIZED = memStoreTSAndTagsCombination(); /** Compression algorithms to use in testing */ public static final Compression.Algorithm[] COMPRESSION_ALGORITHMS ={ Compression.Algorithm.NONE, Compression.Algorithm.GZ @@ -1518,7 +1518,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { desc.addFamily(hcd); } getHBaseAdmin().createTable(desc, splitKeys); - // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned + // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are + // assigned waitUntilAllRegionsAssigned(tableName); return (HTable) getConnection().getTable(tableName); } @@ -1555,7 +1556,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { desc.addFamily(hcd); } getHBaseAdmin().createTable(desc); - // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned + // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are + // assigned waitUntilAllRegionsAssigned(tableName); return (HTable) getConnection().getTable(tableName); } @@ -1573,7 +1575,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { desc.addCoprocessor(cpName); } getHBaseAdmin().createTable(desc); - // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned + // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are + // assigned waitUntilAllRegionsAssigned(tableName); return (HTable) getConnection().getTable(tableName); } @@ -1598,7 +1601,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { i++; } getHBaseAdmin().createTable(desc); - // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned + // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are + // assigned waitUntilAllRegionsAssigned(tableName); return (HTable) getConnection().getTable(tableName); } @@ -1617,7 +1621,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { HColumnDescriptor hcd = new HColumnDescriptor(family); desc.addFamily(hcd); getHBaseAdmin().createTable(desc, splitRows); - // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned + // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are + // assigned waitUntilAllRegionsAssigned(tableName); return (HTable) getConnection().getTable(tableName); } @@ -1829,10 +1834,27 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { */ public HRegion createLocalHRegion(TableName tableName, byte[] startKey, byte[] stopKey, boolean isReadOnly, Durability durability, WAL wal, byte[]... families) throws IOException { + return createLocalHRegionWithInMemoryFlags(tableName,startKey, stopKey, isReadOnly, + durability, wal, null, families); + } + + public HRegion createLocalHRegionWithInMemoryFlags(TableName tableName, byte[] startKey, + byte[] stopKey, + boolean isReadOnly, Durability durability, WAL wal, boolean[] compactedMemStore, + byte[]... families) + throws IOException { HTableDescriptor htd = new HTableDescriptor(tableName); htd.setReadOnly(isReadOnly); + int i=0; for (byte[] family : families) { HColumnDescriptor hcd = new HColumnDescriptor(family); + if(compactedMemStore != null && i < compactedMemStore.length) { + hcd.setMemStoreClass("org.apache.hadoop.hbase.regionserver.CompactingMemStore"); + } else { + hcd.setMemStoreClass("org.apache.hadoop.hbase.regionserver.DefaultMemStore"); + + } + i++; // Set default to be three versions. hcd.setMaxVersions(Integer.MAX_VALUE); htd.addFamily(hcd); @@ -1872,7 +1894,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { * @param preserveRegions keep the existing split points * @return HTable for the new table */ - public HTable truncateTable(final TableName tableName, final boolean preserveRegions) throws IOException { + public HTable truncateTable(final TableName tableName, final boolean preserveRegions) throws + IOException { Admin admin = getHBaseAdmin(); if (!admin.isTableDisabled(tableName)) { admin.disableTable(tableName); @@ -1947,7 +1970,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { * @return Count of rows loaded. * @throws IOException */ - public int loadTable(final Table t, final byte[][] f, byte[] value, boolean writeToWAL) throws IOException { + public int loadTable(final Table t, final byte[][] f, byte[] value, + boolean writeToWAL) throws IOException { List puts = new ArrayList<>(); for (byte[] row : HBaseTestingUtility.ROWS) { Put put = new Put(row); @@ -2005,7 +2029,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { } if (count != expectedCount) { String row = new String(new byte[] {b1,b2,b3}); - throw new RuntimeException("Row:" + row + " has a seen count of " + count + " instead of " + expectedCount); + throw new RuntimeException("Row:" + row + " has a seen count of " + count + " " + + "instead of " + expectedCount); } } } @@ -2515,7 +2540,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { * Create a stubbed out RegionServerService, mainly for getting FS. * This version is used by TestTokenAuthentication */ - public RegionServerServices createMockRegionServerService(RpcServerInterface rpc) throws IOException { + public RegionServerServices createMockRegionServerService(RpcServerInterface rpc) throws + IOException { final MockRegionServerServices rss = new MockRegionServerServices(getZooKeeperWatcher()); rss.setFileSystem(getTestFileSystem()); rss.setRpcServer(rpc); @@ -3124,7 +3150,10 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { * Set maxRecoveryErrorCount in DFSClient. In 0.20 pre-append its hard-coded to 5 and * makes tests linger. Here is the exception you'll see: *
-   * 2010-06-15 11:52:28,511 WARN  [DataStreamer for file /hbase/.logs/wal.1276627923013 block blk_928005470262850423_1021] hdfs.DFSClient$DFSOutputStream(2657): Error Recovery for block blk_928005470262850423_1021 failed  because recovery from primary datanode 127.0.0.1:53683 failed 4 times.  Pipeline was 127.0.0.1:53687, 127.0.0.1:53683. Will retry...
+   * 2010-06-15 11:52:28,511 WARN  [DataStreamer for file /hbase/.logs/wal.1276627923013 block
+   * blk_928005470262850423_1021] hdfs.DFSClient$DFSOutputStream(2657): Error Recovery for block
+   * blk_928005470262850423_1021 failed  because recovery from primary datanode 127.0.0.1:53683
+   * failed 4 times.  Pipeline was 127.0.0.1:53687, 127.0.0.1:53683. Will retry...
    * 
* @param stream A DFSClient.DFSOutputStream. * @param max diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java index d8363d4..3c10ddc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.regionserver.MemStore; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.Store; @@ -208,6 +209,10 @@ public class TestIOFencing { @Override public void finalizeFlush() { } + + @Override public MemStore getMemStore() { + return null; + } } private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java index 5c79d72..4a4b0e9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java @@ -305,19 +305,15 @@ public class TestHeapSize { // DefaultMemStore Deep Overhead actual = DefaultMemStore.DEEP_OVERHEAD; expected = ClassSize.estimateBase(cl, false); - expected += (2 * ClassSize.estimateBase(AtomicLong.class, false)); - expected += (2 * ClassSize.estimateBase(CellSet.class, false)); - expected += (2 * ClassSize.estimateBase(ConcurrentSkipListMap.class, false)); - expected += (2 * ClassSize.estimateBase(TimeRangeTracker.class, false)); + expected += ClassSize.estimateBase(AtomicLong.class, false); + expected += ClassSize.estimateBase(CellSet.class, false); + expected += ClassSize.estimateBase(ConcurrentSkipListMap.class, false); + expected += ClassSize.estimateBase(TimeRangeTracker.class, false); if(expected != actual) { ClassSize.estimateBase(cl, true); ClassSize.estimateBase(AtomicLong.class, true); - ClassSize.estimateBase(AtomicLong.class, true); - ClassSize.estimateBase(CellSet.class, true); ClassSize.estimateBase(CellSet.class, true); ClassSize.estimateBase(ConcurrentSkipListMap.class, true); - ClassSize.estimateBase(ConcurrentSkipListMap.class, true); - ClassSize.estimateBase(TimeRangeTracker.class, true); ClassSize.estimateBase(TimeRangeTracker.class, true); assertEquals(expected, actual); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java new file mode 100644 index 0000000..5c0e42b --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java @@ -0,0 +1,729 @@ +/* + * + * 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; + +import java.io.IOException; +import java.lang.management.ManagementFactory; +import java.lang.management.MemoryMXBean; +import java.util.ArrayList; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.KeepDeletedCells; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValueTestUtil; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdge; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.Threads; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * compacted memstore test case + */ +@Category({RegionServerTests.class, MediumTests.class}) +public class TestCompactingMemStore extends TestDefaultMemStore { + + private static final Log LOG = LogFactory.getLog(TestCompactingMemStore.class); + private static MemStoreChunkPool chunkPool; + private HRegion region; + private RegionServicesForStores regionServicesForStores; + private HStore store; + + ////////////////////////////////////////////////////////////////////////////// + // Helpers + ////////////////////////////////////////////////////////////////////////////// + private static byte[] makeQualifier(final int i1, final int i2) { + return Bytes.toBytes(Integer.toString(i1) + ";" + + Integer.toString(i2)); + } + + @After + public void tearDown() throws Exception { + chunkPool.clearChunks(); + } + + @Before + public void setUp() throws Exception { + super.internalSetUp(); + Configuration conf = new Configuration(); + conf.setBoolean(SegmentFactory.USEMSLAB_KEY, true); + conf.setFloat(MemStoreChunkPool.CHUNK_POOL_MAXSIZE_KEY, 0.2f); + conf.setInt(HRegion.MEMSTORE_PERIODIC_FLUSH_INTERVAL, 1000); + HBaseTestingUtility hbaseUtility = HBaseTestingUtility.createLocalHTU(conf); + HColumnDescriptor hcd = new HColumnDescriptor(FAMILY); + this.region = hbaseUtility.createTestRegion("foobar", hcd); + this.regionServicesForStores = region.getRegionServicesForStores(); + this.store = new HStore(region, hcd, conf); + this.memstore = new CompactingMemStore(HBaseConfiguration.create(), CellComparator.COMPARATOR, + store, regionServicesForStores); + chunkPool = MemStoreChunkPool.getPool(conf); + assertTrue(chunkPool != null); + } + + + /** + * A simple test which verifies the 3 possible states when scanning across snapshot. + * + * @throws IOException + * @throws CloneNotSupportedException + */ + @Test + public void testScanAcrossSnapshot2() throws IOException, CloneNotSupportedException { + // we are going to the scanning across snapshot with two kvs + // kv1 should always be returned before kv2 + final byte[] one = Bytes.toBytes(1); + final byte[] two = Bytes.toBytes(2); + final byte[] f = Bytes.toBytes("f"); + final byte[] q = Bytes.toBytes("q"); + final byte[] v = Bytes.toBytes(3); + + final KeyValue kv1 = new KeyValue(one, f, q, 10, v); + final KeyValue kv2 = new KeyValue(two, f, q, 10, v); + + // use case 1: both kvs in kvset + this.memstore.add(kv1.clone()); + this.memstore.add(kv2.clone()); + verifyScanAcrossSnapshot2(kv1, kv2); + + // use case 2: both kvs in snapshot + this.memstore.snapshot(); + verifyScanAcrossSnapshot2(kv1, kv2); + + // use case 3: first in snapshot second in kvset + this.memstore = new CompactingMemStore(HBaseConfiguration.create(), + CellComparator.COMPARATOR, store, regionServicesForStores); + this.memstore.add(kv1.clone()); + // As compaction is starting in the background the repetition + // of the k1 might be removed BUT the scanners created earlier + // should look on the OLD MutableCellSetSegment, so this should be OK... + this.memstore.snapshot(); + this.memstore.add(kv2.clone()); + verifyScanAcrossSnapshot2(kv1,kv2); + } + + /** + * Test memstore snapshots + * @throws IOException + */ + @Test + public void testSnapshotting() throws IOException { + final int snapshotCount = 5; + // Add some rows, run a snapshot. Do it a few times. + for (int i = 0; i < snapshotCount; i++) { + addRows(this.memstore); + runSnapshot(this.memstore, true); + assertEquals("History not being cleared", 0, this.memstore.getSnapshot().getCellsCount()); + } + } + + + ////////////////////////////////////////////////////////////////////////////// + // Get tests + ////////////////////////////////////////////////////////////////////////////// + + /** Test getNextRow from memstore + * @throws InterruptedException + */ + @Test + public void testGetNextRow() throws Exception { + addRows(this.memstore); + // Add more versions to make it a little more interesting. + Thread.sleep(1); + addRows(this.memstore); + Cell closestToEmpty = ((CompactingMemStore)this.memstore).getNextRow(KeyValue.LOWESTKEY); + assertTrue(KeyValue.COMPARATOR.compareRows(closestToEmpty, + new KeyValue(Bytes.toBytes(0), System.currentTimeMillis())) == 0); + for (int i = 0; i < ROW_COUNT; i++) { + Cell nr = ((CompactingMemStore)this.memstore).getNextRow(new KeyValue(Bytes.toBytes(i), + System.currentTimeMillis())); + if (i + 1 == ROW_COUNT) { + assertEquals(nr, null); + } else { + assertTrue(KeyValue.COMPARATOR.compareRows(nr, + new KeyValue(Bytes.toBytes(i + 1), System.currentTimeMillis())) == 0); + } + } + //starting from each row, validate results should contain the starting row + Configuration conf = HBaseConfiguration.create(); + for (int startRowId = 0; startRowId < ROW_COUNT; startRowId++) { + ScanInfo scanInfo = new ScanInfo(conf, FAMILY, 0, 1, Integer.MAX_VALUE, + KeepDeletedCells.FALSE, 0, this.memstore.getComparator()); + ScanType scanType = ScanType.USER_SCAN; + InternalScanner scanner = new StoreScanner(new Scan( + Bytes.toBytes(startRowId)), scanInfo, scanType, null, + memstore.getScanners(0)); + List results = new ArrayList(); + for (int i = 0; scanner.next(results); i++) { + int rowId = startRowId + i; + Cell left = results.get(0); + byte[] row1 = Bytes.toBytes(rowId); + assertTrue("Row name", + CellComparator.COMPARATOR.compareRows(left, row1, 0, row1.length) == 0); + assertEquals("Count of columns", QUALIFIER_COUNT, results.size()); + List row = new ArrayList(); + for (Cell kv : results) { + row.add(kv); + } + isExpectedRowWithoutTimestamps(rowId, row); + // Clear out set. Otherwise row results accumulate. + results.clear(); + } + } + } + + @Test + public void testGet_memstoreAndSnapShot() throws IOException { + byte[] row = Bytes.toBytes("testrow"); + byte[] fam = Bytes.toBytes("testfamily"); + byte[] qf1 = Bytes.toBytes("testqualifier1"); + byte[] qf2 = Bytes.toBytes("testqualifier2"); + byte[] qf3 = Bytes.toBytes("testqualifier3"); + byte[] qf4 = Bytes.toBytes("testqualifier4"); + byte[] qf5 = Bytes.toBytes("testqualifier5"); + byte[] val = Bytes.toBytes("testval"); + + //Setting up memstore + memstore.add(new KeyValue(row, fam, qf1, val)); + memstore.add(new KeyValue(row, fam, qf2, val)); + memstore.add(new KeyValue(row, fam, qf3, val)); + //Pushing to pipeline + ((CompactingMemStore)memstore).flushInMemory(); + assertEquals(0, memstore.getSnapshot().getCellsCount()); + //Creating a snapshot + memstore.snapshot(); + assertEquals(3, memstore.getSnapshot().getCellsCount()); + //Adding value to "new" memstore + assertEquals(0, memstore.getActive().getCellsCount()); + memstore.add(new KeyValue(row, fam, qf4, val)); + memstore.add(new KeyValue(row, fam, qf5, val)); + assertEquals(2, memstore.getActive().getCellsCount()); + } + + + //////////////////////////////////// + //Test for upsert with MSLAB + //////////////////////////////////// + + /** + * Test a pathological pattern that shows why we can't currently + * use the MSLAB for upsert workloads. This test inserts data + * in the following pattern: + * + * - row0001 through row1000 (fills up one 2M Chunk) + * - row0002 through row1001 (fills up another 2M chunk, leaves one reference + * to the first chunk + * - row0003 through row1002 (another chunk, another dangling reference) + * + * This causes OOME pretty quickly if we use MSLAB for upsert + * since each 2M chunk is held onto by a single reference. + */ + @Test + public void testUpsertMSLAB() throws Exception { + + int ROW_SIZE = 2048; + byte[] qualifier = new byte[ROW_SIZE - 4]; + + MemoryMXBean bean = ManagementFactory.getMemoryMXBean(); + for (int i = 0; i < 3; i++) { System.gc(); } + long usageBefore = bean.getHeapMemoryUsage().getUsed(); + + long size = 0; + long ts=0; + + for (int newValue = 0; newValue < 1000; newValue++) { + for (int row = newValue; row < newValue + 1000; row++) { + byte[] rowBytes = Bytes.toBytes(row); + size += memstore.updateColumnValue(rowBytes, FAMILY, qualifier, newValue, ++ts); + } + } + System.out.println("Wrote " + ts + " vals"); + for (int i = 0; i < 3; i++) { System.gc(); } + long usageAfter = bean.getHeapMemoryUsage().getUsed(); + System.out.println("Memory used: " + (usageAfter - usageBefore) + + " (heapsize: " + memstore.heapSize() + + " size: " + size + ")"); + } + + //////////////////////////////////// + // Test for periodic memstore flushes + // based on time of oldest edit + //////////////////////////////////// + + /** + * Add keyvalues with a fixed memstoreTs, and checks that memstore size is decreased + * as older keyvalues are deleted from the memstore. + * + * @throws Exception + */ + @Test + public void testUpsertMemstoreSize() throws Exception { + long oldSize = memstore.size(); + + List l = new ArrayList(); + KeyValue kv1 = KeyValueTestUtil.create("r", "f", "q", 100, "v"); + KeyValue kv2 = KeyValueTestUtil.create("r", "f", "q", 101, "v"); + KeyValue kv3 = KeyValueTestUtil.create("r", "f", "q", 102, "v"); + + kv1.setSequenceId(1); + kv2.setSequenceId(1); + kv3.setSequenceId(1); + l.add(kv1); + l.add(kv2); + l.add(kv3); + + this.memstore.upsert(l, 2);// readpoint is 2 + long newSize = this.memstore.size(); + assert (newSize > oldSize); + //The kv1 should be removed. + assert (memstore.getActive().getCellsCount() == 2); + + KeyValue kv4 = KeyValueTestUtil.create("r", "f", "q", 104, "v"); + kv4.setSequenceId(1); + l.clear(); + l.add(kv4); + this.memstore.upsert(l, 3); + assertEquals(newSize, this.memstore.size()); + //The kv2 should be removed. + assert (memstore.getActive().getCellsCount() == 2); + //this.memstore = null; + } + + /** + * Tests that the timeOfOldestEdit is updated correctly for the + * various edit operations in memstore. + * @throws Exception + */ + @Test + public void testUpdateToTimeOfOldestEdit() throws Exception { + try { + EnvironmentEdgeForMemstoreTest edge = new EnvironmentEdgeForMemstoreTest(); + EnvironmentEdgeManager.injectEdge(edge); + long t = memstore.timeOfOldestEdit(); + assertEquals(t, Long.MAX_VALUE); + + // test the case that the timeOfOldestEdit is updated after a KV add + memstore.add(KeyValueTestUtil.create("r", "f", "q", 100, "v")); + t = memstore.timeOfOldestEdit(); + assertTrue(t == 1234); + // The method will also assert + // the value is reset to Long.MAX_VALUE + t = runSnapshot(memstore, true); + + // test the case that the timeOfOldestEdit is updated after a KV delete + memstore.delete(KeyValueTestUtil.create("r", "f", "q", 100, "v")); + t = memstore.timeOfOldestEdit(); + assertTrue(t == 1234); + t = runSnapshot(memstore, true); + + // test the case that the timeOfOldestEdit is updated after a KV upsert + List l = new ArrayList(); + KeyValue kv1 = KeyValueTestUtil.create("r", "f", "q", 100, "v"); + kv1.setSequenceId(100); + l.add(kv1); + memstore.upsert(l, 1000); + t = memstore.timeOfOldestEdit(); + assertTrue(t == 1234); + } finally { + EnvironmentEdgeManager.reset(); + } + } + + private long runSnapshot(final AbstractMemStore hmc, boolean useForce) + throws IOException { + // Save off old state. + long oldHistorySize = hmc.getSnapshot().getSize(); + long prevTimeStamp = hmc.timeOfOldestEdit(); + + hmc.snapshot(); + MemStoreSnapshot snapshot = hmc.snapshot(); + if (useForce) { + // Make some assertions about what just happened. + assertTrue("History size has not increased", oldHistorySize < snapshot.getSize()); + long t = hmc.timeOfOldestEdit(); + assertTrue("Time of oldest edit is not Long.MAX_VALUE", t == Long.MAX_VALUE); + hmc.clearSnapshot(snapshot.getId()); + } else { + long t = hmc.timeOfOldestEdit(); + assertTrue("Time of oldest edit didn't remain the same", t == prevTimeStamp); + } + return prevTimeStamp; + } + + private void isExpectedRowWithoutTimestamps(final int rowIndex, + List kvs) { + int i = 0; + for (Cell kv : kvs) { + byte[] expectedColname = makeQualifier(rowIndex, i++); + assertTrue("Column name", CellUtil.matchingQualifier(kv, expectedColname)); + // Value is column name as bytes. Usually result is + // 100 bytes in size at least. This is the default size + // for BytesWriteable. For comparison, convert bytes to + // String and trim to remove trailing null bytes. + assertTrue("Content", CellUtil.matchingValue(kv, expectedColname)); + } + } + + @Test + public void testPuttingBackChunksAfterFlushing() throws IOException { + byte[] row = Bytes.toBytes("testrow"); + byte[] fam = Bytes.toBytes("testfamily"); + byte[] qf1 = Bytes.toBytes("testqualifier1"); + byte[] qf2 = Bytes.toBytes("testqualifier2"); + byte[] qf3 = Bytes.toBytes("testqualifier3"); + byte[] qf4 = Bytes.toBytes("testqualifier4"); + byte[] qf5 = Bytes.toBytes("testqualifier5"); + byte[] val = Bytes.toBytes("testval"); + + // Setting up memstore + memstore.add(new KeyValue(row, fam, qf1, val)); + memstore.add(new KeyValue(row, fam, qf2, val)); + memstore.add(new KeyValue(row, fam, qf3, val)); + + // Creating a snapshot + MemStoreSnapshot snapshot = memstore.snapshot(); + assertEquals(3, memstore.getSnapshot().getCellsCount()); + + // Adding value to "new" memstore + assertEquals(0, memstore.getActive().getCellsCount()); + memstore.add(new KeyValue(row, fam, qf4, val)); + memstore.add(new KeyValue(row, fam, qf5, val)); + assertEquals(2, memstore.getActive().getCellsCount()); + memstore.clearSnapshot(snapshot.getId()); + + int chunkCount = chunkPool.getPoolSize(); + assertTrue(chunkCount > 0); + + } + + @Test + public void testPuttingBackChunksWithOpeningScanner() + throws IOException { + byte[] row = Bytes.toBytes("testrow"); + byte[] fam = Bytes.toBytes("testfamily"); + byte[] qf1 = Bytes.toBytes("testqualifier1"); + byte[] qf2 = Bytes.toBytes("testqualifier2"); + byte[] qf3 = Bytes.toBytes("testqualifier3"); + byte[] qf4 = Bytes.toBytes("testqualifier4"); + byte[] qf5 = Bytes.toBytes("testqualifier5"); + byte[] qf6 = Bytes.toBytes("testqualifier6"); + byte[] qf7 = Bytes.toBytes("testqualifier7"); + byte[] val = Bytes.toBytes("testval"); + + // Setting up memstore + memstore.add(new KeyValue(row, fam, qf1, val)); + memstore.add(new KeyValue(row, fam, qf2, val)); + memstore.add(new KeyValue(row, fam, qf3, val)); + + // Creating a snapshot + MemStoreSnapshot snapshot = memstore.snapshot(); + assertEquals(3, memstore.getSnapshot().getCellsCount()); + + // Adding value to "new" memstore + assertEquals(0, memstore.getActive().getCellsCount()); + memstore.add(new KeyValue(row, fam, qf4, val)); + memstore.add(new KeyValue(row, fam, qf5, val)); + assertEquals(2, memstore.getActive().getCellsCount()); + + // opening scanner before clear the snapshot + List scanners = memstore.getScanners(0); + // Shouldn't putting back the chunks to pool,since some scanners are opening + // based on their data + memstore.clearSnapshot(snapshot.getId()); + + assertTrue(chunkPool.getPoolSize() == 0); + + // Chunks will be put back to pool after close scanners; + for (KeyValueScanner scanner : scanners) { + scanner.close(); + } + assertTrue(chunkPool.getPoolSize() > 0); + + // clear chunks + chunkPool.clearChunks(); + + // Creating another snapshot + + snapshot = memstore.snapshot(); + // Adding more value + memstore.add(new KeyValue(row, fam, qf6, val)); + memstore.add(new KeyValue(row, fam, qf7, val)); + // opening scanners + scanners = memstore.getScanners(0); + // close scanners before clear the snapshot + for (KeyValueScanner scanner : scanners) { + scanner.close(); + } + // Since no opening scanner, the chunks of snapshot should be put back to + // pool + memstore.clearSnapshot(snapshot.getId()); + assertTrue(chunkPool.getPoolSize() > 0); + } + + @Test + public void testPuttingBackChunksWithOpeningPipelineScanner() + throws IOException { + byte[] row = Bytes.toBytes("testrow"); + byte[] fam = Bytes.toBytes("testfamily"); + byte[] qf1 = Bytes.toBytes("testqualifier1"); + byte[] qf2 = Bytes.toBytes("testqualifier2"); + byte[] qf3 = Bytes.toBytes("testqualifier3"); + byte[] val = Bytes.toBytes("testval"); + + // Setting up memstore + memstore.add(new KeyValue(row, fam, qf1, 1, val)); + memstore.add(new KeyValue(row, fam, qf2, 1, val)); + memstore.add(new KeyValue(row, fam, qf3, 1, val)); + + // Creating a pipeline + ((CompactingMemStore)memstore).disableCompaction(); + ((CompactingMemStore)memstore).flushInMemory(); + + // Adding value to "new" memstore + assertEquals(0, memstore.getActive().getCellsCount()); + memstore.add(new KeyValue(row, fam, qf1, 2, val)); + memstore.add(new KeyValue(row, fam, qf2, 2, val)); + assertEquals(2, memstore.getActive().getCellsCount()); + + // pipeline bucket 2 + ((CompactingMemStore)memstore).flushInMemory(); + // opening scanner before force flushing + List scanners = memstore.getScanners(0); + // Shouldn't putting back the chunks to pool,since some scanners are opening + // based on their data + ((CompactingMemStore)memstore).enableCompaction(); + // trigger compaction + ((CompactingMemStore)memstore).flushInMemory(); + + // Adding value to "new" memstore + assertEquals(0, memstore.getActive().getCellsCount()); + memstore.add(new KeyValue(row, fam, qf3, 3, val)); + memstore.add(new KeyValue(row, fam, qf2, 3, val)); + memstore.add(new KeyValue(row, fam, qf1, 3, val)); + assertEquals(3, memstore.getActive().getCellsCount()); + + while (((CompactingMemStore)memstore).isMemStoreFlushingInMemory()) { + Threads.sleep(10); + } + + assertTrue(chunkPool.getPoolSize() == 0); + + // Chunks will be put back to pool after close scanners; + for (KeyValueScanner scanner : scanners) { + scanner.close(); + } + assertTrue(chunkPool.getPoolSize() > 0); + + // clear chunks + chunkPool.clearChunks(); + + // Creating another snapshot + + MemStoreSnapshot snapshot = memstore.snapshot(); + memstore.clearSnapshot(snapshot.getId()); + + snapshot = memstore.snapshot(); + // Adding more value + memstore.add(new KeyValue(row, fam, qf2, 4, val)); + memstore.add(new KeyValue(row, fam, qf3, 4, val)); + // opening scanners + scanners = memstore.getScanners(0); + // close scanners before clear the snapshot + for (KeyValueScanner scanner : scanners) { + scanner.close(); + } + // Since no opening scanner, the chunks of snapshot should be put back to + // pool + memstore.clearSnapshot(snapshot.getId()); + assertTrue(chunkPool.getPoolSize() > 0); + } + + ////////////////////////////////////////////////////////////////////////////// + // Compaction tests + ////////////////////////////////////////////////////////////////////////////// + @Test + public void testCompaction1Bucket() throws IOException { + + String[] keys1 = { "A", "A", "B", "C" }; //A1, A2, B3, C4 + + // test 1 bucket + addRowsByKeys(memstore, keys1); + assertEquals(704, regionServicesForStores.getGlobalMemstoreTotalSize()); + + long size = memstore.getFlushableSize(); + ((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline and compact + while (((CompactingMemStore)memstore).isMemStoreFlushingInMemory()) { + Threads.sleep(10); + } + assertEquals(0, memstore.getSnapshot().getCellsCount()); + assertEquals(528, regionServicesForStores.getGlobalMemstoreTotalSize()); + + size = memstore.getFlushableSize(); + MemStoreSnapshot snapshot = memstore.snapshot(); // push keys to snapshot + region.addAndGetGlobalMemstoreSize(-size); // simulate flusher + ImmutableSegment s = memstore.getSnapshot(); + assertEquals(3, s.getCellsCount()); + assertEquals(0, regionServicesForStores.getGlobalMemstoreTotalSize()); + + memstore.clearSnapshot(snapshot.getId()); + } + + @Test + public void testCompaction2Buckets() throws IOException { + + String[] keys1 = { "A", "A", "B", "C" }; + String[] keys2 = { "A", "B", "D" }; + + addRowsByKeys(memstore, keys1); + assertEquals(704, regionServicesForStores.getGlobalMemstoreTotalSize()); + + long size = memstore.getFlushableSize(); + ((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline and compact + while (((CompactingMemStore)memstore).isMemStoreFlushingInMemory()) { + Threads.sleep(1000); + } + assertEquals(0, memstore.getSnapshot().getCellsCount()); + assertEquals(528, regionServicesForStores.getGlobalMemstoreTotalSize()); + + addRowsByKeys(memstore, keys2); + assertEquals(1056, regionServicesForStores.getGlobalMemstoreTotalSize()); + + size = memstore.getFlushableSize(); + ((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline and compact + while (((CompactingMemStore)memstore).isMemStoreFlushingInMemory()) { + Threads.sleep(10); + } + assertEquals(0, memstore.getSnapshot().getCellsCount()); + assertEquals(704, regionServicesForStores.getGlobalMemstoreTotalSize()); + + size = memstore.getFlushableSize(); + MemStoreSnapshot snapshot = memstore.snapshot(); // push keys to snapshot + region.addAndGetGlobalMemstoreSize(-size); // simulate flusher + ImmutableSegment s = memstore.getSnapshot(); + assertEquals(4, s.getCellsCount()); + assertEquals(0, regionServicesForStores.getGlobalMemstoreTotalSize()); + + memstore.clearSnapshot(snapshot.getId()); + } + + @Test + public void testCompaction3Buckets() throws IOException { + + String[] keys1 = { "A", "A", "B", "C" }; + String[] keys2 = { "A", "B", "D" }; + String[] keys3 = { "D", "B", "B" }; + + addRowsByKeys(memstore, keys1); + assertEquals(704, region.getMemstoreSize()); + + long size = memstore.getFlushableSize(); + ((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline and compact + + String tstStr = "\n\nFlushable size after first flush in memory:" + size + + ". Is MemmStore in compaction?:" + ((CompactingMemStore)memstore).isMemStoreFlushingInMemory(); + while (((CompactingMemStore)memstore).isMemStoreFlushingInMemory()) { + Threads.sleep(10); + } + assertEquals(0, memstore.getSnapshot().getCellsCount()); + assertEquals(528, regionServicesForStores.getGlobalMemstoreTotalSize()); + + addRowsByKeys(memstore, keys2); + + tstStr += " After adding second part of the keys. Memstore size: " + + region.getMemstoreSize() + ", Memstore Total Size: " + + regionServicesForStores.getGlobalMemstoreTotalSize() + "\n\n"; + + assertEquals(1056, regionServicesForStores.getGlobalMemstoreTotalSize()); + + ((CompactingMemStore)memstore).disableCompaction(); + size = memstore.getFlushableSize(); + ((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline without compaction + assertEquals(0, memstore.getSnapshot().getCellsCount()); + assertEquals(1056, regionServicesForStores.getGlobalMemstoreTotalSize()); + + addRowsByKeys(memstore, keys3); + assertEquals(1584, regionServicesForStores.getGlobalMemstoreTotalSize()); + + ((CompactingMemStore)memstore).enableCompaction(); + size = memstore.getFlushableSize(); + ((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline and compact + while (((CompactingMemStore)memstore).isMemStoreFlushingInMemory()) { + Threads.sleep(10); + } + assertEquals(0, memstore.getSnapshot().getCellsCount()); + assertEquals(704, regionServicesForStores.getGlobalMemstoreTotalSize()); + + size = memstore.getFlushableSize(); + MemStoreSnapshot snapshot = memstore.snapshot(); // push keys to snapshot + region.addAndGetGlobalMemstoreSize(-size); // simulate flusher + ImmutableSegment s = memstore.getSnapshot(); + assertEquals(4, s.getCellsCount()); + assertEquals(0, regionServicesForStores.getGlobalMemstoreTotalSize()); + + memstore.clearSnapshot(snapshot.getId()); + + //assertTrue(tstStr, false); + } + + private void addRowsByKeys(final AbstractMemStore hmc, String[] keys) { + byte[] fam = Bytes.toBytes("testfamily"); + byte[] qf = Bytes.toBytes("testqualifier"); + for (int i = 0; i < keys.length; i++) { + long timestamp = System.currentTimeMillis(); + Threads.sleep(1); // to make sure each kv gets a different ts + byte[] row = Bytes.toBytes(keys[i]); + byte[] val = Bytes.toBytes(keys[i] + i); + KeyValue kv = new KeyValue(row, fam, qf, timestamp, val); + hmc.add(kv); + LOG.debug("added kv: " + kv.getKeyString() + ", timestamp" + kv.getTimestamp()); + long size = AbstractMemStore.heapSizeChange(kv, true); + regionServicesForStores.addAndGetGlobalMemstoreSize(size); + } + } + + private class EnvironmentEdgeForMemstoreTest implements EnvironmentEdge { + long t = 1234; + + @Override + public long currentTime() { + return t; + } + public void setCurrentTimeMillis(long t) { + this.t = t; + } + } + +} 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 5e6007d..0c4029d 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 @@ -21,11 +21,11 @@ package org.apache.hadoop.hbase.regionserver; import com.google.common.base.Joiner; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import junit.framework.TestCase; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellUtil; @@ -48,7 +48,17 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdge; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.wal.WALFactory; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertNotNull; import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; +import org.junit.rules.TestRule; import java.io.IOException; import java.lang.management.ManagementFactory; @@ -61,27 +71,38 @@ import java.util.concurrent.atomic.AtomicReference; /** memstore test case */ @Category({RegionServerTests.class, MediumTests.class}) -public class TestDefaultMemStore extends TestCase { +public class TestDefaultMemStore { private static final Log LOG = LogFactory.getLog(TestDefaultMemStore.class); - private DefaultMemStore memstore; - private static final int ROW_COUNT = 10; - private static final int QUALIFIER_COUNT = ROW_COUNT; - private static final byte [] FAMILY = Bytes.toBytes("column"); - private MultiVersionConcurrencyControl mvcc; - private AtomicLong startSeqNum = new AtomicLong(0); - - @Override + @Rule public TestName name = new TestName(); + @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). + withLookingForStuckThread(true).build(); + protected AbstractMemStore memstore; + protected static final int ROW_COUNT = 10; + protected static final int QUALIFIER_COUNT = ROW_COUNT; + protected static final byte[] FAMILY = Bytes.toBytes("column"); + protected MultiVersionConcurrencyControl mvcc; + protected AtomicLong startSeqNum = new AtomicLong(0); + + private String getName() { + return this.name.getMethodName(); + } + + @Before public void setUp() throws Exception { - super.setUp(); - this.mvcc = new MultiVersionConcurrencyControl(); + internalSetUp(); this.memstore = new DefaultMemStore(); } + protected void internalSetUp() throws Exception { + this.mvcc = new MultiVersionConcurrencyControl(); + } + + @Test public void testPutSameKey() { - byte [] bytes = Bytes.toBytes(getName()); + byte[] bytes = Bytes.toBytes(getName()); KeyValue kv = new KeyValue(bytes, bytes, bytes, bytes); this.memstore.add(kv); - byte [] other = Bytes.toBytes("somethingelse"); + byte[] other = Bytes.toBytes("somethingelse"); KeyValue samekey = new KeyValue(bytes, bytes, bytes, other); this.memstore.add(samekey); Cell found = this.memstore.getActive().first(); @@ -93,6 +114,7 @@ public class TestDefaultMemStore extends TestCase { * Test memstore snapshot happening while scanning. * @throws IOException */ + @Test public void testScanAcrossSnapshot() throws IOException { int rowCount = addRows(this.memstore); List memstorescanners = this.memstore.getScanners(0); @@ -180,6 +202,7 @@ public class TestDefaultMemStore extends TestCase { * @throws IOException * @throws CloneNotSupportedException */ + @Test public void testScanAcrossSnapshot2() throws IOException, CloneNotSupportedException { // we are going to the scanning across snapshot with two kvs // kv1 should always be returned before kv2 @@ -209,7 +232,7 @@ public class TestDefaultMemStore extends TestCase { verifyScanAcrossSnapshot2(kv1, kv2); } - private void verifyScanAcrossSnapshot2(KeyValue kv1, KeyValue kv2) + protected void verifyScanAcrossSnapshot2(KeyValue kv1, KeyValue kv2) throws IOException { List memstorescanners = this.memstore.getScanners(mvcc.getReadPoint()); assertEquals(1, memstorescanners.size()); @@ -220,7 +243,7 @@ public class TestDefaultMemStore extends TestCase { assertNull(scanner.next()); } - private void assertScannerResults(KeyValueScanner scanner, KeyValue[] expected) + protected void assertScannerResults(KeyValueScanner scanner, KeyValue[] expected) throws IOException { scanner.seek(KeyValueUtil.createFirstOnRow(new byte[]{})); List returned = Lists.newArrayList(); @@ -238,6 +261,7 @@ public class TestDefaultMemStore extends TestCase { assertNull(scanner.peek()); } + @Test public void testMemstoreConcurrentControl() throws IOException { final byte[] row = Bytes.toBytes(1); final byte[] f = Bytes.toBytes("family"); @@ -280,6 +304,7 @@ public class TestDefaultMemStore extends TestCase { * the same timestamp, we still need to provide consistent reads * for the same scanner. */ + @Test public void testMemstoreEditsVisibilityWithSameKey() throws IOException { final byte[] row = Bytes.toBytes(1); final byte[] f = Bytes.toBytes("family"); @@ -334,6 +359,7 @@ public class TestDefaultMemStore extends TestCase { * the same timestamp, we still need to provide consistent reads * for the same scanner. */ + @Test public void testMemstoreDeletesVisibilityWithSameKey() throws IOException { final byte[] row = Bytes.toBytes(1); final byte[] f = Bytes.toBytes("family"); @@ -438,6 +464,7 @@ public class TestDefaultMemStore extends TestCase { } } + @Test public void testReadOwnWritesUnderConcurrency() throws Throwable { int NUM_THREADS = 8; @@ -463,6 +490,7 @@ public class TestDefaultMemStore extends TestCase { * Test memstore snapshots * @throws IOException */ + @Test public void testSnapshotting() throws IOException { final int snapshotCount = 5; // Add some rows, run a snapshot. Do it a few times. @@ -473,6 +501,7 @@ public class TestDefaultMemStore extends TestCase { } } + @Test public void testMultipleVersionsSimple() throws Exception { DefaultMemStore m = new DefaultMemStore(new Configuration(), CellComparator.COMPARATOR); byte [] row = Bytes.toBytes("testRow"); @@ -500,53 +529,56 @@ public class TestDefaultMemStore extends TestCase { /** Test getNextRow from memstore * @throws InterruptedException */ + @Test public void testGetNextRow() throws Exception { addRows(this.memstore); // Add more versions to make it a little more interesting. Thread.sleep(1); addRows(this.memstore); - Cell closestToEmpty = this.memstore.getNextRow(KeyValue.LOWESTKEY); + Cell closestToEmpty = ((DefaultMemStore) this.memstore).getNextRow(KeyValue.LOWESTKEY); assertTrue(CellComparator.COMPARATOR.compareRows(closestToEmpty, - new KeyValue(Bytes.toBytes(0), System.currentTimeMillis())) == 0); + new KeyValue(Bytes.toBytes(0), System.currentTimeMillis())) == 0); for (int i = 0; i < ROW_COUNT; i++) { - Cell nr = this.memstore.getNextRow(new KeyValue(Bytes.toBytes(i), - System.currentTimeMillis())); + Cell nr = ((DefaultMemStore) this.memstore).getNextRow(new KeyValue(Bytes.toBytes(i), + System.currentTimeMillis())); if (i + 1 == ROW_COUNT) { assertEquals(nr, null); } else { assertTrue(CellComparator.COMPARATOR.compareRows(nr, - new KeyValue(Bytes.toBytes(i + 1), System.currentTimeMillis())) == 0); + new KeyValue(Bytes.toBytes(i + 1), System.currentTimeMillis())) == 0); } } //starting from each row, validate results should contain the starting row Configuration conf = HBaseConfiguration.create(); for (int startRowId = 0; startRowId < ROW_COUNT; startRowId++) { ScanInfo scanInfo = new ScanInfo(conf, FAMILY, 0, 1, Integer.MAX_VALUE, - KeepDeletedCells.FALSE, 0, this.memstore.getComparator()); + KeepDeletedCells.FALSE, 0, this.memstore.getComparator()); ScanType scanType = ScanType.USER_SCAN; - InternalScanner scanner = new StoreScanner(new Scan( + try (InternalScanner scanner = new StoreScanner(new Scan( Bytes.toBytes(startRowId)), scanInfo, scanType, null, - memstore.getScanners(0)); - List results = new ArrayList(); - for (int i = 0; scanner.next(results); i++) { - int rowId = startRowId + i; - Cell left = results.get(0); - byte[] row1 = Bytes.toBytes(rowId); - assertTrue( - "Row name", - CellComparator.COMPARATOR.compareRows(left, row1, 0, row1.length) == 0); - assertEquals("Count of columns", QUALIFIER_COUNT, results.size()); - List row = new ArrayList(); - for (Cell kv : results) { - row.add(kv); + memstore.getScanners(0))) { + List results = new ArrayList(); + for (int i = 0; scanner.next(results); i++) { + int rowId = startRowId + i; + Cell left = results.get(0); + byte[] row1 = Bytes.toBytes(rowId); + assertTrue( + "Row name", + CellComparator.COMPARATOR.compareRows(left, row1, 0, row1.length) == 0); + assertEquals("Count of columns", QUALIFIER_COUNT, results.size()); + List row = new ArrayList(); + for (Cell kv : results) { + row.add(kv); + } + isExpectedRowWithoutTimestamps(rowId, row); + // Clear out set. Otherwise row results accumulate. + results.clear(); } - isExpectedRowWithoutTimestamps(rowId, row); - // Clear out set. Otherwise row results accumulate. - results.clear(); } } } + @Test public void testGet_memstoreAndSnapShot() throws IOException { byte [] row = Bytes.toBytes("testrow"); byte [] fam = Bytes.toBytes("testfamily"); @@ -558,9 +590,9 @@ public class TestDefaultMemStore extends TestCase { byte [] val = Bytes.toBytes("testval"); //Setting up memstore - memstore.add(new KeyValue(row, fam ,qf1, val)); - memstore.add(new KeyValue(row, fam ,qf2, val)); - memstore.add(new KeyValue(row, fam ,qf3, val)); + memstore.add(new KeyValue(row, fam, qf1, val)); + memstore.add(new KeyValue(row, fam, qf2, val)); + memstore.add(new KeyValue(row, fam, qf3, val)); //Creating a snapshot memstore.snapshot(); assertEquals(3, memstore.getSnapshot().getCellsCount()); @@ -574,6 +606,7 @@ public class TestDefaultMemStore extends TestCase { ////////////////////////////////////////////////////////////////////////////// // Delete tests ////////////////////////////////////////////////////////////////////////////// + @Test public void testGetWithDelete() throws IOException { byte [] row = Bytes.toBytes("testrow"); byte [] fam = Bytes.toBytes("testfamily"); @@ -584,7 +617,7 @@ public class TestDefaultMemStore extends TestCase { KeyValue put1 = new KeyValue(row, fam, qf1, ts1, val); long ts2 = ts1 + 1; KeyValue put2 = new KeyValue(row, fam, qf1, ts2, val); - long ts3 = ts2 +1; + long ts3 = ts2 + 1; KeyValue put3 = new KeyValue(row, fam, qf1, ts3, val); memstore.add(put1); memstore.add(put2); @@ -608,6 +641,7 @@ public class TestDefaultMemStore extends TestCase { } } + @Test public void testGetWithDeleteColumn() throws IOException { byte [] row = Bytes.toBytes("testrow"); byte [] fam = Bytes.toBytes("testfamily"); @@ -618,7 +652,7 @@ public class TestDefaultMemStore extends TestCase { KeyValue put1 = new KeyValue(row, fam, qf1, ts1, val); long ts2 = ts1 + 1; KeyValue put2 = new KeyValue(row, fam, qf1, ts2, val); - long ts3 = ts2 +1; + long ts3 = ts2 + 1; KeyValue put3 = new KeyValue(row, fam, qf1, ts3, val); memstore.add(put1); memstore.add(put2); @@ -636,15 +670,14 @@ public class TestDefaultMemStore extends TestCase { expected.add(put2); expected.add(put1); - assertEquals(4, memstore.getActive().getCellsCount()); int i = 0; - for (Cell cell: memstore.getActive().getCellSet()) { + for (Cell cell : memstore.getActive().getCellSet()) { assertEquals(expected.get(i++), cell); } } - + @Test public void testGetWithDeleteFamily() throws IOException { byte [] row = Bytes.toBytes("testrow"); byte [] fam = Bytes.toBytes("testfamily"); @@ -675,15 +708,14 @@ public class TestDefaultMemStore extends TestCase { expected.add(put4); expected.add(put3); - - assertEquals(5, memstore.getActive().getCellsCount()); int i = 0; - for (Cell cell: memstore.getActive().getCellSet()) { + for (Cell cell : memstore.getActive().getCellSet()) { assertEquals(expected.get(i++), cell); } } + @Test public void testKeepDeleteInmemstore() { byte [] row = Bytes.toBytes("testrow"); byte [] fam = Bytes.toBytes("testfamily"); @@ -697,6 +729,7 @@ public class TestDefaultMemStore extends TestCase { assertEquals(delete, memstore.getActive().first()); } + @Test public void testRetainsDeleteVersion() throws IOException { // add a put to memstore memstore.add(KeyValueTestUtil.create("row1", "fam", "a", 100, "dont-care")); @@ -709,6 +742,8 @@ public class TestDefaultMemStore extends TestCase { assertEquals(2, memstore.getActive().getCellsCount()); assertEquals(delete, memstore.getActive().first()); } + + @Test public void testRetainsDeleteColumn() throws IOException { // add a put to memstore memstore.add(KeyValueTestUtil.create("row1", "fam", "a", 100, "dont-care")); @@ -721,6 +756,8 @@ public class TestDefaultMemStore extends TestCase { assertEquals(2, memstore.getActive().getCellsCount()); assertEquals(delete, memstore.getActive().first()); } + + @Test public void testRetainsDeleteFamily() throws IOException { // add a put to memstore memstore.add(KeyValueTestUtil.create("row1", "fam", "a", 100, "dont-care")); @@ -751,6 +788,7 @@ public class TestDefaultMemStore extends TestCase { * This causes OOME pretty quickly if we use MSLAB for upsert * since each 2M chunk is held onto by a single reference. */ + @Test public void testUpsertMSLAB() throws Exception { Configuration conf = HBaseConfiguration.create(); conf.setBoolean(SegmentFactory.USEMSLAB_KEY, true); @@ -793,6 +831,7 @@ public class TestDefaultMemStore extends TestCase { * as older keyvalues are deleted from the memstore. * @throws Exception */ + @Test public void testUpsertMemstoreSize() throws Exception { Configuration conf = HBaseConfiguration.create(); memstore = new DefaultMemStore(conf, CellComparator.COMPARATOR); @@ -832,6 +871,7 @@ public class TestDefaultMemStore extends TestCase { * various edit operations in memstore. * @throws Exception */ + @Test public void testUpdateToTimeOfOldestEdit() throws Exception { try { EnvironmentEdgeForMemstoreTest edge = new EnvironmentEdgeForMemstoreTest(); @@ -874,6 +914,7 @@ public class TestDefaultMemStore extends TestCase { * false. * @throws Exception */ + @Test public void testShouldFlush() throws Exception { Configuration conf = new Configuration(); conf.setInt(HRegion.MEMSTORE_PERIODIC_FLUSH_INTERVAL, 1000); @@ -883,7 +924,7 @@ public class TestDefaultMemStore extends TestCase { checkShouldFlush(conf, false); } - private void checkShouldFlush(Configuration conf, boolean expected) throws Exception { + protected void checkShouldFlush(Configuration conf, boolean expected) throws Exception { try { EnvironmentEdgeForMemstoreTest edge = new EnvironmentEdgeForMemstoreTest(); EnvironmentEdgeManager.injectEdge(edge); @@ -898,7 +939,7 @@ public class TestDefaultMemStore extends TestCase { s.add(KeyValueTestUtil.create("r", "f", "q", 100, "v")); edge.setCurrentTimeMillis(1234 + 100); StringBuffer sb = new StringBuffer(); - assertTrue(region.shouldFlush(sb) == false); + assertTrue(!region.shouldFlush(sb)); edge.setCurrentTimeMillis(1234 + 10000); assertTrue(region.shouldFlush(sb) == expected); } finally { @@ -906,6 +947,7 @@ public class TestDefaultMemStore extends TestCase { } } + @Test public void testShouldFlushMeta() throws Exception { // write an edit in the META and ensure the shouldFlush (that the periodic memstore // flusher invokes) returns true after SYSTEM_CACHE_FLUSH_INTERVAL (even though @@ -954,7 +996,7 @@ public class TestDefaultMemStore extends TestCase { * @return How many rows we added. * @throws IOException */ - private int addRows(final MemStore hmc) { + protected int addRows(final AbstractMemStore hmc) { return addRows(hmc, HConstants.LATEST_TIMESTAMP); } @@ -964,10 +1006,10 @@ public class TestDefaultMemStore extends TestCase { * @return How many rows we added. * @throws IOException */ - private int addRows(final MemStore hmc, final long ts) { + protected int addRows(final MemStore hmc, final long ts) { for (int i = 0; i < ROW_COUNT; i++) { - long timestamp = ts == HConstants.LATEST_TIMESTAMP? - System.currentTimeMillis(): ts; + long timestamp = ts == HConstants.LATEST_TIMESTAMP ? + System.currentTimeMillis() : ts; for (int ii = 0; ii < QUALIFIER_COUNT; ii++) { byte [] row = Bytes.toBytes(i); byte [] qf = makeQualifier(i, ii); @@ -977,7 +1019,7 @@ public class TestDefaultMemStore extends TestCase { return ROW_COUNT; } - private long runSnapshot(final DefaultMemStore hmc) throws UnexpectedStateException { + private long runSnapshot(final AbstractMemStore hmc) throws UnexpectedStateException { // Save off old state. int oldHistorySize = hmc.getSnapshot().getCellsCount(); MemStoreSnapshot snapshot = hmc.snapshot(); @@ -993,7 +1035,7 @@ public class TestDefaultMemStore extends TestCase { private void isExpectedRowWithoutTimestamps(final int rowIndex, List kvs) { int i = 0; - for (Cell kv: kvs) { + for (Cell kv : kvs) { byte[] expectedColname = makeQualifier(rowIndex, i++); assertTrue("Column name", CellUtil.matchingQualifier(kv, expectedColname)); // Value is column name as bytes. Usually result is @@ -1023,7 +1065,6 @@ public class TestDefaultMemStore extends TestCase { } } - static void doScan(MemStore ms, int iteration) throws IOException { long nanos = System.nanoTime(); KeyValueScanner s = ms.getScanners(0).get(0); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java index 713c8e5..4a14124 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -22,6 +22,29 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.protobuf.ByteString; +import java.io.IOException; +import java.io.InterruptedIOException; +import java.security.PrivilegedExceptionAction; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.TreeMap; +import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + import org.apache.commons.lang.RandomStringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -132,29 +155,6 @@ import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.io.IOException; -import java.io.InterruptedIOException; -import java.security.PrivilegedExceptionAction; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.TreeMap; -import java.util.UUID; -import java.util.concurrent.Callable; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; - import static org.apache.hadoop.hbase.HBaseTestingUtility.COLUMNS; import static org.apache.hadoop.hbase.HBaseTestingUtility.FIRST_CHAR; import static org.apache.hadoop.hbase.HBaseTestingUtility.LAST_CHAR; @@ -198,7 +198,7 @@ public class TestHRegion { HRegion region = null; // Do not run unit tests in parallel (? Why not? It don't work? Why not? St.Ack) - private static HBaseTestingUtility TEST_UTIL; + protected static HBaseTestingUtility TEST_UTIL; public static Configuration CONF ; private String dir; private static FileSystem FILESYSTEM; @@ -2448,7 +2448,7 @@ public class TestHRegion { // extract the key values out the memstore: // This is kinda hacky, but better than nothing... long now = System.currentTimeMillis(); - DefaultMemStore memstore = (DefaultMemStore) ((HStore) region.getStore(fam1)).memstore; + AbstractMemStore memstore = (AbstractMemStore)((HStore) region.getStore(fam1)).memstore; Cell firstCell = memstore.getActive().first(); assertTrue(firstCell.getTimestamp() <= now); now = firstCell.getTimestamp(); @@ -5141,7 +5141,7 @@ public class TestHRegion { * @return A region on which you must call * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when done. */ - private static HRegion initHRegion(TableName tableName, String callingMethod, Configuration conf, + protected HRegion initHRegion(TableName tableName, String callingMethod, Configuration conf, byte[]... families) throws IOException { return initHRegion(tableName, null, null, callingMethod, conf, false, families); } @@ -5150,12 +5150,12 @@ public class TestHRegion { * @return A region on which you must call * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when done. */ - private static HRegion initHRegion(TableName tableName, String callingMethod, Configuration conf, + protected HRegion initHRegion(TableName tableName, String callingMethod, Configuration conf, boolean isReadOnly, byte[]... families) throws IOException { return initHRegion(tableName, null, null, callingMethod, conf, isReadOnly, families); } - public static HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey, + protected HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey, String callingMethod, Configuration conf, boolean isReadOnly, byte[]... families) throws IOException { Path logDir = TEST_UTIL.getDataTestDirOnTestFS(callingMethod + ".log"); @@ -5169,7 +5169,7 @@ public class TestHRegion { * @return A region on which you must call * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when done. */ - public static HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey, + public HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey, String callingMethod, Configuration conf, boolean isReadOnly, Durability durability, WAL wal, byte[]... families) throws IOException { return TEST_UTIL.createLocalHRegion(tableName, startKey, stopKey, @@ -6600,7 +6600,7 @@ public class TestHRegion { qual2, 0, qual2.length)); } - static HRegion initHRegion(TableName tableName, String callingMethod, + HRegion initHRegion(TableName tableName, String callingMethod, byte[]... families) throws IOException { return initHRegion(tableName, callingMethod, HBaseConfiguration.create(), families); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionWithInMemoryFlush.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionWithInMemoryFlush.java new file mode 100644 index 0000000..e9c6b6f --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionWithInMemoryFlush.java @@ -0,0 +1,61 @@ +/** + * + * 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; + +import java.io.IOException; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Durability; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests; +import org.apache.hadoop.hbase.wal.WAL; +import org.junit.experimental.categories.Category; + +/** + * A test similar to TestHRegion, but with in-memory flush families. + * Also checks wal truncation after in-memory compaction. + */ +@Category({VerySlowRegionServerTests.class, LargeTests.class}) +@SuppressWarnings("deprecation") +public class TestHRegionWithInMemoryFlush extends TestHRegion{ + // Do not spin up clusters in here. If you need to spin up a cluster, do it + // over in TestHRegionOnCluster. + private static final Log LOG = LogFactory.getLog(TestHRegionWithInMemoryFlush.class); + + /** + * @return A region on which you must call + * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when done. + */ + @Override + public HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey, + String callingMethod, Configuration conf, boolean isReadOnly, Durability durability, + WAL wal, byte[]... families) throws IOException { + boolean[] inMemory = new boolean[families.length]; + for(int i = 0; i < inMemory.length; i++) { + inMemory[i] = true; + } + return TEST_UTIL.createLocalHRegionWithInMemoryFlags(tableName, startKey, stopKey, + isReadOnly, durability, wal, inMemory, families); + } + +} + diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java index 624f4a5..ae98f25 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java @@ -17,16 +17,7 @@ */ package org.apache.hadoop.hbase.regionserver; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; - -import java.io.IOException; -import java.util.Arrays; -import java.util.List; -import java.util.Random; - +import com.google.common.hash.Hashing; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -59,7 +50,15 @@ import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; -import com.google.common.hash.Hashing; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Random; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; /** * This test verifies the correctness of the Per Column Family flushing strategy @@ -127,7 +126,7 @@ public class TestPerColumnFamilyFlush { // Set up the configuration Configuration conf = HBaseConfiguration.create(); conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 200 * 1024); - conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushLargeStoresPolicy.class.getName()); + conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushAllLargeStoresPolicy.class.getName()); conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN, 100 * 1024); // Intialize the region @@ -336,7 +335,7 @@ public class TestPerColumnFamilyFlush { Configuration conf = TEST_UTIL.getConfiguration(); conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 20000); // Carefully chosen limits so that the memstore just flushes when we're done - conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushLargeStoresPolicy.class.getName()); + conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushAllLargeStoresPolicy.class.getName()); conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN, 10000); final int numRegionServers = 4; try { @@ -450,7 +449,7 @@ public class TestPerColumnFamilyFlush { TableName tableName = TableName.valueOf("testFlushingWhenLogRolling"); Configuration conf = TEST_UTIL.getConfiguration(); conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 128 * 1024 * 1024); - conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushLargeStoresPolicy.class.getName()); + conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushAllLargeStoresPolicy.class.getName()); long cfFlushSizeLowerBound = 2048; conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN, cfFlushSizeLowerBound); @@ -607,7 +606,7 @@ public class TestPerColumnFamilyFlush { } LOG.info("==============Test with selective flush enabled==============="); - conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushLargeStoresPolicy.class.getName()); + conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushAllLargeStoresPolicy.class.getName()); // default value of per-cf flush lower bound is too big, set to a small enough value conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN, 0); try { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java new file mode 100644 index 0000000..6291d99 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java @@ -0,0 +1,566 @@ +/** + * 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; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.regionserver.wal.FSHLog; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.JVMClusterUtil; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.util.Threads; +import org.apache.hadoop.hbase.wal.WAL; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** + * This test verifies the correctness of the Per Column Family flushing strategy + * when part of the memstores are compacted memstores + */ +@Category({ RegionServerTests.class, LargeTests.class }) +public class TestWalAndCompactingMemStoreFlush { + + private static final Log LOG = LogFactory.getLog(TestWalAndCompactingMemStoreFlush.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private static final Path DIR = TEST_UTIL.getDataTestDir("TestHRegion"); + public static final TableName TABLENAME = TableName.valueOf("TestWalAndCompactingMemStoreFlush", + "t1"); + + public static final byte[][] FAMILIES = { Bytes.toBytes("f1"), Bytes.toBytes("f2"), + Bytes.toBytes("f3"), Bytes.toBytes("f4"), Bytes.toBytes("f5") }; + + public static final byte[] FAMILY1 = FAMILIES[0]; + public static final byte[] FAMILY2 = FAMILIES[1]; + public static final byte[] FAMILY3 = FAMILIES[2]; + + + + private HRegion initHRegion(String callingMethod, Configuration conf) throws IOException { + int i=0; + HTableDescriptor htd = new HTableDescriptor(TABLENAME); + for (byte[] family : FAMILIES) { + HColumnDescriptor hcd = new HColumnDescriptor(family); + // even column families are going to have compacted memstore + if(i%2 == 0) hcd.setMemStoreClass("org.apache.hadoop.hbase.regionserver" + + ".CompactingMemStore"); + htd.addFamily(hcd); + i++; + } + + HRegionInfo info = new HRegionInfo(TABLENAME, null, null, false); + Path path = new Path(DIR, callingMethod); + return HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd); + } + + + + // A helper function to create puts. + private Put createPut(int familyNum, int putNum) { + byte[] qf = Bytes.toBytes("q" + familyNum); + byte[] row = Bytes.toBytes("row" + familyNum + "-" + putNum); + byte[] val = Bytes.toBytes("val" + familyNum + "-" + putNum); + Put p = new Put(row); + p.addColumn(FAMILIES[familyNum - 1], qf, val); + return p; + } + + + // A helper function to create double puts, so something can be compacted later. + private Put createDoublePut(int familyNum, int putNum) { + byte[] qf = Bytes.toBytes("q" + familyNum); + byte[] row = Bytes.toBytes("row" + familyNum + "-" + putNum); + byte[] val = Bytes.toBytes("val" + familyNum + "-" + putNum); + Put p = new Put(row); + // add twice with different timestamps + p.addColumn(FAMILIES[familyNum - 1], qf, 10, val); + p.addColumn(FAMILIES[familyNum - 1], qf, 20, val); + return p; + } + + + // A helper function to create gets. + private Get createGet(int familyNum, int putNum) { + byte[] row = Bytes.toBytes("row" + familyNum + "-" + putNum); + return new Get(row); + } + + + + + // A helper function to verify edits. + void verifyEdit(int familyNum, int putNum, Table table) throws IOException { + Result r = table.get(createGet(familyNum, putNum)); + byte[] family = FAMILIES[familyNum - 1]; + byte[] qf = Bytes.toBytes("q" + familyNum); + byte[] val = Bytes.toBytes("val" + familyNum + "-" + putNum); + assertNotNull(("Missing Put#" + putNum + " for CF# " + familyNum), r.getFamilyMap(family)); + assertNotNull(("Missing Put#" + putNum + " for CF# " + familyNum), + r.getFamilyMap(family).get(qf)); + assertTrue(("Incorrect value for Put#" + putNum + " for CF# " + familyNum), + Arrays.equals(r.getFamilyMap(family).get(qf), val)); + } + + + + + + @Test(timeout = 180000) + public void testSelectiveFlushWhenEnabled() throws IOException { + + // Set up the configuration + Configuration conf = HBaseConfiguration.create(); + conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 600 * 1024); + conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushNonSloppyStoresFirstPolicy.class + .getName()); + conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN, 300 * + 1024); + conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.5); + + // Intialize the region + Region region = initHRegion("testSelectiveFlushWhenEnabled", conf); + + // Add 1200 entries for CF1, 100 for CF2 and 50 for CF3 + for (int i = 1; i <= 1200; i++) { + region.put(createPut(1, i)); // compacted memstore + + if (i <= 100) { + region.put(createPut(2, i)); + if (i <= 50) { + region.put(createDoublePut(3, i)); // subject for in-memory compaction + } + } + } + + // Now add more puts for CF2, so that we only flush CF2 to disk + for (int i = 100; i < 2000; i++) { + region.put(createPut(2, i)); + } + + long totalMemstoreSize = region.getMemstoreSize(); + + // Find the smallest LSNs for edits wrt to each CF. + long smallestSeqCF1PhaseI = region.getOldestSeqIdOfStore(FAMILY1); + long smallestSeqCF2PhaseI = region.getOldestSeqIdOfStore(FAMILY2); + long smallestSeqCF3PhaseI = region.getOldestSeqIdOfStore(FAMILY3); + + // Find the sizes of the memstores of each CF. + long cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getMemStoreSize(); + long cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getMemStoreSize(); + long cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getMemStoreSize(); + + // Get the overall smallest LSN in the region's memstores. + long smallestSeqInRegionCurrentMemstorePhaseI = getWAL(region) + .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); + + String s = "\n\n----------------------------------\n" + + "Upon initial insert and before any flush, size of CF1 is:" + + cf1MemstoreSizePhaseI + ", is CF1 compacted memstore?:" + + region.getStore(FAMILY1).getMemStore().isSloppy() + ". Size of CF2 is:" + + cf2MemstoreSizePhaseI + ", is CF2 compacted memstore?:" + + region.getStore(FAMILY2).getMemStore().isSloppy() + ". Size of CF3 is:" + + cf3MemstoreSizePhaseI + ", is CF3 compacted memstore?:" + + region.getStore(FAMILY3).getMemStore().isSloppy() + "\n"; + + // The overall smallest LSN in the region's memstores should be the same as + // the LSN of the smallest edit in CF1 + assertEquals(smallestSeqCF1PhaseI, smallestSeqInRegionCurrentMemstorePhaseI); + + // Some other sanity checks. + assertTrue(smallestSeqCF1PhaseI < smallestSeqCF2PhaseI); + assertTrue(smallestSeqCF2PhaseI < smallestSeqCF3PhaseI); + assertTrue(cf1MemstoreSizePhaseI > 0); + assertTrue(cf2MemstoreSizePhaseI > 0); + assertTrue(cf3MemstoreSizePhaseI > 0); + + // The total memstore size should be the same as the sum of the sizes of + // memstores of CF1, CF2 and CF3. + String msg = "totalMemstoreSize="+totalMemstoreSize + + " DefaultMemStore.DEEP_OVERHEAD="+DefaultMemStore.DEEP_OVERHEAD + + " DEEP_OVERHEAD_PER_PIPELINE_ITEM="+CompactingMemStore.DEEP_OVERHEAD_PER_PIPELINE_ITEM + + " cf1MemstoreSizePhaseI="+cf1MemstoreSizePhaseI + + " cf2MemstoreSizePhaseI="+cf2MemstoreSizePhaseI + + " cf3MemstoreSizePhaseI="+cf3MemstoreSizePhaseI ; + assertEquals(msg,totalMemstoreSize + 3 * DefaultMemStore.DEEP_OVERHEAD, + cf1MemstoreSizePhaseI + cf2MemstoreSizePhaseI + cf3MemstoreSizePhaseI); + + // Flush!!!!!!!!!!!!!!!!!!!!!! + // We have big compacting memstore CF1 and two small memstores: + // CF2 (not compacted) and CF3 (compacting) + // All together they are above the flush size lower bound. + // Since CF1 and CF3 should be flushed to memory (not to disk), + // CF2 is going to be flushed to disk. + // CF1 - nothing to compact, CF3 - should be twice compacted + ((CompactingMemStore) region.getStore(FAMILY1).getMemStore()).flushInMemory(); + ((CompactingMemStore) region.getStore(FAMILY3).getMemStore()).flushInMemory(); + region.flush(false); + + // CF3 should be compacted so wait here to be sure the compaction is done + while (((CompactingMemStore) region.getStore(FAMILY3).getMemStore()) + .isMemStoreFlushingInMemory()) + Threads.sleep(10); + + // Recalculate everything + long cf1MemstoreSizePhaseII = region.getStore(FAMILY1).getMemStoreSize(); + long cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getMemStoreSize(); + long cf3MemstoreSizePhaseII = region.getStore(FAMILY3).getMemStoreSize(); + + long smallestSeqInRegionCurrentMemstorePhaseII = getWAL(region) + .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); + // Find the smallest LSNs for edits wrt to each CF. + long smallestSeqCF1PhaseII = region.getOldestSeqIdOfStore(FAMILY1); + long smallestSeqCF2PhaseII = region.getOldestSeqIdOfStore(FAMILY2); + long smallestSeqCF3PhaseII = region.getOldestSeqIdOfStore(FAMILY3); + + s = s + "DefaultMemStore DEEP_OVERHEAD is:" + DefaultMemStore.DEEP_OVERHEAD + + ", CompactingMemStore DEEP_OVERHEAD is:" + CompactingMemStore.DEEP_OVERHEAD + + ", CompactingMemStore DEEP_OVERHEAD_PER_PIPELINE_ITEM is:" + CompactingMemStore + .DEEP_OVERHEAD_PER_PIPELINE_ITEM + + "\n----After first flush! CF1 should be flushed to memory, but not compacted.---\n" + + "Size of CF1 is:" + cf1MemstoreSizePhaseII + ", size of CF2 is:" + cf2MemstoreSizePhaseII + + ", size of CF3 is:" + cf3MemstoreSizePhaseII + "\n"; + + // CF1 was flushed to memory, but there is nothing to compact, should + // remain the same size plus renewed empty skip-list + assertEquals(s, cf1MemstoreSizePhaseII, + cf1MemstoreSizePhaseI + CompactingMemStore.DEEP_OVERHEAD_PER_PIPELINE_ITEM); + + // CF2 should become empty + assertEquals(DefaultMemStore.DEEP_OVERHEAD, cf2MemstoreSizePhaseII); + + // verify that CF3 was flushed to memory and was compacted (this is approximation check) + assertTrue(cf3MemstoreSizePhaseI/2+DefaultMemStore.DEEP_OVERHEAD + + CompactingMemStore.DEEP_OVERHEAD_PER_PIPELINE_ITEM > + cf3MemstoreSizePhaseII); + assertTrue(cf3MemstoreSizePhaseI/2 < cf3MemstoreSizePhaseII); + + + // Now the smallest LSN in the region should be the same as the smallest + // LSN in the memstore of CF1. + assertEquals(smallestSeqInRegionCurrentMemstorePhaseII, smallestSeqCF1PhaseI); + + // Now add more puts for CF1, so that we also flush CF1 to disk instead of + // memory in next flush + for (int i = 1200; i < 2000; i++) { + region.put(createPut(1, i)); + } + + s = s + "The smallest sequence in region WAL is: " + smallestSeqInRegionCurrentMemstorePhaseII + + ", the smallest sequence in CF1:" + smallestSeqCF1PhaseII + ", " + + "the smallest sequence in CF2:" + + smallestSeqCF2PhaseII +", the smallest sequence in CF3:" + smallestSeqCF3PhaseII + "\n"; + + // How much does the CF1 memstore occupy? Will be used later. + long cf1MemstoreSizePhaseIII = region.getStore(FAMILY1).getMemStoreSize(); + long smallestSeqCF1PhaseIII = region.getOldestSeqIdOfStore(FAMILY1); + + s = s + "----After more puts into CF1 its size is:" + cf1MemstoreSizePhaseIII + + ", and its sequence is:" + smallestSeqCF1PhaseIII + " ----\n" ; + + + // Flush!!!!!!!!!!!!!!!!!!!!!! + // Flush again, CF1 is flushed to disk + // CF2 is flushed to disk, because it is not in-memory compacted memstore + // CF3 is flushed empty to memory (actually nothing happens to CF3) + region.flush(false); + + // Recalculate everything + long cf1MemstoreSizePhaseIV = region.getStore(FAMILY1).getMemStoreSize(); + long cf2MemstoreSizePhaseIV = region.getStore(FAMILY2).getMemStoreSize(); + long cf3MemstoreSizePhaseIV = region.getStore(FAMILY3).getMemStoreSize(); + + long smallestSeqInRegionCurrentMemstorePhaseIV = getWAL(region) + .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); + long smallestSeqCF1PhaseIV = region.getOldestSeqIdOfStore(FAMILY1); + long smallestSeqCF2PhaseIV = region.getOldestSeqIdOfStore(FAMILY2); + long smallestSeqCF3PhaseIV = region.getOldestSeqIdOfStore(FAMILY3); + + s = s + "----After SECOND FLUSH, CF1 size is:" + cf1MemstoreSizePhaseIV + ", CF2 size is:" + + cf2MemstoreSizePhaseIV + " and CF3 size is:" + cf3MemstoreSizePhaseIV + + "\n"; + + s = s + "The smallest sequence in region WAL is: " + smallestSeqInRegionCurrentMemstorePhaseIV + + ", the smallest sequence in CF1:" + smallestSeqCF1PhaseIV + ", " + + "the smallest sequence in CF2:" + + smallestSeqCF2PhaseIV +", the smallest sequence in CF3:" + smallestSeqCF3PhaseIV + + "\n"; + + // CF1's pipeline component (inserted before first flush) should be flushed to disk + // CF2 should be flushed to disk + assertEquals(cf1MemstoreSizePhaseIII - cf1MemstoreSizePhaseI + DefaultMemStore.DEEP_OVERHEAD, + cf1MemstoreSizePhaseIV); + assertEquals(DefaultMemStore.DEEP_OVERHEAD, cf2MemstoreSizePhaseIV); + + // CF3 shouldn't have been touched. + assertEquals(cf3MemstoreSizePhaseIV, cf3MemstoreSizePhaseII); + + // the smallest LSN of CF3 shouldn't change + assertEquals(smallestSeqCF3PhaseII, smallestSeqCF3PhaseIV); + + // CF3 should be bottleneck for WAL + assertEquals(s, smallestSeqInRegionCurrentMemstorePhaseIV, smallestSeqCF3PhaseIV); + + // Flush!!!!!!!!!!!!!!!!!!!!!! + // Clearing the existing memstores, CF2 all flushed to disk. The single + // memstore segment in the compaction pipeline of CF1 and CF3 should be flushed to disk. + // Note that active sets of CF1 and CF3 are empty + region.flush(true); + + // Recalculate everything + long cf1MemstoreSizePhaseV = region.getStore(FAMILY1).getMemStoreSize(); + long cf2MemstoreSizePhaseV = region.getStore(FAMILY2).getMemStoreSize(); + long cf3MemstoreSizePhaseV = region.getStore(FAMILY3).getMemStoreSize(); + long smallestSeqInRegionCurrentMemstorePhaseV = getWAL(region) + .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); + + assertEquals(DefaultMemStore.DEEP_OVERHEAD, cf1MemstoreSizePhaseV); + assertEquals(DefaultMemStore.DEEP_OVERHEAD, cf2MemstoreSizePhaseV); + assertEquals(DefaultMemStore.DEEP_OVERHEAD, cf3MemstoreSizePhaseV); + + // Because there is nothing in any memstore the WAL's LSN should be -1 + assertEquals(smallestSeqInRegionCurrentMemstorePhaseV, HConstants.NO_SEQNUM); + + // What happens when we hit the memstore limit, but we are not able to find + // any Column Family above the threshold? + // In that case, we should flush all the CFs. + + // The memstore limit is 200*1024 and the column family flush threshold is + // around 50*1024. We try to just hit the memstore limit with each CF's + // memstore being below the CF flush threshold. + for (int i = 1; i <= 300; i++) { + region.put(createPut(1, i)); + region.put(createPut(2, i)); + region.put(createPut(3, i)); + region.put(createPut(4, i)); + region.put(createPut(5, i)); + } + + region.flush(false); + + s = s + "----AFTER THIRD AND FORTH FLUSH, The smallest sequence in region WAL is: " + + smallestSeqInRegionCurrentMemstorePhaseV + + ". After additional inserts and last flush, the entire region size is:" + region + .getMemstoreSize() + + "\n----------------------------------\n"; + + // Since we won't find any CF above the threshold, and hence no specific + // store to flush, we should flush all the memstores + // Also compacted memstores are flushed to disk. + assertEquals(0, region.getMemstoreSize()); + System.out.println(s); + HBaseTestingUtility.closeRegionAndWAL(region); + } + + + + + + + + + + @Test(timeout = 180000) + public void testSelectiveFlushWhenEnabledAndWALinCompaction() throws IOException { + // Set up the configuration + Configuration conf = HBaseConfiguration.create(); + conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 600 * 1024); + conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushNonSloppyStoresFirstPolicy.class + .getName()); + conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN, 300 * + 1024); + conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.5); + + // Intialize the HRegion + HRegion region = initHRegion("testSelectiveFlushWhenNotEnabled", conf); + // Add 1200 entries for CF1, 100 for CF2 and 50 for CF3 + for (int i = 1; i <= 1200; i++) { + region.put(createPut(1, i)); + if (i <= 100) { + region.put(createPut(2, i)); + if (i <= 50) { + region.put(createPut(3, i)); + } + } + } + // Now add more puts for CF2, so that we only flush CF2 to disk + for (int i = 100; i < 2000; i++) { + region.put(createPut(2, i)); + } + + long totalMemstoreSize = region.getMemstoreSize(); + + // Find the sizes of the memstores of each CF. + long cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getMemStoreSize(); + long cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getMemStoreSize(); + long cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getMemStoreSize(); + + // Some other sanity checks. + assertTrue(cf1MemstoreSizePhaseI > 0); + assertTrue(cf2MemstoreSizePhaseI > 0); + assertTrue(cf3MemstoreSizePhaseI > 0); + + // The total memstore size should be the same as the sum of the sizes of + // memstores of CF1, CF2 and CF3. + String msg = "totalMemstoreSize="+totalMemstoreSize + + " DefaultMemStore.DEEP_OVERHEAD="+DefaultMemStore.DEEP_OVERHEAD + + " DEEP_OVERHEAD_PER_PIPELINE_ITEM="+CompactingMemStore.DEEP_OVERHEAD_PER_PIPELINE_ITEM + + " cf1MemstoreSizePhaseI="+cf1MemstoreSizePhaseI + + " cf2MemstoreSizePhaseI="+cf2MemstoreSizePhaseI + + " cf3MemstoreSizePhaseI="+cf3MemstoreSizePhaseI ; + assertEquals(msg, totalMemstoreSize + 3 * DefaultMemStore.DEEP_OVERHEAD, + cf1MemstoreSizePhaseI + cf2MemstoreSizePhaseI + cf3MemstoreSizePhaseI); + + // Flush! + ((CompactingMemStore) region.getStore(FAMILY1).getMemStore()).flushInMemory(); + ((CompactingMemStore) region.getStore(FAMILY3).getMemStore()).flushInMemory(); + // CF1 and CF3 should be compacted so wait here to be sure the compaction is done + while (((CompactingMemStore) region.getStore(FAMILY1).getMemStore()) + .isMemStoreFlushingInMemory()) + Threads.sleep(10); + while (((CompactingMemStore) region.getStore(FAMILY3).getMemStore()) + .isMemStoreFlushingInMemory()) + Threads.sleep(10); + region.flush(false); + + long cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getMemStoreSize(); + + long smallestSeqInRegionCurrentMemstorePhaseII = + region.getWAL().getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); + long smallestSeqCF1PhaseII = region.getOldestSeqIdOfStore(FAMILY1); + long smallestSeqCF2PhaseII = region.getOldestSeqIdOfStore(FAMILY2); + long smallestSeqCF3PhaseII = region.getOldestSeqIdOfStore(FAMILY3); + + // CF2 should have been cleared + assertEquals(DefaultMemStore.DEEP_OVERHEAD, cf2MemstoreSizePhaseII); + + String s = "\n\n----------------------------------\n" + + "Upon initial insert and flush, LSN of CF1 is:" + + smallestSeqCF1PhaseII + ". LSN of CF2 is:" + + smallestSeqCF2PhaseII + ". LSN of CF3 is:" + + smallestSeqCF3PhaseII + ", smallestSeqInRegionCurrentMemstore:" + + smallestSeqInRegionCurrentMemstorePhaseII + "\n"; + + // Add same entries to compact them later + for (int i = 1; i <= 1200; i++) { + region.put(createPut(1, i)); + if (i <= 100) { + region.put(createPut(2, i)); + if (i <= 50) { + region.put(createPut(3, i)); + } + } + } + // Now add more puts for CF2, so that we only flush CF2 to disk + for (int i = 100; i < 2000; i++) { + region.put(createPut(2, i)); + } + + long smallestSeqInRegionCurrentMemstorePhaseIII = + region.getWAL().getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); + long smallestSeqCF1PhaseIII = region.getOldestSeqIdOfStore(FAMILY1); + long smallestSeqCF2PhaseIII = region.getOldestSeqIdOfStore(FAMILY2); + long smallestSeqCF3PhaseIII = region.getOldestSeqIdOfStore(FAMILY3); + + s = s + "The smallest sequence in region WAL is: " + smallestSeqInRegionCurrentMemstorePhaseIII + + ", the smallest sequence in CF1:" + smallestSeqCF1PhaseIII + ", " + + "the smallest sequence in CF2:" + + smallestSeqCF2PhaseIII +", the smallest sequence in CF3:" + smallestSeqCF3PhaseIII + "\n"; + + // Flush! + ((CompactingMemStore) region.getStore(FAMILY1).getMemStore()).flushInMemory(); + ((CompactingMemStore) region.getStore(FAMILY3).getMemStore()).flushInMemory(); + // CF1 and CF3 should be compacted so wait here to be sure the compaction is done + while (((CompactingMemStore) region.getStore(FAMILY1).getMemStore()) + .isMemStoreFlushingInMemory()) + Threads.sleep(10); + while (((CompactingMemStore) region.getStore(FAMILY3).getMemStore()) + .isMemStoreFlushingInMemory()) + Threads.sleep(10); + region.flush(false); + + long smallestSeqInRegionCurrentMemstorePhaseIV = + region.getWAL().getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); + long smallestSeqCF1PhaseIV = region.getOldestSeqIdOfStore(FAMILY1); + long smallestSeqCF2PhaseIV = region.getOldestSeqIdOfStore(FAMILY2); + long smallestSeqCF3PhaseIV = region.getOldestSeqIdOfStore(FAMILY3); + + s = s + "The smallest sequence in region WAL is: " + smallestSeqInRegionCurrentMemstorePhaseIV + + ", the smallest sequence in CF1:" + smallestSeqCF1PhaseIV + ", " + + "the smallest sequence in CF2:" + + smallestSeqCF2PhaseIV +", the smallest sequence in CF3:" + smallestSeqCF3PhaseIV + "\n"; + + // now check that the LSN of the entire WAL, of CF1 and of CF3 has progressed due to compaction + assertTrue(s, smallestSeqInRegionCurrentMemstorePhaseIV > + smallestSeqInRegionCurrentMemstorePhaseIII); + assertTrue(smallestSeqCF1PhaseIV > smallestSeqCF1PhaseIII); + assertTrue(smallestSeqCF3PhaseIV > smallestSeqCF3PhaseIII); + + HBaseTestingUtility.closeRegionAndWAL(region); + } + + + + + + // Find the (first) region which has the specified name. + private static Pair getRegionWithName(TableName tableName) { + MiniHBaseCluster cluster = TEST_UTIL.getMiniHBaseCluster(); + List rsts = cluster.getRegionServerThreads(); + for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) { + HRegionServer hrs = rsts.get(i).getRegionServer(); + for (Region region : hrs.getOnlineRegions(tableName)) { + return Pair.newPair(region, hrs); + } + } + return null; + } + + + private WAL getWAL(Region region) { + return ((HRegion)region).getWAL(); + } + + private int getNumRolledLogFiles(Region region) { + return ((FSHLog)getWAL(region)).getNumRolledLogFiles(); + } + + +} diff --git a/hbase-shell/src/main/ruby/hbase.rb b/hbase-shell/src/main/ruby/hbase.rb index 21f88f9..1ddd38b 100644 --- a/hbase-shell/src/main/ruby/hbase.rb +++ b/hbase-shell/src/main/ruby/hbase.rb @@ -39,6 +39,7 @@ module HBaseConstants NAME = org.apache.hadoop.hbase.HConstants::NAME VERSIONS = org.apache.hadoop.hbase.HConstants::VERSIONS IN_MEMORY = org.apache.hadoop.hbase.HConstants::IN_MEMORY + IN_MEMORY_COMPACTION = org.apache.hadoop.hbase.HConstants::IN_MEMORY_COMPACTION METADATA = org.apache.hadoop.hbase.HConstants::METADATA STOPROW = "STOPROW" STARTROW = "STARTROW" diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb index b45a210..81799f2 100644 --- a/hbase-shell/src/main/ruby/hbase/admin.rb +++ b/hbase-shell/src/main/ruby/hbase/admin.rb @@ -862,6 +862,7 @@ module Hbase family.setScope(JInteger.valueOf(arg.delete(org.apache.hadoop.hbase.HColumnDescriptor::REPLICATION_SCOPE))) if arg.include?(org.apache.hadoop.hbase.HColumnDescriptor::REPLICATION_SCOPE) family.setCacheDataOnWrite(JBoolean.valueOf(arg.delete(org.apache.hadoop.hbase.HColumnDescriptor::CACHE_DATA_ON_WRITE))) if arg.include?(org.apache.hadoop.hbase.HColumnDescriptor::CACHE_DATA_ON_WRITE) family.setInMemory(JBoolean.valueOf(arg.delete(org.apache.hadoop.hbase.HColumnDescriptor::IN_MEMORY))) if arg.include?(org.apache.hadoop.hbase.HColumnDescriptor::IN_MEMORY) + family.setCompacted(JBoolean.valueOf(arg.delete(org.apache.hadoop.hbase.HColumnDescriptor::IN_MEMORY_COMPACTION))) if arg.include?(org.apache.hadoop.hbase.HColumnDescriptor::IN_MEMORY_COMPACTION) family.setTimeToLive(arg.delete(org.apache.hadoop.hbase.HColumnDescriptor::TTL)) if arg.include?(org.apache.hadoop.hbase.HColumnDescriptor::TTL) family.setDataBlockEncoding(org.apache.hadoop.hbase.io.encoding.DataBlockEncoding.valueOf(arg.delete(org.apache.hadoop.hbase.HColumnDescriptor::DATA_BLOCK_ENCODING))) if arg.include?(org.apache.hadoop.hbase.HColumnDescriptor::DATA_BLOCK_ENCODING) family.setBlocksize(JInteger.valueOf(arg.delete(org.apache.hadoop.hbase.HColumnDescriptor::BLOCKSIZE))) if arg.include?(org.apache.hadoop.hbase.HColumnDescriptor::BLOCKSIZE) -- 1.7.10.2 (Apple Git-33)