HBASE-11882 Row level consistency may not be maintained with bulk load and

compaction (Jerry He)
Signed-off-by: Ramkrishna <ramkrishna.s.vasudevan@intel.com>
This commit is contained in:
Ramkrishna 2014-09-05 08:55:53 +05:30
parent 7a699b9041
commit 8de30d32d4
2 changed files with 12 additions and 3 deletions

View File

@ -136,9 +136,17 @@ public abstract class Compactor {
fd.maxKeyCount += keyCount; fd.maxKeyCount += keyCount;
// calculate the latest MVCC readpoint in any of the involved store files // calculate the latest MVCC readpoint in any of the involved store files
Map<byte[], byte[]> fileInfo = r.loadFileInfo(); Map<byte[], byte[]> fileInfo = r.loadFileInfo();
byte tmp[] = fileInfo.get(HFileWriterV2.MAX_MEMSTORE_TS_KEY); byte tmp[] = null;
if (tmp != null) { // Get and set the real MVCCReadpoint for bulk loaded files, which is the
fd.maxMVCCReadpoint = Math.max(fd.maxMVCCReadpoint, Bytes.toLong(tmp)); // SeqId number.
if (r.isBulkLoaded()) {
fd.maxMVCCReadpoint = Math.max(fd.maxMVCCReadpoint, r.getSequenceID());
}
else {
tmp = fileInfo.get(HFileWriterV2.MAX_MEMSTORE_TS_KEY);
if (tmp != null) {
fd.maxMVCCReadpoint = Math.max(fd.maxMVCCReadpoint, Bytes.toLong(tmp));
}
} }
tmp = fileInfo.get(FileInfo.MAX_TAGS_LEN); tmp = fileInfo.get(FileInfo.MAX_TAGS_LEN);
if (tmp != null) { if (tmp != null) {

View File

@ -115,6 +115,7 @@ public class TestHRegionServerBulkLoad {
KeyValue kv = new KeyValue(rowkey(i), family, qualifier, now, value); KeyValue kv = new KeyValue(rowkey(i), family, qualifier, now, value);
writer.append(kv); writer.append(kv);
} }
writer.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY, Bytes.toBytes(now));
} finally { } finally {
writer.close(); writer.close();
} }