Skip to content

Commit

Permalink
HBASE-23741 Data loss when WAL split to HFile enabled (apache#1254)
Browse files Browse the repository at this point in the history
Signed-off-by: Duo Zhang <[email protected]>
  • Loading branch information
infraio committed Mar 25, 2020
1 parent c5045bd commit 54158c3
Show file tree
Hide file tree
Showing 5 changed files with 171 additions and 69 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -5515,7 +5515,7 @@ private void replayFlushInStores(FlushDescriptor flush, PrepareFlushResult prepa
}

private long loadRecoveredHFilesIfAny(Collection<HStore> stores) throws IOException {
Path regionDir = getWALRegionDir();
Path regionDir = fs.getRegionDir();
long maxSeqId = -1;
for (HStore store : stores) {
String familyName = store.getColumnFamilyName();
Expand All @@ -5528,17 +5528,13 @@ private long loadRecoveredHFilesIfAny(Collection<HStore> stores) throws IOExcept
if (isZeroLengthThenDelete(fs.getFileSystem(), file, filePath)) {
continue;
}

try {
store.assertBulkLoadHFileOk(filePath);
HStoreFile storefile = store.tryCommitRecoveredHFile(file.getPath());
maxSeqId = Math.max(maxSeqId, storefile.getReader().getSequenceID());
} catch (IOException e) {
handleException(fs.getFileSystem(), filePath, e);
continue;
}
Pair<Path, Path> pair = store.preBulkLoadHFile(filePath.toString(), -1);
store.bulkLoadHFile(Bytes.toBytes(familyName), pair.getFirst().toString(),
pair.getSecond());
maxSeqId = Math.max(maxSeqId, WALSplitUtil.getSeqIdForRecoveredHFile(filePath.getName()));
}
if (this.rsServices != null && store.needsCompaction()) {
this.rsServices.getCompactionRequestor()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1086,6 +1086,42 @@ protected List<Path> flushCache(final long logCacheFlushId, MemStoreSnapshot sna
throw lastException;
}

public HStoreFile tryCommitRecoveredHFile(Path path) throws IOException {
LOG.info("Validating recovered hfile at {} for inclusion in store {} region {}", path, this,
getRegionInfo().getRegionNameAsString());
FileSystem srcFs = path.getFileSystem(conf);
srcFs.access(path, FsAction.READ_WRITE);
try (HFile.Reader reader =
HFile.createReader(srcFs, path, cacheConf, isPrimaryReplicaStore(), conf)) {
Optional<byte[]> firstKey = reader.getFirstRowKey();
Preconditions.checkState(firstKey.isPresent(), "First key can not be null");
Optional<Cell> lk = reader.getLastKey();
Preconditions.checkState(lk.isPresent(), "Last key can not be null");
byte[] lastKey = CellUtil.cloneRow(lk.get());
if (!this.getRegionInfo().containsRange(firstKey.get(), lastKey)) {
throw new WrongRegionException("Recovered hfile " + path.toString() +
" does not fit inside region " + this.getRegionInfo().getRegionNameAsString());
}
}

Path dstPath = fs.commitStoreFile(getColumnFamilyName(), path);
HStoreFile sf = createStoreFileAndReader(dstPath);
StoreFileReader r = sf.getReader();
this.storeSize.addAndGet(r.length());
this.totalUncompressedBytes.addAndGet(r.getTotalUncompressedBytes());

this.lock.writeLock().lock();
try {
this.storeEngine.getStoreFileManager().insertNewFiles(Lists.newArrayList(sf));
} finally {
this.lock.writeLock().unlock();
}

LOG.info("Loaded recovered hfile to {}, entries={}, sequenceid={}, filesize={}", sf,
r.getEntries(), r.getSequenceID(), TraditionalBinaryPrefix.long2String(r.length(), "B", 1));
return sf;
}

/**
* @param path The pathname of the tmp file into which the store was flushed
* @return store file created.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,15 +32,17 @@

import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellComparatorImpl;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFileContext;
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
import org.apache.hadoop.hbase.regionserver.CellSet;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.wal.EntryBuffers.RegionEntryBuffer;
Expand Down Expand Up @@ -85,11 +87,14 @@ public void append(RegionEntryBuffer buffer) throws IOException {
if (CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
continue;
}
PrivateCellUtil.setSequenceId(cell, seqId);
String familyName = Bytes.toString(CellUtil.cloneFamily(cell));
// comparator need to be specified for meta
familyCells.computeIfAbsent(familyName, key -> new CellSet(
isMetaTable ? CellComparatorImpl.META_COMPARATOR : CellComparator.getInstance()))
.add(cell);
familyCells
.computeIfAbsent(familyName,
key -> new CellSet(
isMetaTable ? CellComparatorImpl.META_COMPARATOR : CellComparatorImpl.COMPARATOR))
.add(cell);
familySeqIds.compute(familyName, (k, v) -> v == null ? seqId : Math.max(v, seqId));
}
}
Expand All @@ -105,6 +110,8 @@ public void append(RegionEntryBuffer buffer) throws IOException {
for (Cell cell : cellsEntry.getValue()) {
writer.append(cell);
}
// Append the max seqid to hfile, used when recovery.
writer.appendMetadata(familySeqIds.get(familyName), false);
regionEditsWrittenMap.compute(Bytes.toString(buffer.encodedRegionName),
(k, v) -> v == null ? buffer.entries.size() : v + buffer.entries.size());
splits.add(writer.getPath());
Expand Down Expand Up @@ -181,44 +188,32 @@ public boolean keepRegionEvent(Entry entry) {

private StoreFileWriter createRecoveredHFileWriter(TableName tableName, String regionName,
long seqId, String familyName, boolean isMetaTable) throws IOException {
Path outputFile = WALSplitUtil
.getRegionRecoveredHFilePath(tableName, regionName, familyName, seqId,
walSplitter.getFileBeingSplit().getPath().getName(), walSplitter.conf, walSplitter.rootFS);
checkPathValid(outputFile);
Path outputDir = WALSplitUtil.tryCreateRecoveredHFilesDir(walSplitter.rootFS, walSplitter.conf,
tableName, regionName, familyName);
StoreFileWriter.Builder writerBuilder =
new StoreFileWriter.Builder(walSplitter.conf, CacheConfig.DISABLED, walSplitter.rootFS)
.withFilePath(outputFile);
HFileContextBuilder hFileContextBuilder = new HFileContextBuilder();
if (isMetaTable) {
hFileContextBuilder.withCellComparator(CellComparatorImpl.META_COMPARATOR);
} else {
configContextForNonMetaWriter(tableName, familyName, hFileContextBuilder, writerBuilder);
}
return writerBuilder.withFileContext(hFileContextBuilder.build()).build();
}
.withOutputDir(outputDir);

private void configContextForNonMetaWriter(TableName tableName, String familyName,
HFileContextBuilder hFileContextBuilder, StoreFileWriter.Builder writerBuilder)
throws IOException {
TableDescriptor tableDesc =
tableDescCache.computeIfAbsent(tableName, t -> getTableDescriptor(t));
if (tableDesc == null) {
throw new IOException("Failed to get table descriptor for table " + tableName);
}
ColumnFamilyDescriptor cfd = tableDesc.getColumnFamily(Bytes.toBytesBinary(familyName));
hFileContextBuilder.withCompression(cfd.getCompressionType()).withBlockSize(cfd.getBlocksize())
.withCompressTags(cfd.isCompressTags()).withDataBlockEncoding(cfd.getDataBlockEncoding())
.withCellComparator(CellComparatorImpl.COMPARATOR);
writerBuilder.withBloomType(cfd.getBloomFilterType());
HFileContext hFileContext = createFileContext(cfd, isMetaTable);
return writerBuilder.withFileContext(hFileContext).withBloomType(cfd.getBloomFilterType())
.build();
}

private void checkPathValid(Path outputFile) throws IOException {
if (walSplitter.rootFS.exists(outputFile)) {
LOG.warn("this file {} may be left after last failed split ", outputFile);
if (!walSplitter.rootFS.delete(outputFile, false)) {
LOG.warn("delete old generated HFile {} failed", outputFile);
}
}
private HFileContext createFileContext(ColumnFamilyDescriptor cfd, boolean isMetaTable)
throws IOException {
return new HFileContextBuilder().withCompression(cfd.getCompressionType())
.withChecksumType(HStore.getChecksumType(walSplitter.conf))
.withBytesPerCheckSum(HStore.getBytesPerChecksum(walSplitter.conf))
.withBlockSize(cfd.getBlocksize()).withCompressTags(cfd.isCompressTags())
.withDataBlockEncoding(cfd.getDataBlockEncoding()).withCellComparator(
isMetaTable ? CellComparatorImpl.META_COMPARATOR : CellComparatorImpl.COMPARATOR)
.build();
}

private TableDescriptor getTableDescriptor(TableName tableName) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -565,56 +565,43 @@ public static List<MutationReplay> getMutationsFromWALEntry(AdminProtos.WALEntry
}

/**
* Path to a file under recovered.hfiles directory of the region's column family: e.g.
* /hbase/some_table/2323432434/cf/recovered.hfiles/2332-wal. This method also ensures existence
* of recovered.hfiles directory under the region's column family, creating it if necessary.
*
* @param tableName the table name
* @param encodedRegionName the encoded region name
* @param familyName the column family name
* @param seqId the sequence id which used to generate file name
* Return path to recovered.hfiles directory of the region's column family: e.g.
* /hbase/some_table/2323432434/cf/recovered.hfiles/. This method also ensures existence of
* recovered.hfiles directory under the region's column family, creating it if necessary.
* @param rootFS the root file system
* @param conf configuration
* @param tableName the table name
* @param encodedRegionName the encoded region name
* @param familyName the column family name
* @param seqId the sequence id which used to generate file name
* @param fileNameBeingSplit the file being split currently. Used to generate tmp file name
* @param conf configuration
* @param rootFS the root file system
* @return Path to file into which to dump split log edits.
* @return Path to recovered.hfiles directory of the region's column family.
*/
static Path getRegionRecoveredHFilePath(TableName tableName, String encodedRegionName,
String familyName, long seqId, String fileNameBeingSplit, Configuration conf, FileSystem rootFS)
throws IOException {
static Path tryCreateRecoveredHFilesDir(FileSystem rootFS, Configuration conf,
TableName tableName, String encodedRegionName, String familyName) throws IOException {
Path rootDir = FSUtils.getRootDir(conf);
Path regionDir =
FSUtils.getRegionDirFromTableDir(FSUtils.getTableDir(rootDir, tableName), encodedRegionName);
Path dir = getStoreDirRecoveredHFilesDir(regionDir, familyName);

Path regionDir = FSUtils.getRegionDirFromTableDir(FSUtils.getTableDir(rootDir, tableName),
encodedRegionName);
Path dir = getRecoveredHFilesDir(regionDir, familyName);
if (!rootFS.exists(dir) && !rootFS.mkdirs(dir)) {
LOG.warn("mkdir failed on {}, region {}, column family {}", dir, encodedRegionName,
familyName);
}

String fileName = formatRecoveredHFileName(seqId, fileNameBeingSplit);
return new Path(dir, fileName);
}

private static String formatRecoveredHFileName(long seqId, String fileNameBeingSplit) {
return String.format("%019d", seqId) + "-" + fileNameBeingSplit;
}

public static long getSeqIdForRecoveredHFile(String fileName) {
return Long.parseLong(fileName.split("-")[0]);
return dir;
}

/**
* @param regionDir This regions directory in the filesystem
* @param familyName The column family name
* @return The directory that holds recovered hfiles for the region's column family
*/
private static Path getStoreDirRecoveredHFilesDir(final Path regionDir, String familyName) {
private static Path getRecoveredHFilesDir(final Path regionDir, String familyName) {
return new Path(new Path(regionDir, familyName), HConstants.RECOVERED_HFILES_DIR);
}

public static FileStatus[] getRecoveredHFiles(final FileSystem rootFS,
final Path regionDir, String familyName) throws IOException {
Path dir = getStoreDirRecoveredHFilesDir(regionDir, familyName);
Path dir = getRecoveredHFilesDir(regionDir, familyName);
return FSUtils.listStatus(rootFS, dir);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,9 @@
import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
Expand Down Expand Up @@ -252,6 +254,92 @@ public void testCorruptRecoveredHFile() throws Exception {
}
}

@Test
public void testPutWithSameTimestamp() throws Exception {
Pair<TableDescriptor, RegionInfo> pair = setupTableAndRegion();
TableDescriptor td = pair.getFirst();
RegionInfo ri = pair.getSecond();

WAL wal = createWAL(this.conf, rootDir, logName);
HRegion region = HRegion.openHRegion(this.conf, this.fs, rootDir, ri, td, wal);
final long timestamp = this.ee.currentTime();
// Write data and flush
for (ColumnFamilyDescriptor cfd : td.getColumnFamilies()) {
region.put(new Put(ROW).addColumn(cfd.getName(), Bytes.toBytes("x"), timestamp, VALUE1));
}
region.flush(true);

// Now assert edits made it in.
Result result1 = region.get(new Get(ROW));
assertEquals(td.getColumnFamilies().length, result1.size());
for (ColumnFamilyDescriptor cfd : td.getColumnFamilies()) {
assertTrue(Bytes.equals(VALUE1, result1.getValue(cfd.getName(), Bytes.toBytes("x"))));
}

// Write data with same timestamp and do not flush
for (ColumnFamilyDescriptor cfd : td.getColumnFamilies()) {
region.put(new Put(ROW).addColumn(cfd.getName(), Bytes.toBytes("x"), timestamp, VALUE2));
}
// Now close the region (without flush)
region.close(true);
wal.shutdown();
// split the log
WALSplitter.split(rootDir, logDir, oldLogDir, FileSystem.get(this.conf), this.conf, wals);

// reopen the region
WAL wal2 = createWAL(this.conf, rootDir, logName);
HRegion region2 = HRegion.openHRegion(conf, this.fs, rootDir, ri, td, wal2);
Result result2 = region2.get(new Get(ROW));
assertEquals(td.getColumnFamilies().length, result2.size());
for (ColumnFamilyDescriptor cfd : td.getColumnFamilies()) {
assertTrue(Bytes.equals(VALUE2, result2.getValue(cfd.getName(), Bytes.toBytes("x"))));
}
}

@Test
public void testRecoverSequenceId() throws Exception {
Pair<TableDescriptor, RegionInfo> pair = setupTableAndRegion();
TableDescriptor td = pair.getFirst();
RegionInfo ri = pair.getSecond();

WAL wal = createWAL(this.conf, rootDir, logName);
HRegion region = HRegion.openHRegion(this.conf, this.fs, rootDir, ri, td, wal);
Map<Integer, Map<String, Long>> seqIdMap = new HashMap<>();
// Write data and do not flush
for (int i = 0; i < countPerFamily; i++) {
for (ColumnFamilyDescriptor cfd : td.getColumnFamilies()) {
region.put(new Put(Bytes.toBytes(i)).addColumn(cfd.getName(), Bytes.toBytes("x"), VALUE1));
Result result = region.get(new Get(Bytes.toBytes(i)).addFamily(cfd.getName()));
assertTrue(Bytes.equals(VALUE1, result.getValue(cfd.getName(), Bytes.toBytes("x"))));
List<Cell> cells = result.listCells();
assertEquals(1, cells.size());
seqIdMap.computeIfAbsent(i, r -> new HashMap<>()).put(cfd.getNameAsString(),
cells.get(0).getSequenceId());
}
}

// Now close the region (without flush)
region.close(true);
wal.shutdown();
// split the log
WALSplitter.split(rootDir, logDir, oldLogDir, FileSystem.get(this.conf), this.conf, wals);

// reopen the region
WAL wal2 = createWAL(this.conf, rootDir, logName);
HRegion region2 = HRegion.openHRegion(conf, this.fs, rootDir, ri, td, wal2);
// assert the seqid was recovered
for (int i = 0; i < countPerFamily; i++) {
for (ColumnFamilyDescriptor cfd : td.getColumnFamilies()) {
Result result = region2.get(new Get(Bytes.toBytes(i)).addFamily(cfd.getName()));
assertTrue(Bytes.equals(VALUE1, result.getValue(cfd.getName(), Bytes.toBytes("x"))));
List<Cell> cells = result.listCells();
assertEquals(1, cells.size());
assertEquals((long) seqIdMap.get(i).get(cfd.getNameAsString()),
cells.get(0).getSequenceId());
}
}
}

/**
* Test writing edits into an HRegion, closing it, splitting logs, opening
* Region again. Verify seqids.
Expand Down

0 comments on commit 54158c3

Please sign in to comment.