HBASE-4797 [availability] Skip recovered.edits files with edits we know older than what region currently has (Jimmy Jiang)
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1205290 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
ef52263373
commit
351e75629f
|
@ -168,7 +168,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
// Members
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
private final ConcurrentHashMap<HashedBytes, CountDownLatch> lockedRows =
|
||||
private final ConcurrentHashMap<HashedBytes, CountDownLatch> lockedRows =
|
||||
new ConcurrentHashMap<HashedBytes, CountDownLatch>();
|
||||
private final ConcurrentHashMap<Integer, HashedBytes> lockIds =
|
||||
new ConcurrentHashMap<Integer, HashedBytes>();
|
||||
|
@ -487,10 +487,10 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
*/
|
||||
public long initialize(final CancelableProgressable reporter)
|
||||
throws IOException {
|
||||
|
||||
|
||||
MonitoredTask status = TaskMonitor.get().createStatus(
|
||||
"Initializing region " + this);
|
||||
|
||||
|
||||
if (coprocessorHost != null) {
|
||||
status.setStatus("Running coprocessor pre-open hook");
|
||||
coprocessorHost.preOpen();
|
||||
|
@ -558,17 +558,17 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
this.writestate.setReadOnly(this.htableDescriptor.isReadOnly());
|
||||
|
||||
this.writestate.compacting = 0;
|
||||
|
||||
|
||||
// Initialize split policy
|
||||
this.splitPolicy = RegionSplitPolicy.create(this, conf);
|
||||
|
||||
|
||||
this.lastFlushTime = EnvironmentEdgeManager.currentTimeMillis();
|
||||
// Use maximum of log sequenceid or that which was found in stores
|
||||
// (particularly if no recovered edits, seqid will be -1).
|
||||
long nextSeqid = maxSeqId + 1;
|
||||
LOG.info("Onlined " + this.toString() + "; next sequenceid=" + nextSeqid);
|
||||
|
||||
|
||||
|
||||
if (coprocessorHost != null) {
|
||||
status.setStatus("Running coprocessor post-open hooks");
|
||||
coprocessorHost.postOpen();
|
||||
|
@ -605,7 +605,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* This function will return the HDFS blocks distribution based on the data
|
||||
* captured when HFile is created
|
||||
|
@ -642,7 +642,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
Path tablePath = FSUtils.getTablePath(FSUtils.getRootDir(conf),
|
||||
tableDescriptor.getName());
|
||||
FileSystem fs = tablePath.getFileSystem(conf);
|
||||
|
||||
|
||||
for (HColumnDescriptor family: tableDescriptor.getFamilies()) {
|
||||
Path storeHomeDir = Store.getStoreHomedir(tablePath, regionEncodedName,
|
||||
family.getName());
|
||||
|
@ -660,27 +660,27 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
}
|
||||
return hdfsBlocksDistribution;
|
||||
}
|
||||
|
||||
|
||||
public AtomicLong getMemstoreSize() {
|
||||
return memstoreSize;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Increase the size of mem store in this region and the size of global mem
|
||||
* Increase the size of mem store in this region and the size of global mem
|
||||
* store
|
||||
* @param memStoreSize
|
||||
* @return the size of memstore in this region
|
||||
*/
|
||||
public long addAndGetGlobalMemstoreSize(long memStoreSize) {
|
||||
if (this.rsServices != null) {
|
||||
RegionServerAccounting rsAccounting =
|
||||
RegionServerAccounting rsAccounting =
|
||||
this.rsServices.getRegionServerAccounting();
|
||||
|
||||
|
||||
if (rsAccounting != null) {
|
||||
rsAccounting.addAndGetGlobalMemstoreSize(memStoreSize);
|
||||
}
|
||||
}
|
||||
return this.memstoreSize.getAndAdd(memStoreSize);
|
||||
return this.memstoreSize.getAndAdd(memStoreSize);
|
||||
}
|
||||
|
||||
/*
|
||||
|
@ -795,7 +795,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
MonitoredTask status = TaskMonitor.get().createStatus(
|
||||
"Closing region " + this +
|
||||
(abort ? " due to abort" : ""));
|
||||
|
||||
|
||||
status.setStatus("Waiting for close lock");
|
||||
try {
|
||||
synchronized (closeLock) {
|
||||
|
@ -1180,7 +1180,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
}
|
||||
}
|
||||
boolean result = internalFlushcache(status);
|
||||
|
||||
|
||||
if (coprocessorHost != null) {
|
||||
status.setStatus("Running post-flush coprocessor hooks");
|
||||
coprocessorHost.postFlush();
|
||||
|
@ -1228,7 +1228,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
* routes.
|
||||
*
|
||||
* <p> This method may block for some time.
|
||||
* @param status
|
||||
* @param status
|
||||
*
|
||||
* @return true if the region needs compacting
|
||||
*
|
||||
|
@ -1245,7 +1245,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
* @param wal Null if we're NOT to go via hlog/wal.
|
||||
* @param myseqid The seqid to use if <code>wal</code> is null writing out
|
||||
* flush file.
|
||||
* @param status
|
||||
* @param status
|
||||
* @return true if the region needs compacting
|
||||
* @throws IOException
|
||||
* @see #internalFlushcache(MonitoredTask)
|
||||
|
@ -1791,7 +1791,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
|
||||
/**
|
||||
* Perform a batch of puts.
|
||||
*
|
||||
*
|
||||
* @param putsAndLocks
|
||||
* the list of puts paired with their requested lock IDs.
|
||||
* @return an array of OperationStatus which internally contains the
|
||||
|
@ -2015,7 +2015,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
// STEP 7. Sync wal.
|
||||
// -------------------------
|
||||
if (walEdit.size() > 0 &&
|
||||
(this.regionInfo.isMetaRegion() ||
|
||||
(this.regionInfo.isMetaRegion() ||
|
||||
!this.htableDescriptor.isDeferredLogFlush())) {
|
||||
this.log.sync(txid);
|
||||
}
|
||||
|
@ -2378,7 +2378,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
|
||||
/**
|
||||
* Remove all the keys listed in the map from the memstore. This method is
|
||||
* called when a Put has updated memstore but subequently fails to update
|
||||
* called when a Put has updated memstore but subequently fails to update
|
||||
* the wal. This method is then invoked to rollback the memstore.
|
||||
*/
|
||||
private void rollbackMemstore(BatchOperationInProgress<Pair<Put, Integer>> batchOp,
|
||||
|
@ -2392,13 +2392,13 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
continue;
|
||||
}
|
||||
|
||||
// Rollback all the kvs for this row.
|
||||
Map<byte[], List<KeyValue>> familyMap = familyMaps[i];
|
||||
// Rollback all the kvs for this row.
|
||||
Map<byte[], List<KeyValue>> familyMap = familyMaps[i];
|
||||
for (Map.Entry<byte[], List<KeyValue>> e : familyMap.entrySet()) {
|
||||
byte[] family = e.getKey();
|
||||
List<KeyValue> edits = e.getValue();
|
||||
|
||||
// Remove those keys from the memstore that matches our
|
||||
// Remove those keys from the memstore that matches our
|
||||
// key's (row, cf, cq, timestamp, memstoreTS). The interesting part is
|
||||
// that even the memstoreTS has to match for keys that will be rolleded-back.
|
||||
Store store = getStore(family);
|
||||
|
@ -2502,16 +2502,36 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
protected long replayRecoveredEditsIfAny(final Path regiondir,
|
||||
final long minSeqId, final CancelableProgressable reporter,
|
||||
final MonitoredTask status)
|
||||
throws UnsupportedEncodingException, IOException {
|
||||
throws UnsupportedEncodingException, IOException {
|
||||
long seqid = minSeqId;
|
||||
NavigableSet<Path> files = HLog.getSplitEditFilesSorted(this.fs, regiondir);
|
||||
if (files == null || files.isEmpty()) return seqid;
|
||||
boolean checkSafeToSkip = true;
|
||||
for (Path edits: files) {
|
||||
if (edits == null || !this.fs.exists(edits)) {
|
||||
LOG.warn("Null or non-existent edits file: " + edits);
|
||||
continue;
|
||||
}
|
||||
if (isZeroLengthThenDelete(this.fs, edits)) continue;
|
||||
|
||||
if (checkSafeToSkip) {
|
||||
Path higher = files.higher(edits);
|
||||
long maxSeqId = Long.MAX_VALUE;
|
||||
if (higher != null) {
|
||||
// Edit file name pattern, HLog.EDITFILES_NAME_PATTERN: "-?[0-9]+"
|
||||
String fileName = higher.getName();
|
||||
maxSeqId = Math.abs(Long.parseLong(fileName));
|
||||
}
|
||||
if (maxSeqId <= minSeqId) {
|
||||
String msg = "Maximum possible sequenceid for this log is " + maxSeqId
|
||||
+ ", skipped the whole file, path=" + edits;
|
||||
LOG.debug(msg);
|
||||
continue;
|
||||
} else {
|
||||
checkSafeToSkip = false;
|
||||
}
|
||||
}
|
||||
|
||||
try {
|
||||
seqid = replayRecoveredEdits(edits, seqid, reporter);
|
||||
} catch (IOException e) {
|
||||
|
@ -2556,139 +2576,139 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
minSeqId + "; path=" + edits;
|
||||
LOG.info(msg);
|
||||
MonitoredTask status = TaskMonitor.get().createStatus(msg);
|
||||
|
||||
|
||||
status.setStatus("Opening logs");
|
||||
HLog.Reader reader = HLog.getReader(this.fs, edits, conf);
|
||||
try {
|
||||
long currentEditSeqId = minSeqId;
|
||||
long firstSeqIdInLog = -1;
|
||||
long skippedEdits = 0;
|
||||
long editsCount = 0;
|
||||
long intervalEdits = 0;
|
||||
HLog.Entry entry;
|
||||
Store store = null;
|
||||
boolean reported_once = false;
|
||||
long currentEditSeqId = minSeqId;
|
||||
long firstSeqIdInLog = -1;
|
||||
long skippedEdits = 0;
|
||||
long editsCount = 0;
|
||||
long intervalEdits = 0;
|
||||
HLog.Entry entry;
|
||||
Store store = null;
|
||||
boolean reported_once = false;
|
||||
|
||||
try {
|
||||
// How many edits seen before we check elapsed time
|
||||
int interval = this.conf.getInt("hbase.hstore.report.interval.edits",
|
||||
2000);
|
||||
// How often to send a progress report (default 1/2 master timeout)
|
||||
int period = this.conf.getInt("hbase.hstore.report.period",
|
||||
this.conf.getInt("hbase.master.assignment.timeoutmonitor.timeout",
|
||||
180000) / 2);
|
||||
long lastReport = EnvironmentEdgeManager.currentTimeMillis();
|
||||
try {
|
||||
// How many edits seen before we check elapsed time
|
||||
int interval = this.conf.getInt("hbase.hstore.report.interval.edits",
|
||||
2000);
|
||||
// How often to send a progress report (default 1/2 master timeout)
|
||||
int period = this.conf.getInt("hbase.hstore.report.period",
|
||||
this.conf.getInt("hbase.master.assignment.timeoutmonitor.timeout",
|
||||
180000) / 2);
|
||||
long lastReport = EnvironmentEdgeManager.currentTimeMillis();
|
||||
|
||||
while ((entry = reader.next()) != null) {
|
||||
HLogKey key = entry.getKey();
|
||||
WALEdit val = entry.getEdit();
|
||||
while ((entry = reader.next()) != null) {
|
||||
HLogKey key = entry.getKey();
|
||||
WALEdit val = entry.getEdit();
|
||||
|
||||
if (reporter != null) {
|
||||
intervalEdits += val.size();
|
||||
if (intervalEdits >= interval) {
|
||||
// Number of edits interval reached
|
||||
intervalEdits = 0;
|
||||
long cur = EnvironmentEdgeManager.currentTimeMillis();
|
||||
if (lastReport + period <= cur) {
|
||||
status.setStatus("Replaying edits..." +
|
||||
" skipped=" + skippedEdits +
|
||||
" edits=" + editsCount);
|
||||
// Timeout reached
|
||||
if(!reporter.progress()) {
|
||||
msg = "Progressable reporter failed, stopping replay";
|
||||
LOG.warn(msg);
|
||||
status.abort(msg);
|
||||
throw new IOException(msg);
|
||||
if (reporter != null) {
|
||||
intervalEdits += val.size();
|
||||
if (intervalEdits >= interval) {
|
||||
// Number of edits interval reached
|
||||
intervalEdits = 0;
|
||||
long cur = EnvironmentEdgeManager.currentTimeMillis();
|
||||
if (lastReport + period <= cur) {
|
||||
status.setStatus("Replaying edits..." +
|
||||
" skipped=" + skippedEdits +
|
||||
" edits=" + editsCount);
|
||||
// Timeout reached
|
||||
if(!reporter.progress()) {
|
||||
msg = "Progressable reporter failed, stopping replay";
|
||||
LOG.warn(msg);
|
||||
status.abort(msg);
|
||||
throw new IOException(msg);
|
||||
}
|
||||
reported_once = true;
|
||||
lastReport = cur;
|
||||
}
|
||||
reported_once = true;
|
||||
lastReport = cur;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Start coprocessor replay here. The coprocessor is for each WALEdit
|
||||
// instead of a KeyValue.
|
||||
if (coprocessorHost != null) {
|
||||
status.setStatus("Running pre-WAL-restore hook in coprocessors");
|
||||
if (coprocessorHost.preWALRestore(this.getRegionInfo(), key, val)) {
|
||||
// if bypass this log entry, ignore it ...
|
||||
continue;
|
||||
// Start coprocessor replay here. The coprocessor is for each WALEdit
|
||||
// instead of a KeyValue.
|
||||
if (coprocessorHost != null) {
|
||||
status.setStatus("Running pre-WAL-restore hook in coprocessors");
|
||||
if (coprocessorHost.preWALRestore(this.getRegionInfo(), key, val)) {
|
||||
// if bypass this log entry, ignore it ...
|
||||
continue;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (firstSeqIdInLog == -1) {
|
||||
firstSeqIdInLog = key.getLogSeqNum();
|
||||
}
|
||||
// Now, figure if we should skip this edit.
|
||||
if (key.getLogSeqNum() <= currentEditSeqId) {
|
||||
skippedEdits++;
|
||||
continue;
|
||||
}
|
||||
currentEditSeqId = key.getLogSeqNum();
|
||||
boolean flush = false;
|
||||
for (KeyValue kv: val.getKeyValues()) {
|
||||
// Check this edit is for me. Also, guard against writing the special
|
||||
// METACOLUMN info such as HBASE::CACHEFLUSH entries
|
||||
if (kv.matchingFamily(HLog.METAFAMILY) ||
|
||||
!Bytes.equals(key.getEncodedRegionName(), this.regionInfo.getEncodedNameAsBytes())) {
|
||||
skippedEdits++;
|
||||
continue;
|
||||
}
|
||||
// Figure which store the edit is meant for.
|
||||
if (store == null || !kv.matchingFamily(store.getFamily().getName())) {
|
||||
store = this.stores.get(kv.getFamily());
|
||||
if (firstSeqIdInLog == -1) {
|
||||
firstSeqIdInLog = key.getLogSeqNum();
|
||||
}
|
||||
if (store == null) {
|
||||
// This should never happen. Perhaps schema was changed between
|
||||
// crash and redeploy?
|
||||
LOG.warn("No family for " + kv);
|
||||
// Now, figure if we should skip this edit.
|
||||
if (key.getLogSeqNum() <= currentEditSeqId) {
|
||||
skippedEdits++;
|
||||
continue;
|
||||
}
|
||||
// Once we are over the limit, restoreEdit will keep returning true to
|
||||
// flush -- but don't flush until we've played all the kvs that make up
|
||||
// the WALEdit.
|
||||
flush = restoreEdit(store, kv);
|
||||
editsCount++;
|
||||
}
|
||||
if (flush) internalFlushcache(null, currentEditSeqId, status);
|
||||
currentEditSeqId = key.getLogSeqNum();
|
||||
boolean flush = false;
|
||||
for (KeyValue kv: val.getKeyValues()) {
|
||||
// Check this edit is for me. Also, guard against writing the special
|
||||
// METACOLUMN info such as HBASE::CACHEFLUSH entries
|
||||
if (kv.matchingFamily(HLog.METAFAMILY) ||
|
||||
!Bytes.equals(key.getEncodedRegionName(), this.regionInfo.getEncodedNameAsBytes())) {
|
||||
skippedEdits++;
|
||||
continue;
|
||||
}
|
||||
// Figure which store the edit is meant for.
|
||||
if (store == null || !kv.matchingFamily(store.getFamily().getName())) {
|
||||
store = this.stores.get(kv.getFamily());
|
||||
}
|
||||
if (store == null) {
|
||||
// This should never happen. Perhaps schema was changed between
|
||||
// crash and redeploy?
|
||||
LOG.warn("No family for " + kv);
|
||||
skippedEdits++;
|
||||
continue;
|
||||
}
|
||||
// Once we are over the limit, restoreEdit will keep returning true to
|
||||
// flush -- but don't flush until we've played all the kvs that make up
|
||||
// the WALEdit.
|
||||
flush = restoreEdit(store, kv);
|
||||
editsCount++;
|
||||
}
|
||||
if (flush) internalFlushcache(null, currentEditSeqId, status);
|
||||
|
||||
if (coprocessorHost != null) {
|
||||
coprocessorHost.postWALRestore(this.getRegionInfo(), key, val);
|
||||
if (coprocessorHost != null) {
|
||||
coprocessorHost.postWALRestore(this.getRegionInfo(), key, val);
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (EOFException eof) {
|
||||
Path p = HLog.moveAsideBadEditsFile(fs, edits);
|
||||
msg = "Encountered EOF. Most likely due to Master failure during " +
|
||||
"log spliting, so we have this data in another edit. " +
|
||||
"Continuing, but renaming " + edits + " as " + p;
|
||||
LOG.warn(msg, eof);
|
||||
status.abort(msg);
|
||||
} catch (IOException ioe) {
|
||||
// If the IOE resulted from bad file format,
|
||||
// then this problem is idempotent and retrying won't help
|
||||
if (ioe.getCause() instanceof ParseException) {
|
||||
} catch (EOFException eof) {
|
||||
Path p = HLog.moveAsideBadEditsFile(fs, edits);
|
||||
msg = "File corruption encountered! " +
|
||||
msg = "Encountered EOF. Most likely due to Master failure during " +
|
||||
"log spliting, so we have this data in another edit. " +
|
||||
"Continuing, but renaming " + edits + " as " + p;
|
||||
LOG.warn(msg, ioe);
|
||||
status.setStatus(msg);
|
||||
} else {
|
||||
status.abort(StringUtils.stringifyException(ioe));
|
||||
// other IO errors may be transient (bad network connection,
|
||||
// checksum exception on one datanode, etc). throw & retry
|
||||
throw ioe;
|
||||
LOG.warn(msg, eof);
|
||||
status.abort(msg);
|
||||
} catch (IOException ioe) {
|
||||
// If the IOE resulted from bad file format,
|
||||
// then this problem is idempotent and retrying won't help
|
||||
if (ioe.getCause() instanceof ParseException) {
|
||||
Path p = HLog.moveAsideBadEditsFile(fs, edits);
|
||||
msg = "File corruption encountered! " +
|
||||
"Continuing, but renaming " + edits + " as " + p;
|
||||
LOG.warn(msg, ioe);
|
||||
status.setStatus(msg);
|
||||
} else {
|
||||
status.abort(StringUtils.stringifyException(ioe));
|
||||
// other IO errors may be transient (bad network connection,
|
||||
// checksum exception on one datanode, etc). throw & retry
|
||||
throw ioe;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (reporter != null && !reported_once) {
|
||||
reporter.progress();
|
||||
}
|
||||
msg = "Applied " + editsCount + ", skipped " + skippedEdits +
|
||||
", firstSequenceidInLog=" + firstSeqIdInLog +
|
||||
", maxSequenceidInLog=" + currentEditSeqId + ", path=" + edits;
|
||||
status.markComplete(msg);
|
||||
LOG.debug(msg);
|
||||
return currentEditSeqId;
|
||||
if (reporter != null && !reported_once) {
|
||||
reporter.progress();
|
||||
}
|
||||
msg = "Applied " + editsCount + ", skipped " + skippedEdits +
|
||||
", firstSequenceidInLog=" + firstSeqIdInLog +
|
||||
", maxSequenceidInLog=" + currentEditSeqId + ", path=" + edits;
|
||||
status.markComplete(msg);
|
||||
LOG.debug(msg);
|
||||
return currentEditSeqId;
|
||||
} finally {
|
||||
reader.close();
|
||||
status.cleanup();
|
||||
|
@ -2712,7 +2732,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
* @throws IOException
|
||||
*/
|
||||
private static boolean isZeroLengthThenDelete(final FileSystem fs, final Path p)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
FileStatus stat = fs.getFileStatus(p);
|
||||
if (stat.getLen() > 0) return false;
|
||||
LOG.warn("File " + p + " is zero-length, deleting.");
|
||||
|
@ -2721,7 +2741,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
}
|
||||
|
||||
protected Store instantiateHStore(Path tableDir, HColumnDescriptor c)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
return new Store(tableDir, this, c, this.fs, this.conf);
|
||||
}
|
||||
|
||||
|
@ -2801,7 +2821,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
try {
|
||||
HashedBytes rowKey = new HashedBytes(row);
|
||||
CountDownLatch rowLatch = new CountDownLatch(1);
|
||||
|
||||
|
||||
// loop until we acquire the row lock (unless !waitForLock)
|
||||
while (true) {
|
||||
CountDownLatch existingLatch = lockedRows.putIfAbsent(rowKey, rowLatch);
|
||||
|
@ -2822,7 +2842,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
// loop until we generate an unused lock id
|
||||
while (true) {
|
||||
Integer lockId = lockIdGenerator.incrementAndGet();
|
||||
|
@ -2848,7 +2868,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
HashedBytes rowKey = lockIds.get(lockid);
|
||||
return rowKey == null ? null : rowKey.getBytes();
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Release the row lock!
|
||||
* @param lockId The lock ID to release.
|
||||
|
@ -2899,7 +2919,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
}
|
||||
return lid;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Determines whether multiple column families are present
|
||||
* Precondition: familyPaths is not null
|
||||
|
@ -2994,7 +3014,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
try {
|
||||
store.bulkLoadHFile(path);
|
||||
} catch (IOException ioe) {
|
||||
// a failure here causes an atomicity violation that we currently
|
||||
// a failure here causes an atomicity violation that we currently
|
||||
// cannot recover from since it is likely a failed hdfs operation.
|
||||
|
||||
// TODO Need a better story for reverting partial failures due to HDFS.
|
||||
|
@ -3303,9 +3323,9 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
|
||||
/**
|
||||
* Convenience method creating new HRegions. Used by createTable.
|
||||
* The {@link HLog} for the created region needs to be closed explicitly.
|
||||
* The {@link HLog} for the created region needs to be closed explicitly.
|
||||
* Use {@link HRegion#getLog()} to get access.
|
||||
*
|
||||
*
|
||||
* @param info Info for region to create.
|
||||
* @param rootDir Root directory for HBase instance
|
||||
* @param conf
|
||||
|
@ -3332,14 +3352,14 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
HLog effectiveHLog = hlog;
|
||||
if (hlog == null) {
|
||||
effectiveHLog = new HLog(fs, new Path(regionDir, HConstants.HREGION_LOGDIR_NAME),
|
||||
new Path(regionDir, HConstants.HREGION_OLDLOGDIR_NAME), conf);
|
||||
new Path(regionDir, HConstants.HREGION_OLDLOGDIR_NAME), conf);
|
||||
}
|
||||
HRegion region = HRegion.newHRegion(tableDir,
|
||||
effectiveHLog, fs, conf, info, hTableDescriptor, null);
|
||||
region.initialize();
|
||||
return region;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Open a Region.
|
||||
* @param info Info for region to be opened.
|
||||
|
@ -3932,12 +3952,12 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
// TODO: There's a lot of boiler plate code identical
|
||||
// to increment... See how to better unify that.
|
||||
/**
|
||||
*
|
||||
*
|
||||
* Perform one or more append operations on a row.
|
||||
* <p>
|
||||
* Appends performed are done under row lock but reads do not take locks out
|
||||
* so this can be seen partially complete by gets and scans.
|
||||
*
|
||||
*
|
||||
* @param append
|
||||
* @param lockid
|
||||
* @param writeToWAL
|
||||
|
@ -4308,7 +4328,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
public static final long DEEP_OVERHEAD = FIXED_OVERHEAD +
|
||||
ClassSize.OBJECT + // closeLock
|
||||
(2 * ClassSize.ATOMIC_BOOLEAN) + // closed, closing
|
||||
ClassSize.ATOMIC_LONG + // memStoreSize
|
||||
ClassSize.ATOMIC_LONG + // memStoreSize
|
||||
ClassSize.ATOMIC_INTEGER + // lockIdGenerator
|
||||
(3 * ClassSize.CONCURRENT_HASHMAP) + // lockedRows, lockIds, scannerReadPoints
|
||||
WriteState.HEAP_SIZE + // writestate
|
||||
|
@ -4533,13 +4553,13 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
if (this.explicitSplitPoint != null) {
|
||||
return this.explicitSplitPoint;
|
||||
}
|
||||
|
||||
|
||||
if (!splitPolicy.shouldSplit()) {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
||||
byte[] ret = splitPolicy.getSplitPoint();
|
||||
|
||||
|
||||
if (ret != null) {
|
||||
try {
|
||||
checkRow(ret, "calculated split");
|
||||
|
@ -4547,7 +4567,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
LOG.error("Ignoring invalid split", e);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
|
|
|
@ -34,16 +34,30 @@ import java.util.concurrent.atomic.AtomicReference;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.*;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestCase;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
|
||||
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
||||
import org.apache.hadoop.hbase.MultithreadedTestUtil;
|
||||
import org.apache.hadoop.hbase.MultithreadedTestUtil.TestThread;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.filter.BinaryComparator;
|
||||
import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
|
||||
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
|
||||
|
@ -52,9 +66,13 @@ import org.apache.hadoop.hbase.filter.FilterList;
|
|||
import org.apache.hadoop.hbase.filter.NullComparator;
|
||||
import org.apache.hadoop.hbase.filter.PrefixFilter;
|
||||
import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
|
||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
|
||||
import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
|
||||
|
@ -64,9 +82,9 @@ import org.apache.hadoop.hbase.util.Pair;
|
|||
import org.apache.hadoop.hbase.util.PairOfSameType;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
|
||||
/**
|
||||
|
@ -1296,7 +1314,7 @@ public class TestHRegion extends HBaseTestCase {
|
|||
LOG.info("" + addContent(region, fam3));
|
||||
region.flushcache();
|
||||
region.compactStores();
|
||||
byte [] splitRow = region.checkSplit();
|
||||
byte [] splitRow = region.checkSplit();
|
||||
assertNotNull(splitRow);
|
||||
LOG.info("SplitRow: " + Bytes.toString(splitRow));
|
||||
HRegion [] subregions = splitRegion(region, splitRow);
|
||||
|
@ -2170,7 +2188,7 @@ public class TestHRegion extends HBaseTestCase {
|
|||
} catch (Exception exception) {
|
||||
// Expected.
|
||||
}
|
||||
|
||||
|
||||
|
||||
assertICV(row1, fam1, qual1, row1Field1);
|
||||
assertICV(row1, fam1, qual2, row1Field2);
|
||||
|
@ -2302,7 +2320,7 @@ public class TestHRegion extends HBaseTestCase {
|
|||
LOG.info("" + addContent(region, fam3));
|
||||
region.flushcache();
|
||||
region.compactStores();
|
||||
byte [] splitRow = region.checkSplit();
|
||||
byte [] splitRow = region.checkSplit();
|
||||
assertNotNull(splitRow);
|
||||
LOG.info("SplitRow: " + Bytes.toString(splitRow));
|
||||
HRegion [] regions = splitRegion(region, splitRow);
|
||||
|
@ -2337,7 +2355,7 @@ public class TestHRegion extends HBaseTestCase {
|
|||
// To make regions splitable force compaction.
|
||||
for (int i = 0; i < regions.length; i++) {
|
||||
regions[i].compactStores();
|
||||
midkeys[i] = regions[i].checkSplit();
|
||||
midkeys[i] = regions[i].checkSplit();
|
||||
}
|
||||
|
||||
TreeMap<String, HRegion> sortedMap = new TreeMap<String, HRegion>();
|
||||
|
@ -2809,6 +2827,115 @@ public class TestHRegion extends HBaseTestCase {
|
|||
region.get(g, null);
|
||||
}
|
||||
|
||||
public void testSkipRecoveredEditsReplay() throws Exception {
|
||||
String method = "testSkipRecoveredEditsReplay";
|
||||
byte[] tableName = Bytes.toBytes(method);
|
||||
byte[] family = Bytes.toBytes("family");
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
initHRegion(tableName, method, conf, family);
|
||||
Path regiondir = region.getRegionDir();
|
||||
FileSystem fs = region.getFilesystem();
|
||||
byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
|
||||
|
||||
Path recoveredEditsDir = HLog.getRegionDirRecoveredEditsDir(regiondir);
|
||||
|
||||
long maxSeqId = 1050;
|
||||
long minSeqId = 1000;
|
||||
|
||||
for (long i = minSeqId; i <= maxSeqId; i += 10) {
|
||||
Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i));
|
||||
HLog.Writer writer = HLog.createWriter(fs, recoveredEdits, conf);
|
||||
|
||||
long time = System.nanoTime();
|
||||
WALEdit edit = new WALEdit();
|
||||
edit.add(new KeyValue(row, family, Bytes.toBytes(i),
|
||||
time, KeyValue.Type.Put, Bytes.toBytes(i)));
|
||||
writer.append(new HLog.Entry(new HLogKey(regionName, tableName,
|
||||
i, time, HConstants.DEFAULT_CLUSTER_ID), edit));
|
||||
|
||||
writer.close();
|
||||
}
|
||||
MonitoredTask status = TaskMonitor.get().createStatus(method);
|
||||
long seqId = region.replayRecoveredEditsIfAny(regiondir, minSeqId-1, null, status);
|
||||
assertEquals(maxSeqId, seqId);
|
||||
Get get = new Get(row);
|
||||
Result result = region.get(get, null);
|
||||
for (long i = minSeqId; i <= maxSeqId; i += 10) {
|
||||
List<KeyValue> kvs = result.getColumn(family, Bytes.toBytes(i));
|
||||
assertEquals(1, kvs.size());
|
||||
assertEquals(Bytes.toBytes(i), kvs.get(0).getValue());
|
||||
}
|
||||
}
|
||||
|
||||
public void testSkipRecoveredEditsReplaySomeIgnored() throws Exception {
|
||||
String method = "testSkipRecoveredEditsReplaySomeIgnored";
|
||||
byte[] tableName = Bytes.toBytes(method);
|
||||
byte[] family = Bytes.toBytes("family");
|
||||
initHRegion(tableName, method, HBaseConfiguration.create(), family);
|
||||
Path regiondir = region.getRegionDir();
|
||||
FileSystem fs = region.getFilesystem();
|
||||
byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
|
||||
|
||||
Path recoveredEditsDir = HLog.getRegionDirRecoveredEditsDir(regiondir);
|
||||
|
||||
long maxSeqId = 1050;
|
||||
long minSeqId = 1000;
|
||||
|
||||
for (long i = minSeqId; i <= maxSeqId; i += 10) {
|
||||
Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i));
|
||||
HLog.Writer writer = HLog.createWriter(fs, recoveredEdits, conf);
|
||||
|
||||
long time = System.nanoTime();
|
||||
WALEdit edit = new WALEdit();
|
||||
edit.add(new KeyValue(row, family, Bytes.toBytes(i),
|
||||
time, KeyValue.Type.Put, Bytes.toBytes(i)));
|
||||
writer.append(new HLog.Entry(new HLogKey(regionName, tableName,
|
||||
i, time, HConstants.DEFAULT_CLUSTER_ID), edit));
|
||||
|
||||
writer.close();
|
||||
}
|
||||
long recoverSeqId = 1030;
|
||||
MonitoredTask status = TaskMonitor.get().createStatus(method);
|
||||
long seqId = region.replayRecoveredEditsIfAny(regiondir, recoverSeqId-1, null, status);
|
||||
assertEquals(maxSeqId, seqId);
|
||||
Get get = new Get(row);
|
||||
Result result = region.get(get, null);
|
||||
for (long i = minSeqId; i <= maxSeqId; i += 10) {
|
||||
List<KeyValue> kvs = result.getColumn(family, Bytes.toBytes(i));
|
||||
if (i < recoverSeqId) {
|
||||
assertEquals(0, kvs.size());
|
||||
} else {
|
||||
assertEquals(1, kvs.size());
|
||||
assertEquals(Bytes.toBytes(i), kvs.get(0).getValue());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void testSkipRecoveredEditsReplayAllIgnored() throws Exception {
|
||||
String method = "testSkipRecoveredEditsReplayAllIgnored";
|
||||
byte[] tableName = Bytes.toBytes(method);
|
||||
byte[] family = Bytes.toBytes("family");
|
||||
initHRegion(tableName, method, HBaseConfiguration.create(), family);
|
||||
Path regiondir = region.getRegionDir();
|
||||
FileSystem fs = region.getFilesystem();
|
||||
|
||||
Path recoveredEditsDir = HLog.getRegionDirRecoveredEditsDir(regiondir);
|
||||
for (int i = 1000; i < 1050; i += 10) {
|
||||
Path recoveredEdits = new Path(
|
||||
recoveredEditsDir, String.format("%019d", i));
|
||||
FSDataOutputStream dos= fs.create(recoveredEdits);
|
||||
dos.writeInt(i);
|
||||
dos.close();
|
||||
}
|
||||
long minSeqId = 2000;
|
||||
Path recoveredEdits = new Path(
|
||||
recoveredEditsDir, String.format("%019d", minSeqId-1));
|
||||
FSDataOutputStream dos= fs.create(recoveredEdits);
|
||||
dos.close();
|
||||
long seqId = region.replayRecoveredEditsIfAny(regiondir, minSeqId, null, null);
|
||||
assertEquals(minSeqId, seqId);
|
||||
}
|
||||
|
||||
public void testIndexesScanWithOneDeletedRow() throws IOException {
|
||||
byte[] tableName = Bytes.toBytes("testIndexesScanWithOneDeletedRow");
|
||||
byte[] family = Bytes.toBytes("family");
|
||||
|
@ -2864,13 +2991,13 @@ public class TestHRegion extends HBaseTestCase {
|
|||
HColumnDescriptor hcd = new HColumnDescriptor(fam1, Integer.MAX_VALUE,
|
||||
HColumnDescriptor.DEFAULT_COMPRESSION, false, true,
|
||||
HColumnDescriptor.DEFAULT_TTL, "rowcol");
|
||||
|
||||
|
||||
HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||
htd.addFamily(hcd);
|
||||
HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
|
||||
Path path = new Path(DIR + "testBloomFilterSize");
|
||||
region = HRegion.createHRegion(info, path, conf, htd);
|
||||
|
||||
|
||||
int num_unique_rows = 10;
|
||||
int duplicate_multiplier =2;
|
||||
int num_storefiles = 4;
|
||||
|
@ -2887,7 +3014,7 @@ public class TestHRegion extends HBaseTestCase {
|
|||
}
|
||||
region.flushcache();
|
||||
}
|
||||
//before compaction
|
||||
//before compaction
|
||||
Store store = region.getStore(fam1);
|
||||
List<StoreFile> storeFiles = store.getStorefiles();
|
||||
for (StoreFile storefile : storeFiles) {
|
||||
|
@ -2897,10 +3024,10 @@ public class TestHRegion extends HBaseTestCase {
|
|||
assertEquals(num_unique_rows*duplicate_multiplier, reader.getEntries());
|
||||
assertEquals(num_unique_rows, reader.getFilterEntries());
|
||||
}
|
||||
|
||||
region.compactStores(true);
|
||||
|
||||
//after compaction
|
||||
|
||||
region.compactStores(true);
|
||||
|
||||
//after compaction
|
||||
storeFiles = store.getStorefiles();
|
||||
for (StoreFile storefile : storeFiles) {
|
||||
StoreFile.Reader reader = storefile.getReader();
|
||||
|
@ -2909,9 +3036,9 @@ public class TestHRegion extends HBaseTestCase {
|
|||
assertEquals(num_unique_rows*duplicate_multiplier*num_storefiles,
|
||||
reader.getEntries());
|
||||
assertEquals(num_unique_rows, reader.getFilterEntries());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public void testAllColumnsWithBloomFilter() throws IOException {
|
||||
byte [] TABLE = Bytes.toBytes("testAllColumnsWithBloomFilter");
|
||||
byte [] FAMILY = Bytes.toBytes("family");
|
||||
|
@ -3002,13 +3129,13 @@ public class TestHRegion extends HBaseTestCase {
|
|||
final int DEFAULT_BLOCK_SIZE = 1024;
|
||||
htu.getConfiguration().setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
|
||||
htu.getConfiguration().setInt("dfs.replication", 2);
|
||||
|
||||
|
||||
|
||||
|
||||
// set up a cluster with 3 nodes
|
||||
MiniHBaseCluster cluster;
|
||||
String dataNodeHosts[] = new String[] { "host1", "host2", "host3" };
|
||||
int regionServersCount = 3;
|
||||
|
||||
|
||||
try {
|
||||
cluster = htu.startMiniCluster(1, regionServersCount, dataNodeHosts);
|
||||
byte [][] families = {fam1, fam2};
|
||||
|
@ -3018,31 +3145,31 @@ public class TestHRegion extends HBaseTestCase {
|
|||
byte row[] = Bytes.toBytes("row1");
|
||||
byte col[] = Bytes.toBytes("col1");
|
||||
|
||||
Put put = new Put(row);
|
||||
Put put = new Put(row);
|
||||
put.add(fam1, col, 1, Bytes.toBytes("test1"));
|
||||
put.add(fam2, col, 1, Bytes.toBytes("test2"));
|
||||
ht.put(put);
|
||||
|
||||
|
||||
HRegion firstRegion = htu.getHBaseCluster().
|
||||
getRegions(Bytes.toBytes(this.getName())).get(0);
|
||||
firstRegion.flushcache();
|
||||
HDFSBlocksDistribution blocksDistribution1 =
|
||||
firstRegion.getHDFSBlocksDistribution();
|
||||
|
||||
|
||||
// given the default replication factor is 2 and we have 2 HFiles,
|
||||
// we will have total of 4 replica of blocks on 3 datanodes; thus there
|
||||
// must be at least one host that have replica for 2 HFiles. That host's
|
||||
// weight will be equal to the unique block weight.
|
||||
long uniqueBlocksWeight1 =
|
||||
blocksDistribution1.getUniqueBlocksTotalWeight();
|
||||
|
||||
|
||||
String topHost = blocksDistribution1.getTopHosts().get(0);
|
||||
long topHostWeight = blocksDistribution1.getWeight(topHost);
|
||||
assertTrue(uniqueBlocksWeight1 == topHostWeight);
|
||||
|
||||
|
||||
// use the static method to compute the value, it should be the same.
|
||||
// static method is used by load balancer or other components
|
||||
HDFSBlocksDistribution blocksDistribution2 =
|
||||
HDFSBlocksDistribution blocksDistribution2 =
|
||||
HRegion.computeHDFSBlocksDistribution(htu.getConfiguration(),
|
||||
firstRegion.getTableDesc(),
|
||||
firstRegion.getRegionInfo().getEncodedName());
|
||||
|
@ -3054,7 +3181,7 @@ public class TestHRegion extends HBaseTestCase {
|
|||
htu.shutdownMiniCluster();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private void putData(int startRow, int numRows, byte [] qf,
|
||||
byte [] ...families)
|
||||
throws IOException {
|
||||
|
|
Loading…
Reference in New Issue