diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index 25e2ef42926..3a5dd39f490 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -415,7 +415,7 @@ public final class HConstants { /** * The hbase:meta table's name. - * + * @deprecated For upgrades of 0.94 to 0.96 */ @Deprecated // for compat from 0.94 -> 0.96. public static final byte[] META_TABLE_NAME = TableName.META_TABLE_NAME.getName(); @@ -605,7 +605,7 @@ public final class HConstants { * 1, 2, 3, 5, 10, 20, 40, 100, 100, 100. * With 100ms, a back-off of 200 means 20s */ - public static final int RETRY_BACKOFF[] = {1, 2, 3, 5, 10, 20, 40, 100, 100, 100, 100, 200, 200}; + public static final int [] RETRY_BACKOFF = {1, 2, 3, 5, 10, 20, 40, 100, 100, 100, 100, 200, 200}; public static final String REGION_IMPL = "hbase.hregion.impl"; diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/NamespacesInstanceModel.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/NamespacesInstanceModel.java index d8528efd1e3..b31ecf96807 100644 --- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/NamespacesInstanceModel.java +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/NamespacesInstanceModel.java @@ -67,7 +67,7 @@ public class NamespacesInstanceModel implements Serializable, ProtobufMessageHan /** * Constructor to use if namespace does not exist in HBASE. - * @param namespaceName: the namespace name. + * @param namespaceName the namespace name. * @throws IOException */ public NamespacesInstanceModel(String namespaceName) throws IOException { @@ -76,8 +76,8 @@ public class NamespacesInstanceModel implements Serializable, ProtobufMessageHan /** * Constructor - * @param admin: the administrative API - * @param namespaceName: the namespace name. + * @param admin the administrative API + * @param namespaceName the namespace name. * @throws IOException */ public NamespacesInstanceModel(Admin admin, String namespaceName) throws IOException { @@ -95,7 +95,7 @@ public class NamespacesInstanceModel implements Serializable, ProtobufMessageHan /** * Add property to the namespace. - * @param key: attribute name + * @param key attribute name * @param value attribute value */ public void addProperty(String key, String value) { diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/NamespacesModel.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/NamespacesModel.java index 7b8f3b7c993..d6a56852b59 100644 --- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/NamespacesModel.java +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/NamespacesModel.java @@ -61,7 +61,7 @@ public class NamespacesModel implements Serializable, ProtobufMessageHandler { /** * Constructor - * @param admin: the administrative API + * @param admin the administrative API * @throws IOException */ public NamespacesModel(Admin admin) throws IOException { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java index 7925cb0da15..8b2772b6110 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java @@ -878,13 +878,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements * @return true when distributed log replay is turned on */ private boolean isDistributedLogReplay(Configuration conf) { - boolean dlr = - conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, - HConstants.DEFAULT_DISTRIBUTED_LOG_REPLAY_CONFIG); - if (LOG.isDebugEnabled()) { - LOG.debug("Distributed log replay=" + dlr); - } - return dlr; + return false; } private boolean resubmit(ServerName serverName, String path, int version) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java index 6f800c20b79..75f1f82ed80 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java @@ -55,7 +55,7 @@ import org.apache.hadoop.util.StringUtils; /** * Provides functionality to write ({@link BlockIndexWriter}) and read - * ({@link BlockIndexReader}) + * BlockIndexReader * single-level and multi-level block indexes. * * Examples of how to use the block index writer can be found in 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 2c145b46a54..a8ffa8d8f72 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 @@ -46,7 +46,6 @@ import java.util.concurrent.CompletionService; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentSkipListMap; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorCompletionService; import java.util.concurrent.ExecutorService; @@ -61,6 +60,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.commons.logging.Log; @@ -77,7 +77,6 @@ import org.apache.hadoop.hbase.CompoundConfiguration; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.DroppedSnapshotException; import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants.OperationStatusCode; @@ -147,7 +146,6 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.Stor import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.EventType; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor; -import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl.WriteEntry; import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope; import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState; import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; @@ -203,13 +201,15 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi private static final Log LOG = LogFactory.getLog(HRegion.class); public static final String LOAD_CFS_ON_DEMAND_CONFIG_KEY = - "hbase.hregion.scan.loadColumnFamiliesOnDemand"; + "hbase.hregion.scan.loadColumnFamiliesOnDemand"; /** * Longest time we'll wait on a sequenceid. * Sequenceid comes up out of the WAL subsystem. WAL subsystem can go bad or a test might use - * it without cleanup previous usage properly; generally, a WAL roll is needed. - * Key to use changing the default of 30000ms. + * it without cleaning up previous usage properly; generally, a WAL roll is needed. The timeout + * is for a latch in WALKey. There is no global accounting of outstanding WALKeys; intentionally + * to avoid contention, but it makes it so if an abort or problem, we could be stuck waiting + * on the WALKey latch. Revisit. */ private final int maxWaitForSeqId; private static final String MAX_WAIT_FOR_SEQ_ID_KEY = "hbase.hregion.max.wait.for.sequenceid.ms"; @@ -222,6 +222,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi private static final Durability DEFAULT_DURABILITY = Durability.SYNC_WAL; final AtomicBoolean closed = new AtomicBoolean(false); + /* Closing can take some time; use the closing flag if there is stuff we don't * want to do while in closing state; e.g. like offer this region up to the * master as a region to close if the carrying regionserver is overloaded. @@ -241,19 +242,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi * {@link #maxFlushedSeqId} will be older than the oldest edit in memory. */ private volatile long lastFlushOpSeqId = HConstants.NO_SEQNUM; - /** - * Region scoped edit sequence Id. Edits to this region are GUARANTEED to appear in the WAL - * file in this sequence id's order; i.e. edit #2 will be in the WAL after edit #1. - * Its default value is -1L. This default is used as a marker to indicate - * that the region hasn't opened yet. Once it is opened, it is set to the derived - * #openSeqNum, the largest sequence id of all hfiles opened under this Region. - * - *

Control of this sequence is handed off to the WAL implementation. It is responsible - * for tagging edits with the correct sequence id since it is responsible for getting the - * edits into the WAL files. It controls updating the sequence id value. DO NOT UPDATE IT - * OUTSIDE OF THE WAL. The value you get will not be what you think it is. - */ - private final AtomicLong sequenceId = new AtomicLong(-1L); /** * The sequence id of the last replayed open region event from the primary region. This is used @@ -371,7 +359,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi /** * @return The smallest mvcc readPoint across all the scanners in this - * region. Writes older than this readPoint, are included in every + * region. Writes older than this readPoint, are included in every * read operation. */ public long getSmallestReadPoint() { @@ -380,7 +368,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // no new RegionScanners can grab a readPoint that we are unaware of. // We achieve this by synchronizing on the scannerReadPoints object. synchronized(scannerReadPoints) { - minimumReadPoint = mvcc.memstoreReadPoint(); + minimumReadPoint = mvcc.getReadPoint(); for (Long readPoint: this.scannerReadPoints.values()) { if (readPoint < minimumReadPoint) { @@ -592,8 +580,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi private boolean splitRequest; private byte[] explicitSplitPoint = null; - private final MultiVersionConcurrencyControl mvcc = - new MultiVersionConcurrencyControl(); + private final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(); // Coprocessor host private RegionCoprocessorHost coprocessorHost; @@ -629,6 +616,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi * @deprecated Use other constructors. */ @Deprecated + @VisibleForTesting public HRegion(final Path tableDir, final WAL wal, final FileSystem fs, final Configuration confParam, final HRegionInfo regionInfo, final HTableDescriptor htd, final RegionServerServices rsServices) { @@ -819,7 +807,15 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // Initialize all the HStores status.setStatus("Initializing all the Stores"); - long maxSeqId = initializeRegionStores(reporter, status, false); + long maxSeqId = initializeStores(reporter, status); + this.mvcc.advanceTo(maxSeqId); + if (ServerRegionReplicaUtil.shouldReplayRecoveredEdits(this)) { + // Recover any edits if available. + maxSeqId = Math.max(maxSeqId, + replayRecoveredEditsIfAny(this.fs.getRegionDir(), maxSeqIdInStores, reporter, status)); + // Make sure mvcc is up to max. + this.mvcc.advanceTo(maxSeqId); + } this.lastReplayedOpenRegionSeqId = maxSeqId; this.writestate.setReadOnly(ServerRegionReplicaUtil.isReadOnly(this)); @@ -882,10 +878,15 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi return nextSeqid; } - private long initializeRegionStores(final CancelableProgressable reporter, MonitoredTask status, - boolean warmupOnly) - throws IOException { - + /** + * Open all Stores. + * @param reporter + * @param status + * @return Highest sequenceId found out in a Store. + * @throws IOException + */ + private long initializeStores(final CancelableProgressable reporter, MonitoredTask status) + throws IOException { // Load in all the HStores. long maxSeqId = -1; @@ -947,14 +948,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } } } - if (ServerRegionReplicaUtil.shouldReplayRecoveredEdits(this) && !warmupOnly) { - // Recover any edits if available. - maxSeqId = Math.max(maxSeqId, replayRecoveredEditsIfAny( - this.fs.getRegionDir(), maxSeqIdInStores, reporter, status)); - } - maxSeqId = Math.max(maxSeqId, maxMemstoreTS + 1); - mvcc.initialize(maxSeqId); - return maxSeqId; + return Math.max(maxSeqId, maxMemstoreTS + 1); } private void initializeWarmup(final CancelableProgressable reporter) throws IOException { @@ -962,7 +956,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // Initialize all the HStores status.setStatus("Warming up all the Stores"); - initializeRegionStores(reporter, status, true); + initializeStores(reporter, status); } private void writeRegionOpenMarker(WAL wal, long openSeqId) throws IOException { @@ -978,8 +972,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi RegionEventDescriptor regionOpenDesc = ProtobufUtil.toRegionEventDescriptor( RegionEventDescriptor.EventType.REGION_OPEN, getRegionInfo(), openSeqId, getRegionServerServices().getServerName(), storeFiles); - WALUtil.writeRegionEventMarker(wal, getTableDesc(), getRegionInfo(), regionOpenDesc, - getSequenceId()); + WALUtil.writeRegionEventMarker(wal, getTableDesc(), getRegionInfo(), regionOpenDesc); } private void writeRegionCloseMarker(WAL wal) throws IOException { @@ -993,17 +986,16 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } RegionEventDescriptor regionEventDesc = ProtobufUtil.toRegionEventDescriptor( - RegionEventDescriptor.EventType.REGION_CLOSE, getRegionInfo(), getSequenceId().get(), + RegionEventDescriptor.EventType.REGION_CLOSE, getRegionInfo(), mvcc.getReadPoint(), getRegionServerServices().getServerName(), storeFiles); - WALUtil.writeRegionEventMarker(wal, getTableDesc(), getRegionInfo(), regionEventDesc, - getSequenceId()); + WALUtil.writeRegionEventMarker(wal, getTableDesc(), getRegionInfo(), regionEventDesc); // Store SeqId in HDFS when a region closes // checking region folder exists is due to many tests which delete the table folder while a // table is still online if (this.fs.getFileSystem().exists(this.fs.getRegionDir())) { WALSplitter.writeRegionSequenceIdFile(this.fs.getFileSystem(), this.fs.getRegionDir(), - getSequenceId().get(), 0); + mvcc.getReadPoint(), 0); } } @@ -1275,7 +1267,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // This scan can read even uncommitted transactions return Long.MAX_VALUE; } - return mvcc.memstoreReadPoint(); + return mvcc.getReadPoint(); } @Override @@ -1955,11 +1947,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi boolean shouldFlushStore(Store store) { long earliest = this.wal.getEarliestMemstoreSeqNum(getRegionInfo().getEncodedNameAsBytes(), store.getFamily().getName()) - 1; - if (earliest > 0 && earliest + flushPerChanges < sequenceId.get()) { + if (earliest > 0 && earliest + flushPerChanges < mvcc.getReadPoint()) { if (LOG.isDebugEnabled()) { LOG.debug("Flush column family " + store.getColumnFamilyName() + " of " + getRegionInfo().getEncodedName() + " because unflushed sequenceid=" + earliest + - " is > " + this.flushPerChanges + " from current=" + sequenceId.get()); + " is > " + this.flushPerChanges + " from current=" + mvcc.getReadPoint()); } return true; } @@ -1985,7 +1977,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi whyFlush.setLength(0); // This is a rough measure. if (this.maxFlushedSeqId > 0 - && (this.maxFlushedSeqId + this.flushPerChanges < this.sequenceId.get())) { + && (this.maxFlushedSeqId + this.flushPerChanges < this.mvcc.getReadPoint())) { whyFlush.append("more than max edits, " + this.flushPerChanges + ", since last flush"); return true; } @@ -2075,11 +2067,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } } - protected PrepareFlushResult internalPrepareFlushCache( - final WAL wal, final long myseqid, final Collection storesToFlush, - MonitoredTask status, boolean writeFlushWalMarker) - throws IOException { - + protected PrepareFlushResult internalPrepareFlushCache(final WAL wal, final long myseqid, + final Collection storesToFlush, MonitoredTask status, boolean writeFlushWalMarker) + throws IOException { if (this.rsServices != null && this.rsServices.isAborted()) { // Don't flush when server aborting, it's unsafe throw new IOException("Aborting flush because server is aborted..."); @@ -2089,7 +2079,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi if (this.memstoreSize.get() <= 0) { // Take an update lock because am about to change the sequence id and we want the sequence id // to be at the border of the empty memstore. - MultiVersionConcurrencyControl.WriteEntry w = null; + MultiVersionConcurrencyControl.WriteEntry writeEntry = null; this.updatesLock.writeLock().lock(); try { if (this.memstoreSize.get() <= 0) { @@ -2097,29 +2087,34 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // this region out in the WAL subsystem so no need to do any trickery clearing out // edits in the WAL system. Up the sequence number so the resulting flush id is for // sure just beyond the last appended region edit (useful as a marker when bulk loading, - // etc.) - // wal can be null replaying edits. + // etc.). NOTE: The writeEntry write number is NOT in the WAL.. there is no WAL writing + // here. if (wal != null) { - w = mvcc.beginMemstoreInsert(); - long flushOpSeqId = getNextSequenceId(wal); + writeEntry = mvcc.begin(); + long flushOpSeqId = writeEntry.getWriteNumber(); FlushResult flushResult = new FlushResultImpl( - FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY, flushOpSeqId, "Nothing to flush", - writeFlushRequestMarkerToWAL(wal, writeFlushWalMarker)); - w.setWriteNumber(flushOpSeqId); - mvcc.waitForPreviousTransactionsComplete(w); - w = null; + FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY, + flushOpSeqId, + "Nothing to flush", + writeFlushRequestMarkerToWAL(wal, writeFlushWalMarker)); + // TODO: Lets see if we hang here, if there is a scenario where an outstanding reader + // with a read point is in advance of this write point. + mvcc.completeAndWait(writeEntry); + writeEntry = null; return new PrepareFlushResult(flushResult, myseqid); } else { return new PrepareFlushResult( - new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY, - "Nothing to flush", false), + new FlushResultImpl( + FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY, + "Nothing to flush", + false), myseqid); } } } finally { this.updatesLock.writeLock().unlock(); - if (w != null) { - mvcc.advanceMemstore(w); + if (writeEntry != null) { + mvcc.complete(writeEntry); } } } @@ -2130,10 +2125,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi if (!isAllFamilies(storesToFlush)) { perCfExtras = new StringBuilder(); for (Store store: storesToFlush) { - perCfExtras.append("; "); - perCfExtras.append(store.getColumnFamilyName()); - perCfExtras.append("="); - perCfExtras.append(StringUtils.byteDesc(store.getMemStoreSize())); + perCfExtras.append("; ").append(store.getColumnFamilyName()); + perCfExtras.append("=").append(StringUtils.byteDesc(store.getMemStoreSize())); } } LOG.info("Flushing " + + storesToFlush.size() + "/" + stores.size() + @@ -2178,7 +2171,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi long trxId = 0; try { try { - writeEntry = mvcc.beginMemstoreInsert(); + writeEntry = mvcc.begin(); if (wal != null) { Long earliestUnflushedSequenceIdForTheRegion = wal.startCacheFlush(encodedRegionName, flushedFamilyNames); @@ -2213,7 +2206,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi getRegionInfo(), flushOpSeqId, committedFiles); // no sync. Sync is below where we do not hold the updates lock trxId = WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(), - desc, sequenceId, false); + desc, false, mvcc); } // Prepare flush (take a snapshot) @@ -2227,7 +2220,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.ABORT_FLUSH, getRegionInfo(), flushOpSeqId, committedFiles); WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(), - desc, sequenceId, false); + desc, false, mvcc); } catch (Throwable t) { LOG.warn("Received unexpected exception trying to write ABORT_FLUSH marker to WAL:" + StringUtils.stringifyException(t)); @@ -2261,18 +2254,18 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // uncommitted transactions from being written into HFiles. // We have to block before we start the flush, otherwise keys that // were removed via a rollbackMemstore could be written to Hfiles. - writeEntry.setWriteNumber(flushOpSeqId); - mvcc.waitForPreviousTransactionsComplete(writeEntry); - // set w to null to prevent mvcc.advanceMemstore from being called again inside finally block + mvcc.completeAndWait(writeEntry); + // set writeEntry to null to prevent mvcc.complete from being called again inside finally + // block writeEntry = null; } finally { if (writeEntry != null) { - // in case of failure just mark current writeEntry as complete - mvcc.advanceMemstore(writeEntry); + // In case of failure just mark current writeEntry as complete. + mvcc.complete(writeEntry); } } - return new PrepareFlushResult(storeFlushCtxs, committedFiles, storeFlushableSize, startTime, flushOpSeqId, - flushedSeqId, totalFlushableSizeOfFlushableStores); + return new PrepareFlushResult(storeFlushCtxs, committedFiles, storeFlushableSize, startTime, + flushOpSeqId, flushedSeqId, totalFlushableSizeOfFlushableStores); } /** @@ -2292,10 +2285,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi private boolean writeFlushRequestMarkerToWAL(WAL wal, boolean writeFlushWalMarker) { if (writeFlushWalMarker && wal != null && !writestate.readOnly) { FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.CANNOT_FLUSH, - getRegionInfo(), -1, new TreeMap>()); + getRegionInfo(), -1, new TreeMap>(Bytes.BYTES_COMPARATOR)); try { WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(), - desc, sequenceId, true); + desc, true, mvcc); return true; } catch (IOException e) { LOG.warn(getRegionInfo().getEncodedName() + " : " @@ -2364,7 +2357,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.COMMIT_FLUSH, getRegionInfo(), flushOpSeqId, committedFiles); WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(), - desc, sequenceId, true); + desc, true, mvcc); } } catch (Throwable t) { // An exception here means that the snapshot was not persisted. @@ -2378,7 +2371,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.ABORT_FLUSH, getRegionInfo(), flushOpSeqId, committedFiles); WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(), - desc, sequenceId, false); + desc, false, mvcc); } catch (Throwable ex) { LOG.warn(getRegionInfo().getEncodedName() + " : " + "Received unexpected exception trying to write ABORT_FLUSH marker to WAL:" @@ -2456,7 +2449,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // a timeout. May happen in tests after we tightened the semantic via HBASE-14317. // Also, the getSequenceId blocks on a latch. There is no global list of outstanding latches // so if an abort or stop, there is no way to call them in. - WALKey key = this.appendEmptyEdit(wal, null); + WALKey key = this.appendEmptyEdit(wal); + mvcc.complete(key.getWriteEntry()); return key.getSequenceId(this.maxWaitForSeqId); } @@ -2885,7 +2879,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi List acquiredRowLocks = Lists.newArrayListWithCapacity(batchOp.operations.length); // reference family maps directly so coprocessors can mutate them if desired Map>[] familyMaps = new Map[batchOp.operations.length]; - List memstoreCells = new ArrayList(); // We try to set up a batch in the range [firstIndex,lastIndexExclusive) int firstIndex = batchOp.nextIndexToProcess; int lastIndexExclusive = firstIndex; @@ -2950,17 +2943,16 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // If we haven't got any rows in our batch, we should block to // get the next one. - boolean shouldBlock = numReadyToWrite == 0; RowLock rowLock = null; try { - rowLock = getRowLockInternal(mutation.getRow(), shouldBlock); + rowLock = getRowLock(mutation.getRow(), true); } catch (IOException ioe) { LOG.warn("Failed getting lock in batch put, row=" + Bytes.toStringBinary(mutation.getRow()), ioe); } if (rowLock == null) { // We failed to grab another lock - assert !shouldBlock : "Should never fail to get lock when blocking"; + assert false: "Should never fail to get lock when blocking"; break; // stop acquiring more rows for this batch } else { acquiredRowLocks.add(rowLock); @@ -3020,16 +3012,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi lock(this.updatesLock.readLock(), numReadyToWrite); locked = true; - if(isInReplay) { - mvccNum = batchOp.getReplaySequenceId(); - } else { - mvccNum = MultiVersionConcurrencyControl.getPreAssignedWriteNumber(this.sequenceId); - } - // - // ------------------------------------ - // Acquire the latest mvcc number - // ---------------------------------- - writeEntry = mvcc.beginMemstoreInsertWithSeqNum(mvccNum); // calling the pre CP hook for batch mutation if (!isInReplay && coprocessorHost != null) { @@ -3040,35 +3022,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } // ------------------------------------ - // STEP 3. Write back to memstore - // Write to memstore. It is ok to write to memstore - // first without updating the WAL because we do not roll - // forward the memstore MVCC. The MVCC will be moved up when - // the complete operation is done. These changes are not yet - // visible to scanners till we update the MVCC. The MVCC is - // moved only when the sync is complete. - // ---------------------------------- - long addedSize = 0; - for (int i = firstIndex; i < lastIndexExclusive; i++) { - if (batchOp.retCodeDetails[i].getOperationStatusCode() - != OperationStatusCode.NOT_RUN) { - continue; - } - doRollBackMemstore = true; // If we have a failure, we need to clean what we wrote - addedSize += applyFamilyMapToMemstore(familyMaps[i], mvccNum, memstoreCells, isInReplay); - } - - // ------------------------------------ - // STEP 4. Build WAL edit + // STEP 3. Build WAL edit // ---------------------------------- Durability durability = Durability.USE_DEFAULT; for (int i = firstIndex; i < lastIndexExclusive; i++) { // Skip puts that were determined to be invalid during preprocessing - if (batchOp.retCodeDetails[i].getOperationStatusCode() - != OperationStatusCode.NOT_RUN) { + if (batchOp.retCodeDetails[i].getOperationStatusCode() != OperationStatusCode.NOT_RUN) { continue; } - batchOp.retCodeDetails[i] = OperationStatus.SUCCESS; Mutation m = batchOp.getMutation(i); Durability tmpDur = getEffectiveDurability(m.getDurability()); @@ -3094,9 +3055,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // we use HLogKey here instead of WALKey directly to support legacy coprocessors. walKey = new ReplayHLogKey(this.getRegionInfo().getEncodedNameAsBytes(), this.htableDescriptor.getTableName(), now, m.getClusterIds(), - currentNonceGroup, currentNonce); + currentNonceGroup, currentNonce, mvcc); txid = this.wal.append(this.htableDescriptor, this.getRegionInfo(), walKey, - walEdit, getSequenceId(), true, null); + walEdit, true); walEdit = new WALEdit(isInReplay); walKey = null; } @@ -3115,38 +3076,57 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } // ------------------------- - // STEP 5. Append the final edit to WAL. Do not sync wal. + // STEP 4. Append the final edit to WAL. Do not sync wal. // ------------------------- Mutation mutation = batchOp.getMutation(firstIndex); if (isInReplay) { // use wal key from the original walKey = new ReplayHLogKey(this.getRegionInfo().getEncodedNameAsBytes(), this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, now, - mutation.getClusterIds(), currentNonceGroup, currentNonce); + mutation.getClusterIds(), currentNonceGroup, currentNonce, mvcc); long replaySeqId = batchOp.getReplaySequenceId(); walKey.setOrigLogSeqNum(replaySeqId); - - // ensure that the sequence id of the region is at least as big as orig log seq id - while (true) { - long seqId = getSequenceId().get(); - if (seqId >= replaySeqId) break; - if (getSequenceId().compareAndSet(seqId, replaySeqId)) break; - } } if (walEdit.size() > 0) { if (!isInReplay) { // we use HLogKey here instead of WALKey directly to support legacy coprocessors. walKey = new HLogKey(this.getRegionInfo().getEncodedNameAsBytes(), this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, now, - mutation.getClusterIds(), currentNonceGroup, currentNonce); + mutation.getClusterIds(), currentNonceGroup, currentNonce, mvcc); } - - txid = this.wal.append(this.htableDescriptor, this.getRegionInfo(), walKey, walEdit, - getSequenceId(), true, memstoreCells); + txid = this.wal.append(this.htableDescriptor, this.getRegionInfo(), walKey, walEdit, true); } - if (walKey == null){ - // Append a faked WALEdit in order for SKIP_WAL updates to get mvcc assigned - walKey = this.appendEmptyEdit(this.wal, memstoreCells); + // ------------------------------------ + // Acquire the latest mvcc number + // ---------------------------------- + if (walKey == null) { + // If this is a skip wal operation just get the read point from mvcc + walKey = this.appendEmptyEdit(this.wal); + } + if (!isInReplay) { + writeEntry = walKey.getWriteEntry(); + mvccNum = writeEntry.getWriteNumber(); + } else { + mvccNum = batchOp.getReplaySequenceId(); + } + + // ------------------------------------ + // STEP 5. Write back to memstore + // Write to memstore. It is ok to write to memstore + // first without syncing the WAL because we do not roll + // forward the memstore MVCC. The MVCC will be moved up when + // the complete operation is done. These changes are not yet + // visible to scanners till we update the MVCC. The MVCC is + // moved only when the sync is complete. + // ---------------------------------- + long addedSize = 0; + for (int i = firstIndex; i < lastIndexExclusive; i++) { + if (batchOp.retCodeDetails[i].getOperationStatusCode() + != OperationStatusCode.NOT_RUN) { + continue; + } + doRollBackMemstore = true; // If we have a failure, we need to clean what we wrote + addedSize += applyFamilyMapToMemstore(familyMaps[i], mvccNum, isInReplay); } // ------------------------------- @@ -3174,13 +3154,21 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi coprocessorHost.postBatchMutate(miniBatchOp); } - // ------------------------------------------------------------------ // STEP 8. Advance mvcc. This will make this put visible to scanners and getters. // ------------------------------------------------------------------ if (writeEntry != null) { - mvcc.completeMemstoreInsertWithSeqNum(writeEntry, walKey); + mvcc.completeAndWait(writeEntry); writeEntry = null; + } else if (isInReplay) { + // ensure that the sequence id of the region is at least as big as orig log seq id + mvcc.advanceTo(mvccNum); + } + + for (int i = firstIndex; i < lastIndexExclusive; i ++) { + if (batchOp.retCodeDetails[i] == OperationStatus.NOT_RUN) { + batchOp.retCodeDetails[i] = OperationStatus.SUCCESS; + } } // ------------------------------------ @@ -3208,10 +3196,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } finally { // if the wal sync was unsuccessful, remove keys from memstore if (doRollBackMemstore) { - rollbackMemstore(memstoreCells); - if (writeEntry != null) mvcc.cancelMemstoreInsert(writeEntry); + for (int j = 0; j < familyMaps.length; j++) { + for(List cells:familyMaps[j].values()) { + rollbackMemstore(cells); + } + } + if (writeEntry != null) mvcc.complete(writeEntry); } else if (writeEntry != null) { - mvcc.completeMemstoreInsertWithSeqNum(writeEntry, walKey); + mvcc.completeAndWait(writeEntry); } if (locked) { @@ -3298,7 +3290,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // Lock row - note that doBatchMutate will relock this row if called RowLock rowLock = getRowLock(get.getRow()); // wait for all previous transactions to complete (with lock held) - mvcc.waitForPreviousTransactionsComplete(); + mvcc.await(); try { if (this.getCoprocessorHost() != null) { Boolean processed = null; @@ -3407,7 +3399,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // Lock row - note that doBatchMutate will relock this row if called RowLock rowLock = getRowLock(get.getRow()); // wait for all previous transactions to complete (with lock held) - mvcc.waitForPreviousTransactionsComplete(); + mvcc.await(); try { List result = get(get, false); @@ -3484,7 +3476,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi private void doBatchMutate(Mutation mutation) throws IOException { // Currently this is only called for puts and deletes, so no nonces. - OperationStatus[] batchMutate = this.batchMutate(new Mutation[] { mutation }); + OperationStatus[] batchMutate = this.batchMutate(new Mutation[]{mutation}); if (batchMutate[0].getOperationStatusCode().equals(OperationStatusCode.SANITY_CHECK_FAILURE)) { throw new FailedSanityCheckException(batchMutate[0].getExceptionMsg()); } else if (batchMutate[0].getOperationStatusCode().equals(OperationStatusCode.BAD_FAMILY)) { @@ -3669,7 +3661,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi * new entries. */ private long applyFamilyMapToMemstore(Map> familyMap, - long mvccNum, List memstoreCells, boolean isInReplay) throws IOException { + long mvccNum, boolean isInReplay) throws IOException { long size = 0; for (Map.Entry> e : familyMap.entrySet()) { @@ -3680,10 +3672,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi int listSize = cells.size(); for (int i=0; i < listSize; i++) { Cell cell = cells.get(i); - CellUtil.setSequenceId(cell, mvccNum); + if (cell.getSequenceId() == 0) { + CellUtil.setSequenceId(cell, mvccNum); + } Pair ret = store.add(cell); size += ret.getFirst(); - memstoreCells.add(ret.getSecond()); if(isInReplay) { // set memstore newly added cells with replay mvcc number CellUtil.setSequenceId(ret.getSecond(), mvccNum); @@ -4440,12 +4433,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi this.maxFlushedSeqId = flush.getFlushSequenceNumber(); // advance the mvcc read point so that the new flushed file is visible. - // there may be some in-flight transactions, but they won't be made visible since they are - // either greater than flush seq number or they were already dropped via flush. - // TODO: If we are using FlushAllStoresPolicy, then this can make edits visible from other - // stores while they are still in flight because the flush commit marker will not contain - // flushes from ALL stores. - getMVCC().advanceMemstoreReadPointIfNeeded(flush.getFlushSequenceNumber()); + mvcc.advanceTo(flush.getFlushSequenceNumber()); } catch (FileNotFoundException ex) { LOG.warn(getRegionInfo().getEncodedName() + " : " @@ -4512,15 +4500,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi /** * Drops the memstore contents after replaying a flush descriptor or region open event replay * if the memstore edits have seqNums smaller than the given seq id - * @param flush the flush descriptor * @throws IOException */ private long dropMemstoreContentsForSeqId(long seqId, Store store) throws IOException { long totalFreedSize = 0; this.updatesLock.writeLock().lock(); try { - mvcc.waitForPreviousTransactionsComplete(); - long currentSeqId = getSequenceId().get(); + + long currentSeqId = mvcc.getReadPoint(); if (seqId >= currentSeqId) { // then we can drop the memstore contents since everything is below this seqId LOG.info(getRegionInfo().getEncodedName() + " : " @@ -4683,9 +4670,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi dropPrepareFlushIfPossible(); // advance the mvcc read point so that the new flushed file is visible. - // there may be some in-flight transactions, but they won't be made visible since they are - // either greater than flush seq number or they were already dropped via flush. - getMVCC().advanceMemstoreReadPointIfNeeded(this.maxFlushedSeqId); + mvcc.await(); // If we were waiting for observing a flush or region opening event for not showing partial // data after a secondary region crash, we can allow reads now. @@ -4776,7 +4761,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } } if (bulkLoadEvent.getBulkloadSeqNum() > 0) { - getMVCC().advanceMemstoreReadPointIfNeeded(bulkLoadEvent.getBulkloadSeqNum()); + mvcc.advanceTo(bulkLoadEvent.getBulkloadSeqNum()); } } finally { closeBulkRegionOperation(); @@ -4875,11 +4860,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi dropPrepareFlushIfPossible(); // advance the mvcc read point so that the new flushed files are visible. - // there may be some in-flight transactions, but they won't be made visible since they are - // either greater than flush seq number or they were already picked up via flush. - for (Store s : getStores()) { - getMVCC().advanceMemstoreReadPointIfNeeded(s.getMaxMemstoreTS()); - } + // either greater than flush seq number or they were already picked up via flush. + for (Store s : getStores()) { + mvcc.advanceTo(s.getMaxMemstoreTS()); + } + // smallestSeqIdInStores is the seqId that we have a corresponding hfile for. We can safely // skip all edits that are to be replayed in the future with that has a smaller seqId @@ -5037,75 +5022,91 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } } - @Override - public RowLock getRowLock(byte[] row, boolean waitForLock) throws IOException { - startRegionOperation(); - try { - return getRowLockInternal(row, waitForLock); - } finally { - closeRegionOperation(); - } - } /** - * A version of getRowLock(byte[], boolean) to use when a region operation has already been - * started (the calling thread has already acquired the region-close-guard lock). - */ - protected RowLock getRowLockInternal(byte[] row, boolean waitForLock) throws IOException { - HashedBytes rowKey = new HashedBytes(row); - RowLockContext rowLockContext = new RowLockContext(rowKey); - - // loop until we acquire the row lock (unless !waitForLock) - while (true) { - RowLockContext existingContext = lockedRows.putIfAbsent(rowKey, rowLockContext); - if (existingContext == null) { - // Row is not already locked by any thread, use newly created context. - break; - } else if (existingContext.ownedByCurrentThread()) { - // Row is already locked by current thread, reuse existing context instead. - rowLockContext = existingContext; - break; - } else { - if (!waitForLock) { - return null; - } - TraceScope traceScope = null; - try { - if (Trace.isTracing()) { - traceScope = Trace.startSpan("HRegion.getRowLockInternal"); - } - // Row is already locked by some other thread, give up or wait for it - if (!existingContext.latch.await(this.rowLockWaitDuration, TimeUnit.MILLISECONDS)) { - if(traceScope != null) { - traceScope.getSpan().addTimelineAnnotation("Failed to get row lock"); - } - throw new IOException("Timed out waiting for lock for row: " + rowKey); - } - if (traceScope != null) traceScope.close(); - traceScope = null; - } catch (InterruptedException ie) { - LOG.warn("Thread interrupted waiting for lock on row: " + rowKey); - InterruptedIOException iie = new InterruptedIOException(); - iie.initCause(ie); - throw iie; - } finally { - if (traceScope != null) traceScope.close(); - } - } - } - - // allocate new lock for this thread - return rowLockContext.newLock(); - } - - /** - * Acquires a lock on the given row. - * The same thread may acquire multiple locks on the same row. - * @return the acquired row lock - * @throws IOException if the lock could not be acquired after waiting + * Get an exclusive ( write lock ) lock on a given row. + * @param row Which row to lock. + * @return A locked RowLock. The lock is exclusive and already aqquired. + * @throws IOException */ public RowLock getRowLock(byte[] row) throws IOException { - return getRowLock(row, true); + return getRowLock(row, false); + } + + /** + * + * Get a row lock for the specified row. All locks are reentrant. + * + * Before calling this function make sure that a region operation has already been + * started (the calling thread has already acquired the region-close-guard lock). + * @param row The row actions will be performed against + * @param readLock is the lock reader or writer. True indicates that a non-exlcusive + * lock is requested + */ + public RowLock getRowLock(byte[] row, boolean readLock) throws IOException { + // Make sure the row is inside of this region before getting the lock for it. + checkRow(row, "row lock"); + // create an object to use a a key in the row lock map + HashedBytes rowKey = new HashedBytes(row); + + RowLockContext rowLockContext = null; + RowLockImpl result = null; + TraceScope traceScope = null; + + // If we're tracing start a span to show how long this took. + if (Trace.isTracing()) { + traceScope = Trace.startSpan("HRegion.getRowLock"); + traceScope.getSpan().addTimelineAnnotation("Getting a " + (readLock?"readLock":"writeLock")); + } + + try { + // Keep trying until we have a lock or error out. + // TODO: do we need to add a time component here? + while (result == null) { + + // Try adding a RowLockContext to the lockedRows. + // If we can add it then there's no other transactions currently running. + rowLockContext = new RowLockContext(rowKey); + RowLockContext existingContext = lockedRows.putIfAbsent(rowKey, rowLockContext); + + // if there was a running transaction then there's already a context. + if (existingContext != null) { + rowLockContext = existingContext; + } + + // Now try an get the lock. + // + // This can fail as + if (readLock) { + result = rowLockContext.newReadLock(); + } else { + result = rowLockContext.newWriteLock(); + } + } + if (!result.getLock().tryLock(this.rowLockWaitDuration, TimeUnit.MILLISECONDS)) { + if (traceScope != null) { + traceScope.getSpan().addTimelineAnnotation("Failed to get row lock"); + } + result = null; + // Clean up the counts just in case this was the thing keeping the context alive. + rowLockContext.cleanUp(); + throw new IOException("Timed out waiting for lock for row: " + rowKey); + } + return result; + } catch (InterruptedException ie) { + LOG.warn("Thread interrupted waiting for lock on row: " + rowKey); + InterruptedIOException iie = new InterruptedIOException(); + iie.initCause(ie); + if (traceScope != null) { + traceScope.getSpan().addTimelineAnnotation("Interrupted exception getting row lock"); + } + Thread.currentThread().interrupt(); + throw iie; + } finally { + if (traceScope != null) { + traceScope.close(); + } + } } @Override @@ -5118,6 +5119,97 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } } + @VisibleForTesting + class RowLockContext { + private final HashedBytes row; + final ReadWriteLock readWriteLock = new ReentrantReadWriteLock(true); + final AtomicBoolean usable = new AtomicBoolean(true); + final AtomicInteger count = new AtomicInteger(0); + final Object lock = new Object(); + + RowLockContext(HashedBytes row) { + this.row = row; + } + + RowLockImpl newWriteLock() { + Lock l = readWriteLock.writeLock(); + return getRowLock(l); + } + RowLockImpl newReadLock() { + Lock l = readWriteLock.readLock(); + return getRowLock(l); + } + + private RowLockImpl getRowLock(Lock l) { + count.incrementAndGet(); + synchronized (lock) { + if (usable.get()) { + return new RowLockImpl(this, l); + } else { + return null; + } + } + } + + void cleanUp() { + long c = count.decrementAndGet(); + if (c <= 0) { + synchronized (lock) { + if (count.get() <= 0 ){ + usable.set(false); + RowLockContext removed = lockedRows.remove(row); + assert removed == this: "we should never remove a different context"; + } + } + } + } + + @Override + public String toString() { + return "RowLockContext{" + + "row=" + row + + ", readWriteLock=" + readWriteLock + + ", count=" + count + + '}'; + } + } + + /** + * Class used to represent a lock on a row. + */ + public static class RowLockImpl implements RowLock { + private final RowLockContext context; + private final Lock lock; + + public RowLockImpl(RowLockContext context, Lock lock) { + this.context = context; + this.lock = lock; + } + + public Lock getLock() { + return lock; + } + + @VisibleForTesting + public RowLockContext getContext() { + return context; + } + + @Override + public void release() { + lock.unlock(); + context.cleanUp(); + } + + @Override + public String toString() { + return "RowLockImpl{" + + "context=" + context + + ", lock=" + lock + + '}'; + } + } + /** * Determines whether multiple column families are present * Precondition: familyPaths is not null @@ -5263,7 +5355,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi this.getRegionInfo().getTable(), ByteStringer.wrap(this.getRegionInfo().getEncodedNameAsBytes()), storeFiles, seqId); WALUtil.writeBulkLoadMarkerAndSync(wal, this.htableDescriptor, getRegionInfo(), - loadDescriptor, sequenceId); + loadDescriptor); } catch (IOException ioe) { if (this.rsServices != null) { // Have to abort region server because some hfiles has been loaded but we can't write @@ -6010,13 +6102,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi FileSystem fs = FileSystem.get(conf); Path tableDir = FSUtils.getTableDir(rootDir, info.getTable()); HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, info); - HRegion region = HRegion.newHRegion(tableDir, - wal, fs, conf, info, hTableDescriptor, null); - if (initialize) { - // If initializing, set the sequenceId. It is also required by WALPerformanceEvaluation when - // verifying the WALEdits. - region.setSequenceId(region.initialize(null)); - } + HRegion region = HRegion.newHRegion(tableDir, wal, fs, conf, info, hTableDescriptor, null); + if (initialize) region.initialize(null); return region; } @@ -6229,7 +6316,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // Refuse to open the region if a required class cannot be loaded checkClassLoading(); this.openSeqNum = initialize(reporter); - this.setSequenceId(openSeqNum); + this.mvcc.advanceTo(openSeqNum); if (wal != null && getRegionServerServices() != null && !writestate.readOnly && !recovering) { // Only write the region open event marker to WAL if (1) we are not read-only @@ -6664,7 +6751,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi List acquiredRowLocks; long addedSize = 0; List mutations = new ArrayList(); - List memstoreCells = new ArrayList(); Collection rowsToLock = processor.getRowsToLock(); long mvccNum = 0; WALKey walKey = null; @@ -6673,13 +6759,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi acquiredRowLocks = new ArrayList(rowsToLock.size()); for (byte[] row : rowsToLock) { // Attempt to lock all involved rows, throw if any lock times out + // use a writer lock for mixed reads and writes acquiredRowLocks.add(getRowLock(row)); } // 3. Region lock lock(this.updatesLock.readLock(), acquiredRowLocks.size() == 0 ? 1 : acquiredRowLocks.size()); locked = true; - // Get a mvcc write number - mvccNum = MultiVersionConcurrencyControl.getPreAssignedWriteNumber(this.sequenceId); long now = EnvironmentEdgeManager.currentTime(); try { @@ -6689,11 +6774,33 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi processor, now, this, mutations, walEdit, timeout); if (!mutations.isEmpty()) { - // 5. Start mvcc transaction - writeEntry = mvcc.beginMemstoreInsertWithSeqNum(mvccNum); - // 6. Call the preBatchMutate hook + + // 5. Call the preBatchMutate hook processor.preBatchMutate(this, walEdit); - // 7. Apply to memstore + + long txid = 0; + // 6. Append no sync + if (!walEdit.isEmpty()) { + // we use HLogKey here instead of WALKey directly to support legacy coprocessors. + walKey = new HLogKey(this.getRegionInfo().getEncodedNameAsBytes(), + this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, now, + processor.getClusterIds(), nonceGroup, nonce, mvcc); + txid = this.wal.append(this.htableDescriptor, this.getRegionInfo(), + walKey, walEdit, false); + } + if(walKey == null){ + // since we use wal sequence Id as mvcc, for SKIP_WAL changes we need a "faked" WALEdit + // to get a sequence id assigned which is done by FSWALEntry#stampRegionSequenceId + walKey = this.appendEmptyEdit(this.wal); + } + + // 7. Start mvcc transaction + writeEntry = walKey.getWriteEntry(); + mvccNum = walKey.getSequenceId(); + + + + // 8. Apply to memstore for (Mutation m : mutations) { // Handle any tag based cell features rewriteCellTags(m.getFamilyCellMap(), m); @@ -6708,25 +6815,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } Pair ret = store.add(cell); addedSize += ret.getFirst(); - memstoreCells.add(ret.getSecond()); } } - long txid = 0; - // 8. Append no sync - if (!walEdit.isEmpty()) { - // we use HLogKey here instead of WALKey directly to support legacy coprocessors. - walKey = new HLogKey(this.getRegionInfo().getEncodedNameAsBytes(), - this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, now, - processor.getClusterIds(), nonceGroup, nonce); - txid = this.wal.append(this.htableDescriptor, this.getRegionInfo(), - walKey, walEdit, getSequenceId(), true, memstoreCells); - } - if(walKey == null){ - // since we use wal sequence Id as mvcc, for SKIP_WAL changes we need a "faked" WALEdit - // to get a sequence id assigned which is done by FSWALEntry#stampRegionSequenceId - walKey = this.appendEmptyEdit(this.wal, memstoreCells); - } // 9. Release region lock if (locked) { this.updatesLock.readLock().unlock(); @@ -6759,13 +6850,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } } if (writeEntry != null) { - mvcc.cancelMemstoreInsert(writeEntry); + mvcc.complete(writeEntry); writeEntry = null; } } // 13. Roll mvcc forward if (writeEntry != null) { - mvcc.completeMemstoreInsertWithSeqNum(writeEntry, walKey); + mvcc.completeAndWait(writeEntry); } if (locked) { this.updatesLock.readLock().unlock(); @@ -6836,6 +6927,46 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } } + /** + * @param cell + * @param tags + * @return The passed-in List but with the tags from cell added. + */ + private static List carryForwardTags(final Cell cell, final List tags) { + if (cell.getTagsLength() <= 0) return tags; + List newTags = tags == null? new ArrayList(): /*Append Tags*/tags; + Iterator i = + CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength()); + while (i.hasNext()) newTags.add(i.next()); + return newTags; + } + + /** + * Run a Get against passed in store on passed row, etc. + * @param store + * @param row + * @param family + * @param tr + * @return Get result. + * @throws IOException + */ + private List doGet(final Store store, final byte [] row, + final Map.Entry> family, final TimeRange tr) + throws IOException { + // Sort the cells so that they match the order that they + // appear in the Get results. Otherwise, we won't be able to + // find the existing values if the cells are not specified + // in order by the client since cells are in an array list. + Collections.sort(family.getValue(), store.getComparator()); + // Get previous values for all columns in this family + Get get = new Get(row); + for (Cell cell : family.getValue()) { + get.addColumn(family.getKey(), CellUtil.cloneQualifier(cell)); + } + if (tr != null) get.setTimeRange(tr.getMin(), tr.getMax()); + return get(get, false); + } + public Result append(Append append) throws IOException { return append(append, HConstants.NO_NONCE, HConstants.NO_NONCE); } @@ -6845,64 +6976,50 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // transactions, so all stores only go through one code path for puts. @Override - public Result append(Append append, long nonceGroup, long nonce) throws IOException { - byte[] row = append.getRow(); - checkRow(row, "append"); + public Result append(Append mutate, long nonceGroup, long nonce) throws IOException { + Operation op = Operation.APPEND; + byte[] row = mutate.getRow(); + checkRow(row, op.toString()); boolean flush = false; - Durability durability = getEffectiveDurability(append.getDurability()); + Durability durability = getEffectiveDurability(mutate.getDurability()); boolean writeToWAL = durability != Durability.SKIP_WAL; WALEdit walEdits = null; - List allKVs = new ArrayList(append.size()); + List allKVs = new ArrayList(mutate.size()); Map> tempMemstore = new HashMap>(); long size = 0; long txid = 0; - checkReadOnly(); checkResources(); // Lock row - startRegionOperation(Operation.APPEND); + startRegionOperation(op); this.writeRequestsCount.increment(); - long mvccNum = 0; - WriteEntry writeEntry = null; - WALKey walKey = null; RowLock rowLock = null; - List memstoreCells = new ArrayList(); + WALKey walKey = null; + MultiVersionConcurrencyControl.WriteEntry writeEntry = null; boolean doRollBackMemstore = false; try { rowLock = getRowLock(row); + assert rowLock != null; try { lock(this.updatesLock.readLock()); try { - // wait for all prior MVCC transactions to finish - while we hold the row lock - // (so that we are guaranteed to see the latest state) - mvcc.waitForPreviousTransactionsComplete(); + // Wait for all prior MVCC transactions to finish - while we hold the row lock + // (so that we are guaranteed to see the latest state when we do our Get) + mvcc.await(); if (this.coprocessorHost != null) { - Result r = this.coprocessorHost.preAppendAfterRowLock(append); - if(r!= null) { + Result r = this.coprocessorHost.preAppendAfterRowLock(mutate); + if (r!= null) { return r; } } - // now start my own transaction - mvccNum = MultiVersionConcurrencyControl.getPreAssignedWriteNumber(this.sequenceId); - writeEntry = mvcc.beginMemstoreInsertWithSeqNum(mvccNum); long now = EnvironmentEdgeManager.currentTime(); // Process each family - for (Map.Entry> family : append.getFamilyCellMap().entrySet()) { - + for (Map.Entry> family : mutate.getFamilyCellMap().entrySet()) { Store store = stores.get(family.getKey()); List kvs = new ArrayList(family.getValue().size()); - // Sort the cells so that they match the order that they - // appear in the Get results. Otherwise, we won't be able to - // find the existing values if the cells are not specified - // in order by the client since cells are in an array list. - Collections.sort(family.getValue(), store.getComparator()); - // Get previous values for all columns in this family - Get get = new Get(row); - for (Cell cell : family.getValue()) { - get.addColumn(family.getKey(), CellUtil.cloneQualifier(cell)); - } - List results = get(get, false); + List results = doGet(store, row, family, null); + // Iterate the input columns and update existing values if they were // found, otherwise add new column initialized to the append value @@ -6919,30 +7036,15 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi long ts = Math.max(now, oldCell.getTimestamp()); // Process cell tags - List newTags = new ArrayList(); - // Make a union of the set of tags in the old and new KVs - - if (oldCell.getTagsLength() > 0) { - Iterator i = CellUtil.tagsIterator(oldCell.getTagsArray(), - oldCell.getTagsOffset(), oldCell.getTagsLength()); - while (i.hasNext()) { - newTags.add(i.next()); - } - } - if (cell.getTagsLength() > 0) { - Iterator i = CellUtil.tagsIterator(cell.getTagsArray(), - cell.getTagsOffset(), cell.getTagsLength()); - while (i.hasNext()) { - newTags.add(i.next()); - } - } + List newTags = carryForwardTags(oldCell, new ArrayList()); + newTags = carryForwardTags(cell, newTags); // Cell TTL handling - if (append.getTTL() != Long.MAX_VALUE) { + if (mutate.getTTL() != Long.MAX_VALUE) { // Add the new TTL tag - newTags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(append.getTTL()))); + newTags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(mutate.getTTL()))); } // Rebuild tags @@ -6978,9 +7080,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // Cell TTL handling - if (append.getTTL() != Long.MAX_VALUE) { + if (mutate.getTTL() != Long.MAX_VALUE) { List newTags = new ArrayList(1); - newTags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(append.getTTL()))); + newTags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(mutate.getTTL()))); // Add the new TTL tag newCell = new TagRewriteCell(cell, Tag.fromList(newTags)); } else { @@ -6988,11 +7090,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } } - CellUtil.setSequenceId(newCell, mvccNum); // Give coprocessors a chance to update the new cell if (coprocessorHost != null) { newCell = coprocessorHost.postMutationBeforeWAL(RegionObserver.MutationType.APPEND, - append, oldCell, newCell); + mutate, oldCell, newCell); } kvs.add(newCell); @@ -7009,47 +7110,64 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi tempMemstore.put(store, kvs); } - //Actually write to Memstore now - for (Map.Entry> entry : tempMemstore.entrySet()) { - Store store = entry.getKey(); - if (store.getFamily().getMaxVersions() == 1) { - // upsert if VERSIONS for this CF == 1 - size += store.upsert(entry.getValue(), getSmallestReadPoint()); - memstoreCells.addAll(entry.getValue()); - } else { - // otherwise keep older versions around - for (Cell cell: entry.getValue()) { - Pair ret = store.add(cell); - size += ret.getFirst(); - memstoreCells.add(ret.getSecond()); - doRollBackMemstore = true; - } - } - allKVs.addAll(entry.getValue()); - } - // Actually write to WAL now - if (writeToWAL) { - // Using default cluster id, as this can only happen in the originating - // cluster. A slave cluster receives the final value (not the delta) - // as a Put. - // we use HLogKey here instead of WALKey directly to support legacy coprocessors. - walKey = new HLogKey(getRegionInfo().getEncodedNameAsBytes(), - this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, nonceGroup, nonce); - txid = this.wal.append(this.htableDescriptor, getRegionInfo(), walKey, walEdits, - this.sequenceId, true, memstoreCells); - } else { - recordMutationWithoutWal(append.getFamilyCellMap()); + if (walEdits != null && !walEdits.isEmpty()) { + if (writeToWAL) { + // Using default cluster id, as this can only happen in the originating + // cluster. A slave cluster receives the final value (not the delta) + // as a Put. + // we use HLogKey here instead of WALKey directly to support legacy coprocessors. + walKey = new HLogKey( + getRegionInfo().getEncodedNameAsBytes(), + this.htableDescriptor.getTableName(), + WALKey.NO_SEQUENCE_ID, + nonceGroup, + nonce, + mvcc); + txid = + this.wal.append(this.htableDescriptor, getRegionInfo(), walKey, walEdits, true); + } else { + recordMutationWithoutWal(mutate.getFamilyCellMap()); + } } if (walKey == null) { // Append a faked WALEdit in order for SKIP_WAL updates to get mvcc assigned - walKey = this.appendEmptyEdit(this.wal, memstoreCells); + walKey = this.appendEmptyEdit(this.wal); + } + + // now start my own transaction + writeEntry = walKey.getWriteEntry(); + + + // Actually write to Memstore now + if (!tempMemstore.isEmpty()) { + for (Map.Entry> entry : tempMemstore.entrySet()) { + Store store = entry.getKey(); + if (store.getFamily().getMaxVersions() == 1) { + // upsert if VERSIONS for this CF == 1 + // Is this right? It immediately becomes visible? St.Ack 20150907 + size += store.upsert(entry.getValue(), getSmallestReadPoint()); + } else { + // otherwise keep older versions around + for (Cell cell: entry.getValue()) { + CellUtil.setSequenceId(cell, writeEntry.getWriteNumber()); + Pair ret = store.add(cell); + size += ret.getFirst(); + doRollBackMemstore = true; + } + } + // We add to all KVs here whereas when doing increment, we do it + // earlier... why? + allKVs.addAll(entry.getValue()); + } + + size = this.addAndGetGlobalMemstoreSize(size); + flush = isFlushSize(size); } - size = this.addAndGetGlobalMemstoreSize(size); - flush = isFlushSize(size); } finally { this.updatesLock.readLock().unlock(); } + } finally { rowLock.release(); rowLock = null; @@ -7065,13 +7183,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } // if the wal sync was unsuccessful, remove keys from memstore if (doRollBackMemstore) { - rollbackMemstore(memstoreCells); - if (writeEntry != null) mvcc.cancelMemstoreInsert(writeEntry); + rollbackMemstore(allKVs); + if (writeEntry != null) mvcc.complete(writeEntry); } else if (writeEntry != null) { - mvcc.completeMemstoreInsertWithSeqNum(writeEntry, walKey); + mvcc.completeAndWait(writeEntry); } - closeRegionOperation(Operation.APPEND); + closeRegionOperation(op); } if (this.metricsRegion != null) { @@ -7083,8 +7201,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi requestFlush(); } - - return append.isReturnResults() ? Result.create(allKVs) : null; + return mutate.isReturnResults() ? Result.create(allKVs) : null; } public Result increment(Increment increment) throws IOException { @@ -7095,89 +7212,73 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // We should refactor append and increment as local get-mutate-put // transactions, so all stores only go through one code path for puts. + // They are subtley different in quiet a few ways. This came out only + // after study. I am not sure that many of the differences are intentional. + // TODO: St.Ack 20150907 + @Override - public Result increment(Increment increment, long nonceGroup, long nonce) + public Result increment(Increment mutation, long nonceGroup, long nonce) throws IOException { - byte [] row = increment.getRow(); - checkRow(row, "increment"); - TimeRange tr = increment.getTimeRange(); + Operation op = Operation.INCREMENT; + byte [] row = mutation.getRow(); + checkRow(row, op.toString()); boolean flush = false; - Durability durability = getEffectiveDurability(increment.getDurability()); + Durability durability = getEffectiveDurability(mutation.getDurability()); boolean writeToWAL = durability != Durability.SKIP_WAL; WALEdit walEdits = null; - List allKVs = new ArrayList(increment.size()); + List allKVs = new ArrayList(mutation.size()); + Map> tempMemstore = new HashMap>(); - long size = 0; long txid = 0; - checkReadOnly(); checkResources(); // Lock row - startRegionOperation(Operation.INCREMENT); + startRegionOperation(op); this.writeRequestsCount.increment(); RowLock rowLock = null; - WriteEntry writeEntry = null; WALKey walKey = null; - long mvccNum = 0; - List memstoreCells = new ArrayList(); + MultiVersionConcurrencyControl.WriteEntry writeEntry = null; boolean doRollBackMemstore = false; + TimeRange tr = mutation.getTimeRange(); try { rowLock = getRowLock(row); + assert rowLock != null; try { lock(this.updatesLock.readLock()); try { // wait for all prior MVCC transactions to finish - while we hold the row lock // (so that we are guaranteed to see the latest state) - mvcc.waitForPreviousTransactionsComplete(); + mvcc.await(); if (this.coprocessorHost != null) { - Result r = this.coprocessorHost.preIncrementAfterRowLock(increment); + Result r = this.coprocessorHost.preIncrementAfterRowLock(mutation); if (r != null) { return r; } } - // now start my own transaction - mvccNum = MultiVersionConcurrencyControl.getPreAssignedWriteNumber(this.sequenceId); - writeEntry = mvcc.beginMemstoreInsertWithSeqNum(mvccNum); long now = EnvironmentEdgeManager.currentTime(); // Process each family - for (Map.Entry> family: - increment.getFamilyCellMap().entrySet()) { - + for (Map.Entry> family: mutation.getFamilyCellMap().entrySet()) { Store store = stores.get(family.getKey()); List kvs = new ArrayList(family.getValue().size()); - // Sort the cells so that they match the order that they - // appear in the Get results. Otherwise, we won't be able to - // find the existing values if the cells are not specified - // in order by the client since cells are in an array list. - Collections.sort(family.getValue(), store.getComparator()); - // Get previous values for all columns in this family - Get get = new Get(row); - for (Cell cell: family.getValue()) { - get.addColumn(family.getKey(), CellUtil.cloneQualifier(cell)); - } - get.setTimeRange(tr.getMin(), tr.getMax()); - List results = get(get, false); + List results = doGet(store, row, family, tr); // Iterate the input columns and update existing values if they were // found, otherwise add new column initialized to the increment amount + + // Avoid as much copying as possible. We may need to rewrite and + // consolidate tags. Bytes are only copied once. + // Would be nice if KeyValue had scatter/gather logic int idx = 0; + // HERE WE DIVERGE FROM APPEND List edits = family.getValue(); for (int i = 0; i < edits.size(); i++) { Cell cell = edits.get(i); long amount = Bytes.toLong(CellUtil.cloneValue(cell)); boolean noWriteBack = (amount == 0); - List newTags = new ArrayList(); - // Carry forward any tags that might have been added by a coprocessor - if (cell.getTagsLength() > 0) { - Iterator itr = CellUtil.tagsIterator(cell.getTagsArray(), - cell.getTagsOffset(), cell.getTagsLength()); - while (itr.hasNext()) { - newTags.add(itr.next()); - } - } + List newTags = carryForwardTags(cell, new ArrayList()); Cell c = null; long ts = now; @@ -7192,15 +7293,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi "Attempted to increment field that isn't 64 bits wide"); } // Carry tags forward from previous version - if (c.getTagsLength() > 0) { - Iterator itr = CellUtil.tagsIterator(c.getTagsArray(), - c.getTagsOffset(), c.getTagsLength()); - while (itr.hasNext()) { - newTags.add(itr.next()); - } - } - if (i < (edits.size() - 1) && !CellUtil.matchingQualifier(cell, edits.get(i + 1))) + newTags = carryForwardTags(c, newTags); + if (i < (edits.size() - 1) && !CellUtil.matchingQualifier(cell, edits.get(i + 1))) { idx++; + } } // Append new incremented KeyValue to list @@ -7208,8 +7304,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi byte[] val = Bytes.toBytes(amount); // Add the TTL tag if the mutation carried one - if (increment.getTTL() != Long.MAX_VALUE) { - newTags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(increment.getTTL()))); + if (mutation.getTTL() != Long.MAX_VALUE) { + newTags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(mutation.getTTL()))); } Cell newKV = new KeyValue(row, 0, row.length, @@ -7220,12 +7316,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi val, 0, val.length, newTags); - CellUtil.setSequenceId(newKV, mvccNum); - // Give coprocessors a chance to update the new cell if (coprocessorHost != null) { newKV = coprocessorHost.postMutationBeforeWAL( - RegionObserver.MutationType.INCREMENT, increment, c, newKV); + RegionObserver.MutationType.INCREMENT, mutation, c, newKV); } allKVs.add(newKV); @@ -7248,28 +7342,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } } - //Actually write to Memstore now - if (!tempMemstore.isEmpty()) { - for (Map.Entry> entry : tempMemstore.entrySet()) { - Store store = entry.getKey(); - if (store.getFamily().getMaxVersions() == 1) { - // upsert if VERSIONS for this CF == 1 - size += store.upsert(entry.getValue(), getSmallestReadPoint()); - memstoreCells.addAll(entry.getValue()); - } else { - // otherwise keep older versions around - for (Cell cell : entry.getValue()) { - Pair ret = store.add(cell); - size += ret.getFirst(); - memstoreCells.add(ret.getSecond()); - doRollBackMemstore = true; - } - } - } - size = this.addAndGetGlobalMemstoreSize(size); - flush = isFlushSize(size); - } - // Actually write to WAL now if (walEdits != null && !walEdits.isEmpty()) { if (writeToWAL) { @@ -7278,16 +7350,45 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // as a Put. // we use HLogKey here instead of WALKey directly to support legacy coprocessors. walKey = new HLogKey(this.getRegionInfo().getEncodedNameAsBytes(), - this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, nonceGroup, nonce); + this.htableDescriptor.getTableName(), + WALKey.NO_SEQUENCE_ID, + nonceGroup, + nonce, + mvcc); txid = this.wal.append(this.htableDescriptor, this.getRegionInfo(), - walKey, walEdits, getSequenceId(), true, memstoreCells); + walKey, walEdits, true); } else { - recordMutationWithoutWal(increment.getFamilyCellMap()); + recordMutationWithoutWal(mutation.getFamilyCellMap()); } } - if(walKey == null){ + if (walKey == null) { // Append a faked WALEdit in order for SKIP_WAL updates to get mvccNum assigned - walKey = this.appendEmptyEdit(this.wal, memstoreCells); + walKey = this.appendEmptyEdit(this.wal); + } + + // now start my own transaction + writeEntry = walKey.getWriteEntry(); + + // Actually write to Memstore now + if (!tempMemstore.isEmpty()) { + for (Map.Entry> entry : tempMemstore.entrySet()) { + Store store = entry.getKey(); + if (store.getFamily().getMaxVersions() == 1) { + // upsert if VERSIONS for this CF == 1 + // Is this right? It immediately becomes visible? St.Ack 20150907 + size += store.upsert(entry.getValue(), getSmallestReadPoint()); + } else { + // otherwise keep older versions around + for (Cell cell : entry.getValue()) { + CellUtil.setSequenceId(cell, writeEntry.getWriteNumber()); + Pair ret = store.add(cell); + size += ret.getFirst(); + doRollBackMemstore = true; + } + } + } + size = this.addAndGetGlobalMemstoreSize(size); + flush = isFlushSize(size); } } finally { this.updatesLock.readLock().unlock(); @@ -7307,10 +7408,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } // if the wal sync was unsuccessful, remove keys from memstore if (doRollBackMemstore) { - rollbackMemstore(memstoreCells); - if (writeEntry != null) mvcc.cancelMemstoreInsert(writeEntry); + for(List cells: tempMemstore.values()) { + rollbackMemstore(cells); + } + if (writeEntry != null) mvcc.complete(writeEntry); } else if (writeEntry != null) { - mvcc.completeMemstoreInsertWithSeqNum(writeEntry, walKey); + mvcc.completeAndWait(writeEntry); } closeRegionOperation(Operation.INCREMENT); if (this.metricsRegion != null) { @@ -7322,7 +7425,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // Request a cache flush. Do it outside update lock. requestFlush(); } - return increment.isReturnResults() ? Result.create(allKVs) : null; + return mutation.isReturnResults() ? Result.create(allKVs) : null; } // @@ -7341,7 +7444,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi public static final long FIXED_OVERHEAD = ClassSize.align( ClassSize.OBJECT + ClassSize.ARRAY + - 44 * ClassSize.REFERENCE + 3 * Bytes.SIZEOF_INT + + 43 * ClassSize.REFERENCE + 3 * Bytes.SIZEOF_INT + (14 * Bytes.SIZEOF_LONG) + 5 * Bytes.SIZEOF_BOOLEAN); @@ -7487,7 +7590,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi throw new IOException("Not a known catalog table: " + p.toString()); } try { - region.initialize(null); + region.mvcc.advanceTo(region.initialize(null)); if (majorCompact) { region.compact(true); } else { @@ -7905,110 +8008,32 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } /** - * Do not change this sequence id. See {@link #sequenceId} comment. + * Do not change this sequence id. * @return sequenceId */ @VisibleForTesting - public AtomicLong getSequenceId() { - return this.sequenceId; + public long getSequenceId() { + return this.mvcc.getReadPoint(); } - /** - * sets this region's sequenceId. - * @param value new value - */ - private void setSequenceId(long value) { - this.sequenceId.set(value); - } - - @VisibleForTesting class RowLockContext { - private final HashedBytes row; - private final CountDownLatch latch = new CountDownLatch(1); - private final Thread thread; - private int lockCount = 0; - - RowLockContext(HashedBytes row) { - this.row = row; - this.thread = Thread.currentThread(); - } - - boolean ownedByCurrentThread() { - return thread == Thread.currentThread(); - } - - RowLock newLock() { - lockCount++; - RowLockImpl rl = new RowLockImpl(); - rl.setContext(this); - return rl; - } - - @Override - public String toString() { - Thread t = this.thread; - return "Thread=" + (t == null? "null": t.getName()) + ", row=" + this.row + - ", lockCount=" + this.lockCount; - } - - void releaseLock() { - if (!ownedByCurrentThread()) { - throw new IllegalArgumentException("Lock held by thread: " + thread - + " cannot be released by different thread: " + Thread.currentThread()); - } - lockCount--; - if (lockCount == 0) { - // no remaining locks by the thread, unlock and allow other threads to access - RowLockContext existingContext = lockedRows.remove(row); - if (existingContext != this) { - throw new RuntimeException( - "Internal row lock state inconsistent, should not happen, row: " + row); - } - latch.countDown(); - } - } - } - - public static class RowLockImpl implements RowLock { - private RowLockContext context; - private boolean released = false; - - @VisibleForTesting - public RowLockContext getContext() { - return context; - } - - @VisibleForTesting - public void setContext(RowLockContext context) { - this.context = context; - } - - @Override - public void release() { - if (!released) { - context.releaseLock(); - } - released = true; - } - } /** * Append a faked WALEdit in order to get a long sequence number and wal syncer will just ignore * the WALEdit append later. * @param wal - * @param cells list of Cells inserted into memstore. Those Cells are passed in order to - * be updated with right mvcc values(their wal sequence number) * @return Return the key used appending with no sync and no append. * @throws IOException */ - private WALKey appendEmptyEdit(final WAL wal, List cells) throws IOException { + private WALKey appendEmptyEdit(final WAL wal) throws IOException { // we use HLogKey here instead of WALKey directly to support legacy coprocessors. @SuppressWarnings("deprecation") - WALKey key = new HLogKey(getRegionInfo().getEncodedNameAsBytes(), getRegionInfo().getTable(), - WALKey.NO_SEQUENCE_ID, 0, null, HConstants.NO_NONCE, HConstants.NO_NONCE); + WALKey key = new HLogKey(getRegionInfo().getEncodedNameAsBytes(), + getRegionInfo().getTable(), WALKey.NO_SEQUENCE_ID, 0, null, + HConstants.NO_NONCE, HConstants.NO_NONCE, getMVCC()); + // Call append but with an empty WALEdit. The returned seqeunce id will not be associated // with any edit and we can be sure it went in after all outstanding appends. - wal.append(getTableDesc(), getRegionInfo(), key, WALEdit.EMPTY_WALEDIT, getSequenceId(), false, - cells); + wal.append(getTableDesc(), getRegionInfo(), key, WALEdit.EMPTY_WALEDIT, false); return key; } 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 13413974b25..cfda1c62930 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 @@ -651,7 +651,7 @@ public class HStore implements Store { // readers might pick it up. This assumes that the store is not getting any writes (otherwise // in-flight transactions might be made visible) if (!toBeAddedFiles.isEmpty()) { - region.getMVCC().advanceMemstoreReadPointIfNeeded(this.getMaxSequenceId()); + region.getMVCC().advanceTo(this.getMaxSequenceId()); } // notify scanners, close file readers, and recompute store size @@ -1308,7 +1308,7 @@ public class HStore implements Store { CompactionDescriptor compactionDescriptor = ProtobufUtil.toCompactionDescriptor(info, family.getName(), inputPaths, outputPaths, fs.getStoreDir(getFamily().getNameAsString())); WALUtil.writeCompactionMarker(region.getWAL(), this.region.getTableDesc(), - this.region.getRegionInfo(), compactionDescriptor, this.region.getSequenceId()); + this.region.getRegionInfo(), compactionDescriptor, region.getMVCC()); } @VisibleForTesting diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiVersionConcurrencyControl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiVersionConcurrencyControl.java index 2d653873ba7..d101f7b393c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiVersionConcurrencyControl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiVersionConcurrencyControl.java @@ -18,239 +18,198 @@ */ package org.apache.hadoop.hbase.regionserver; -import java.io.IOException; import java.util.LinkedList; import java.util.concurrent.atomic.AtomicLong; +import com.google.common.annotations.VisibleForTesting; + import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; /** - * Manages the read/write consistency within memstore. This provides - * an interface for readers to determine what entries to ignore, and - * a mechanism for writers to obtain new write numbers, then "commit" + * Manages the read/write consistency. This provides an interface for readers to determine what + * entries to ignore, and a mechanism for writers to obtain new write numbers, then "commit" * the new writes for readers to read (thus forming atomic transactions). */ @InterfaceAudience.Private public class MultiVersionConcurrencyControl { - private static final long NO_WRITE_NUMBER = 0; - private volatile long memstoreRead = 0; + final AtomicLong readPoint = new AtomicLong(0); + final AtomicLong writePoint = new AtomicLong(0); private final Object readWaiters = new Object(); + /** + * Represents no value, or not set. + */ + private static final long NONE = -1; // This is the pending queue of writes. - private final LinkedList writeQueue = - new LinkedList(); + // + // TODO(eclark): Should this be an array of fixed size to + // reduce the number of allocations on the write path? + // This could be equal to the number of handlers + a small number. + // TODO: St.Ack 20150903 Sounds good to me. + private final LinkedList writeQueue = new LinkedList(); - /** - * Default constructor. Initializes the memstoreRead/Write points to 0. - */ public MultiVersionConcurrencyControl() { + super(); } /** - * Initializes the memstoreRead/Write points appropriately. - * @param startPoint + * Construct and set read point. Write point is uninitialized. */ - public void initialize(long startPoint) { - synchronized (writeQueue) { - writeQueue.clear(); - memstoreRead = startPoint; + public MultiVersionConcurrencyControl(long startPoint) { + tryAdvanceTo(startPoint, NONE); + } + + /** + * Step the MVCC forward on to a new read/write basis. + * @param newStartPoint + */ + public void advanceTo(long newStartPoint) { + while (true) { + long seqId = this.getWritePoint(); + if (seqId >= newStartPoint) break; + if (this.tryAdvanceTo(/* newSeqId = */ newStartPoint, /* expected = */ seqId)) break; } } /** - * - * @param initVal The value we used initially and expected it'll be reset later - * @return WriteEntry instance. + * Step the MVCC forward on to a new read/write basis. + * @param newStartPoint Point to move read and write points to. + * @param expected If not -1 (#NONE) + * @return Returns false if expected is not equal to the + * current readPoint or if startPoint is less than current + * readPoint */ - WriteEntry beginMemstoreInsert() { - return beginMemstoreInsertWithSeqNum(NO_WRITE_NUMBER); - } - - /** - * Get a mvcc write number before an actual one(its log sequence Id) being assigned - * @param sequenceId - * @return long a faked write number which is bigger enough not to be seen by others before a real - * one is assigned - */ - public static long getPreAssignedWriteNumber(AtomicLong sequenceId) { - // the 1 billion is just an arbitrary big number to guard no scanner will reach it before - // current MVCC completes. Theoretically the bump only needs to be 2 * the number of handlers - // because each handler could increment sequence num twice and max concurrent in-flight - // transactions is the number of RPC handlers. - // We can't use Long.MAX_VALUE because we still want to maintain the ordering when multiple - // changes touch same row key. - // If for any reason, the bumped value isn't reset due to failure situations, we'll reset - // curSeqNum to NO_WRITE_NUMBER in order NOT to advance memstore read point at all. - // St.Ack 20150901 Where is the reset to NO_WRITE_NUMBER done? - return sequenceId.incrementAndGet() + 1000000000; - } - - /** - * This function starts a MVCC transaction with current region's log change sequence number. Since - * we set change sequence number when flushing current change to WAL(late binding), the flush - * order may differ from the order to start a MVCC transaction. For example, a change begins a - * MVCC firstly may complete later than a change which starts MVCC at a later time. Therefore, we - * add a safe bumper to the passed in sequence number to start a MVCC so that no other concurrent - * transactions will reuse the number till current MVCC completes(success or fail). The "faked" - * big number is safe because we only need it to prevent current change being seen and the number - * will be reset to real sequence number(set in log sync) right before we complete a MVCC in order - * for MVCC to align with flush sequence. - * @param curSeqNum - * @return WriteEntry a WriteEntry instance with the passed in curSeqNum - */ - public WriteEntry beginMemstoreInsertWithSeqNum(long curSeqNum) { - WriteEntry e = new WriteEntry(curSeqNum); + boolean tryAdvanceTo(long newStartPoint, long expected) { synchronized (writeQueue) { + long currentRead = this.readPoint.get(); + long currentWrite = this.writePoint.get(); + if (currentRead != currentWrite) { + throw new RuntimeException("Already used this mvcc; currentRead=" + currentRead + + ", currentWrite=" + currentWrite + "; too late to tryAdvanceTo"); + } + if (expected != NONE && expected != currentRead) { + return false; + } + + if (newStartPoint < currentRead) { + return false; + } + + readPoint.set(newStartPoint); + writePoint.set(newStartPoint); + } + return true; + } + + /** + * Start a write transaction. Create a new {@link WriteEntry} with a new write number and add it + * to our queue of ongoing writes. Return this WriteEntry instance. + * To complete the write transaction and wait for it to be visible, call + * {@link #completeAndWait(WriteEntry)}. If the write failed, call + * {@link #complete(WriteEntry)} so we can clean up AFTER removing ALL trace of the failed write + * transaction. + * @see #complete(WriteEntry) + * @see #completeAndWait(WriteEntry) + */ + public WriteEntry begin() { + synchronized (writeQueue) { + long nextWriteNumber = writePoint.incrementAndGet(); + WriteEntry e = new WriteEntry(nextWriteNumber); writeQueue.add(e); return e; } } /** - * Complete a {@link WriteEntry} that was created by - * {@link #beginMemstoreInsertWithSeqNum(long)}. At the end of this call, the global read - * point is at least as large as the write point of the passed in WriteEntry. Thus, the write is - * visible to MVCC readers. - * @throws IOException + * Wait until the read point catches up to the write point; i.e. wait on all outstanding mvccs + * to complete. */ - public void completeMemstoreInsertWithSeqNum(WriteEntry e, SequenceId seqId) - throws IOException { - if(e == null) return; - if (seqId != null) { - e.setWriteNumber(seqId.getSequenceId()); - } else { - // set the value to NO_WRITE_NUMBER in order NOT to advance memstore readpoint inside - // function beginMemstoreInsertWithSeqNum in case of failures - e.setWriteNumber(NO_WRITE_NUMBER); - } - waitForPreviousTransactionsComplete(e); + public void await() { + // Add a write and then wait on reads to catch up to it. + completeAndWait(begin()); } /** - * Cancel a write insert that failed. - * Removes the write entry without advancing read point or without interfering with write - * entries queued behind us. It is like #advanceMemstore(WriteEntry) only this method - * will move the read point to the sequence id that is in WriteEntry even if it ridiculous (see - * the trick in HRegion where we call {@link #getPreAssignedWriteNumber(AtomicLong)} just to mark - * it as for special handling). - * @param writeEntry Failed attempt at write. Does cleanup. + * Complete a {@link WriteEntry} that was created by {@link #begin()} then wait until the + * read point catches up to our write. + * + * At the end of this call, the global read point is at least as large as the write point + * of the passed in WriteEntry. Thus, the write is visible to MVCC readers. */ - public void cancelMemstoreInsert(WriteEntry writeEntry) { - // I'm not clear on how this voodoo all works but setting write number to -1 does NOT advance - // readpoint and gets my little writeEntry completed and removed from queue of outstanding - // events which seems right. St.Ack 20150901. - writeEntry.setWriteNumber(NO_WRITE_NUMBER); - advanceMemstore(writeEntry); + public void completeAndWait(WriteEntry e) { + complete(e); + waitForRead(e); } /** - * Complete a {@link WriteEntry} that was created by {@link #beginMemstoreInsert()}. At the - * end of this call, the global read point is at least as large as the write point of the passed - * in WriteEntry. Thus, the write is visible to MVCC readers. - */ - public void completeMemstoreInsert(WriteEntry e) { - waitForPreviousTransactionsComplete(e); - } - - /** - * Mark the {@link WriteEntry} as complete and advance the read point as - * much as possible. + * Mark the {@link WriteEntry} as complete and advance the read point as much as possible. + * Call this even if the write has FAILED (AFTER backing out the write transaction + * changes completely) so we can clean up the outstanding transaction. * * How much is the read point advanced? - * Let S be the set of all write numbers that are completed and where all previous write numbers - * are also completed. Then, the read point is advanced to the supremum of S. + * + * Let S be the set of all write numbers that are completed. Set the read point to the highest + * numbered write of S. + * + * @param writeEntry * - * @param e * @return true if e is visible to MVCC readers (that is, readpoint >= e.writeNumber) */ - boolean advanceMemstore(WriteEntry e) { - long nextReadValue = -1; + public boolean complete(WriteEntry writeEntry) { synchronized (writeQueue) { - e.markCompleted(); + writeEntry.markCompleted(); + long nextReadValue = NONE; + boolean ranOnce = false; while (!writeQueue.isEmpty()) { + ranOnce = true; WriteEntry queueFirst = writeQueue.getFirst(); + + if (nextReadValue > 0) { + if (nextReadValue + 1 != queueFirst.getWriteNumber()) { + throw new RuntimeException("Invariant in complete violated, nextReadValue=" + + nextReadValue + ", writeNumber=" + queueFirst.getWriteNumber()); + } + } + if (queueFirst.isCompleted()) { - // Using Max because Edit complete in WAL sync order not arriving order - nextReadValue = Math.max(nextReadValue, queueFirst.getWriteNumber()); + nextReadValue = queueFirst.getWriteNumber(); writeQueue.removeFirst(); } else { break; } } - if (nextReadValue > memstoreRead) { - memstoreRead = nextReadValue; + if (!ranOnce) { + throw new RuntimeException("There is no first!"); } - // notify waiters on writeQueue before return - writeQueue.notifyAll(); - } - - if (nextReadValue > 0) { - synchronized (readWaiters) { - readWaiters.notifyAll(); - } - } - - if (memstoreRead >= e.getWriteNumber()) { - return true; - } - return false; - } - - /** - * Advances the current read point to be given seqNum if it is smaller than - * that. - */ - void advanceMemstoreReadPointIfNeeded(long seqNum) { - synchronized (writeQueue) { - if (this.memstoreRead < seqNum) { - memstoreRead = seqNum; - } - } - } - - /** - * Wait for all previous MVCC transactions complete - */ - public void waitForPreviousTransactionsComplete() { - WriteEntry w = beginMemstoreInsert(); - waitForPreviousTransactionsComplete(w); - } - - public void waitForPreviousTransactionsComplete(WriteEntry waitedEntry) { - boolean interrupted = false; - WriteEntry w = waitedEntry; - - try { - WriteEntry firstEntry = null; - do { - synchronized (writeQueue) { - // writeQueue won't be empty at this point, the following is just a safety check - if (writeQueue.isEmpty()) { - break; - } - firstEntry = writeQueue.getFirst(); - if (firstEntry == w) { - // all previous in-flight transactions are done - break; - } - try { - writeQueue.wait(0); - } catch (InterruptedException ie) { - // We were interrupted... finish the loop -- i.e. cleanup --and then - // on our way out, reset the interrupt flag. - interrupted = true; - break; - } + if (nextReadValue > 0) { + synchronized (readWaiters) { + readPoint.set(nextReadValue); + readWaiters.notifyAll(); + } + } + return readPoint.get() >= writeEntry.getWriteNumber(); + } + } + + /** + * Wait for the global readPoint to advance up to the passed in write entry number. + */ + void waitForRead(WriteEntry e) { + boolean interrupted = false; + synchronized (readWaiters) { + while (readPoint.get() < e.getWriteNumber()) { + try { + readWaiters.wait(0); + } catch (InterruptedException ie) { + // We were interrupted... finish the loop -- i.e. cleanup --and then + // on our way out, reset the interrupt flag. + interrupted = true; } - } while (firstEntry != null); - } finally { - if (w != null) { - advanceMemstore(w); } } if (interrupted) { @@ -258,34 +217,43 @@ public class MultiVersionConcurrencyControl { } } - public long memstoreReadPoint() { - return memstoreRead; + public long getReadPoint() { + return readPoint.get(); } + @VisibleForTesting + public long getWritePoint() { + return writePoint.get(); + } + + /** + * Write number and whether write has completed given out at start of a write transaction. + * Every created WriteEntry must be completed by calling mvcc#complete or #completeAndWait. + */ + @InterfaceAudience.Private public static class WriteEntry { - private long writeNumber; - private volatile boolean completed = false; + private final long writeNumber; + private boolean completed = false; WriteEntry(long writeNumber) { this.writeNumber = writeNumber; } + void markCompleted() { this.completed = true; } + boolean isCompleted() { return this.completed; } - long getWriteNumber() { + + public long getWriteNumber() { return this.writeNumber; } - void setWriteNumber(long val){ - this.writeNumber = val; - } } public static final long FIXED_SIZE = ClassSize.align( ClassSize.OBJECT + 2 * Bytes.SIZEOF_LONG + 2 * ClassSize.REFERENCE); - -} +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java index 1c298273cd9..0e4a585dade 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java @@ -33,7 +33,6 @@ import java.util.List; import java.util.Map; import java.util.NavigableMap; import java.util.Set; -import java.util.UUID; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentSkipListMap; @@ -63,7 +62,6 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; @@ -111,7 +109,7 @@ import com.lmax.disruptor.dsl.ProducerType; * *

To read an WAL, call {@link WALFactory#createReader(org.apache.hadoop.fs.FileSystem, * org.apache.hadoop.fs.Path)}. - * + * *

Failure Semantic

* If an exception on append or sync, roll the WAL because the current WAL is now a lame duck; * any more appends or syncs will fail also with the same original exception. If we have made @@ -141,7 +139,7 @@ public class FSHLog implements WAL { // Calls to append now also wait until the append has been done on the consumer side of the // disruptor. We used to not wait but it makes the implemenation easier to grok if we have // the region edit/sequence id after the append returns. - // + // // TODO: Handlers need to coordinate appending AND syncing. Can we have the threads contend // once only? Probably hard given syncs take way longer than an append. // @@ -232,7 +230,7 @@ public class FSHLog implements WAL { private final String logFilePrefix; /** - * Suffix included on generated wal file names + * Suffix included on generated wal file names */ private final String logFileSuffix; @@ -249,13 +247,14 @@ public class FSHLog implements WAL { protected final Configuration conf; /** Listeners that are called on WAL events. */ - private final List listeners = new CopyOnWriteArrayList(); + private final List listeners = + new CopyOnWriteArrayList(); @Override public void registerWALActionsListener(final WALActionsListener listener) { this.listeners.add(listener); } - + @Override public boolean unregisterWALActionsListener(final WALActionsListener listener) { return this.listeners.remove(listener); @@ -618,7 +617,7 @@ public class FSHLog implements WAL { /** * Tell listeners about pre log roll. - * @throws IOException + * @throws IOException */ private void tellListenersAboutPreLogRoll(final Path oldPath, final Path newPath) throws IOException { @@ -631,7 +630,7 @@ public class FSHLog implements WAL { /** * Tell listeners about post log roll. - * @throws IOException + * @throws IOException */ private void tellListenersAboutPostLogRoll(final Path oldPath, final Path newPath) throws IOException { @@ -1059,27 +1058,11 @@ public class FSHLog implements WAL { } } - /** - * @param now - * @param encodedRegionName Encoded name of the region as returned by - * HRegionInfo#getEncodedNameAsBytes(). - * @param tableName - * @param clusterIds that have consumed the change - * @return New log key. - */ - @SuppressWarnings("deprecation") - protected WALKey makeKey(byte[] encodedRegionName, TableName tableName, long seqnum, - long now, List clusterIds, long nonceGroup, long nonce) { - // we use HLogKey here instead of WALKey directly to support legacy coprocessors. - return new HLogKey(encodedRegionName, tableName, seqnum, now, clusterIds, nonceGroup, nonce); - } - @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NP_NULL_ON_SOME_PATH_EXCEPTION", justification="Will never be null") @Override public long append(final HTableDescriptor htd, final HRegionInfo hri, final WALKey key, - final WALEdit edits, final AtomicLong sequenceId, final boolean inMemstore, - final List memstoreCells) throws IOException { + final WALEdit edits, final boolean inMemstore) throws IOException { if (this.closed) throw new IOException("Cannot append; log is closed"); // Make a trace scope for the append. It is closed on other side of the ring buffer by the // single consuming thread. Don't have to worry about it. @@ -1093,9 +1076,9 @@ public class FSHLog implements WAL { try { RingBufferTruck truck = this.disruptor.getRingBuffer().get(sequence); // Construction of FSWALEntry sets a latch. The latch is thrown just after we stamp the - // edit with its edit/sequence id. The below entry.getRegionSequenceId will wait on the - // latch to be thrown. TODO: reuse FSWALEntry as we do SyncFuture rather create per append. - entry = new FSWALEntry(sequence, key, edits, sequenceId, inMemstore, htd, hri, memstoreCells); + // edit with its edit/sequence id. + // TODO: reuse FSWALEntry as we do SyncFuture rather create per append. + entry = new FSWALEntry(sequence, key, edits, htd, hri, inMemstore); truck.loadPayload(entry, scope.detach()); } finally { this.disruptor.getRingBuffer().publish(sequence); @@ -1122,9 +1105,9 @@ public class FSHLog implements WAL { private volatile long sequence; // Keep around last exception thrown. Clear on successful sync. private final BlockingQueue syncFutures; - + /** - * UPDATE! + * UPDATE! * @param syncs the batch of calls to sync that arrived as this thread was starting; when done, * we will put the result of the actual hdfs sync call as the result. * @param sequence The sequence number on the ring buffer when this thread was set running. @@ -1172,7 +1155,7 @@ public class FSHLog implements WAL { // This function releases one sync future only. return 1; } - + /** * Release all SyncFutures whose sequence is <= currentSequence. * @param currentSequence @@ -1604,7 +1587,7 @@ public class FSHLog implements WAL { * 'safe point' while the orchestrating thread does some work that requires the first thread * paused: e.g. holding the WAL writer while its WAL is swapped out from under it by another * thread. - * + * *

Thread A signals Thread B to hold when it gets to a 'safe point'. Thread A wait until * Thread B gets there. When the 'safe point' has been attained, Thread B signals Thread A. * Thread B then holds at the 'safe point'. Thread A on notification that Thread B is paused, @@ -1612,7 +1595,7 @@ public class FSHLog implements WAL { * it flags B and then Thread A and Thread B continue along on their merry way. Pause and * signalling 'zigzags' between the two participating threads. We use two latches -- one the * inverse of the other -- pausing and signaling when states are achieved. - * + * *

To start up the drama, Thread A creates an instance of this class each time it would do * this zigzag dance and passes it to Thread B (these classes use Latches so it is one shot * only). Thread B notices the new instance (via reading a volatile reference or how ever) and it @@ -1634,7 +1617,7 @@ public class FSHLog implements WAL { * Latch to wait on. Will be released when we can proceed. */ private volatile CountDownLatch safePointReleasedLatch = new CountDownLatch(1); - + /** * For Thread A to call when it is ready to wait on the 'safe point' to be attained. * Thread A will be held in here until Thread B calls {@link #safePointAttained()} @@ -1643,7 +1626,7 @@ public class FSHLog implements WAL { * @throws InterruptedException * @throws ExecutionException * @return The passed syncFuture - * @throws FailedSyncBeforeLogCloseException + * @throws FailedSyncBeforeLogCloseException */ SyncFuture waitSafePoint(final SyncFuture syncFuture) throws InterruptedException, FailedSyncBeforeLogCloseException { @@ -1655,7 +1638,7 @@ public class FSHLog implements WAL { } return syncFuture; } - + /** * Called by Thread B when it attains the 'safe point'. In this method, Thread B signals * Thread A it can proceed. Thread B will be held in here until {@link #releaseSafePoint()} @@ -1893,9 +1876,8 @@ public class FSHLog implements WAL { // here inside this single appending/writing thread. Events are ordered on the ringbuffer // so region sequenceids will also be in order. regionSequenceId = entry.stampRegionSequenceId(); - - // Edits are empty, there is nothing to append. Maybe empty when we are looking for a - // region sequence id only, a region edit/sequence id that is not associated with an actual + // Edits are empty, there is nothing to append. Maybe empty when we are looking for a + // region sequence id only, a region edit/sequence id that is not associated with an actual // edit. It has to go through all the rigmarole to be sure we have the right ordering. if (entry.getEdit().isEmpty()) { return; @@ -2053,4 +2035,4 @@ public class FSHLog implements WAL { } return new DatanodeInfo[0]; } -} \ No newline at end of file +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java index a7686608a06..7f3eb61a015 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java @@ -21,15 +21,14 @@ package org.apache.hadoop.hbase.regionserver.wal; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; -import java.util.List; import java.util.Set; -import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CollectionUtils; @@ -51,23 +50,18 @@ class FSWALEntry extends Entry { // The below data members are denoted 'transient' just to highlight these are not persisted; // they are only in memory and held here while passing over the ring buffer. private final transient long sequence; - private final transient AtomicLong regionSequenceIdReference; private final transient boolean inMemstore; private final transient HTableDescriptor htd; private final transient HRegionInfo hri; - private final transient List memstoreCells; private final Set familyNames; FSWALEntry(final long sequence, final WALKey key, final WALEdit edit, - final AtomicLong referenceToRegionSequenceId, final boolean inMemstore, - final HTableDescriptor htd, final HRegionInfo hri, List memstoreCells) { + final HTableDescriptor htd, final HRegionInfo hri, final boolean inMemstore) { super(key, edit); - this.regionSequenceIdReference = referenceToRegionSequenceId; this.inMemstore = inMemstore; this.htd = htd; this.hri = hri; this.sequence = sequence; - this.memstoreCells = memstoreCells; if (inMemstore) { // construct familyNames here to reduce the work of log sinker. ArrayList cells = this.getEdit().getCells(); @@ -111,24 +105,30 @@ class FSWALEntry extends Entry { } /** - * Stamp this edit with a region edit/sequence id. - * Call when safe to do so: i.e. the context is such that the increment on the passed in - * {@link #regionSequenceIdReference} is guaranteed aligned w/ how appends are going into the - * WAL. This method works with {@link #getRegionSequenceId()}. It will block waiting on this - * method to be called. - * @return The region edit/sequence id we set for this edit. + * Here is where a WAL edit gets its sequenceid. + * @return The sequenceid we stamped on this edit. * @throws IOException - * @see #getRegionSequenceId() */ long stampRegionSequenceId() throws IOException { - long regionSequenceId = this.regionSequenceIdReference.incrementAndGet(); - if (!this.getEdit().isReplay() && !CollectionUtils.isEmpty(memstoreCells)) { - for (Cell cell : this.memstoreCells) { - CellUtil.setSequenceId(cell, regionSequenceId); + long regionSequenceId = WALKey.NO_SEQUENCE_ID; + MultiVersionConcurrencyControl mvcc = getKey().getMvcc(); + MultiVersionConcurrencyControl.WriteEntry we = null; + + if (mvcc != null) { + we = mvcc.begin(); + regionSequenceId = we.getWriteNumber(); + } + + if (!this.getEdit().isReplay() && inMemstore) { + for (Cell c:getEdit().getCells()) { + CellUtil.setSequenceId(c, regionSequenceId); } } + + // This has to stay in this order WALKey key = getKey(); key.setLogSeqNum(regionSequenceId); + key.setWriteEntry(we); return regionSequenceId; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java index 5218981a6e8..1302d8c83e5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.io.Writable; @@ -73,6 +74,13 @@ public class HLogKey extends WALKey implements Writable { super(encodedRegionName, tablename, now); } + public HLogKey(final byte[] encodedRegionName, + final TableName tablename, + final long now, + final MultiVersionConcurrencyControl mvcc) { + super(encodedRegionName, tablename, now, mvcc); + } + /** * Create the log key for writing to somewhere. * We maintain the tablename mainly for debugging purposes. @@ -86,9 +94,16 @@ public class HLogKey extends WALKey implements Writable { * @param now Time at which this edit was written. * @param clusterIds the clusters that have consumed the change(used in Replication) */ - public HLogKey(final byte [] encodedRegionName, final TableName tablename, - long logSeqNum, final long now, List clusterIds, long nonceGroup, long nonce) { - super(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce); + public HLogKey( + final byte[] encodedRegionName, + final TableName tablename, + long logSeqNum, + final long now, + List clusterIds, + long nonceGroup, + long nonce, + MultiVersionConcurrencyControl mvcc) { + super(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce, mvcc); } /** @@ -104,9 +119,14 @@ public class HLogKey extends WALKey implements Writable { * @param nonceGroup * @param nonce */ - public HLogKey(final byte [] encodedRegionName, final TableName tablename, - final long now, List clusterIds, long nonceGroup, long nonce) { - super(encodedRegionName, tablename, now, clusterIds, nonceGroup, nonce); + public HLogKey(final byte[] encodedRegionName, + final TableName tablename, + final long now, + List clusterIds, + long nonceGroup, + long nonce, + final MultiVersionConcurrencyControl mvcc) { + super(encodedRegionName, tablename, now, clusterIds, nonceGroup, nonce, mvcc); } /** @@ -122,8 +142,8 @@ public class HLogKey extends WALKey implements Writable { * @param nonce */ public HLogKey(final byte [] encodedRegionName, final TableName tablename, long logSeqNum, - long nonceGroup, long nonce) { - super(encodedRegionName, tablename, logSeqNum, nonceGroup, nonce); + long nonceGroup, long nonce, MultiVersionConcurrencyControl mvcc) { + super(encodedRegionName, tablename, logSeqNum, nonceGroup, nonce, mvcc); } /** @@ -141,7 +161,8 @@ public class HLogKey extends WALKey implements Writable { Compressor.writeCompressed(this.encodedRegionName, 0, this.encodedRegionName.length, out, compressionContext.regionDict); - Compressor.writeCompressed(this.tablename.getName(), 0, this.tablename.getName().length, out, + Compressor.writeCompressed(this.tablename.getName(), 0, + this.tablename.getName().length, out, compressionContext.tableDict); } out.writeLong(this.logSeqNum); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReplayHLogKey.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReplayHLogKey.java index cb8934691c9..f7ae20806e5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReplayHLogKey.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReplayHLogKey.java @@ -24,6 +24,7 @@ import java.util.UUID; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; /** * An HLogKey specific to WalEdits coming from replay. @@ -32,13 +33,15 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; public class ReplayHLogKey extends HLogKey { public ReplayHLogKey(final byte [] encodedRegionName, final TableName tablename, - final long now, List clusterIds, long nonceGroup, long nonce) { - super(encodedRegionName, tablename, now, clusterIds, nonceGroup, nonce); + final long now, List clusterIds, long nonceGroup, long nonce, + MultiVersionConcurrencyControl mvcc) { + super(encodedRegionName, tablename, now, clusterIds, nonceGroup, nonce, mvcc); } public ReplayHLogKey(final byte [] encodedRegionName, final TableName tablename, - long logSeqNum, final long now, List clusterIds, long nonceGroup, long nonce) { - super(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce); + long logSeqNum, final long now, List clusterIds, long nonceGroup, long nonce, + MultiVersionConcurrencyControl mvcc) { + super(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce, mvcc); } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java index be39873a594..e41e1c38ed8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java @@ -37,7 +37,8 @@ import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.SequenceFile.Metadata; import org.apache.hadoop.io.Text; -@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX, HBaseInterfaceAudience.CONFIG}) +@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX, + HBaseInterfaceAudience.CONFIG}) public class SequenceFileLogReader extends ReaderBase { private static final Log LOG = LogFactory.getLog(SequenceFileLogReader.class); @@ -273,8 +274,10 @@ public class SequenceFileLogReader extends ReaderBase { end = fEnd.getLong(this.reader); } catch(NoSuchFieldException nfe) { /* reflection failure, keep going */ + if (LOG.isTraceEnabled()) LOG.trace(nfe); } catch(IllegalAccessException iae) { /* reflection failure, keep going */ + if (LOG.isTraceEnabled()) LOG.trace(iae); } catch(Exception e) { /* All other cases. Should we handle it more aggressively? */ LOG.warn("Unexpected exception when accessing the end field", e); @@ -293,8 +296,10 @@ public class SequenceFileLogReader extends ReaderBase { .initCause(ioe); } catch(NoSuchMethodException nfe) { /* reflection failure, keep going */ + if (LOG.isTraceEnabled()) LOG.trace(nfe); } catch(IllegalAccessException iae) { /* reflection failure, keep going */ + if (LOG.isTraceEnabled()) LOG.trace(iae); } catch(Exception e) { /* All other cases. Should we handle it more aggressively? */ LOG.warn("Unexpected exception when accessing the end field", e); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java index a752ff19517..3b774ef154e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java @@ -73,7 +73,7 @@ import com.google.common.annotations.VisibleForTesting; * where, the WALEdit is serialized as: * <-1, # of edits, <KeyValue>, <KeyValue>, ... > * For example: - * <-1, 3, <Keyvalue-for-edit-c1>, <KeyValue-for-edit-c2>, <KeyValue-for-edit-c3>> + * <-1, 3, <KV-for-edit-c1>, <KV-for-edit-c2>, <KV-for-edit-c3>> * * The -1 marker is just a special way of being backward compatible with * an old WAL which would have contained a single <KeyValue>. @@ -104,6 +104,9 @@ public class WALEdit implements Writable, HeapSize { public static final WALEdit EMPTY_WALEDIT = new WALEdit(); // Only here for legacy writable deserialization + /** + * @deprecated Legacy + */ @Deprecated private NavigableMap scopes; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java index 399623fcac7..27182950e69 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java @@ -20,12 +20,9 @@ package org.apache.hadoop.hbase.regionserver.wal; import java.io.IOException; -import java.util.ArrayList; -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.classification.InterfaceAudience; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; @@ -34,6 +31,7 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor; +import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.WALKey; @@ -48,19 +46,27 @@ import com.google.protobuf.TextFormat; public class WALUtil { private static final Log LOG = LogFactory.getLog(WALUtil.class); + private WALUtil() { + // Shut down construction of this class. + } + /** * Write the marker that a compaction has succeeded and is about to be committed. * This provides info to the HMaster to allow it to recover the compaction if * this regionserver dies in the middle (This part is not yet implemented). It also prevents * the compaction from finishing if this regionserver has already lost its lease on the log. - * @param sequenceId Used by WAL to get sequence Id for the waledit. + * @param mvcc Used by WAL to get sequence Id for the waledit. */ - public static void writeCompactionMarker(WAL log, HTableDescriptor htd, HRegionInfo info, - final CompactionDescriptor c, AtomicLong sequenceId) throws IOException { + public static void writeCompactionMarker(WAL log, + HTableDescriptor htd, + HRegionInfo info, + final CompactionDescriptor c, + MultiVersionConcurrencyControl mvcc) throws IOException { TableName tn = TableName.valueOf(c.getTableName().toByteArray()); // we use HLogKey here instead of WALKey directly to support legacy coprocessors. - WALKey key = new HLogKey(info.getEncodedNameAsBytes(), tn); - log.append(htd, info, key, WALEdit.createCompaction(info, c), sequenceId, false, null); + WALKey key = new HLogKey(info.getEncodedNameAsBytes(), tn, System.currentTimeMillis(), mvcc); + log.append(htd, info, key, WALEdit.createCompaction(info, c), false); + mvcc.complete(key.getWriteEntry()); log.sync(); if (LOG.isTraceEnabled()) { LOG.trace("Appended compaction marker " + TextFormat.shortDebugString(c)); @@ -70,13 +76,17 @@ public class WALUtil { /** * Write a flush marker indicating a start / abort or a complete of a region flush */ - public static long writeFlushMarker(WAL log, HTableDescriptor htd, HRegionInfo info, - final FlushDescriptor f, AtomicLong sequenceId, boolean sync) throws IOException { + public static long writeFlushMarker(WAL log, + HTableDescriptor htd, + HRegionInfo info, + final FlushDescriptor f, + boolean sync, + MultiVersionConcurrencyControl mvcc) throws IOException { TableName tn = TableName.valueOf(f.getTableName().toByteArray()); // we use HLogKey here instead of WALKey directly to support legacy coprocessors. - WALKey key = new HLogKey(info.getEncodedNameAsBytes(), tn); - long trx = log.append(htd, info, key, WALEdit.createFlushWALEdit(info, f), sequenceId, false, - null); + WALKey key = new HLogKey(info.getEncodedNameAsBytes(), tn, System.currentTimeMillis(), mvcc); + long trx = log.append(htd, info, key, WALEdit.createFlushWALEdit(info, f), false); + mvcc.complete(key.getWriteEntry()); if (sync) log.sync(trx); if (LOG.isTraceEnabled()) { LOG.trace("Appended flush marker " + TextFormat.shortDebugString(f)); @@ -88,12 +98,11 @@ public class WALUtil { * Write a region open marker indicating that the region is opened */ public static long writeRegionEventMarker(WAL log, HTableDescriptor htd, HRegionInfo info, - final RegionEventDescriptor r, AtomicLong sequenceId) throws IOException { + final RegionEventDescriptor r) throws IOException { TableName tn = TableName.valueOf(r.getTableName().toByteArray()); // we use HLogKey here instead of WALKey directly to support legacy coprocessors. WALKey key = new HLogKey(info.getEncodedNameAsBytes(), tn); - long trx = log.append(htd, info, key, WALEdit.createRegionEventWALEdit(info, r), - sequenceId, false, null); + long trx = log.append(htd, info, key, WALEdit.createRegionEventWALEdit(info, r), false); log.sync(trx); if (LOG.isTraceEnabled()) { LOG.trace("Appended region event marker " + TextFormat.shortDebugString(r)); @@ -108,27 +117,22 @@ public class WALUtil { * @param htd A description of the table that we are bulk loading into. * @param info A description of the region in the table that we are bulk loading into. * @param descriptor A protocol buffers based description of the client's bulk loading request - * @param sequenceId The current sequenceId in the log at the time when we were to write the - * bulk load marker. * @return txid of this transaction or if nothing to do, the last txid * @throws IOException We will throw an IOException if we can not append to the HLog. */ public static long writeBulkLoadMarkerAndSync(final WAL wal, final HTableDescriptor htd, final HRegionInfo info, - final WALProtos.BulkLoadDescriptor descriptor, - final AtomicLong sequenceId) throws IOException { + final WALProtos.BulkLoadDescriptor descriptor) + throws IOException { TableName tn = info.getTable(); WALKey key = new HLogKey(info.getEncodedNameAsBytes(), tn); // Add it to the log but the false specifies that we don't need to add it to the memstore long trx = wal.append(htd, - info, - key, - WALEdit.createBulkLoadEvent(info, descriptor), - sequenceId, - false, - new ArrayList()); + info, + key, + WALEdit.createBulkLoadEvent(info, descriptor), false); wal.sync(trx); if (LOG.isTraceEnabled()) { @@ -136,5 +140,4 @@ public class WALUtil { } return trx; } - } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HashedBytes.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HashedBytes.java index f628cee654f..84d61285416 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HashedBytes.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HashedBytes.java @@ -55,7 +55,7 @@ public class HashedBytes { if (obj == null || getClass() != obj.getClass()) return false; HashedBytes other = (HashedBytes) obj; - return Arrays.equals(bytes, other.bytes); + return (hashCode == other.hashCode) && Arrays.equals(bytes, other.bytes); } @Override 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 04045ec6d70..191d54691c0 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 @@ -156,7 +156,7 @@ class DisabledWALProvider implements WALProvider { @Override public long append(HTableDescriptor htd, HRegionInfo info, WALKey key, WALEdit edits, - AtomicLong sequenceId, boolean inMemstore, List memstoreKVs) { + boolean inMemstore) { if (!this.listeners.isEmpty()) { final long start = System.nanoTime(); long len = 0; 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 4844487cc99..ce34c989bc7 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 @@ -114,19 +114,16 @@ public interface WAL { * @param key Modified by this call; we add to it this edits region edit/sequence id. * @param edits Edits to append. MAY CONTAIN NO EDITS for case where we want to get an edit * sequence id that is after all currently appended edits. - * @param htd used to give scope for replication TODO refactor out in favor of table name and info - * @param sequenceId A reference to the atomic long the info region is using as - * source of its incrementing edits sequence id. Inside in this call we will increment it and - * attach the sequence to the edit we apply the WAL. + * @param htd used to give scope for replication TODO refactor out in favor of table name and + * info * @param inMemstore Always true except for case where we are writing a compaction completion * record into the WAL; in this case the entry is just so we can finish an unfinished compaction * -- it is not an edit for memstore. - * @param memstoreKVs list of KVs added into memstore * @return Returns a 'transaction id' and key will have the region edit/sequence id * in it. */ long append(HTableDescriptor htd, HRegionInfo info, WALKey key, WALEdit edits, - AtomicLong sequenceId, boolean inMemstore, List memstoreKVs) + boolean inMemstore) throws IOException; /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java index 74284e071a4..48ede4c8dad 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java @@ -32,6 +32,7 @@ import java.util.UUID; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -68,13 +69,47 @@ import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec; * * Note that protected members marked @InterfaceAudience.Private are only protected * to support the legacy HLogKey class, which is in a different package. + * + *

*/ // TODO: Key and WALEdit are never used separately, or in one-to-many relation, for practical // purposes. They need to be merged into WALEntry. +// TODO: Cleanup. We have logSeqNum and then WriteEntry, both are sequence id'ing. Fix. @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION) public class WALKey implements SequenceId, Comparable { private static final Log LOG = LogFactory.getLog(WALKey.class); + @InterfaceAudience.Private // For internal use only. + public MultiVersionConcurrencyControl getMvcc() { + return mvcc; + } + + /** + * Will block until a write entry has been assigned by they WAL subsystem. + * @return A WriteEntry gotten from local WAL subsystem. Must be completed by calling + * mvcc#complete or mvcc#completeAndWait. + * @throws InterruptedIOException + * @see + * #setWriteEntry(org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl.WriteEntry) + */ + @InterfaceAudience.Private // For internal use only. + public MultiVersionConcurrencyControl.WriteEntry getWriteEntry() throws InterruptedIOException { + try { + this.seqNumAssignedLatch.await(); + } catch (InterruptedException ie) { + InterruptedIOException iie = new InterruptedIOException(); + iie.initCause(ie); + throw iie; + } + return writeEntry; + } + + @InterfaceAudience.Private // For internal use only. + public void setWriteEntry(MultiVersionConcurrencyControl.WriteEntry writeEntry) { + this.writeEntry = writeEntry; + this.seqNumAssignedLatch.countDown(); + } + // should be < 0 (@see HLogKey#readFields(DataInput)) // version 2 supports WAL compression // public members here are only public because of HLogKey @@ -151,7 +186,9 @@ public class WALKey implements SequenceId, Comparable { private long nonceGroup = HConstants.NO_NONCE; private long nonce = HConstants.NO_NONCE; - static final List EMPTY_UUIDS = Collections.unmodifiableList(new ArrayList()); + private MultiVersionConcurrencyControl mvcc; + private MultiVersionConcurrencyControl.WriteEntry writeEntry; + public static final List EMPTY_UUIDS = Collections.unmodifiableList(new ArrayList()); // visible for deprecated HLogKey @InterfaceAudience.Private @@ -159,16 +196,17 @@ public class WALKey implements SequenceId, Comparable { public WALKey() { init(null, null, 0L, HConstants.LATEST_TIMESTAMP, - new ArrayList(), HConstants.NO_NONCE, HConstants.NO_NONCE); + new ArrayList(), HConstants.NO_NONCE, HConstants.NO_NONCE, null); } @VisibleForTesting - public WALKey(final byte[] encodedRegionName, final TableName tablename, long logSeqNum, + public WALKey(final byte[] encodedRegionName, final TableName tablename, + long logSeqNum, final long now, UUID clusterId) { List clusterIds = new ArrayList(); clusterIds.add(clusterId); init(encodedRegionName, tablename, logSeqNum, now, clusterIds, - HConstants.NO_NONCE, HConstants.NO_NONCE); + HConstants.NO_NONCE, HConstants.NO_NONCE, null); } public WALKey(final byte[] encodedRegionName, final TableName tablename) { @@ -176,8 +214,28 @@ public class WALKey implements SequenceId, Comparable { } public WALKey(final byte[] encodedRegionName, final TableName tablename, final long now) { - init(encodedRegionName, tablename, NO_SEQUENCE_ID, now, - EMPTY_UUIDS, HConstants.NO_NONCE, HConstants.NO_NONCE); + init(encodedRegionName, + tablename, + NO_SEQUENCE_ID, + now, + EMPTY_UUIDS, + HConstants.NO_NONCE, + HConstants.NO_NONCE, + null); + } + + public WALKey(final byte[] encodedRegionName, + final TableName tablename, + final long now, + MultiVersionConcurrencyControl mvcc) { + init(encodedRegionName, + tablename, + NO_SEQUENCE_ID, + now, + EMPTY_UUIDS, + HConstants.NO_NONCE, + HConstants.NO_NONCE, + mvcc); } /** @@ -187,15 +245,21 @@ public class WALKey implements SequenceId, Comparable { *

Used by log splitting and snapshots. * * @param encodedRegionName Encoded name of the region as returned by - * HRegionInfo#getEncodedNameAsBytes(). - * @param tablename - name of table - * @param logSeqNum - log sequence number - * @param now Time at which this edit was written. - * @param clusterIds the clusters that have consumed the change(used in Replication) + * HRegionInfo#getEncodedNameAsBytes(). + * @param tablename - name of table + * @param logSeqNum - log sequence number + * @param now Time at which this edit was written. + * @param clusterIds the clusters that have consumed the change(used in Replication) */ - public WALKey(final byte [] encodedRegionName, final TableName tablename, - long logSeqNum, final long now, List clusterIds, long nonceGroup, long nonce) { - init(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce); + public WALKey(final byte[] encodedRegionName, + final TableName tablename, + long logSeqNum, + final long now, + List clusterIds, + long nonceGroup, + long nonce, + MultiVersionConcurrencyControl mvcc) { + init(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce, mvcc); } /** @@ -204,17 +268,18 @@ public class WALKey implements SequenceId, Comparable { * A regionName is always a sub-table object. * * @param encodedRegionName Encoded name of the region as returned by - * HRegionInfo#getEncodedNameAsBytes(). + * HRegionInfo#getEncodedNameAsBytes(). * @param tablename - * @param now Time at which this edit was written. - * @param clusterIds the clusters that have consumed the change(used in Replication) + * @param now Time at which this edit was written. + * @param clusterIds the clusters that have consumed the change(used in Replication) * @param nonceGroup * @param nonce + * @param mvcc mvcc control used to generate sequence numbers and control read/write points */ - public WALKey(final byte [] encodedRegionName, final TableName tablename, - final long now, List clusterIds, long nonceGroup, long nonce) { - init(encodedRegionName, tablename, NO_SEQUENCE_ID, now, clusterIds, - nonceGroup, nonce); + public WALKey(final byte[] encodedRegionName, final TableName tablename, + final long now, List clusterIds, long nonceGroup, + final long nonce, final MultiVersionConcurrencyControl mvcc) { + init(encodedRegionName, tablename, NO_SEQUENCE_ID, now, clusterIds, nonceGroup, nonce, mvcc); } /** @@ -223,21 +288,37 @@ public class WALKey implements SequenceId, Comparable { * A regionName is always a sub-table object. * * @param encodedRegionName Encoded name of the region as returned by - * HRegionInfo#getEncodedNameAsBytes(). + * HRegionInfo#getEncodedNameAsBytes(). * @param tablename * @param logSeqNum * @param nonceGroup * @param nonce */ - public WALKey(final byte [] encodedRegionName, final TableName tablename, long logSeqNum, - long nonceGroup, long nonce) { - init(encodedRegionName, tablename, logSeqNum, EnvironmentEdgeManager.currentTime(), - EMPTY_UUIDS, nonceGroup, nonce); + public WALKey(final byte[] encodedRegionName, + final TableName tablename, + long logSeqNum, + long nonceGroup, + long nonce, + final MultiVersionConcurrencyControl mvcc) { + init(encodedRegionName, + tablename, + logSeqNum, + EnvironmentEdgeManager.currentTime(), + EMPTY_UUIDS, + nonceGroup, + nonce, + mvcc); } @InterfaceAudience.Private - protected void init(final byte [] encodedRegionName, final TableName tablename, - long logSeqNum, final long now, List clusterIds, long nonceGroup, long nonce) { + protected void init(final byte[] encodedRegionName, + final TableName tablename, + long logSeqNum, + final long now, + List clusterIds, + long nonceGroup, + long nonce, + MultiVersionConcurrencyControl mvcc) { this.logSeqNum = logSeqNum; this.writeTime = now; this.clusterIds = clusterIds; @@ -245,6 +326,7 @@ public class WALKey implements SequenceId, Comparable { this.tablename = tablename; this.nonceGroup = nonceGroup; this.nonce = nonce; + this.mvcc = mvcc; } /** @@ -270,15 +352,14 @@ public class WALKey implements SequenceId, Comparable { } /** - * Allow that the log sequence id to be set post-construction and release all waiters on assigned - * sequence number. + * Allow that the log sequence id to be set post-construction * Only public for org.apache.hadoop.hbase.regionserver.wal.FSWALEntry * @param sequence */ @InterfaceAudience.Private public void setLogSeqNum(final long sequence) { this.logSeqNum = sequence; - this.seqNumAssignedLatch.countDown(); + } /** @@ -492,21 +573,22 @@ public class WALKey implements SequenceId, Comparable { this.encodedRegionName = encodedRegionName; } - public org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey.Builder getBuilder(WALCellCodec.ByteStringCompressor compressor) - throws IOException { - org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey.Builder builder = org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey.newBuilder(); + public org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey.Builder getBuilder( + WALCellCodec.ByteStringCompressor compressor) throws IOException { + org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey.Builder builder = + org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey.newBuilder(); if (compressionContext == null) { builder.setEncodedRegionName(ByteStringer.wrap(this.encodedRegionName)); builder.setTableName(ByteStringer.wrap(this.tablename.getName())); } else { builder.setEncodedRegionName(compressor.compress(this.encodedRegionName, - compressionContext.regionDict)); + compressionContext.regionDict)); builder.setTableName(compressor.compress(this.tablename.getName(), - compressionContext.tableDict)); + compressionContext.tableDict)); } builder.setLogSequenceNumber(this.logSeqNum); builder.setWriteTime(writeTime); - if(this.origLogSeqNum > 0) { + if (this.origLogSeqNum > 0) { builder.setOrigSequenceNumber(this.origLogSeqNum); } if (this.nonce != HConstants.NO_NONCE) { @@ -532,8 +614,9 @@ public class WALKey implements SequenceId, Comparable { return builder; } - public void readFieldsFromPb( - org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey walKey, WALCellCodec.ByteStringUncompressor uncompressor) throws IOException { + public void readFieldsFromPb(org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey walKey, + WALCellCodec.ByteStringUncompressor uncompressor) + throws IOException { if (this.compressionContext != null) { this.encodedRegionName = uncompressor.uncompress( walKey.getEncodedRegionName(), compressionContext.regionDict); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java index 51043e4268c..3741cdfa2dc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java @@ -2301,7 +2301,7 @@ public class WALSplitter { // we use HLogKey here instead of WALKey directly to support legacy coprocessors. key = new HLogKey(walKeyProto.getEncodedRegionName().toByteArray(), TableName.valueOf( walKeyProto.getTableName().toByteArray()), replaySeqId, walKeyProto.getWriteTime(), - clusterIds, walKeyProto.getNonceGroup(), walKeyProto.getNonce()); + clusterIds, walKeyProto.getNonceGroup(), walKeyProto.getNonce(), null); logEntry.setFirst(key); logEntry.setSecond(val); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java index 7fd8902e22a..1665e6619c1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java @@ -22,7 +22,6 @@ package org.apache.hadoop.hbase; import static org.junit.Assert.assertEquals; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MiscTests; 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 6223b159c75..bb216b68d8f 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 @@ -24,7 +24,6 @@ import java.io.IOException; import java.util.Collection; import java.util.List; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -46,12 +45,12 @@ import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputController; -import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.regionserver.wal.WALUtil; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; +import org.apache.hadoop.hbase.wal.WAL; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -225,8 +224,7 @@ public class TestIOFencing { */ @Test public void testFencingAroundCompaction() throws Exception { - doTest(BlockCompactionsInPrepRegion.class, false); - doTest(BlockCompactionsInPrepRegion.class, true); + doTest(BlockCompactionsInPrepRegion.class); } /** @@ -237,13 +235,11 @@ public class TestIOFencing { */ @Test public void testFencingAroundCompactionAfterWALSync() throws Exception { - doTest(BlockCompactionsInCompletionRegion.class, false); - doTest(BlockCompactionsInCompletionRegion.class, true); + doTest(BlockCompactionsInCompletionRegion.class); } - public void doTest(Class regionClass, boolean distributedLogReplay) throws Exception { + public void doTest(Class regionClass) throws Exception { Configuration c = TEST_UTIL.getConfiguration(); - c.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, distributedLogReplay); // Insert our custom region c.setClass(HConstants.REGION_IMPL, regionClass, HRegion.class); c.setBoolean("dfs.support.append", true); @@ -285,7 +281,7 @@ public class TestIOFencing { FAMILY, Lists.newArrayList(new Path("/a")), Lists.newArrayList(new Path("/b")), new Path("store_dir")); WALUtil.writeCompactionMarker(compactingRegion.getWAL(), table.getTableDescriptor(), - oldHri, compactionDescriptor, new AtomicLong(Long.MAX_VALUE-100)); + oldHri, compactionDescriptor, compactingRegion.getMVCC()); // Wait till flush has happened, otherwise there won't be multiple store files long startWaitTime = System.currentTimeMillis(); @@ -356,4 +352,4 @@ public class TestIOFencing { TEST_UTIL.shutdownMiniCluster(); } } -} +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java index 37e98e83920..a064bccb0ec 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java @@ -376,7 +376,6 @@ public class TestReplicasClient { } } - @Test public void testFlushTable() throws Exception { openRegion(hriSecondary); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java index 78deed9f109..0a4ca16bdb8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java @@ -141,6 +141,4 @@ public class TestRegionObserverStacking extends TestCase { assertTrue(idB < idC); HBaseTestingUtility.closeRegionAndWAL(region); } - -} - +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java index 8993255df10..7772664ee85 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java @@ -29,7 +29,6 @@ import java.security.PrivilegedExceptionAction; import java.util.Arrays; import java.util.List; import java.util.Map; -import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -49,6 +48,7 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.WALCoprocessorHost; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; @@ -188,7 +188,6 @@ public class TestWALObserver { Path basedir = new Path(this.hbaseRootDir, Bytes.toString(TEST_TABLE)); deleteDir(basedir); fs.mkdirs(new Path(basedir, hri.getEncodedName())); - final AtomicLong sequenceId = new AtomicLong(0); // TEST_FAMILY[0] shall be removed from WALEdit. // TEST_FAMILY[1] value shall be changed. @@ -237,7 +236,7 @@ public class TestWALObserver { long now = EnvironmentEdgeManager.currentTime(); // we use HLogKey here instead of WALKey directly to support legacy coprocessors. long txid = log.append(htd, hri, new HLogKey(hri.getEncodedNameAsBytes(), hri.getTable(), now), - edit, sequenceId, true, null); + edit, true); log.sync(txid); // the edit shall have been change now by the coprocessor. @@ -273,7 +272,7 @@ public class TestWALObserver { final HTableDescriptor htd = createBasic3FamilyHTD(Bytes .toString(TEST_TABLE)); final HRegionInfo hri = new HRegionInfo(tableName, null, null); - final AtomicLong sequenceId = new AtomicLong(0); + MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(); fs.mkdirs(new Path(FSUtils.getTableDir(hbaseRootDir, tableName), hri.getEncodedName())); @@ -300,7 +299,7 @@ public class TestWALObserver { final int countPerFamily = 5; for (HColumnDescriptor hcd : htd.getFamilies()) { addWALEdits(tableName, hri, TEST_ROW, hcd.getName(), countPerFamily, - EnvironmentEdgeManager.getDelegate(), wal, htd, sequenceId); + EnvironmentEdgeManager.getDelegate(), wal, htd, mvcc); } LOG.debug("Verify that only the non-legacy CP saw edits."); @@ -324,7 +323,7 @@ public class TestWALObserver { final WALEdit edit = new WALEdit(); final byte[] nonce = Bytes.toBytes("1772"); edit.add(new KeyValue(TEST_ROW, TEST_FAMILY[0], nonce, now, nonce)); - final long txid = wal.append(htd, hri, legacyKey, edit, sequenceId, true, null); + final long txid = wal.append(htd, hri, legacyKey, edit, true); wal.sync(txid); LOG.debug("Make sure legacy cps can see supported edits after having been skipped."); @@ -349,7 +348,7 @@ public class TestWALObserver { public void testEmptyWALEditAreNotSeen() throws Exception { final HRegionInfo hri = createBasic3FamilyHRegionInfo(Bytes.toString(TEST_TABLE)); final HTableDescriptor htd = createBasic3FamilyHTD(Bytes.toString(TEST_TABLE)); - final AtomicLong sequenceId = new AtomicLong(0); + final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(); WAL log = wals.getWAL(UNSPECIFIED_REGION, null); try { @@ -361,8 +360,9 @@ public class TestWALObserver { assertFalse(cp.isPostWALWriteCalled()); final long now = EnvironmentEdgeManager.currentTime(); - long txid = log.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), hri.getTable(), now), - new WALEdit(), sequenceId, true, null); + long txid = log.append(htd, hri, + new WALKey(hri.getEncodedNameAsBytes(), hri.getTable(), now, mvcc), + new WALEdit(), true); log.sync(txid); assertFalse("Empty WALEdit should skip coprocessor evaluation.", cp.isPreWALWriteCalled()); @@ -381,7 +381,7 @@ public class TestWALObserver { // ultimately called by HRegion::initialize() TableName tableName = TableName.valueOf("testWALCoprocessorReplay"); final HTableDescriptor htd = getBasic3FamilyHTableDescriptor(tableName); - final AtomicLong sequenceId = new AtomicLong(0); + MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(); // final HRegionInfo hri = // createBasic3FamilyHRegionInfo(Bytes.toString(tableName)); // final HRegionInfo hri1 = @@ -405,10 +405,9 @@ public class TestWALObserver { // for (HColumnDescriptor hcd: hri.getTableDesc().getFamilies()) { for (HColumnDescriptor hcd : htd.getFamilies()) { addWALEdits(tableName, hri, TEST_ROW, hcd.getName(), countPerFamily, - EnvironmentEdgeManager.getDelegate(), wal, htd, sequenceId); + EnvironmentEdgeManager.getDelegate(), wal, htd, mvcc); } - wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName, now), edit, sequenceId, - true, null); + wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName, now, mvcc), edit, true); // sync to fs. wal.sync(); @@ -528,7 +527,7 @@ public class TestWALObserver { private void addWALEdits(final TableName tableName, final HRegionInfo hri, final byte[] rowName, final byte[] family, final int count, EnvironmentEdge ee, final WAL wal, - final HTableDescriptor htd, final AtomicLong sequenceId) throws IOException { + final HTableDescriptor htd, final MultiVersionConcurrencyControl mvcc) throws IOException { String familyStr = Bytes.toString(family); long txid = -1; for (int j = 0; j < count; j++) { @@ -539,7 +538,7 @@ public class TestWALObserver { // uses WALKey instead of HLogKey on purpose. will only work for tests where we don't care // about legacy coprocessors txid = wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName, - ee.currentTime()), edit, sequenceId, true, null); + ee.currentTime(), mvcc), edit, true); } if (-1 != txid) { wal.sync(txid); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java index d82f36bae20..5fa588b1290 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java @@ -32,8 +32,8 @@ import org.junit.experimental.categories.Category; public class TestHLogRecordReader extends TestWALRecordReader { @Override - protected WALKey getWalKey(final long sequenceid) { - return new HLogKey(info.getEncodedNameAsBytes(), tableName, sequenceid); + protected WALKey getWalKey(final long time) { + return new HLogKey(info.getEncodedNameAsBytes(), tableName, time, mvcc); } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java index 64ef8fdf663..a4381c88e5f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java @@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.mapreduce.WALInputFormat.WALKeyRecordReader; import org.apache.hadoop.hbase.mapreduce.WALInputFormat.WALRecordReader; +import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.WALFactory; @@ -75,6 +76,7 @@ public class TestWALRecordReader { private static final byte [] value = Bytes.toBytes("value"); private static HTableDescriptor htd; private static Path logDir; + protected MultiVersionConcurrencyControl mvcc; private static String getName() { return "TestWALRecordReader"; @@ -82,6 +84,7 @@ public class TestWALRecordReader { @Before public void setUp() throws Exception { + mvcc = new MultiVersionConcurrencyControl(); FileStatus[] entries = fs.listStatus(hbaseDir); for (FileStatus dir : entries) { fs.delete(dir.getPath(), true); @@ -124,13 +127,11 @@ public class TestWALRecordReader { // being millisecond based. long ts = System.currentTimeMillis(); WALEdit edit = new WALEdit(); - final AtomicLong sequenceId = new AtomicLong(0); edit.add(new KeyValue(rowName, family, Bytes.toBytes("1"), ts, value)); - log.append(htd, info, getWalKey(ts), edit, sequenceId, true, null); + log.append(htd, info, getWalKey(ts), edit, true); edit = new WALEdit(); edit.add(new KeyValue(rowName, family, Bytes.toBytes("2"), ts+1, value)); - log.append(htd, info, getWalKey(ts+1), edit, sequenceId, - true, null); + log.append(htd, info, getWalKey(ts+1), edit, true); log.sync(); LOG.info("Before 1st WAL roll " + log.toString()); log.rollWriter(); @@ -141,12 +142,10 @@ public class TestWALRecordReader { edit = new WALEdit(); edit.add(new KeyValue(rowName, family, Bytes.toBytes("3"), ts1+1, value)); - log.append(htd, info, getWalKey(ts1+1), edit, sequenceId, - true, null); + log.append(htd, info, getWalKey(ts1+1), edit, true); edit = new WALEdit(); edit.add(new KeyValue(rowName, family, Bytes.toBytes("4"), ts1+2, value)); - log.append(htd, info, getWalKey(ts1+2), edit, sequenceId, - true, null); + log.append(htd, info, getWalKey(ts1+2), edit, true); log.sync(); log.shutdown(); walfactory.shutdown(); @@ -188,8 +187,7 @@ public class TestWALRecordReader { WALEdit edit = new WALEdit(); edit.add(new KeyValue(rowName, family, Bytes.toBytes("1"), System.currentTimeMillis(), value)); - long txid = log.append(htd, info, getWalKey(System.currentTimeMillis()), edit, sequenceId, true, - null); + long txid = log.append(htd, info, getWalKey(System.currentTimeMillis()), edit, true); log.sync(txid); Thread.sleep(1); // make sure 2nd log gets a later timestamp @@ -199,8 +197,7 @@ public class TestWALRecordReader { edit = new WALEdit(); edit.add(new KeyValue(rowName, family, Bytes.toBytes("2"), System.currentTimeMillis(), value)); - txid = log.append(htd, info, getWalKey(System.currentTimeMillis()), edit, sequenceId, true, - null); + txid = log.append(htd, info, getWalKey(System.currentTimeMillis()), edit, true); log.sync(txid); log.shutdown(); walfactory.shutdown(); @@ -239,8 +236,8 @@ public class TestWALRecordReader { testSplit(splits.get(1)); } - protected WALKey getWalKey(final long sequenceid) { - return new WALKey(info.getEncodedNameAsBytes(), tableName, sequenceid); + protected WALKey getWalKey(final long time) { + return new WALKey(info.getEncodedNameAsBytes(), tableName, time, mvcc); } protected WALRecordReader getReader() { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java index 58ffb866951..25a5f412767 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java @@ -114,6 +114,7 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -132,7 +133,7 @@ public class TestDistributedLogSplitting { } - // Start a cluster with 2 masters and 5 regionservers + // Start a cluster with 2 masters and 6 regionservers static final int NUM_MASTERS = 2; static final int NUM_RS = 5; @@ -203,7 +204,7 @@ public class TestDistributedLogSplitting { } } - @Test (timeout=300000) + @Ignore("DLR is broken by HBASE-12751") @Test (timeout=300000) public void testRecoveredEdits() throws Exception { LOG.info("testRecoveredEdits"); conf.setLong("hbase.regionserver.hlog.blocksize", 30 * 1024); // create more than one wal @@ -292,7 +293,7 @@ public class TestDistributedLogSplitting { } } - @Test(timeout = 300000) + @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000) public void testLogReplayWithNonMetaRSDown() throws Exception { LOG.info("testLogReplayWithNonMetaRSDown"); conf.setLong("hbase.regionserver.hlog.blocksize", 30 * 1024); // create more than one wal @@ -337,7 +338,7 @@ public class TestDistributedLogSplitting { } } - @Test(timeout = 300000) + @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000) public void testNonceRecovery() throws Exception { LOG.info("testNonceRecovery"); final String TABLE_NAME = "table"; @@ -395,7 +396,7 @@ public class TestDistributedLogSplitting { } } - @Test(timeout = 300000) + @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000) public void testLogReplayWithMetaRSDown() throws Exception { LOG.info("testRecoveredEditsReplayWithMetaRSDown"); conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true); @@ -463,7 +464,7 @@ public class TestDistributedLogSplitting { }); } - @Test(timeout = 300000) + @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000) public void testMasterStartsUpWithLogSplittingWork() throws Exception { LOG.info("testMasterStartsUpWithLogSplittingWork"); conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false); @@ -521,7 +522,7 @@ public class TestDistributedLogSplitting { } } - @Test(timeout = 300000) + @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000) public void testMasterStartsUpWithLogReplayWork() throws Exception { LOG.info("testMasterStartsUpWithLogReplayWork"); conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true); @@ -584,7 +585,7 @@ public class TestDistributedLogSplitting { } - @Test(timeout = 300000) + @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000) public void testLogReplayTwoSequentialRSDown() throws Exception { LOG.info("testRecoveredEditsReplayTwoSequentialRSDown"); conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true); @@ -668,7 +669,7 @@ public class TestDistributedLogSplitting { } } - @Test(timeout = 300000) + @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000) public void testMarkRegionsRecoveringInZK() throws Exception { LOG.info("testMarkRegionsRecoveringInZK"); conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true); @@ -718,7 +719,7 @@ public class TestDistributedLogSplitting { } } - @Test(timeout = 300000) + @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000) public void testReplayCmd() throws Exception { LOG.info("testReplayCmd"); conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true); @@ -764,7 +765,7 @@ public class TestDistributedLogSplitting { } } - @Test(timeout = 300000) + @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000) public void testLogReplayForDisablingTable() throws Exception { LOG.info("testLogReplayForDisablingTable"); conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true); @@ -911,7 +912,7 @@ public class TestDistributedLogSplitting { } } - @Test(timeout = 300000) + @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000) public void testDisallowWritesInRecovering() throws Exception { LOG.info("testDisallowWritesInRecovering"); conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true); @@ -1184,7 +1185,7 @@ public class TestDistributedLogSplitting { } } - @Test(timeout = 300000) + @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000) public void testMetaRecoveryInZK() throws Exception { LOG.info("testMetaRecoveryInZK"); conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true); @@ -1233,7 +1234,7 @@ public class TestDistributedLogSplitting { zkw.close(); } - @Test(timeout = 300000) + @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000) public void testSameVersionUpdatesRecovery() throws Exception { LOG.info("testSameVersionUpdatesRecovery"); conf.setLong("hbase.regionserver.hlog.blocksize", 15 * 1024); @@ -1301,7 +1302,7 @@ public class TestDistributedLogSplitting { e.add(new KeyValue(row, family, qualifier, timeStamp, Bytes.toBytes(value))); wal.append(htd, curRegionInfo, new HLogKey(curRegionInfo.getEncodedNameAsBytes(), tableName, System.currentTimeMillis()), - e, sequenceId, true, null); + e, true); } wal.sync(); wal.shutdown(); @@ -1328,7 +1329,7 @@ public class TestDistributedLogSplitting { } } - @Test(timeout = 300000) + @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000) public void testSameVersionUpdatesRecoveryWithCompaction() throws Exception { LOG.info("testSameVersionUpdatesRecoveryWithWrites"); conf.setLong("hbase.regionserver.hlog.blocksize", 15 * 1024); @@ -1396,7 +1397,7 @@ public class TestDistributedLogSplitting { value++; e.add(new KeyValue(row, family, qualifier, timeStamp, Bytes.toBytes(value))); wal.append(htd, curRegionInfo, new HLogKey(curRegionInfo.getEncodedNameAsBytes(), - tableName, System.currentTimeMillis()), e, sequenceId, true, null); + tableName, System.currentTimeMillis()), e, true); } wal.sync(); wal.shutdown(); @@ -1609,7 +1610,7 @@ public class TestDistributedLogSplitting { e.add(new KeyValue(row, family, qualifier, System.currentTimeMillis(), value)); log.append(htd, curRegionInfo, new HLogKey(curRegionInfo.getEncodedNameAsBytes(), fullTName, - System.currentTimeMillis()), e, sequenceId, true, null); + System.currentTimeMillis()), e, true); if (0 == i % syncEvery) { log.sync(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java index 8f551f84298..65c8649a513 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java @@ -78,6 +78,7 @@ import org.apache.zookeeper.ZooDefs.Ids; import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -639,7 +640,7 @@ public class TestSplitLogManager { assertTrue("Recovery regions isn't cleaned", recoveringRegions.isEmpty()); } - @Test(timeout=60000) + @Ignore("DLR is broken by HBASE-12751") @Test(timeout=60000) public void testGetPreviousRecoveryMode() throws Exception { LOG.info("testGetPreviousRecoveryMode"); SplitLogCounters.resetCounters(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerCrashProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerCrashProcedure.java index 97512ce3109..510b017a5ee 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerCrashProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerCrashProcedure.java @@ -25,6 +25,7 @@ import java.util.Arrays; import java.util.Collection; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; @@ -44,7 +45,7 @@ import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; /** - * Runs first with DLS and then with DLR. + * It used to first run with DLS and then DLR but HBASE-12751 broke DLR so we disabled it here. */ @Category(LargeTests.class) @RunWith(Parameterized.class) @@ -53,7 +54,7 @@ public class TestServerCrashProcedure { // to return sequences of two-element arrays. @Parameters(name = "{index}: setting={0}") public static Collection data() { - return Arrays.asList(new Object[] [] {{Boolean.FALSE, -1}, {Boolean.TRUE, -1}}); + return Arrays.asList(new Object[] [] {{Boolean.FALSE, -1}}); } private final HBaseTestingUtility util = new HBaseTestingUtility(); @@ -67,8 +68,12 @@ public class TestServerCrashProcedure { @After public void tearDown() throws Exception { - ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate( - this.util.getHBaseCluster().getMaster().getMasterProcedureExecutor(), false); + MiniHBaseCluster cluster = this.util.getHBaseCluster(); + HMaster master = cluster == null? null: cluster.getMaster(); + if (master != null && master.getMasterProcedureExecutor() != null) { + ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(master.getMasterProcedureExecutor(), + false); + } this.util.shutdownMiniCluster(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java index b4bc76461be..3a770462817 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java @@ -316,7 +316,6 @@ public class TestAtomicOperation { */ @Test public void testRowMutationMultiThreads() throws IOException { - LOG.info("Starting test testRowMutationMultiThreads"); initHRegion(tableName, name.getMethodName(), fam1); @@ -616,30 +615,33 @@ public class TestAtomicOperation { } @Override - public RowLock getRowLockInternal(final byte[] row, boolean waitForLock) throws IOException { + public RowLock getRowLock(final byte[] row, boolean readLock) throws IOException { if (testStep == TestStep.CHECKANDPUT_STARTED) { latch.countDown(); } - return new WrappedRowLock(super.getRowLockInternal(row, waitForLock)); + return new WrappedRowLock(super.getRowLock(row, readLock)); } - public class WrappedRowLock extends RowLockImpl { + public class WrappedRowLock implements RowLock { + + private final RowLock rowLock; private WrappedRowLock(RowLock rowLock) { - setContext(((RowLockImpl)rowLock).getContext()); + this.rowLock = rowLock; } + @Override public void release() { if (testStep == TestStep.INIT) { - super.release(); + this.rowLock.release(); return; } if (testStep == TestStep.PUT_STARTED) { try { testStep = TestStep.PUT_COMPLETED; - super.release(); + this.rowLock.release(); // put has been written to the memstore and the row lock has been released, but the // MVCC has not been advanced. Prior to fixing HBASE-7051, the following order of // operations would cause the non-atomicity to show up: @@ -657,7 +659,7 @@ public class TestAtomicOperation { } } else if (testStep == TestStep.CHECKANDPUT_STARTED) { - super.release(); + this.rowLock.release(); } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java index 4ce228f5475..34278c95d26 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java @@ -96,7 +96,7 @@ public class TestBulkLoad { { oneOf(log).append(with(any(HTableDescriptor.class)), with(any(HRegionInfo.class)), with(any(WALKey.class)), with(bulkLogWalEditType(WALEdit.BULK_LOAD)), - with(any(AtomicLong.class)), with(any(boolean.class)), with(any(List.class))); + with(any(boolean.class))); will(returnValue(0l)); oneOf(log).sync(with(any(long.class))); } @@ -122,8 +122,7 @@ public class TestBulkLoad { Expectations expection = new Expectations() { { oneOf(log).append(with(any(HTableDescriptor.class)), with(any(HRegionInfo.class)), - with(any(WALKey.class)), with(bulkEventMatcher), - with(any(AtomicLong.class)), with(any(boolean.class)), with(any(List.class))); + with(any(WALKey.class)), with(bulkEventMatcher), with(any(boolean.class))); will(returnValue(0l)); oneOf(log).sync(with(any(long.class))); } 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 d9995178284..066bbb38599 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 @@ -124,7 +124,7 @@ public class TestDefaultMemStore extends TestCase { scanner.close(); } - memstorescanners = this.memstore.getScanners(mvcc.memstoreReadPoint()); + memstorescanners = this.memstore.getScanners(mvcc.getReadPoint()); // Now assert can count same number even if a snapshot mid-scan. s = new StoreScanner(scan, scanInfo, scanType, null, memstorescanners); count = 0; @@ -149,7 +149,7 @@ public class TestDefaultMemStore extends TestCase { for (KeyValueScanner scanner : memstorescanners) { scanner.close(); } - memstorescanners = this.memstore.getScanners(mvcc.memstoreReadPoint()); + memstorescanners = this.memstore.getScanners(mvcc.getReadPoint()); // Assert that new values are seen in kvset as we scan. long ts = System.currentTimeMillis(); s = new StoreScanner(scan, scanInfo, scanType, null, memstorescanners); @@ -214,7 +214,7 @@ public class TestDefaultMemStore extends TestCase { private void verifyScanAcrossSnapshot2(KeyValue kv1, KeyValue kv2) throws IOException { - List memstorescanners = this.memstore.getScanners(mvcc.memstoreReadPoint()); + List memstorescanners = this.memstore.getScanners(mvcc.getReadPoint()); assertEquals(1, memstorescanners.size()); final KeyValueScanner scanner = memstorescanners.get(0); scanner.seek(KeyValueUtil.createFirstOnRow(HConstants.EMPTY_START_ROW)); @@ -249,31 +249,31 @@ public class TestDefaultMemStore extends TestCase { final byte[] v = Bytes.toBytes("value"); MultiVersionConcurrencyControl.WriteEntry w = - mvcc.beginMemstoreInsertWithSeqNum(this.startSeqNum.incrementAndGet()); + mvcc.begin(); KeyValue kv1 = new KeyValue(row, f, q1, v); kv1.setSequenceId(w.getWriteNumber()); memstore.add(kv1); - KeyValueScanner s = this.memstore.getScanners(mvcc.memstoreReadPoint()).get(0); + KeyValueScanner s = this.memstore.getScanners(mvcc.getReadPoint()).get(0); assertScannerResults(s, new KeyValue[]{}); - mvcc.completeMemstoreInsert(w); + mvcc.completeAndWait(w); - s = this.memstore.getScanners(mvcc.memstoreReadPoint()).get(0); + s = this.memstore.getScanners(mvcc.getReadPoint()).get(0); assertScannerResults(s, new KeyValue[]{kv1}); - w = mvcc.beginMemstoreInsertWithSeqNum(this.startSeqNum.incrementAndGet()); + w = mvcc.begin(); KeyValue kv2 = new KeyValue(row, f, q2, v); kv2.setSequenceId(w.getWriteNumber()); memstore.add(kv2); - s = this.memstore.getScanners(mvcc.memstoreReadPoint()).get(0); + s = this.memstore.getScanners(mvcc.getReadPoint()).get(0); assertScannerResults(s, new KeyValue[]{kv1}); - mvcc.completeMemstoreInsert(w); + mvcc.completeAndWait(w); - s = this.memstore.getScanners(mvcc.memstoreReadPoint()).get(0); + s = this.memstore.getScanners(mvcc.getReadPoint()).get(0); assertScannerResults(s, new KeyValue[]{kv1, kv2}); } @@ -293,7 +293,7 @@ public class TestDefaultMemStore extends TestCase { // INSERT 1: Write both columns val1 MultiVersionConcurrencyControl.WriteEntry w = - mvcc.beginMemstoreInsertWithSeqNum(this.startSeqNum.incrementAndGet()); + mvcc.begin(); KeyValue kv11 = new KeyValue(row, f, q1, v1); kv11.setSequenceId(w.getWriteNumber()); @@ -302,14 +302,14 @@ public class TestDefaultMemStore extends TestCase { KeyValue kv12 = new KeyValue(row, f, q2, v1); kv12.setSequenceId(w.getWriteNumber()); memstore.add(kv12); - mvcc.completeMemstoreInsert(w); + mvcc.completeAndWait(w); // BEFORE STARTING INSERT 2, SEE FIRST KVS - KeyValueScanner s = this.memstore.getScanners(mvcc.memstoreReadPoint()).get(0); + KeyValueScanner s = this.memstore.getScanners(mvcc.getReadPoint()).get(0); assertScannerResults(s, new KeyValue[]{kv11, kv12}); // START INSERT 2: Write both columns val2 - w = mvcc.beginMemstoreInsertWithSeqNum(this.startSeqNum.incrementAndGet()); + w = mvcc.begin(); KeyValue kv21 = new KeyValue(row, f, q1, v2); kv21.setSequenceId(w.getWriteNumber()); memstore.add(kv21); @@ -319,16 +319,16 @@ public class TestDefaultMemStore extends TestCase { memstore.add(kv22); // BEFORE COMPLETING INSERT 2, SEE FIRST KVS - s = this.memstore.getScanners(mvcc.memstoreReadPoint()).get(0); + s = this.memstore.getScanners(mvcc.getReadPoint()).get(0); assertScannerResults(s, new KeyValue[]{kv11, kv12}); // COMPLETE INSERT 2 - mvcc.completeMemstoreInsert(w); + mvcc.completeAndWait(w); // NOW SHOULD SEE NEW KVS IN ADDITION TO OLD KVS. // See HBASE-1485 for discussion about what we should do with // the duplicate-TS inserts - s = this.memstore.getScanners(mvcc.memstoreReadPoint()).get(0); + s = this.memstore.getScanners(mvcc.getReadPoint()).get(0); assertScannerResults(s, new KeyValue[]{kv21, kv11, kv22, kv12}); } @@ -345,7 +345,7 @@ public class TestDefaultMemStore extends TestCase { final byte[] v1 = Bytes.toBytes("value1"); // INSERT 1: Write both columns val1 MultiVersionConcurrencyControl.WriteEntry w = - mvcc.beginMemstoreInsertWithSeqNum(this.startSeqNum.incrementAndGet()); + mvcc.begin(); KeyValue kv11 = new KeyValue(row, f, q1, v1); kv11.setSequenceId(w.getWriteNumber()); @@ -354,28 +354,28 @@ public class TestDefaultMemStore extends TestCase { KeyValue kv12 = new KeyValue(row, f, q2, v1); kv12.setSequenceId(w.getWriteNumber()); memstore.add(kv12); - mvcc.completeMemstoreInsert(w); + mvcc.completeAndWait(w); // BEFORE STARTING INSERT 2, SEE FIRST KVS - KeyValueScanner s = this.memstore.getScanners(mvcc.memstoreReadPoint()).get(0); + KeyValueScanner s = this.memstore.getScanners(mvcc.getReadPoint()).get(0); assertScannerResults(s, new KeyValue[]{kv11, kv12}); // START DELETE: Insert delete for one of the columns - w = mvcc.beginMemstoreInsertWithSeqNum(this.startSeqNum.incrementAndGet()); + w = mvcc.begin(); KeyValue kvDel = new KeyValue(row, f, q2, kv11.getTimestamp(), KeyValue.Type.DeleteColumn); kvDel.setSequenceId(w.getWriteNumber()); memstore.add(kvDel); // BEFORE COMPLETING DELETE, SEE FIRST KVS - s = this.memstore.getScanners(mvcc.memstoreReadPoint()).get(0); + s = this.memstore.getScanners(mvcc.getReadPoint()).get(0); assertScannerResults(s, new KeyValue[]{kv11, kv12}); // COMPLETE DELETE - mvcc.completeMemstoreInsert(w); + mvcc.completeAndWait(w); // NOW WE SHOULD SEE DELETE - s = this.memstore.getScanners(mvcc.memstoreReadPoint()).get(0); + s = this.memstore.getScanners(mvcc.getReadPoint()).get(0); assertScannerResults(s, new KeyValue[]{kv11, kvDel, kv12}); } @@ -419,7 +419,7 @@ public class TestDefaultMemStore extends TestCase { private void internalRun() throws IOException { for (long i = 0; i < NUM_TRIES && caughtException.get() == null; i++) { MultiVersionConcurrencyControl.WriteEntry w = - mvcc.beginMemstoreInsertWithSeqNum(this.startSeqNum.incrementAndGet()); + mvcc.begin(); // Insert the sequence value (i) byte[] v = Bytes.toBytes(i); @@ -427,10 +427,10 @@ public class TestDefaultMemStore extends TestCase { KeyValue kv = new KeyValue(row, f, q1, i, v); kv.setSequenceId(w.getWriteNumber()); memstore.add(kv); - mvcc.completeMemstoreInsert(w); + mvcc.completeAndWait(w); // Assert that we can read back - KeyValueScanner s = this.memstore.getScanners(mvcc.memstoreReadPoint()).get(0); + KeyValueScanner s = this.memstore.getScanners(mvcc.getReadPoint()).get(0); s.seek(kv); Cell ret = s.next(); 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 e96c6303a97..cb95d6f1346 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 @@ -56,9 +56,9 @@ import java.util.Map; import java.util.NavigableMap; import java.util.TreeMap; import java.util.UUID; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import org.apache.commons.lang.RandomStringUtils; @@ -659,7 +659,7 @@ public class TestHRegion { } long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, status); assertEquals(maxSeqId, seqId); - region.getMVCC().initialize(seqId); + region.getMVCC().advanceTo(seqId); Get get = new Get(row); Result result = region.get(get); for (long i = minSeqId; i <= maxSeqId; i += 10) { @@ -713,7 +713,7 @@ public class TestHRegion { } long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, status); assertEquals(maxSeqId, seqId); - region.getMVCC().initialize(seqId); + region.getMVCC().advanceTo(seqId); Get get = new Get(row); Result result = region.get(get); for (long i = minSeqId; i <= maxSeqId; i += 10) { @@ -881,7 +881,7 @@ public class TestHRegion { .getRegionFileSystem().getStoreDir(Bytes.toString(family))); WALUtil.writeCompactionMarker(region.getWAL(), this.region.getTableDesc(), - this.region.getRegionInfo(), compactionDescriptor, new AtomicLong(1)); + this.region.getRegionInfo(), compactionDescriptor, region.getMVCC()); Path recoveredEditsDir = WALSplitter.getRegionDirRecoveredEditsDir(regiondir); @@ -1536,14 +1536,19 @@ public class TestHRegion { LOG.info("batchPut will have to break into four batches to avoid row locks"); RowLock rowLock1 = region.getRowLock(Bytes.toBytes("row_2")); - RowLock rowLock2 = region.getRowLock(Bytes.toBytes("row_4")); - RowLock rowLock3 = region.getRowLock(Bytes.toBytes("row_6")); + RowLock rowLock2 = region.getRowLock(Bytes.toBytes("row_1")); + RowLock rowLock3 = region.getRowLock(Bytes.toBytes("row_3")); + RowLock rowLock4 = region.getRowLock(Bytes.toBytes("row_3"), true); + MultithreadedTestUtil.TestContext ctx = new MultithreadedTestUtil.TestContext(CONF); final AtomicReference retFromThread = new AtomicReference(); + final CountDownLatch startingPuts = new CountDownLatch(1); + final CountDownLatch startingClose = new CountDownLatch(1); TestThread putter = new TestThread(ctx) { @Override public void doWork() throws IOException { + startingPuts.countDown(); retFromThread.set(region.batchMutate(puts)); } }; @@ -1551,43 +1556,38 @@ public class TestHRegion { ctx.addThread(putter); ctx.startThreads(); - LOG.info("...waiting for put thread to sync 1st time"); - waitForCounter(source, "syncTimeNumOps", syncs + 1); - // Now attempt to close the region from another thread. Prior to HBASE-12565 // this would cause the in-progress batchMutate operation to to fail with // exception because it use to release and re-acquire the close-guard lock // between batches. Caller then didn't get status indicating which writes succeeded. // We now expect this thread to block until the batchMutate call finishes. - Thread regionCloseThread = new Thread() { + Thread regionCloseThread = new TestThread(ctx) { @Override - public void run() { + public void doWork() { try { + startingPuts.await(); + // Give some time for the batch mutate to get in. + // We don't want to race with the mutate + Thread.sleep(10); + startingClose.countDown(); HBaseTestingUtility.closeRegionAndWAL(region); } catch (IOException e) { throw new RuntimeException(e); + } catch (InterruptedException e) { + throw new RuntimeException(e); } } }; regionCloseThread.start(); + startingClose.await(); + startingPuts.await(); + Thread.sleep(100); LOG.info("...releasing row lock 1, which should let put thread continue"); rowLock1.release(); - - LOG.info("...waiting for put thread to sync 2nd time"); - waitForCounter(source, "syncTimeNumOps", syncs + 2); - - LOG.info("...releasing row lock 2, which should let put thread continue"); rowLock2.release(); - - LOG.info("...waiting for put thread to sync 3rd time"); - waitForCounter(source, "syncTimeNumOps", syncs + 3); - - LOG.info("...releasing row lock 3, which should let put thread continue"); rowLock3.release(); - - LOG.info("...waiting for put thread to sync 4th time"); - waitForCounter(source, "syncTimeNumOps", syncs + 4); + waitForCounter(source, "syncTimeNumOps", syncs + 1); LOG.info("...joining on put thread"); ctx.stop(); @@ -1598,6 +1598,7 @@ public class TestHRegion { assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY : OperationStatusCode.SUCCESS, codes[i].getOperationStatusCode()); } + rowLock4.release(); } finally { HBaseTestingUtility.closeRegionAndWAL(this.region); this.region = null; @@ -4750,7 +4751,6 @@ public class TestHRegion { } - @SuppressWarnings("unchecked") private void durabilityTest(String method, Durability tableDurability, Durability mutationDurability, long timeout, boolean expectAppend, final boolean expectSync, final boolean expectSyncFromLogSyncer) throws Exception { @@ -4775,7 +4775,7 @@ public class TestHRegion { //verify append called or not verify(wal, expectAppend ? times(1) : never()) .append((HTableDescriptor)any(), (HRegionInfo)any(), (WALKey)any(), - (WALEdit)any(), (AtomicLong)any(), Mockito.anyBoolean(), (List)any()); + (WALEdit)any(), Mockito.anyBoolean()); // verify sync called or not if (expectSync || expectSyncFromLogSyncer) { @@ -5901,7 +5901,6 @@ public class TestHRegion { } @Test - @SuppressWarnings("unchecked") public void testOpenRegionWrittenToWAL() throws Exception { final ServerName serverName = ServerName.valueOf("testOpenRegionWrittenToWAL", 100, 42); final RegionServerServices rss = spy(TEST_UTIL.createMockRegionServerService(serverName)); @@ -5936,7 +5935,7 @@ public class TestHRegion { TEST_UTIL.getConfiguration(), rss, null); verify(wal, times(1)).append((HTableDescriptor)any(), (HRegionInfo)any(), (WALKey)any() - , editCaptor.capture(), (AtomicLong)any(), anyBoolean(), (List)any()); + , editCaptor.capture(), anyBoolean()); WALEdit edit = editCaptor.getValue(); assertNotNull(edit); @@ -6003,8 +6002,8 @@ public class TestHRegion { ,sf.getReader().getHFileReader().getFileContext().isIncludesTags()); } } + @Test - @SuppressWarnings("unchecked") public void testOpenRegionWrittenToWALForLogReplay() throws Exception { // similar to the above test but with distributed log replay final ServerName serverName = ServerName.valueOf("testOpenRegionWrittenToWALForLogReplay", @@ -6050,7 +6049,7 @@ public class TestHRegion { // verify that we have not appended region open event to WAL because this region is still // recovering verify(wal, times(0)).append((HTableDescriptor)any(), (HRegionInfo)any(), (WALKey)any() - , editCaptor.capture(), (AtomicLong)any(), anyBoolean(), (List)any()); + , editCaptor.capture(), anyBoolean()); // not put the region out of recovering state new FinishRegionRecoveringHandler(rss, region.getRegionInfo().getEncodedName(), "/foo") @@ -6058,7 +6057,7 @@ public class TestHRegion { // now we should have put the entry verify(wal, times(1)).append((HTableDescriptor)any(), (HRegionInfo)any(), (WALKey)any() - , editCaptor.capture(), (AtomicLong)any(), anyBoolean(), (List)any()); + , editCaptor.capture(), anyBoolean()); WALEdit edit = editCaptor.getValue(); assertNotNull(edit); @@ -6094,7 +6093,6 @@ public class TestHRegion { } @Test - @SuppressWarnings("unchecked") public void testCloseRegionWrittenToWAL() throws Exception { final ServerName serverName = ServerName.valueOf("testCloseRegionWrittenToWAL", 100, 42); final RegionServerServices rss = spy(TEST_UTIL.createMockRegionServerService(serverName)); @@ -6122,7 +6120,7 @@ public class TestHRegion { // 2 times, one for region open, the other close region verify(wal, times(2)).append((HTableDescriptor)any(), (HRegionInfo)any(), (WALKey)any(), - editCaptor.capture(), (AtomicLong)any(), anyBoolean(), (List)any()); + editCaptor.capture(), anyBoolean()); WALEdit edit = editCaptor.getAllValues().get(1); assertNotNull(edit); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java index c63f6bed2d3..04e9b56dbf2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java @@ -35,7 +35,6 @@ import java.util.List; import java.util.Map; import java.util.Random; import java.util.UUID; -import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -761,7 +760,7 @@ public class TestHRegionReplayEvents { // ensure all files are visible in secondary for (Store store : secondaryRegion.getStores()) { - assertTrue(store.getMaxSequenceId() <= secondaryRegion.getSequenceId().get()); + assertTrue(store.getMaxSequenceId() <= secondaryRegion.getSequenceId()); } LOG.info("-- Replaying flush commit in secondary" + commitFlushDesc); @@ -1058,7 +1057,7 @@ public class TestHRegionReplayEvents { // TODO: what to do with this? // assert that the newly picked up flush file is visible - long readPoint = secondaryRegion.getMVCC().memstoreReadPoint(); + long readPoint = secondaryRegion.getMVCC().getReadPoint(); assertEquals(flushSeqId, readPoint); // after replay verify that everything is still visible @@ -1076,7 +1075,7 @@ public class TestHRegionReplayEvents { HRegion region = initHRegion(tableName, method, family); try { // replay an entry that is bigger than current read point - long readPoint = region.getMVCC().memstoreReadPoint(); + long readPoint = region.getMVCC().getReadPoint(); long origSeqId = readPoint + 100; Put put = new Put(row).add(family, row, row); @@ -1087,7 +1086,7 @@ public class TestHRegionReplayEvents { assertGet(region, family, row); // region seqId should have advanced at least to this seqId - assertEquals(origSeqId, region.getSequenceId().get()); + assertEquals(origSeqId, region.getSequenceId()); // replay an entry that is smaller than current read point // caution: adding an entry below current read point might cause partial dirty reads. Normal @@ -1116,7 +1115,7 @@ public class TestHRegionReplayEvents { // test for region open and close secondaryRegion = HRegion.openHRegion(secondaryHri, htd, walSecondary, CONF, rss, null); verify(walSecondary, times(0)).append((HTableDescriptor)any(), (HRegionInfo)any(), - (WALKey)any(), (WALEdit)any(), (AtomicLong)any(), anyBoolean(), (List) any()); + (WALKey)any(), (WALEdit)any(), anyBoolean()); // test for replay prepare flush putDataByReplay(secondaryRegion, 0, 10, cq, families); @@ -1130,11 +1129,11 @@ public class TestHRegionReplayEvents { .build()); verify(walSecondary, times(0)).append((HTableDescriptor)any(), (HRegionInfo)any(), - (WALKey)any(), (WALEdit)any(), (AtomicLong)any(), anyBoolean(), (List) any()); + (WALKey)any(), (WALEdit)any(), anyBoolean()); secondaryRegion.close(); verify(walSecondary, times(0)).append((HTableDescriptor)any(), (HRegionInfo)any(), - (WALKey)any(), (WALEdit)any(), (AtomicLong)any(), anyBoolean(), (List) any()); + (WALKey)any(), (WALEdit)any(), anyBoolean()); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java index 9286e0d5dda..d19d7094abe 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver; import static org.apache.hadoop.hbase.HBaseTestingUtility.COLUMNS; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -246,6 +247,14 @@ public class TestKeepDeletes { Put p = new Put(T1, ts); p.add(c0, c0, T1); region.put(p); + + Get gOne = new Get(T1); + gOne.setMaxVersions(); + gOne.setTimeRange(0L, ts + 1); + Result rOne = region.get(gOne); + assertFalse(rOne.isEmpty()); + + Delete d = new Delete(T1, ts+2); d.deleteColumn(c0, c0, ts); region.delete(d); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiVersionConcurrencyControl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiVersionConcurrencyControl.java index c811cda5379..a639e2c8b2d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiVersionConcurrencyControl.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiVersionConcurrencyControl.java @@ -50,7 +50,7 @@ public class TestMultiVersionConcurrencyControl extends TestCase { AtomicLong startPoint = new AtomicLong(); while (!finished.get()) { MultiVersionConcurrencyControl.WriteEntry e = - mvcc.beginMemstoreInsertWithSeqNum(startPoint.incrementAndGet()); + mvcc.begin(); // System.out.println("Begin write: " + e.getWriteNumber()); // 10 usec - 500usec (including 0) int sleepTime = rnd.nextInt(500); @@ -61,7 +61,7 @@ public class TestMultiVersionConcurrencyControl extends TestCase { } catch (InterruptedException e1) { } try { - mvcc.completeMemstoreInsert(e); + mvcc.completeAndWait(e); } catch (RuntimeException ex) { // got failure System.out.println(ex.toString()); @@ -84,9 +84,9 @@ public class TestMultiVersionConcurrencyControl extends TestCase { final AtomicLong failedAt = new AtomicLong(); Runnable reader = new Runnable() { public void run() { - long prev = mvcc.memstoreReadPoint(); + long prev = mvcc.getReadPoint(); while (!finished.get()) { - long newPrev = mvcc.memstoreReadPoint(); + long newPrev = mvcc.getReadPoint(); if (newPrev < prev) { // serious problem. System.out.println("Reader got out of order, prev: " + prev + " next was: " + newPrev); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiVersionConcurrencyControlBasic.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiVersionConcurrencyControlBasic.java index eceb92478a0..b63ca9e4d62 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiVersionConcurrencyControlBasic.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiVersionConcurrencyControlBasic.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hbase.regionserver; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -34,30 +33,13 @@ public class TestMultiVersionConcurrencyControlBasic { @Test public void testSimpleMvccOps() { MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(); - long readPoint = mvcc.memstoreReadPoint(); - MultiVersionConcurrencyControl.WriteEntry writeEntry = mvcc.beginMemstoreInsert(); - mvcc.completeMemstoreInsert(writeEntry); - long readPoint2 = mvcc.memstoreReadPoint(); - assertEquals(readPoint, readPoint2); - long seqid = 238; - writeEntry = mvcc.beginMemstoreInsertWithSeqNum(seqid); - mvcc.completeMemstoreInsert(writeEntry); - assertEquals(seqid, mvcc.memstoreReadPoint()); - writeEntry = mvcc.beginMemstoreInsertWithSeqNum(seqid + 1); - assertTrue(mvcc.advanceMemstore(writeEntry)); - assertEquals(seqid + 1, mvcc.memstoreReadPoint()); + long readPoint = mvcc.getReadPoint(); + MultiVersionConcurrencyControl.WriteEntry writeEntry = mvcc.begin(); + mvcc.completeAndWait(writeEntry); + assertEquals(readPoint + 1, mvcc.getReadPoint()); + writeEntry = mvcc.begin(); + // The write point advances even though we may have 'failed'... call complete on fail. + mvcc.complete(writeEntry); + assertEquals(readPoint + 2, mvcc.getWritePoint()); } - - @Test - public void testCancel() { - long seqid = 238; - MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(); - MultiVersionConcurrencyControl.WriteEntry writeEntry = - mvcc.beginMemstoreInsertWithSeqNum(seqid); - assertTrue(mvcc.advanceMemstore(writeEntry)); - assertEquals(seqid, mvcc.memstoreReadPoint()); - writeEntry = mvcc.beginMemstoreInsertWithSeqNum(seqid + 1); - mvcc.cancelMemstoreInsert(writeEntry); - assertEquals(seqid, mvcc.memstoreReadPoint()); - } -} +} \ No newline at end of file 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 c6aaf676fcb..5fe7f397c86 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 @@ -56,6 +56,7 @@ 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.wal.WAL; +import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -417,7 +418,7 @@ public class TestPerColumnFamilyFlush { // In distributed log replay, the log splitters ask the master for the // last flushed sequence id for a region. This test would ensure that we // are doing the book-keeping correctly. - @Test(timeout = 180000) + @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 180000) public void testLogReplayWithDistributedReplay() throws Exception { TEST_UTIL.getConfiguration().setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true); doTestLogReplay(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicaFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicaFailover.java index b18a0f48c1e..64cdff2b873 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicaFailover.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicaFailover.java @@ -95,7 +95,7 @@ public class TestRegionReplicaFailover { @Parameters public static Collection getParameters() { Object[][] params = - new Boolean[][] { {true}, {false} }; + new Boolean[][] { /*{true}, Disable DLR!!! It is going to be removed*/ {false} }; return Arrays.asList(params); } @@ -105,6 +105,8 @@ public class TestRegionReplicaFailover { @Before public void before() throws Exception { Configuration conf = HTU.getConfiguration(); + // Up the handlers; this test needs more than usual. + conf.setInt(HConstants.REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT, 10); conf.setBoolean(HConstants.REPLICATION_ENABLE_KEY, true); conf.setBoolean(ServerRegionReplicaUtil.REGION_REPLICA_REPLICATION_CONF_KEY, true); conf.setBoolean(ServerRegionReplicaUtil.REGION_REPLICA_WAIT_FOR_PRIMARY_FLUSH_CONF_KEY, true); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java index 665c5468a0c..18eda709baa 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java @@ -158,7 +158,7 @@ public class TestStoreFileRefresherChore { } } - @Test (timeout = 60000) + @Test public void testIsStale() throws IOException { int period = 0; byte[][] families = new byte[][] {Bytes.toBytes("cf")}; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java index ccf2b15ed81..11f77085040 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java @@ -22,8 +22,6 @@ package org.apache.hadoop.hbase.regionserver; import static org.junit.Assert.assertTrue; import java.io.IOException; -import java.util.ArrayList; -import java.util.List; import java.util.concurrent.CountDownLatch; import org.apache.commons.logging.Log; @@ -31,8 +29,6 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; @@ -212,21 +208,15 @@ public class TestWALLockup { put.addColumn(COLUMN_FAMILY_BYTES, Bytes.toBytes("1"), bytes); WALKey key = new WALKey(region.getRegionInfo().getEncodedNameAsBytes(), htd.getTableName()); WALEdit edit = new WALEdit(); - List cells = new ArrayList(); - for (CellScanner cs = put.cellScanner(); cs.advance();) { - edit.add(cs.current()); - cells.add(cs.current()); - } // Put something in memstore and out in the WAL. Do a big number of appends so we push // out other side of the ringbuffer. If small numbers, stuff doesn't make it to WAL for (int i = 0; i < 1000; i++) { - dodgyWAL.append(htd, region.getRegionInfo(), key, edit, region.getSequenceId(), true, - cells); + dodgyWAL.append(htd, region.getRegionInfo(), key, edit, true); } // Set it so we start throwing exceptions. dodgyWAL.throwException = true; // This append provokes a WAL roll. - dodgyWAL.append(htd, region.getRegionInfo(), key, edit, region.getSequenceId(), true, cells); + dodgyWAL.append(htd, region.getRegionInfo(), key, edit, true); boolean exception = false; try { dodgyWAL.sync(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java index 28ae46adf02..f86bdd5b21d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java @@ -29,7 +29,6 @@ import java.util.Comparator; import java.util.List; import java.util.Set; import java.util.UUID; -import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.lang.mutable.MutableBoolean; import org.apache.commons.logging.Log; @@ -54,7 +53,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.SampleRegionWALObserver; import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; @@ -152,15 +151,21 @@ public class TestFSHLog { } } - protected void addEdits(WAL log, HRegionInfo hri, HTableDescriptor htd, int times, - AtomicLong sequenceId) throws IOException { + protected void addEdits(WAL log, + HRegionInfo hri, + HTableDescriptor htd, + int times, + MultiVersionConcurrencyControl mvcc) + throws IOException { final byte[] row = Bytes.toBytes("row"); for (int i = 0; i < times; i++) { long timestamp = System.currentTimeMillis(); WALEdit cols = new WALEdit(); cols.add(new KeyValue(row, row, row, timestamp, row)); - log.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), htd.getTableName(), timestamp), - cols, sequenceId, true, null); + WALKey key = new WALKey(hri.getEncodedNameAsBytes(), htd.getTableName(), + WALKey.NO_SEQUENCE_ID, timestamp, WALKey.EMPTY_UUIDS, HConstants.NO_NONCE, + HConstants.NO_NONCE, mvcc); + log.append(htd, hri, key, cols, true); } log.sync(); } @@ -253,15 +258,13 @@ public class TestFSHLog { new HRegionInfo(t1.getTableName(), HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); HRegionInfo hri2 = new HRegionInfo(t2.getTableName(), HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); - // variables to mock region sequenceIds - final AtomicLong sequenceId1 = new AtomicLong(1); - final AtomicLong sequenceId2 = new AtomicLong(1); // add edits and roll the wal + MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(); try { - addEdits(wal, hri1, t1, 2, sequenceId1); + addEdits(wal, hri1, t1, 2, mvcc); wal.rollWriter(); // add some more edits and roll the wal. This would reach the log number threshold - addEdits(wal, hri1, t1, 2, sequenceId1); + addEdits(wal, hri1, t1, 2, mvcc); wal.rollWriter(); // with above rollWriter call, the max logs limit is reached. assertTrue(wal.getNumRolledLogFiles() == 2); @@ -272,7 +275,7 @@ public class TestFSHLog { assertEquals(1, regionsToFlush.length); assertEquals(hri1.getEncodedNameAsBytes(), regionsToFlush[0]); // insert edits in second region - addEdits(wal, hri2, t2, 2, sequenceId2); + addEdits(wal, hri2, t2, 2, mvcc); // get the regions to flush, it should still read region1. regionsToFlush = wal.findRegionsToForceFlush(); assertEquals(regionsToFlush.length, 1); @@ -289,12 +292,12 @@ public class TestFSHLog { // no wal should remain now. assertEquals(0, wal.getNumRolledLogFiles()); // add edits both to region 1 and region 2, and roll. - addEdits(wal, hri1, t1, 2, sequenceId1); - addEdits(wal, hri2, t2, 2, sequenceId2); + addEdits(wal, hri1, t1, 2, mvcc); + addEdits(wal, hri2, t2, 2, mvcc); wal.rollWriter(); // add edits and roll the writer, to reach the max logs limit. assertEquals(1, wal.getNumRolledLogFiles()); - addEdits(wal, hri1, t1, 2, sequenceId1); + addEdits(wal, hri1, t1, 2, mvcc); wal.rollWriter(); // it should return two regions to flush, as the oldest wal file has entries // for both regions. @@ -306,7 +309,7 @@ public class TestFSHLog { wal.rollWriter(true); assertEquals(0, wal.getNumRolledLogFiles()); // Add an edit to region1, and roll the wal. - addEdits(wal, hri1, t1, 2, sequenceId1); + addEdits(wal, hri1, t1, 2, mvcc); // tests partial flush: roll on a partial flush, and ensure that wal is not archived. wal.startCacheFlush(hri1.getEncodedNameAsBytes(), t1.getFamiliesKeys()); wal.rollWriter(); @@ -399,18 +402,18 @@ public class TestFSHLog { for (int i = 0; i < countPerFamily; i++) { final HRegionInfo info = region.getRegionInfo(); final WALKey logkey = new WALKey(info.getEncodedNameAsBytes(), tableName, - System.currentTimeMillis(), clusterIds, -1, -1); - wal.append(htd, info, logkey, edits, region.getSequenceId(), true, null); + System.currentTimeMillis(), clusterIds, -1, -1, region.getMVCC()); + wal.append(htd, info, logkey, edits, true); } region.flush(true); // FlushResult.flushSequenceId is not visible here so go get the current sequence id. - long currentSequenceId = region.getSequenceId().get(); + long currentSequenceId = region.getSequenceId(); // Now release the appends goslow.setValue(false); synchronized (goslow) { goslow.notifyAll(); } - assertTrue(currentSequenceId >= region.getSequenceId().get()); + assertTrue(currentSequenceId >= region.getSequenceId()); } finally { region.close(true); wal.close(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java index e990d836d63..8106acfb330 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java @@ -19,8 +19,8 @@ package org.apache.hadoop.hbase.regionserver.wal; import java.io.FileNotFoundException; import java.io.IOException; -import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.junit.Assert; @@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.util.Bytes; @@ -192,8 +191,7 @@ public class TestLogRollAbort { HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); final WAL log = wals.getWAL(regioninfo.getEncodedNameAsBytes(), regioninfo.getTable().getNamespace()); - - final AtomicLong sequenceId = new AtomicLong(1); + MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(1); final int total = 20; for (int i = 0; i < total; i++) { @@ -202,7 +200,7 @@ public class TestLogRollAbort { HTableDescriptor htd = new HTableDescriptor(tableName); htd.addFamily(new HColumnDescriptor("column")); log.append(htd, regioninfo, new WALKey(regioninfo.getEncodedNameAsBytes(), tableName, - System.currentTimeMillis()), kvs, sequenceId, true, null); + System.currentTimeMillis(), mvcc), kvs, true); } // Send the data to HDFS datanodes and close the HDFS writer log.sync(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java index ea565e15816..08c1b153766 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java @@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.regionserver.wal; import static org.junit.Assert.assertFalse; import java.io.IOException; -import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -32,6 +31,7 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -121,7 +121,7 @@ public class TestLogRollingNoCluster { @Override public void run() { this.log.info(getName() +" started"); - final AtomicLong sequenceId = new AtomicLong(1); + final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(); try { for (int i = 0; i < this.count; i++) { long now = System.currentTimeMillis(); @@ -135,7 +135,7 @@ public class TestLogRollingNoCluster { final HRegionInfo hri = HRegionInfo.FIRST_META_REGIONINFO; final HTableDescriptor htd = TEST_UTIL.getMetaTableDescriptor(); final long txid = wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), - TableName.META_TABLE_NAME, now), edit, sequenceId, true, null); + TableName.META_TABLE_NAME, now, mvcc), edit, true); wal.sync(txid); } String msg = getName() + " finished"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java index 390587d8176..82dfa92c3fb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java @@ -88,7 +88,6 @@ public class TestWALActionsListener { list.add(observer); final WALFactory wals = new WALFactory(conf, list, "testActionListener"); DummyWALActionsListener laterobserver = new DummyWALActionsListener(); - final AtomicLong sequenceId = new AtomicLong(1); HRegionInfo hri = new HRegionInfo(TableName.valueOf(SOME_BYTES), SOME_BYTES, SOME_BYTES, false); final WAL wal = wals.getWAL(hri.getEncodedNameAsBytes(), hri.getTable().getNamespace()); @@ -102,7 +101,7 @@ public class TestWALActionsListener { htd.addFamily(new HColumnDescriptor(b)); final long txid = wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), - TableName.valueOf(b), 0), edit, sequenceId, true, null); + TableName.valueOf(b), 0), edit, true); wal.sync(txid); if (i == 10) { wal.registerWALActionsListener(laterobserver); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java index c943d12abf3..1c97a2d7e8a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java @@ -33,7 +33,6 @@ import java.util.List; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -65,17 +64,7 @@ import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; -import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine; -import org.apache.hadoop.hbase.regionserver.DefaultStoreFlusher; -import org.apache.hadoop.hbase.regionserver.FlushRequestListener; -import org.apache.hadoop.hbase.regionserver.FlushRequester; -import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.regionserver.MemStoreSnapshot; -import org.apache.hadoop.hbase.regionserver.Region; -import org.apache.hadoop.hbase.regionserver.RegionScanner; -import org.apache.hadoop.hbase.regionserver.RegionServerServices; -import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.*; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; @@ -281,6 +270,8 @@ public class TestWALReplay { // Ensure edits are replayed properly. final TableName tableName = TableName.valueOf("test2727"); + + MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(); HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName); Path basedir = FSUtils.getTableDir(hbaseRootDir, tableName); deleteDir(basedir); @@ -293,10 +284,10 @@ public class TestWALReplay { WAL wal1 = createWAL(this.conf); // Add 1k to each family. final int countPerFamily = 1000; - final AtomicLong sequenceId = new AtomicLong(1); + for (HColumnDescriptor hcd: htd.getFamilies()) { addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily, ee, - wal1, htd, sequenceId); + wal1, htd, mvcc); } wal1.shutdown(); runWALSplit(this.conf); @@ -305,7 +296,7 @@ public class TestWALReplay { // Add 1k to each family. for (HColumnDescriptor hcd: htd.getFamilies()) { addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily, - ee, wal2, htd, sequenceId); + ee, wal2, htd, mvcc); } wal2.shutdown(); runWALSplit(this.conf); @@ -316,10 +307,10 @@ public class TestWALReplay { long seqid = region.getOpenSeqNum(); // The regions opens with sequenceId as 1. With 6k edits, its sequence number reaches 6k + 1. // When opened, this region would apply 6k edits, and increment the sequenceId by 1 - assertTrue(seqid > sequenceId.get()); - assertEquals(seqid - 1, sequenceId.get()); + assertTrue(seqid > mvcc.getWritePoint()); + assertEquals(seqid - 1, mvcc.getWritePoint()); LOG.debug("region.getOpenSeqNum(): " + region.getOpenSeqNum() + ", wal3.id: " - + sequenceId.get()); + + mvcc.getReadPoint()); // TODO: Scan all. region.close(); @@ -771,6 +762,7 @@ public class TestWALReplay { public void testReplayEditsWrittenIntoWAL() throws Exception { final TableName tableName = TableName.valueOf("testReplayEditsWrittenIntoWAL"); + final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(); final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName); final Path basedir = FSUtils.getTableDir(hbaseRootDir, tableName); deleteDir(basedir); @@ -781,14 +773,13 @@ public class TestWALReplay { final WAL wal = createWAL(this.conf); final byte[] rowName = tableName.getName(); final byte[] regionName = hri.getEncodedNameAsBytes(); - final AtomicLong sequenceId = new AtomicLong(1); // Add 1k to each family. final int countPerFamily = 1000; Set familyNames = new HashSet(); for (HColumnDescriptor hcd: htd.getFamilies()) { addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily, - ee, wal, htd, sequenceId); + ee, wal, htd, mvcc); familyNames.add(hcd.getName()); } @@ -801,16 +792,13 @@ public class TestWALReplay { long now = ee.currentTime(); edit.add(new KeyValue(rowName, Bytes.toBytes("another family"), rowName, now, rowName)); - wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName, now), edit, sequenceId, - true, null); + wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName, now, mvcc), edit, true); // Delete the c family to verify deletes make it over. edit = new WALEdit(); now = ee.currentTime(); - edit.add(new KeyValue(rowName, Bytes.toBytes("c"), null, now, - KeyValue.Type.DeleteFamily)); - wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName, now), edit, sequenceId, - true, null); + edit.add(new KeyValue(rowName, Bytes.toBytes("c"), null, now, KeyValue.Type.DeleteFamily)); + wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName, now, mvcc), edit, true); // Sync. wal.sync(); @@ -842,12 +830,17 @@ public class TestWALReplay { Mockito.mock(MonitoredTask.class), writeFlushWalMarker); flushcount.incrementAndGet(); return fs; - }; + } }; + // The seq id this region has opened up with long seqid = region.initialize(); + + // The mvcc readpoint of from inserting data. + long writePoint = mvcc.getWritePoint(); + // We flushed during init. assertTrue("Flushcount=" + flushcount.get(), flushcount.get() > 0); - assertTrue(seqid - 1 == sequenceId.get()); + assertTrue((seqid - 1) == writePoint); Get get = new Get(rowName); Result result = region.get(get); @@ -889,7 +882,7 @@ public class TestWALReplay { for (HColumnDescriptor hcd : htd.getFamilies()) { addRegionEdits(rowName, hcd.getName(), 5, this.ee, region, "x"); } - long lastestSeqNumber = region.getSequenceId().get(); + long lastestSeqNumber = region.getSequenceId(); // get the current seq no wal.doCompleteCacheFlush = true; // allow complete cache flush with the previous seq number got after first @@ -992,7 +985,7 @@ public class TestWALReplay { private void addWALEdits(final TableName tableName, final HRegionInfo hri, final byte[] rowName, final byte[] family, final int count, EnvironmentEdge ee, final WAL wal, - final HTableDescriptor htd, final AtomicLong sequenceId) + final HTableDescriptor htd, final MultiVersionConcurrencyControl mvcc) throws IOException { String familyStr = Bytes.toString(family); for (int j = 0; j < count; j++) { @@ -1001,8 +994,8 @@ public class TestWALReplay { WALEdit edit = new WALEdit(); edit.add(new KeyValue(rowName, family, qualifierBytes, ee.currentTime(), columnBytes)); - wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName, ee.currentTime()), - edit, sequenceId, true, null); + wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName,999, mvcc), + edit, true); } wal.sync(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java index bbe1495c011..d50522c9343 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java @@ -31,7 +31,6 @@ import java.util.SortedSet; import java.util.TreeSet; import java.util.UUID; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -52,6 +51,7 @@ import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ClusterConnection; +import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.replication.ReplicationFactory; @@ -190,9 +190,9 @@ public class TestReplicationSourceManager { @Test public void testLogRoll() throws Exception { - long seq = 0; long baseline = 1000; long time = baseline; + MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(); KeyValue kv = new KeyValue(r1, f1, r1); WALEdit edit = new WALEdit(); edit.add(kv); @@ -202,7 +202,6 @@ public class TestReplicationSourceManager { final WALFactory wals = new WALFactory(utility.getConfiguration(), listeners, URLEncoder.encode("regionserver:60020", "UTF8")); final WAL wal = wals.getWAL(hri.getEncodedNameAsBytes(), hri.getTable().getNamespace()); - final AtomicLong sequenceId = new AtomicLong(1); manager.init(); HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("tableame")); htd.addFamily(new HColumnDescriptor(f1)); @@ -212,8 +211,11 @@ public class TestReplicationSourceManager { wal.rollWriter(); } LOG.info(i); - final long txid = wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), test, - System.currentTimeMillis()), edit, sequenceId, true ,null); + final long txid = wal.append(htd, + hri, + new WALKey(hri.getEncodedNameAsBytes(), test, System.currentTimeMillis(), mvcc), + edit, + true); wal.sync(txid); } @@ -225,8 +227,10 @@ public class TestReplicationSourceManager { LOG.info(baseline + " and " + time); for (int i = 0; i < 3; i++) { - wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), test, - System.currentTimeMillis()), edit, sequenceId, true, null); + wal.append(htd, hri, + new WALKey(hri.getEncodedNameAsBytes(), test, System.currentTimeMillis(), mvcc), + edit, + true); } wal.sync(); @@ -241,8 +245,10 @@ public class TestReplicationSourceManager { manager.logPositionAndCleanOldLogs(manager.getSources().get(0).getCurrentPath(), "1", 0, false, false); - wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), test, - System.currentTimeMillis()), edit, sequenceId, true, null); + wal.append(htd, hri, + new WALKey(hri.getEncodedNameAsBytes(), test, System.currentTimeMillis(), mvcc), + edit, + true); wal.sync(); assertEquals(1, manager.getWALs().size()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationWALReaderManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationWALReaderManager.java index 41662a81b51..2ad34ea97b5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationWALReaderManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationWALReaderManager.java @@ -17,6 +17,19 @@ */ package org.apache.hadoop.hbase.replication.regionserver; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.fail; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.concurrent.atomic.AtomicLong; +import java.util.List; +import java.util.concurrent.atomic.AtomicLong; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -26,14 +39,15 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; -import org.apache.hadoop.hbase.wal.WAL; -import org.apache.hadoop.hbase.wal.WALFactory; -import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.junit.After; import org.junit.AfterClass; @@ -47,21 +61,12 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; -import static org.junit.Assert.*; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.concurrent.atomic.AtomicLong; - @Category({ReplicationTests.class, LargeTests.class}) @RunWith(Parameterized.class) public class TestReplicationWALReaderManager { private static HBaseTestingUtility TEST_UTIL; private static Configuration conf; - private static Path hbaseDir; private static FileSystem fs; private static MiniDFSCluster cluster; private static final TableName tableName = TableName.valueOf("tablename"); @@ -78,6 +83,7 @@ public class TestReplicationWALReaderManager { private int walEditKVs; private final AtomicLong sequenceId = new AtomicLong(1); @Rule public TestName tn = new TestName(); + private final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(); @Parameters public static Collection parameters() { @@ -106,6 +112,7 @@ public class TestReplicationWALReaderManager { this.walEditKVs = walEditKVs; TEST_UTIL.getConfiguration().setBoolean(HConstants.ENABLE_WAL_COMPRESSION, enableCompression); + mvcc.advanceTo(1); } @BeforeClass @@ -114,7 +121,6 @@ public class TestReplicationWALReaderManager { conf = TEST_UTIL.getConfiguration(); TEST_UTIL.startMiniDFSCluster(3); - hbaseDir = TEST_UTIL.createRootDir(); cluster = TEST_UTIL.getDFSCluster(); fs = cluster.getFileSystem(); } @@ -198,8 +204,9 @@ public class TestReplicationWALReaderManager { } private void appendToLogPlus(int count) throws IOException { - final long txid = log.append(htd, info, new WALKey(info.getEncodedNameAsBytes(), tableName, - System.currentTimeMillis()), getWALEdits(count), sequenceId, true, null); + final long txid = log.append(htd, info, + new WALKey(info.getEncodedNameAsBytes(), tableName, System.currentTimeMillis(), mvcc), + getWALEdits(count), true); log.sync(txid); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDistributedLogReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDistributedLogReplay.java deleted file mode 100644 index 8c00db43039..00000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDistributedLogReplay.java +++ /dev/null @@ -1,54 +0,0 @@ -/** - * 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.security.visibility; - -import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABELS_TABLE_NAME; - -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.testclassification.SecurityTests; -import org.junit.BeforeClass; -import org.junit.experimental.categories.Category; - -/** - * Test class that tests the visibility labels with distributed log replay feature ON. - */ -@Category({SecurityTests.class, MediumTests.class}) -public class TestVisibilityLabelsWithDistributedLogReplay extends - TestVisibilityLabelsWithDefaultVisLabelService { - - @BeforeClass - public static void setupBeforeClass() throws Exception { - // setup configuration - conf = TEST_UTIL.getConfiguration(); - conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true); - VisibilityTestUtil.enableVisiblityLabels(conf); - - conf.setClass(VisibilityUtils.VISIBILITY_LABEL_GENERATOR_CLASS, SimpleScanLabelGenerator.class, - ScanLabelGenerator.class); - conf.set("hbase.superuser", "admin"); - TEST_UTIL.startMiniCluster(2); - SUPERUSER = User.createUserForTesting(conf, "admin", new String[] { "supergroup" }); - USER1 = User.createUserForTesting(conf, "user1", new String[] {}); - - // Wait for the labels table to become available - TEST_UTIL.waitTableEnabled(LABELS_TABLE_NAME.getName(), 50000); - addLabels(); - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java index 32128226775..29540966907 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java @@ -66,11 +66,11 @@ public class FaultyFSLog extends FSHLog { @Override public long append(HTableDescriptor htd, HRegionInfo info, WALKey key, WALEdit edits, - AtomicLong sequenceId, boolean isInMemstore, List cells) throws IOException { + boolean inMemstore) throws IOException { if (this.ft == FailureType.APPEND) { throw new IOException("append"); } - return super.append(htd, info, key, edits, sequenceId, isInMemstore, cells); + return super.append(htd, info, key, edits, inMemstore); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestDefaultWALProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestDefaultWALProvider.java index c48f60bab7d..89c63a6044b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestDefaultWALProvider.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestDefaultWALProvider.java @@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; @@ -66,12 +67,14 @@ public class TestDefaultWALProvider { protected static Configuration conf; protected static FileSystem fs; protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + protected MultiVersionConcurrencyControl mvcc; @Rule public final TestName currentTest = new TestName(); @Before public void setUp() throws Exception { + mvcc = new MultiVersionConcurrencyControl(); FileStatus[] entries = fs.listStatus(new Path("/")); for (FileStatus dir : entries) { fs.delete(dir.getPath(), true); @@ -148,14 +151,14 @@ public class TestDefaultWALProvider { protected void addEdits(WAL log, HRegionInfo hri, HTableDescriptor htd, - int times, AtomicLong sequenceId) throws IOException { + int times) throws IOException { final byte[] row = Bytes.toBytes("row"); for (int i = 0; i < times; i++) { long timestamp = System.currentTimeMillis(); WALEdit cols = new WALEdit(); cols.add(new KeyValue(row, row, row, timestamp, row)); log.append(htd, hri, getWalKey(hri.getEncodedNameAsBytes(), htd.getTableName(), timestamp), - cols, sequenceId, true, null); + cols, true); } log.sync(); } @@ -164,7 +167,7 @@ public class TestDefaultWALProvider { * used by TestDefaultWALProviderWithHLogKey */ WALKey getWalKey(final byte[] info, final TableName tableName, final long timestamp) { - return new WALKey(info, tableName, timestamp); + return new WALKey(info, tableName, timestamp, mvcc); } /** @@ -202,26 +205,26 @@ public class TestDefaultWALProvider { // Add a single edit and make sure that rolling won't remove the file // Before HBASE-3198 it used to delete it - addEdits(log, hri, htd, 1, sequenceId); + addEdits(log, hri, htd, 1); log.rollWriter(); assertEquals(1, DefaultWALProvider.getNumRolledLogFiles(log)); // See if there's anything wrong with more than 1 edit - addEdits(log, hri, htd, 2, sequenceId); + addEdits(log, hri, htd, 2); log.rollWriter(); assertEquals(2, DefaultWALProvider.getNumRolledLogFiles(log)); // Now mix edits from 2 regions, still no flushing - addEdits(log, hri, htd, 1, sequenceId); - addEdits(log, hri2, htd2, 1, sequenceId); - addEdits(log, hri, htd, 1, sequenceId); - addEdits(log, hri2, htd2, 1, sequenceId); + addEdits(log, hri, htd, 1); + addEdits(log, hri2, htd2, 1); + addEdits(log, hri, htd, 1); + addEdits(log, hri2, htd2, 1); log.rollWriter(); assertEquals(3, DefaultWALProvider.getNumRolledLogFiles(log)); // Flush the first region, we expect to see the first two files getting // archived. We need to append something or writer won't be rolled. - addEdits(log, hri2, htd2, 1, sequenceId); + addEdits(log, hri2, htd2, 1); log.startCacheFlush(hri.getEncodedNameAsBytes(), htd.getFamiliesKeys()); log.completeCacheFlush(hri.getEncodedNameAsBytes()); log.rollWriter(); @@ -230,7 +233,7 @@ public class TestDefaultWALProvider { // Flush the second region, which removes all the remaining output files // since the oldest was completely flushed and the two others only contain // flush information - addEdits(log, hri2, htd2, 1, sequenceId); + addEdits(log, hri2, htd2, 1); log.startCacheFlush(hri2.getEncodedNameAsBytes(), htd2.getFamiliesKeys()); log.completeCacheFlush(hri2.getEncodedNameAsBytes()); log.rollWriter(); @@ -277,34 +280,32 @@ public class TestDefaultWALProvider { hri1.setSplit(false); hri2.setSplit(false); // variables to mock region sequenceIds. - final AtomicLong sequenceId1 = new AtomicLong(1); - final AtomicLong sequenceId2 = new AtomicLong(1); // start with the testing logic: insert a waledit, and roll writer - addEdits(wal, hri1, table1, 1, sequenceId1); + addEdits(wal, hri1, table1, 1); wal.rollWriter(); // assert that the wal is rolled assertEquals(1, DefaultWALProvider.getNumRolledLogFiles(wal)); // add edits in the second wal file, and roll writer. - addEdits(wal, hri1, table1, 1, sequenceId1); + addEdits(wal, hri1, table1, 1); wal.rollWriter(); // assert that the wal is rolled assertEquals(2, DefaultWALProvider.getNumRolledLogFiles(wal)); // add a waledit to table1, and flush the region. - addEdits(wal, hri1, table1, 3, sequenceId1); + addEdits(wal, hri1, table1, 3); flushRegion(wal, hri1.getEncodedNameAsBytes(), table1.getFamiliesKeys()); // roll log; all old logs should be archived. wal.rollWriter(); assertEquals(0, DefaultWALProvider.getNumRolledLogFiles(wal)); // add an edit to table2, and roll writer - addEdits(wal, hri2, table2, 1, sequenceId2); + addEdits(wal, hri2, table2, 1); wal.rollWriter(); assertEquals(1, DefaultWALProvider.getNumRolledLogFiles(wal)); // add edits for table1, and roll writer - addEdits(wal, hri1, table1, 2, sequenceId1); + addEdits(wal, hri1, table1, 2); wal.rollWriter(); assertEquals(2, DefaultWALProvider.getNumRolledLogFiles(wal)); // add edits for table2, and flush hri1. - addEdits(wal, hri2, table2, 2, sequenceId2); + addEdits(wal, hri2, table2, 2); flushRegion(wal, hri1.getEncodedNameAsBytes(), table2.getFamiliesKeys()); // the log : region-sequenceId map is // log1: region2 (unflushed) @@ -314,7 +315,7 @@ public class TestDefaultWALProvider { wal.rollWriter(); assertEquals(2, DefaultWALProvider.getNumRolledLogFiles(wal)); // flush region2, and all logs should be archived. - addEdits(wal, hri2, table2, 2, sequenceId2); + addEdits(wal, hri2, table2, 2); flushRegion(wal, hri2.getEncodedNameAsBytes(), table2.getFamiliesKeys()); wal.rollWriter(); assertEquals(0, DefaultWALProvider.getNumRolledLogFiles(wal)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestDefaultWALProviderWithHLogKey.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestDefaultWALProviderWithHLogKey.java index c667e9450d0..1885d87bf36 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestDefaultWALProviderWithHLogKey.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestDefaultWALProviderWithHLogKey.java @@ -29,6 +29,6 @@ import org.apache.hadoop.hbase.regionserver.wal.HLogKey; public class TestDefaultWALProviderWithHLogKey extends TestDefaultWALProvider { @Override WALKey getWalKey(final byte[] info, final TableName tableName, final long timestamp) { - return new HLogKey(info, tableName, timestamp); + return new HLogKey(info, tableName, timestamp, mvcc); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java index a74fd847735..c289663e8f1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java @@ -88,7 +88,6 @@ public class TestSecureWAL { final byte[] value = Bytes.toBytes("Test value"); FileSystem fs = TEST_UTIL.getTestFileSystem(); final WALFactory wals = new WALFactory(TEST_UTIL.getConfiguration(), null, "TestSecureWAL"); - final AtomicLong sequenceId = new AtomicLong(1); // Write the WAL final WAL wal = @@ -98,7 +97,7 @@ public class TestSecureWAL { WALEdit kvs = new WALEdit(); kvs.add(new KeyValue(row, family, Bytes.toBytes(i), value)); wal.append(htd, regioninfo, new WALKey(regioninfo.getEncodedNameAsBytes(), tableName, - System.currentTimeMillis()), kvs, sequenceId, true, null); + System.currentTimeMillis()), kvs, true); } wal.sync(); final Path walPath = DefaultWALProvider.getCurrentFileName(wal); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java index 875492cacbc..747977a91c8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java @@ -30,7 +30,6 @@ import java.io.IOException; import java.lang.reflect.Method; import java.net.BindException; import java.util.List; -import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -52,6 +51,7 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.SampleRegionWALObserver; +import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogReader; import org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogWriter; @@ -165,6 +165,7 @@ public class TestWALFactory { public void testSplit() throws IOException { final TableName tableName = TableName.valueOf(currentTest.getMethodName()); final byte [] rowName = tableName.getName(); + final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(1); final Path logdir = new Path(hbaseDir, DefaultWALProvider.getWALDirectoryName(currentTest.getMethodName())); Path oldLogDir = new Path(hbaseDir, HConstants.HREGION_OLDLOGDIR_NAME); @@ -182,7 +183,6 @@ public class TestWALFactory { htd.addFamily(new HColumnDescriptor("column")); // Add edits for three regions. - final AtomicLong sequenceId = new AtomicLong(1); for (int ii = 0; ii < howmany; ii++) { for (int i = 0; i < howmany; i++) { final WAL log = @@ -195,11 +195,13 @@ public class TestWALFactory { edit.add(new KeyValue(rowName, family, qualifier, System.currentTimeMillis(), column)); LOG.info("Region " + i + ": " + edit); - log.append(htd, infos[i], new WALKey(infos[i].getEncodedNameAsBytes(), tableName, - System.currentTimeMillis()), edit, sequenceId, true, null); + WALKey walKey = new WALKey(infos[i].getEncodedNameAsBytes(), tableName, + System.currentTimeMillis(), mvcc); + log.append(htd, infos[i], walKey, edit, true); + walKey.getWriteEntry(); } log.sync(); - log.rollWriter(); + log.rollWriter(true); } } wals.shutdown(); @@ -214,6 +216,7 @@ public class TestWALFactory { @Test public void Broken_testSync() throws Exception { TableName tableName = TableName.valueOf(currentTest.getMethodName()); + MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(1); // First verify that using streams all works. Path p = new Path(dir, currentTest.getMethodName() + ".fsdos"); FSDataOutputStream out = fs.create(p); @@ -238,7 +241,6 @@ public class TestWALFactory { out.close(); in.close(); - final AtomicLong sequenceId = new AtomicLong(1); final int total = 20; WAL.Reader reader = null; @@ -253,7 +255,7 @@ public class TestWALFactory { WALEdit kvs = new WALEdit(); kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), tableName.getName())); wal.append(htd, info, new WALKey(info.getEncodedNameAsBytes(), tableName, - System.currentTimeMillis()), kvs, sequenceId, true, null); + System.currentTimeMillis(), mvcc), kvs, true); } // Now call sync and try reading. Opening a Reader before you sync just // gives you EOFE. @@ -272,7 +274,7 @@ public class TestWALFactory { WALEdit kvs = new WALEdit(); kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), tableName.getName())); wal.append(htd, info, new WALKey(info.getEncodedNameAsBytes(), tableName, - System.currentTimeMillis()), kvs, sequenceId, true, null); + System.currentTimeMillis(), mvcc), kvs, true); } wal.sync(); reader = wals.createReader(fs, walPath); @@ -294,7 +296,7 @@ public class TestWALFactory { WALEdit kvs = new WALEdit(); kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), value)); wal.append(htd, info, new WALKey(info.getEncodedNameAsBytes(), tableName, - System.currentTimeMillis()), kvs, sequenceId, true, null); + System.currentTimeMillis(), mvcc), kvs, true); } // Now I should have written out lots of blocks. Sync then read. wal.sync(); @@ -364,7 +366,6 @@ public class TestWALFactory { final WAL wal = wals.getWAL(regioninfo.getEncodedNameAsBytes(), regioninfo.getTable().getNamespace()); - final AtomicLong sequenceId = new AtomicLong(1); final int total = 20; HTableDescriptor htd = new HTableDescriptor(tableName); @@ -374,7 +375,7 @@ public class TestWALFactory { WALEdit kvs = new WALEdit(); kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), tableName.getName())); wal.append(htd, regioninfo, new WALKey(regioninfo.getEncodedNameAsBytes(), tableName, - System.currentTimeMillis()), kvs, sequenceId, true, null); + System.currentTimeMillis()), kvs, true); } // Now call sync to send the data to HDFS datanodes wal.sync(); @@ -487,7 +488,7 @@ public class TestWALFactory { final byte [] row = Bytes.toBytes("row"); WAL.Reader reader = null; try { - final AtomicLong sequenceId = new AtomicLong(1); + final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(1); // Write columns named 1, 2, 3, etc. and then values of single byte // 1, 2, 3... @@ -503,8 +504,9 @@ public class TestWALFactory { final WAL log = wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace()); final long txid = log.append(htd, info, - new WALKey(info.getEncodedNameAsBytes(), htd.getTableName(), System.currentTimeMillis()), - cols, sequenceId, true, null); + new WALKey(info.getEncodedNameAsBytes(), htd.getTableName(), System.currentTimeMillis(), + mvcc), + cols, true); log.sync(txid); log.startCacheFlush(info.getEncodedNameAsBytes(), htd.getFamiliesKeys()); log.completeCacheFlush(info.getEncodedNameAsBytes()); @@ -545,7 +547,7 @@ public class TestWALFactory { "column")); final byte [] row = Bytes.toBytes("row"); WAL.Reader reader = null; - final AtomicLong sequenceId = new AtomicLong(1); + final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(1); try { // Write columns named 1, 2, 3, etc. and then values of single byte // 1, 2, 3... @@ -560,8 +562,9 @@ public class TestWALFactory { HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); final WAL log = wals.getWAL(hri.getEncodedNameAsBytes(), hri.getTable().getNamespace()); final long txid = log.append(htd, hri, - new WALKey(hri.getEncodedNameAsBytes(), htd.getTableName(), System.currentTimeMillis()), - cols, sequenceId, true, null); + new WALKey(hri.getEncodedNameAsBytes(), htd.getTableName(), System.currentTimeMillis(), + mvcc), + cols, true); log.sync(txid); log.startCacheFlush(hri.getEncodedNameAsBytes(), htd.getFamiliesKeys()); log.completeCacheFlush(hri.getEncodedNameAsBytes()); @@ -600,7 +603,7 @@ public class TestWALFactory { TableName.valueOf("tablename"); final byte [] row = Bytes.toBytes("row"); final DumbWALActionsListener visitor = new DumbWALActionsListener(); - final AtomicLong sequenceId = new AtomicLong(1); + final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(1); long timestamp = System.currentTimeMillis(); HTableDescriptor htd = new HTableDescriptor(tableName); htd.addFamily(new HColumnDescriptor("column")); @@ -615,7 +618,7 @@ public class TestWALFactory { Bytes.toBytes(Integer.toString(i)), timestamp, new byte[]{(byte) (i + '0')})); log.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName, - System.currentTimeMillis()), cols, sequenceId, true, null); + System.currentTimeMillis(), mvcc), cols, true); } log.sync(); assertEquals(COL_COUNT, visitor.increments); @@ -625,7 +628,7 @@ public class TestWALFactory { Bytes.toBytes(Integer.toString(11)), timestamp, new byte[]{(byte) (11 + '0')})); log.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName, - System.currentTimeMillis()), cols, sequenceId, true, null); + System.currentTimeMillis(), mvcc), cols, true); log.sync(); assertEquals(COL_COUNT, visitor.increments); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java index 415d626b62c..9ae98c634fc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java @@ -21,10 +21,8 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import java.io.IOException; -import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.io.IOUtils; -import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.impl.Log4JLogger; import org.apache.hadoop.conf.Configuration; @@ -38,11 +36,20 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; +import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; +// imports for things that haven't moved from regionserver.wal yet. +import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader; +import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter; +import org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogReader; +import org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogWriter; +import org.apache.hadoop.hbase.regionserver.wal.SecureWALCellCodec; +import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec; +import org.apache.hadoop.hbase.regionserver.wal.WALEdit; +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.FSUtils; import org.apache.hadoop.hbase.zookeeper.ZKSplitLog; @@ -53,21 +60,11 @@ import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -// imports for things that haven't moved from regionserver.wal yet. -import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader; -import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter; -import org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogReader; -import org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogWriter; -import org.apache.hadoop.hbase.regionserver.wal.WALEdit; -import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec; -import org.apache.hadoop.hbase.regionserver.wal.SecureWALCellCodec; - /* * Test that verifies WAL written by SecureProtobufLogWriter is not readable by ProtobufLogReader */ @Category({RegionServerTests.class, MediumTests.class}) public class TestWALReaderOnSecureWAL { - private static final Log LOG = LogFactory.getLog(TestWALReaderOnSecureWAL.class); static { ((Log4JLogger)LogFactory.getLog("org.apache.hadoop.hbase.regionserver.wal")) .getLogger().setLevel(Level.ALL); @@ -104,9 +101,7 @@ public class TestWALReaderOnSecureWAL { final int total = 10; final byte[] row = Bytes.toBytes("row"); final byte[] family = Bytes.toBytes("family"); - FileSystem fs = TEST_UTIL.getTestFileSystem(); - Path logDir = TEST_UTIL.getDataTestDir(tblName); - final AtomicLong sequenceId = new AtomicLong(1); + final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(1); // Write the WAL WAL wal = @@ -115,7 +110,7 @@ public class TestWALReaderOnSecureWAL { WALEdit kvs = new WALEdit(); kvs.add(new KeyValue(row, family, Bytes.toBytes(i), value)); wal.append(htd, regioninfo, new WALKey(regioninfo.getEncodedNameAsBytes(), tableName, - System.currentTimeMillis()), kvs, sequenceId, true, null); + System.currentTimeMillis(), mvcc), kvs, true); } wal.sync(); final Path walPath = DefaultWALProvider.getCurrentFileName(wal); @@ -150,7 +145,7 @@ public class TestWALReaderOnSecureWAL { // Confirm the WAL cannot be read back by ProtobufLogReader try { - WAL.Reader reader = wals.createReader(TEST_UTIL.getTestFileSystem(), walPath); + wals.createReader(TEST_UTIL.getTestFileSystem(), walPath); assertFalse(true); } catch (IOException ioe) { // expected IOE diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java index 64bf319559c..3af853b328e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java @@ -19,14 +19,12 @@ package org.apache.hadoop.hbase.wal; import java.io.IOException; -import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Random; import java.util.Set; -import java.util.UUID; import java.util.concurrent.TimeUnit; import org.apache.commons.logging.Log; @@ -50,16 +48,15 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.LogRoller; +import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; import org.apache.hadoop.hbase.trace.HBaseHTraceConfiguration; import org.apache.hadoop.hbase.trace.SpanReceiverHost; import org.apache.hadoop.hbase.wal.WALProvider.Writer; -import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; -import org.apache.htrace.HTraceConfiguration; import org.apache.htrace.Sampler; import org.apache.htrace.Trace; import org.apache.htrace.TraceScope; @@ -99,6 +96,8 @@ public final class WALPerformanceEvaluation extends Configured implements Tool { private final Histogram latencyHistogram = metrics.newHistogram(WALPerformanceEvaluation.class, "latencyHistogram", "nanos", true); + private final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(); + private HBaseTestingUtility TEST_UTIL; static final String TABLE_NAME = "WALPerformanceEvaluation"; @@ -179,8 +178,9 @@ public final class WALPerformanceEvaluation extends Configured implements Tool { WALEdit walEdit = new WALEdit(); addFamilyMapToWALEdit(put.getFamilyCellMap(), walEdit); HRegionInfo hri = region.getRegionInfo(); - final WALKey logkey = new WALKey(hri.getEncodedNameAsBytes(), hri.getTable(), now); - wal.append(htd, hri, logkey, walEdit, region.getSequenceId(), true, null); + final WALKey logkey = + new WALKey(hri.getEncodedNameAsBytes(), hri.getTable(), now, mvcc); + wal.append(htd, hri, logkey, walEdit, true); if (!this.noSync) { if (++lastSync >= this.syncInterval) { wal.sync();