HBASE-22627 Port HBASE-22617 (Recovered WAL directories not getting cleaned up) to branch-1 (#339)

HBASE-22617 Recovered WAL directories not getting cleaned up (Duo Zhang)

Signed-off-by: Zach York <zyork@apache.org>
This commit is contained in:
Andrew Purtell 2019-06-25 18:14:01 -07:00 committed by GitHub
parent 5023690c84
commit 5a06739d04
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
18 changed files with 131 additions and 117 deletions

View File

@ -32,10 +32,8 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@ -85,7 +83,7 @@ public class HFileArchiver {
throws IOException {
Path rootDir = FSUtils.getRootDir(conf);
archiveRegion(fs, rootDir, FSUtils.getTableDir(rootDir, info.getTable()),
HRegion.getRegionDir(rootDir, info));
FSUtils.getRegionDirFromRootDir(rootDir, info));
}
/**

View File

@ -31,7 +31,6 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
@ -486,7 +485,7 @@ public class HFileLink extends FileLink {
String linkName = createHFileLinkName(FSUtils.getTableName(tablePath),
regionPath.getName(), hfileName);
Path linkTableDir = FSUtils.getTableDir(rootDir, linkTableName);
Path regionDir = HRegion.getRegionDir(linkTableDir, linkRegionName);
Path regionDir = new Path(linkTableDir, linkRegionName);
return new Path(new Path(regionDir, familyPath.getName()), linkName);
}

View File

@ -214,7 +214,7 @@ public class MasterFileSystem {
return this.walFsOk;
}
protected FileSystem getWALFileSystem() {
public FileSystem getWALFileSystem() {
return this.walFs;
}
@ -691,6 +691,4 @@ public class MasterFileSystem {
LOG.warn("Failed archiving meta log for server " + serverName, ie);
}
}
}

View File

@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.exceptions.HBaseException;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.MasterFileSystem;
@ -323,7 +322,7 @@ public class DeleteTableProcedure
for (HRegionInfo hri : regions) {
LOG.debug("Archiving region " + hri.getRegionNameAsString() + " from FS");
HFileArchiver.archiveRegion(fs, mfs.getRootDir(),
tempTableDir, HRegion.getRegionDir(tempTableDir, hri.getEncodedName()));
tempTableDir, new Path(tempTableDir, hri.getEncodedName()));
}
LOG.debug("Table '" + tableName + "' archived!");
}

View File

@ -319,8 +319,7 @@ public class NamespaceUpgrade implements Tool {
}
// Since meta table name has changed rename meta region dir from it's old encoding to new one
Path oldMetaRegionDir = HRegion.getRegionDir(rootDir,
new Path(newMetaDir, "1028785192").toString());
Path oldMetaRegionDir = new Path(rootDir, new Path(newMetaDir, "1028785192").toString());
if (fs.exists(oldMetaRegionDir)) {
LOG.info("Migrating meta region " + oldMetaRegionDir + " to " + newMetaRegionDir);
if (!fs.rename(oldMetaRegionDir, newMetaRegionDir)) {

View File

@ -21,8 +21,10 @@ package org.apache.hadoop.hbase.regionserver;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.google.common.io.Closeables;
import com.google.protobuf.ByteString;
import com.google.protobuf.Descriptors;
@ -4173,7 +4175,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
if (nonExistentList != null) {
for (byte[] family : nonExistentList) {
// Perhaps schema was changed between crash and replay
LOG.info("No family for " + Bytes.toString(family) + " omit from reply.");
LOG.info("No family for " + Bytes.toString(family) + " omit from replay.");
familyMap.remove(family);
}
}
@ -4286,54 +4288,58 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
minSeqIdForTheRegion = maxSeqIdInStore;
}
}
long seqid = minSeqIdForTheRegion;
long seqId = minSeqIdForTheRegion;
FileSystem walFS = getWalFileSystem();
Path regionDir = getWALRegionDir();
FileSystem rootFS = getFilesystem();
Path defaultRegionDir = getRegionDir(FSUtils.getRootDir(conf), getRegionInfo());
Path regionDir = FSUtils.getRegionDirFromRootDir(FSUtils.getRootDir(conf), getRegionInfo());
Path regionWALDir = getWALRegionDir();
Path wrongRegionWALDir = FSUtils.getWrongWALRegionDir(conf, getRegionInfo().getTable(),
getRegionInfo().getEncodedName());
// We made a mistake in HBASE-20734 so we need to do this dirty hack...
NavigableSet<Path> filesUnderWrongRegionWALDir =
WALSplitter.getSplitEditFilesSorted(walFS, wrongRegionWALDir);
seqId = Math.max(seqId, replayRecoveredEditsForPaths(minSeqIdForTheRegion, walFS,
filesUnderWrongRegionWALDir, reporter, regionDir));
// This is to ensure backwards compatability with HBASE-20723 where recovered edits can appear
// under the root dir even if walDir is set.
NavigableSet<Path> filesUnderRootDir = null;
if (!regionDir.equals(defaultRegionDir)) {
filesUnderRootDir =
WALSplitter.getSplitEditFilesSorted(rootFS, defaultRegionDir);
seqid = Math.max(seqid,
replayRecoveredEditsForPaths(minSeqIdForTheRegion, rootFS, filesUnderRootDir, reporter,
defaultRegionDir));
NavigableSet<Path> filesUnderRootDir = Sets.newTreeSet();
if (!regionWALDir.equals(regionDir)) {
filesUnderRootDir = WALSplitter.getSplitEditFilesSorted(rootFS, regionDir);
seqId = Math.max(seqId, replayRecoveredEditsForPaths(minSeqIdForTheRegion, rootFS,
filesUnderRootDir, reporter, regionDir));
}
NavigableSet<Path> files = WALSplitter.getSplitEditFilesSorted(walFS, regionDir);
seqid = Math.max(seqid, replayRecoveredEditsForPaths(minSeqIdForTheRegion, walFS,
files, reporter, regionDir));
if (seqid > minSeqIdForTheRegion) {
NavigableSet<Path> files = WALSplitter.getSplitEditFilesSorted(walFS, regionWALDir);
seqId = Math.max(seqId, replayRecoveredEditsForPaths(minSeqIdForTheRegion, walFS,
files, reporter, regionWALDir));
if (seqId > minSeqIdForTheRegion) {
// Then we added some edits to memory. Flush and cleanup split edit files.
internalFlushcache(null, seqid, stores.values(), status, false);
internalFlushcache(null, seqId, stores.values(), status, false);
}
// Now delete the content of recovered edits. We're done w/ them.
if (files.size() > 0 && this.conf.getBoolean("hbase.region.archive.recovered.edits", false)) {
// Now delete the content of recovered edits. We're done w/ them.
if (conf.getBoolean("hbase.region.archive.recovered.edits", false)) {
// For debugging data loss issues!
// If this flag is set, make use of the hfile archiving by making recovered.edits a fake
// column family. Have to fake out file type too by casting our recovered.edits as storefiles
String fakeFamilyName = WALSplitter.getRegionDirRecoveredEditsDir(regionDir).getName();
Set<StoreFile> fakeStoreFiles = new HashSet<>(files.size());
for (Path file: files) {
fakeStoreFiles.add(
new StoreFile(walFS, file, this.conf, null, null));
String fakeFamilyName = WALSplitter.getRegionDirRecoveredEditsDir(regionWALDir).getName();
Set<StoreFile> fakeStoreFiles = new HashSet<>();
for (Path file: Iterables.concat(files, filesUnderWrongRegionWALDir)) {
fakeStoreFiles.add(new StoreFile(walFS, file, conf, null, null));
}
for (Path file: filesUnderRootDir) {
fakeStoreFiles.add(new StoreFile(rootFS, file, conf, null, null));
}
getRegionWALFileSystem().removeStoreFiles(fakeFamilyName, fakeStoreFiles);
} else {
if (filesUnderRootDir != null) {
for (Path file : filesUnderRootDir) {
if (!rootFS.delete(file, false)) {
LOG.error("Failed delete of {} under root directory." + file);
} else {
LOG.debug("Deleted recovered.edits root directory file=" + file);
}
for (Path file : filesUnderRootDir) {
if (!rootFS.delete(file, false)) {
LOG.error("Failed delete of " + file + " from under the root directory");
} else {
LOG.debug("Deleted recovered.edits under root directory, file=" + file);
}
}
for (Path file: files) {
for (Path file : Iterables.concat(files, filesUnderWrongRegionWALDir)) {
if (!walFS.delete(file, false)) {
LOG.error("Failed delete of " + file);
} else {
@ -4341,7 +4347,17 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
}
}
}
return seqid;
// We have replayed all the recovered edits. Let's delete the wrong directories introduced
// in HBASE-20734, see HBASE-22617 for more details.
FileSystem walFs = getWalFileSystem();
if (walFs.exists(wrongRegionWALDir)) {
if (!walFs.delete(wrongRegionWALDir, true)) {
LOG.warn("Unable to delete " + wrongRegionWALDir);
}
}
return seqId;
}
private long replayRecoveredEditsForPaths(long minSeqIdForTheRegion, FileSystem fs,
@ -7206,34 +7222,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
meta.put(row, HConstants.CATALOG_FAMILY, cells);
}
/**
* Computes the Path of the HRegion
*
* @param tabledir qualified path for table
* @param name ENCODED region name
* @return Path of HRegion directory
* @deprecated For tests only; to be removed.
*/
@Deprecated
public static Path getRegionDir(final Path tabledir, final String name) {
return new Path(tabledir, name);
}
/**
* Computes the Path of the HRegion
*
* @param rootdir qualified path of HBase root directory
* @param info HRegionInfo for the region
* @return qualified path of region directory
* @deprecated For tests only; to be removed.
*/
@Deprecated
@VisibleForTesting
public static Path getRegionDir(final Path rootdir, final HRegionInfo info) {
return new Path(
FSUtils.getTableDir(rootdir, info.getTable()), info.getEncodedName());
}
/**
* Determines if the specified row is within the row range specified by the
* specified HRegionInfo

View File

@ -640,19 +640,26 @@ public class HRegionFileSystem {
/**
* Create the region splits directory.
*/
void createSplitsDir() throws IOException {
void createSplitsDir(HRegionInfo daughterA, HRegionInfo daughterB) throws IOException {
Path splitdir = getSplitsDir();
if (fs.exists(splitdir)) {
LOG.info("The " + splitdir + " directory exists. Hence deleting it to recreate it");
if (!deleteDir(splitdir)) {
throw new IOException("Failed deletion of " + splitdir
+ " before creating them again.");
throw new IOException("Failed deletion of " + splitdir + " before creating them again.");
}
}
// splitDir doesn't exists now. No need to do an exists() call for it.
if (!createDir(splitdir)) {
throw new IOException("Failed create of " + splitdir);
}
Path daughterATmpDir = getSplitsDir(daughterA);
if (!createDir(daughterATmpDir)) {
throw new IOException("Failed create of " + daughterATmpDir);
}
Path daughterBTmpDir = getSplitsDir(daughterB);
if (!createDir(daughterBTmpDir)) {
throw new IOException("Failed create of " + daughterBTmpDir);
}
}
/**

View File

@ -351,7 +351,7 @@ public class SplitTransactionImpl implements SplitTransaction {
hri_b, std);
}
this.parent.getRegionFileSystem().createSplitsDir();
this.parent.getRegionFileSystem().createSplitsDir(hri_a, hri_b);
transition(SplitTransactionPhase.CREATE_SPLIT_DIR);

View File

@ -648,8 +648,9 @@ public class RestoreSnapshotHelper {
if (linkPath != null) {
in = HFileLink.buildFromHFileLinkPattern(conf, linkPath).open(fs);
} else {
linkPath = new Path(new Path(HRegion.getRegionDir(snapshotManifest.getSnapshotDir(),
regionInfo.getEncodedName()), familyDir.getName()), hfileName);
linkPath = new Path(new Path(new Path(snapshotManifest.getSnapshotDir(),
regionInfo.getEncodedName()),
familyDir.getName()), hfileName);
in = fs.open(linkPath);
}
OutputStream out = fs.create(outPath);

View File

@ -84,7 +84,6 @@ import org.apache.hadoop.hbase.security.AccessDeniedException;
import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.FSProtos;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.SequenceFile;
@ -708,26 +707,36 @@ public abstract class FSUtils extends CommonFSUtils {
* @return the region directory used to store WALs under the WALRootDir
* @throws IOException if there is an exception determining the WALRootDir
*/
public static Path getWALRegionDir(final Configuration conf,
final HRegionInfo regionInfo)
throws IOException {
public static Path getWALRegionDir(final Configuration conf, final HRegionInfo regionInfo)
throws IOException {
return new Path(getWALTableDir(conf, regionInfo.getTable()),
regionInfo.getEncodedName());
}
/**
* Returns the WAL region directory based on the region info
* @param conf configuration to determine WALRootDir
* @param tableName the table name
* @param encodedRegionName the encoded region name
* @return the region directory used to store WALs under the WALRootDir
* @throws IOException if there is an exception determining the WALRootDir
*/
public static Path getWALRegionDir(final Configuration conf, final TableName tableName,
final String encodedRegionName) throws IOException {
return new Path(getWALTableDir(conf, tableName), encodedRegionName);
}
/**
* Checks if meta region exists
*
* @param fs file system
* @param rootdir root directory of HBase installation
* @param rootDir root directory of HBase installation
* @return true if exists
* @throws IOException e
*/
@SuppressWarnings("deprecation")
public static boolean metaRegionExists(FileSystem fs, Path rootdir)
throws IOException {
Path metaRegionDir =
HRegion.getRegionDir(rootdir, HRegionInfo.FIRST_META_REGIONINFO);
public static boolean metaRegionExists(FileSystem fs, Path rootDir) throws IOException {
Path metaRegionDir = getRegionDirFromRootDir(rootDir, HRegionInfo.FIRST_META_REGIONINFO);
return fs.exists(metaRegionDir);
}
@ -861,10 +870,24 @@ public abstract class FSUtils extends CommonFSUtils {
*/
public static Path getWALTableDir(final Configuration conf, final TableName tableName)
throws IOException {
return new Path(new Path(getWALRootDir(conf), tableName.getNamespaceAsString()),
Path baseDir = new Path(getWALRootDir(conf), HConstants.BASE_NAMESPACE_DIR);
return new Path(new Path(baseDir, tableName.getNamespaceAsString()),
tableName.getQualifierAsString());
}
/**
* For backward compatibility with HBASE-20734, where we store recovered edits in a wrong
* directory without BASE_NAMESPACE_DIR. See HBASE-22617 for more details.
* @deprecated For compatibility, will be removed in 4.0.0.
*/
@Deprecated
public static Path getWrongWALRegionDir(final Configuration conf, final TableName tableName,
final String encodedRegionName) throws IOException {
Path wrongTableDir = new Path(new Path(getWALRootDir(conf), tableName.getNamespaceAsString()),
tableName.getQualifierAsString());
return new Path(wrongTableDir, encodedRegionName);
}
/**
* A {@link PathFilter} that returns only regular files.
*/
@ -1063,6 +1086,14 @@ public abstract class FSUtils extends CommonFSUtils {
}
}
public static Path getRegionDirFromRootDir(Path rootDir, HRegionInfo region) {
return getRegionDirFromTableDir(getTableDir(rootDir, region.getTable()), region);
}
public static Path getRegionDirFromTableDir(Path tableDir, HRegionInfo region) {
return new Path(tableDir, ServerRegionReplicaUtil.getRegionInfoForFs(region).getEncodedName());
}
/**
* Given a particular table dir, return all the regiondirs inside it, excluding files such as
* .tableinfo

View File

@ -24,7 +24,6 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HStore;
/**
@ -78,15 +77,12 @@ public class HFileArchiveUtil {
* @return {@link Path} to the directory to archive the given region, or <tt>null</tt> if it
* should not be archived
*/
public static Path getRegionArchiveDir(Path rootDir,
TableName tableName,
Path regiondir) {
public static Path getRegionArchiveDir(Path rootDir, TableName tableName, Path regiondir) {
// get the archive directory for a table
Path archiveDir = getTableArchivePath(rootDir, tableName);
// then add on the region path under the archive
String encodedRegionName = regiondir.getName();
return HRegion.getRegionDir(archiveDir, encodedRegionName);
return new Path(archiveDir, encodedRegionName);
}
/**
@ -94,14 +90,15 @@ public class HFileArchiveUtil {
* @param rootDir {@link Path} to the root directory where hbase files are stored (for building
* the archive path)
* @param tableName name of the table to archive. Cannot be null.
* @param encodedRegionName encoded region name
* @return {@link Path} to the directory to archive the given region, or <tt>null</tt> if it
* should not be archived
*/
public static Path getRegionArchiveDir(Path rootDir,
TableName tableName, String encodedRegionName) {
public static Path getRegionArchiveDir(Path rootDir, TableName tableName,
String encodedRegionName) {
// get the archive directory for a table
Path archiveDir = getTableArchivePath(rootDir, tableName);
return HRegion.getRegionDir(archiveDir, encodedRegionName);
return new Path(archiveDir, encodedRegionName);
}
/**

View File

@ -550,9 +550,9 @@ public class WALSplitter {
static Path getRegionSplitEditsPath(final Entry logEntry, String fileNameBeingSplit,
String tmpDirName, Configuration conf) throws IOException {
FileSystem walFS = FSUtils.getWALFileSystem(conf);
Path tableDir = FSUtils.getWALTableDir(conf, logEntry.getKey().getTablename());
TableName tableName = logEntry.getKey().getTablename();
String encodedRegionName = Bytes.toString(logEntry.getKey().getEncodedRegionName());
Path regionDir = HRegion.getRegionDir(tableDir, encodedRegionName);
Path regionDir = FSUtils.getWALRegionDir(conf, tableName, encodedRegionName);
Path dir = getRegionDirRecoveredEditsDir(regionDir);
if (walFS.exists(dir) && walFS.isFile(dir)) {
@ -560,8 +560,7 @@ public class WALSplitter {
if (!walFS.exists(tmp)) {
walFS.mkdirs(tmp);
}
tmp = new Path(tmp,
HConstants.RECOVERED_EDITS_DIR + "_" + encodedRegionName);
tmp = new Path(tmp, HConstants.RECOVERED_EDITS_DIR + "_" + encodedRegionName);
LOG.warn("Found existing old file: " + dir + ". It could be some "
+ "leftover of an old installation. It should be a folder instead. "
+ "So moving it to " + tmp);

View File

@ -136,7 +136,7 @@ public class TestHFileArchiving {
// now attempt to depose the region
Path rootDir = region.getRegionFileSystem().getTableDir().getParent();
Path regionDir = HRegion.getRegionDir(rootDir, region.getRegionInfo());
Path regionDir = FSUtils.getRegionDirFromRootDir(rootDir, region.getRegionInfo());
HFileArchiver.archiveRegion(UTIL.getConfiguration(), fs, region.getRegionInfo());
@ -188,7 +188,7 @@ public class TestHFileArchiving {
// make sure there are some files in the regiondir
Path rootDir = FSUtils.getRootDir(fs.getConf());
Path regionDir = HRegion.getRegionDir(rootDir, region.getRegionInfo());
Path regionDir = FSUtils.getRegionDirFromRootDir(rootDir, region.getRegionInfo());
FileStatus[] regionFiles = FSUtils.listStatus(fs, regionDir, null);
Assert.assertNotNull("No files in the region directory", regionFiles);
if (LOG.isDebugEnabled()) {

View File

@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.io;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.junit.Assert;
@ -31,7 +30,6 @@ import org.junit.experimental.categories.Category;
import java.util.regex.Matcher;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
/**
@ -95,7 +93,7 @@ public class TestHFileLink {
for(TableName refTable : refTables) {
Path refTableDir = FSUtils.getTableDir(archiveDir, refTable);
Path refRegionDir = HRegion.getRegionDir(refTableDir, encodedRegion);
Path refRegionDir = new Path(refTableDir, encodedRegion);
Path refDir = new Path(refRegionDir, cf);
Path refLinkDir = new Path(refDir, linkDir);
String refStoreFileName = refTable.getNameAsString().replace(
@ -107,7 +105,7 @@ public class TestHFileLink {
for( TableName tableName : tableNames) {
Path tableDir = FSUtils.getTableDir(rootDir, tableName);
Path regionDir = HRegion.getRegionDir(tableDir, encodedRegion);
Path regionDir = new Path(tableDir, encodedRegion);
Path cfDir = new Path(regionDir, cf);
//Verify back reference creation

View File

@ -88,7 +88,6 @@ import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
@ -263,7 +262,7 @@ public class TestDistributedLogSplitting {
Path tdir = FSUtils.getTableDir(rootdir, table);
Path editsdir =
WALSplitter.getRegionDirRecoveredEditsDir(
HRegion.getRegionDir(tdir, hri.getEncodedName()));
FSUtils.getRegionDirFromTableDir(tdir, hri));
LOG.debug("checking edits dir " + editsdir);
FileStatus[] files = fs.listStatus(editsdir, new PathFilter() {
@Override
@ -873,7 +872,7 @@ public class TestDistributedLogSplitting {
for (HRegionInfo hri : regions) {
Path editsdir =
WALSplitter.getRegionDirRecoveredEditsDir(
HRegion.getRegionDir(tdir, hri.getEncodedName()));
FSUtils.getRegionDirFromTableDir(tdir, hri));
LOG.debug("checking edits dir " + editsdir);
if(!fs.exists(editsdir)) continue;
FileStatus[] files = fs.listStatus(editsdir, new PathFilter() {
@ -903,7 +902,7 @@ public class TestDistributedLogSplitting {
for (HRegionInfo hri : regions) {
Path editsdir =
WALSplitter.getRegionDirRecoveredEditsDir(
HRegion.getRegionDir(tdir, hri.getEncodedName()));
FSUtils.getRegionDirFromTableDir(tdir, hri));
fs.delete(editsdir, true);
}
disablingHT.close();

View File

@ -319,7 +319,7 @@ public class TestRegionMergeTransaction {
assertEquals(rowCountOfRegionB, rowCountOfRegionB2);
// Assert rollback cleaned up stuff in fs
assertTrue(!this.fs.exists(HRegion.getRegionDir(this.testdir,
assertTrue(!this.fs.exists(FSUtils.getRegionDirFromRootDir(this.testdir,
mt.getMergedRegionInfo())));
assertTrue(!this.region_a.lock.writeLock().isHeldByCurrentThread());

View File

@ -346,8 +346,10 @@ public class TestSplitTransaction {
assertEquals(parentRowCount, parentRowCount2);
// Assert rollback cleaned up stuff in fs
assertTrue(!this.fs.exists(HRegion.getRegionDir(this.testdir, st.getFirstDaughter())));
assertTrue(!this.fs.exists(HRegion.getRegionDir(this.testdir, st.getSecondDaughter())));
assertTrue(!this.fs.exists(FSUtils.getRegionDirFromRootDir(this.testdir,
st.getFirstDaughter())));
assertTrue(!this.fs.exists(FSUtils.getRegionDirFromRootDir(this.testdir,
st.getSecondDaughter())));
assertTrue(!this.parent.lock.writeLock().isHeldByCurrentThread());
// Now retry the split but do not throw an exception this time.

View File

@ -66,7 +66,6 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.wal.FaultySequenceFileLogReader;
import org.apache.hadoop.hbase.regionserver.wal.InstrumentedLogWriter;
import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
@ -1235,7 +1234,7 @@ public class TestWALSplit {
throws IOException {
Path tdir = FSUtils.getWALTableDir(conf, table);
@SuppressWarnings("deprecation")
Path editsdir = WALSplitter.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir,
Path editsdir = WALSplitter.getRegionDirRecoveredEditsDir(new Path(tdir,
Bytes.toString(region.getBytes())));
FileStatus[] files = fs.listStatus(editsdir, new PathFilter() {
@Override