HBASE-12751 Allow RowLock to be reader writer

This commit is contained in:
stack 2015-09-22 17:23:07 -07:00
parent f697d2d312
commit 2c83d8a263
57 changed files with 1302 additions and 1291 deletions

View File

@ -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";

View File

@ -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) {

View File

@ -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 {

View File

@ -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) {

View File

@ -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

View File

@ -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

View File

@ -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<WriteEntry> writeQueue =
new LinkedList<WriteEntry>();
//
// 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<WriteEntry> writeQueue = new LinkedList<WriteEntry>();
/**
* 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 <code>expected</code> is not equal to the
* current <code>readPoint</code> or if <code>startPoint</code> is less than current
* <code>readPoint</code>
*/
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);
}
}

View File

@ -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;
*
* <p>To read an WAL, call {@link WALFactory#createReader(org.apache.hadoop.fs.FileSystem,
* org.apache.hadoop.fs.Path)}.
*
*
* <h2>Failure Semantic</h2>
* 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<WALActionsListener> listeners = new CopyOnWriteArrayList<WALActionsListener>();
private final List<WALActionsListener> listeners =
new CopyOnWriteArrayList<WALActionsListener>();
@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
* <code>HRegionInfo#getEncodedNameAsBytes()</code>.
* @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<UUID> 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<Cell> 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<SyncFuture> 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 <= <code>currentSequence</code>.
* @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.
*
*
* <p>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.
*
*
* <p>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 <code>syncFuture</code>
* @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];
}
}
}

View File

@ -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<Cell> memstoreCells;
private final Set<byte[]> familyNames;
FSWALEntry(final long sequence, final WALKey key, final WALEdit edit,
final AtomicLong referenceToRegionSequenceId, final boolean inMemstore,
final HTableDescriptor htd, final HRegionInfo hri, List<Cell> 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<Cell> 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;
}

View File

@ -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<UUID> 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<UUID> 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<UUID> clusterIds, long nonceGroup, long nonce) {
super(encodedRegionName, tablename, now, clusterIds, nonceGroup, nonce);
public HLogKey(final byte[] encodedRegionName,
final TableName tablename,
final long now,
List<UUID> 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);

View File

@ -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<UUID> clusterIds, long nonceGroup, long nonce) {
super(encodedRegionName, tablename, now, clusterIds, nonceGroup, nonce);
final long now, List<UUID> 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<UUID> clusterIds, long nonceGroup, long nonce) {
super(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce);
long logSeqNum, final long now, List<UUID> clusterIds, long nonceGroup, long nonce,
MultiVersionConcurrencyControl mvcc) {
super(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce, mvcc);
}
/**

View File

@ -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);

View File

@ -73,7 +73,7 @@ import com.google.common.annotations.VisibleForTesting;
* where, the WALEdit is serialized as:
* &lt;-1, # of edits, &lt;KeyValue&gt;, &lt;KeyValue&gt;, ... &gt;
* For example:
* &lt;-1, 3, &lt;Keyvalue-for-edit-c1&gt;, &lt;KeyValue-for-edit-c2&gt;, &lt;KeyValue-for-edit-c3&gt;&gt;
* &lt;-1, 3, &lt;KV-for-edit-c1&gt;, &lt;KV-for-edit-c2&gt;, &lt;KV-for-edit-c3&gt;&gt;
*
* The -1 marker is just a special way of being backward compatible with
* an old WAL which would have contained a single &lt;KeyValue&gt;.
@ -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<byte[], Integer> scopes;

View File

@ -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<Cell>());
info,
key,
WALEdit.createBulkLoadEvent(info, descriptor), false);
wal.sync(trx);
if (LOG.isTraceEnabled()) {
@ -136,5 +140,4 @@ public class WALUtil {
}
return trx;
}
}

View File

@ -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

View File

@ -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<Cell> memstoreKVs) {
boolean inMemstore) {
if (!this.listeners.isEmpty()) {
final long start = System.nanoTime();
long len = 0;

View File

@ -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 <code>info</code> 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 <code>key</code> will have the region edit/sequence id
* in it.
*/
long append(HTableDescriptor htd, HRegionInfo info, WALKey key, WALEdit edits,
AtomicLong sequenceId, boolean inMemstore, List<Cell> memstoreKVs)
boolean inMemstore)
throws IOException;
/**

View File

@ -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.
*
* <p>
*/
// 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<WALKey> {
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<WALKey> {
private long nonceGroup = HConstants.NO_NONCE;
private long nonce = HConstants.NO_NONCE;
static final List<UUID> EMPTY_UUIDS = Collections.unmodifiableList(new ArrayList<UUID>());
private MultiVersionConcurrencyControl mvcc;
private MultiVersionConcurrencyControl.WriteEntry writeEntry;
public static final List<UUID> EMPTY_UUIDS = Collections.unmodifiableList(new ArrayList<UUID>());
// visible for deprecated HLogKey
@InterfaceAudience.Private
@ -159,16 +196,17 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
public WALKey() {
init(null, null, 0L, HConstants.LATEST_TIMESTAMP,
new ArrayList<UUID>(), HConstants.NO_NONCE, HConstants.NO_NONCE);
new ArrayList<UUID>(), 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<UUID> clusterIds = new ArrayList<UUID>();
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<WALKey> {
}
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<WALKey> {
* <p>Used by log splitting and snapshots.
*
* @param encodedRegionName Encoded name of the region as returned by
* <code>HRegionInfo#getEncodedNameAsBytes()</code>.
* @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)
* <code>HRegionInfo#getEncodedNameAsBytes()</code>.
* @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<UUID> 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<UUID> 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<WALKey> {
* A regionName is always a sub-table object.
*
* @param encodedRegionName Encoded name of the region as returned by
* <code>HRegionInfo#getEncodedNameAsBytes()</code>.
* <code>HRegionInfo#getEncodedNameAsBytes()</code>.
* @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<UUID> 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<UUID> 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<WALKey> {
* A regionName is always a sub-table object.
*
* @param encodedRegionName Encoded name of the region as returned by
* <code>HRegionInfo#getEncodedNameAsBytes()</code>.
* <code>HRegionInfo#getEncodedNameAsBytes()</code>.
* @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<UUID> clusterIds, long nonceGroup, long nonce) {
protected void init(final byte[] encodedRegionName,
final TableName tablename,
long logSeqNum,
final long now,
List<UUID> 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<WALKey> {
this.tablename = tablename;
this.nonceGroup = nonceGroup;
this.nonce = nonce;
this.mvcc = mvcc;
}
/**
@ -270,15 +352,14 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
}
/**
* 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<WALKey> {
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<WALKey> {
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);

View File

@ -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);
}

View File

@ -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;

View File

@ -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();
}
}
}
}

View File

@ -376,7 +376,6 @@ public class TestReplicasClient {
}
}
@Test
public void testFlushTable() throws Exception {
openRegion(hriSecondary);

View File

@ -141,6 +141,4 @@ public class TestRegionObserverStacking extends TestCase {
assertTrue(idB < idC);
HBaseTestingUtility.closeRegionAndWAL(region);
}
}
}

View File

@ -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);

View File

@ -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

View File

@ -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() {

View File

@ -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();
}

View File

@ -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();

View File

@ -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<Object []> 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();
}

View File

@ -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();
}
}
}

View File

@ -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)));
}

View File

@ -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<KeyValueScanner> memstorescanners = this.memstore.getScanners(mvcc.memstoreReadPoint());
List<KeyValueScanner> 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();

View File

@ -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<OperationStatus[]> retFromThread = new AtomicReference<OperationStatus[]>();
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<Cell>)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<Cell>)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<Cell>)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<Cell>)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<Cell>)any());
editCaptor.capture(), anyBoolean());
WALEdit edit = editCaptor.getAllValues().get(1);
assertNotNull(edit);

View File

@ -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<Cell>) 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<Cell>) 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<Cell>) any());
(WALKey)any(), (WALEdit)any(), anyBoolean());
}
/**

View File

@ -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);

View File

@ -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);

View File

@ -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());
}
}
}

View File

@ -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();

View File

@ -95,7 +95,7 @@ public class TestRegionReplicaFailover {
@Parameters
public static Collection<Object[]> 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);

View File

@ -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")};

View File

@ -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<Cell> cells = new ArrayList<Cell>();
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();

View File

@ -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();

View File

@ -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();

View File

@ -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";

View File

@ -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);

View File

@ -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<byte[]> familyNames = new HashSet<byte[]>();
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();
}

View File

@ -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());

View File

@ -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<Object[]> 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);
}

View File

@ -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();
}
}

View File

@ -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<Cell> 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);
}
}

View File

@ -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));

View File

@ -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);
}
}

View File

@ -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);

View File

@ -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);
}

View File

@ -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

View File

@ -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();