HBASE-22617 Recovered WAL directories not getting cleaned up (#330)
Signed-off-by: Guanghao Zhang <zghao@apache.org> Signed-off-by: Andrew Purtell <apurtell@apache.org>
This commit is contained in:
parent
a1aab9514f
commit
a172b480fe
|
@ -28,7 +28,6 @@ import java.util.List;
|
|||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
import org.apache.hadoop.HadoopIllegalArgumentException;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
|
@ -429,11 +428,9 @@ public abstract class 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 TableName tableName, final String encodedRegionName)
|
||||
throws IOException {
|
||||
return new Path(getWALTableDir(conf, tableName),
|
||||
encodedRegionName);
|
||||
public static Path getWALRegionDir(final Configuration conf, final TableName tableName,
|
||||
final String encodedRegionName) throws IOException {
|
||||
return new Path(getWALTableDir(conf, tableName), encodedRegionName);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -445,8 +442,22 @@ public abstract class CommonFSUtils {
|
|||
*/
|
||||
public static Path getWALTableDir(final Configuration conf, final TableName tableName)
|
||||
throws IOException {
|
||||
return new Path(new Path(getWALRootDir(conf), tableName.getNamespaceAsString()),
|
||||
tableName.getQualifierAsString());
|
||||
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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1059,5 +1070,4 @@ public abstract class CommonFSUtils {
|
|||
super(message);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -38,7 +38,6 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.HStoreFile;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
|
@ -87,23 +86,21 @@ public class HFileArchiver {
|
|||
public static boolean exists(Configuration conf, FileSystem fs, RegionInfo info)
|
||||
throws IOException {
|
||||
Path rootDir = FSUtils.getRootDir(conf);
|
||||
Path regionDir = HRegion.getRegionDir(rootDir, info);
|
||||
Path regionDir = FSUtils.getRegionDirFromRootDir(rootDir, info);
|
||||
return fs.exists(regionDir);
|
||||
}
|
||||
|
||||
/**
|
||||
* Cleans up all the files for a HRegion by archiving the HFiles to the
|
||||
* archive directory
|
||||
* Cleans up all the files for a HRegion by archiving the HFiles to the archive directory
|
||||
* @param conf the configuration to use
|
||||
* @param fs the file system object
|
||||
* @param info RegionInfo for region to be deleted
|
||||
* @throws IOException
|
||||
*/
|
||||
public static void archiveRegion(Configuration conf, FileSystem fs, RegionInfo info)
|
||||
throws IOException {
|
||||
Path rootDir = FSUtils.getRootDir(conf);
|
||||
archiveRegion(fs, rootDir, FSUtils.getTableDir(rootDir, info.getTable()),
|
||||
HRegion.getRegionDir(rootDir, info));
|
||||
FSUtils.getRegionDirFromRootDir(rootDir, info));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -194,7 +194,9 @@ public class MasterFileSystem {
|
|||
return this.fs;
|
||||
}
|
||||
|
||||
protected FileSystem getWALFileSystem() { return this.walFs; }
|
||||
public FileSystem getWALFileSystem() {
|
||||
return this.walFs;
|
||||
}
|
||||
|
||||
public Configuration getConfiguration() {
|
||||
return this.conf;
|
||||
|
@ -220,7 +222,7 @@ public class MasterFileSystem {
|
|||
* @return the directory for a give {@code region}.
|
||||
*/
|
||||
public Path getRegionDir(RegionInfo region) {
|
||||
return FSUtils.getRegionDir(FSUtils.getTableDir(getRootDir(), region.getTable()), region);
|
||||
return FSUtils.getRegionDirFromRootDir(getRootDir(), region);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -18,22 +18,26 @@
|
|||
package org.apache.hadoop.hbase.master.assignment;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.backup.HFileArchiver;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.favored.FavoredNodesManager;
|
||||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineRegionProcedure;
|
||||
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionState;
|
||||
|
@ -64,46 +68,65 @@ public class GCRegionProcedure extends AbstractStateMachineRegionProcedure<GCReg
|
|||
|
||||
@Override
|
||||
protected Flow executeFromState(MasterProcedureEnv env, GCRegionState state)
|
||||
throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
|
||||
throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + " execute state=" + state);
|
||||
}
|
||||
MasterServices masterServices = env.getMasterServices();
|
||||
try {
|
||||
switch (state) {
|
||||
case GC_REGION_PREPARE:
|
||||
// Nothing to do to prepare.
|
||||
setNextState(GCRegionState.GC_REGION_ARCHIVE);
|
||||
break;
|
||||
case GC_REGION_ARCHIVE:
|
||||
FileSystem fs = masterServices.getMasterFileSystem().getFileSystem();
|
||||
if (HFileArchiver.exists(masterServices.getConfiguration(), fs, getRegion())) {
|
||||
if (LOG.isDebugEnabled()) LOG.debug("Archiving region=" + getRegion().getShortNameToLog());
|
||||
HFileArchiver.archiveRegion(masterServices.getConfiguration(), fs, getRegion());
|
||||
}
|
||||
setNextState(GCRegionState.GC_REGION_PURGE_METADATA);
|
||||
break;
|
||||
case GC_REGION_PURGE_METADATA:
|
||||
// TODO: Purge metadata before removing from HDFS? This ordering is copied
|
||||
// from CatalogJanitor.
|
||||
AssignmentManager am = masterServices.getAssignmentManager();
|
||||
if (am != null) {
|
||||
if (am.getRegionStates() != null) {
|
||||
am.getRegionStates().deleteRegion(getRegion());
|
||||
case GC_REGION_PREPARE:
|
||||
// Nothing to do to prepare.
|
||||
setNextState(GCRegionState.GC_REGION_ARCHIVE);
|
||||
break;
|
||||
case GC_REGION_ARCHIVE:
|
||||
MasterFileSystem mfs = masterServices.getMasterFileSystem();
|
||||
FileSystem fs = mfs.getFileSystem();
|
||||
if (HFileArchiver.exists(masterServices.getConfiguration(), fs, getRegion())) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Archiving region=" + getRegion().getShortNameToLog());
|
||||
}
|
||||
HFileArchiver.archiveRegion(masterServices.getConfiguration(), fs, getRegion());
|
||||
}
|
||||
}
|
||||
MetaTableAccessor.deleteRegion(masterServices.getConnection(), getRegion());
|
||||
masterServices.getServerManager().removeRegion(getRegion());
|
||||
FavoredNodesManager fnm = masterServices.getFavoredNodesManager();
|
||||
if (fnm != null) {
|
||||
fnm.deleteFavoredNodesForRegions(Lists.newArrayList(getRegion()));
|
||||
}
|
||||
return Flow.NO_MORE_STATE;
|
||||
default:
|
||||
throw new UnsupportedOperationException(this + " unhandled state=" + state);
|
||||
FileSystem walFs = mfs.getWALFileSystem();
|
||||
// Cleanup the directories on WAL filesystem also
|
||||
Path regionWALDir = FSUtils.getWALRegionDir(env.getMasterConfiguration(),
|
||||
getRegion().getTable(), getRegion().getEncodedName());
|
||||
if (walFs.exists(regionWALDir)) {
|
||||
if (!walFs.delete(regionWALDir, true)) {
|
||||
LOG.debug("Failed to delete {}", regionWALDir);
|
||||
}
|
||||
}
|
||||
Path wrongRegionWALDir = FSUtils.getWrongWALRegionDir(env.getMasterConfiguration(),
|
||||
getRegion().getTable(), getRegion().getEncodedName());
|
||||
if (walFs.exists(wrongRegionWALDir)) {
|
||||
if (!walFs.delete(wrongRegionWALDir, true)) {
|
||||
LOG.debug("Failed to delete {}", regionWALDir);
|
||||
}
|
||||
}
|
||||
setNextState(GCRegionState.GC_REGION_PURGE_METADATA);
|
||||
break;
|
||||
case GC_REGION_PURGE_METADATA:
|
||||
// TODO: Purge metadata before removing from HDFS? This ordering is copied
|
||||
// from CatalogJanitor.
|
||||
AssignmentManager am = masterServices.getAssignmentManager();
|
||||
if (am != null) {
|
||||
if (am.getRegionStates() != null) {
|
||||
am.getRegionStates().deleteRegion(getRegion());
|
||||
}
|
||||
}
|
||||
MetaTableAccessor.deleteRegion(masterServices.getConnection(), getRegion());
|
||||
masterServices.getServerManager().removeRegion(getRegion());
|
||||
FavoredNodesManager fnm = masterServices.getFavoredNodesManager();
|
||||
if (fnm != null) {
|
||||
fnm.deleteFavoredNodesForRegions(Lists.newArrayList(getRegion()));
|
||||
}
|
||||
return Flow.NO_MORE_STATE;
|
||||
default:
|
||||
throw new UnsupportedOperationException(this + " unhandled state=" + state);
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
// TODO: This is going to spew log?
|
||||
// TODO: This is going to spew log? Add retry backoff
|
||||
LOG.warn("Error trying to GC " + getRegion().getShortNameToLog() + "; retrying...", ioe);
|
||||
}
|
||||
return Flow.HAS_MORE_STATE;
|
||||
|
|
|
@ -843,16 +843,16 @@ public class MergeTableRegionsProcedure
|
|||
}
|
||||
|
||||
private void writeMaxSequenceIdFile(MasterProcedureEnv env) throws IOException {
|
||||
FileSystem walFS = env.getMasterServices().getMasterWalManager().getFileSystem();
|
||||
MasterFileSystem fs = env.getMasterFileSystem();
|
||||
long maxSequenceId = -1L;
|
||||
for (RegionInfo region : regionsToMerge) {
|
||||
maxSequenceId =
|
||||
Math.max(maxSequenceId, WALSplitter.getMaxRegionSequenceId(
|
||||
walFS, getWALRegionDir(env, region)));
|
||||
Math.max(maxSequenceId, WALSplitter.getMaxRegionSequenceId(env.getMasterConfiguration(),
|
||||
region, fs::getFileSystem, fs::getWALFileSystem));
|
||||
}
|
||||
if (maxSequenceId > 0) {
|
||||
WALSplitter.writeRegionSequenceIdFile(walFS, getWALRegionDir(env, mergedRegion),
|
||||
maxSequenceId);
|
||||
WALSplitter.writeRegionSequenceIdFile(fs.getWALFileSystem(),
|
||||
getWALRegionDir(env, mergedRegion), maxSequenceId);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.master.assignment;
|
|||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellBuilderFactory;
|
||||
import org.apache.hadoop.hbase.CellBuilderType;
|
||||
|
@ -35,13 +34,13 @@ import org.apache.hadoop.hbase.client.RegionInfo;
|
|||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
import org.apache.hadoop.hbase.master.RegionState.State;
|
||||
import org.apache.hadoop.hbase.procedure2.Procedure;
|
||||
import org.apache.hadoop.hbase.procedure2.util.StringUtils;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.wal.WALSplitter;
|
||||
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
@ -217,10 +216,9 @@ public class RegionStateStore {
|
|||
}
|
||||
|
||||
private long getOpenSeqNumForParentRegion(RegionInfo region) throws IOException {
|
||||
FileSystem walFS = master.getMasterWalManager().getFileSystem();
|
||||
long maxSeqId =
|
||||
WALSplitter.getMaxRegionSequenceId(walFS, FSUtils.getWALRegionDir(
|
||||
master.getConfiguration(), region.getTable(), region.getEncodedName()));
|
||||
MasterFileSystem fs = master.getMasterFileSystem();
|
||||
long maxSeqId = WALSplitter.getMaxRegionSequenceId(master.getConfiguration(), region,
|
||||
fs::getFileSystem, fs::getWALFileSystem);
|
||||
return maxSeqId > 0 ? maxSeqId + 1 : HConstants.NO_SEQNUM;
|
||||
}
|
||||
|
||||
|
|
|
@ -34,7 +34,6 @@ import java.util.concurrent.Executors;
|
|||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -596,7 +595,7 @@ public class SplitTableRegionProcedure
|
|||
final FileSystem fs = mfs.getFileSystem();
|
||||
HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(
|
||||
env.getMasterConfiguration(), fs, tabledir, getParentRegion(), false);
|
||||
regionFs.createSplitsDir();
|
||||
regionFs.createSplitsDir(daughter_1_RI, daughter_2_RI);
|
||||
|
||||
Pair<Integer, Integer> expectedReferences = splitStoreFiles(env, regionFs);
|
||||
|
||||
|
@ -903,14 +902,14 @@ public class SplitTableRegionProcedure
|
|||
}
|
||||
|
||||
private void writeMaxSequenceIdFile(MasterProcedureEnv env) throws IOException {
|
||||
FileSystem walFS = env.getMasterServices().getMasterWalManager().getFileSystem();
|
||||
long maxSequenceId =
|
||||
WALSplitter.getMaxRegionSequenceId(walFS, getWALRegionDir(env, getParentRegion()));
|
||||
MasterFileSystem fs = env.getMasterFileSystem();
|
||||
long maxSequenceId = WALSplitter.getMaxRegionSequenceId(env.getMasterConfiguration(),
|
||||
getParentRegion(), fs::getFileSystem, fs::getWALFileSystem);
|
||||
if (maxSequenceId > 0) {
|
||||
WALSplitter.writeRegionSequenceIdFile(walFS, getWALRegionDir(env, daughter_1_RI),
|
||||
maxSequenceId);
|
||||
WALSplitter.writeRegionSequenceIdFile(walFS, getWALRegionDir(env, daughter_2_RI),
|
||||
maxSequenceId);
|
||||
WALSplitter.writeRegionSequenceIdFile(fs.getWALFileSystem(),
|
||||
getWALRegionDir(env, daughter_1_RI), maxSequenceId);
|
||||
WALSplitter.writeRegionSequenceIdFile(fs.getWALFileSystem(),
|
||||
getWALRegionDir(env, daughter_2_RI), maxSequenceId);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -314,12 +314,11 @@ public class DeleteTableProcedure
|
|||
|
||||
// Archive regions from FS (temp directory)
|
||||
if (archive) {
|
||||
List<Path> regionDirList = regions.stream()
|
||||
.filter(RegionReplicaUtil::isDefaultReplica)
|
||||
.map(region -> FSUtils.getRegionDir(tempTableDir, region))
|
||||
List<Path> regionDirList = regions.stream().filter(RegionReplicaUtil::isDefaultReplica)
|
||||
.map(region -> FSUtils.getRegionDirFromTableDir(tempTableDir, region))
|
||||
.collect(Collectors.toList());
|
||||
HFileArchiver.archiveRegions(env.getMasterConfiguration(), fs, mfs.getRootDir(),
|
||||
tempTableDir, regionDirList);
|
||||
HFileArchiver.archiveRegions(env.getMasterConfiguration(), fs, mfs.getRootDir(), tempTableDir,
|
||||
regionDirList);
|
||||
LOG.debug("Table '{}' archived!", tableName);
|
||||
}
|
||||
|
||||
|
@ -343,6 +342,13 @@ public class DeleteTableProcedure
|
|||
throw new IOException("Couldn't delete mob dir " + mobTableDir);
|
||||
}
|
||||
}
|
||||
|
||||
// Delete the directory on wal filesystem
|
||||
FileSystem walFs = mfs.getWALFileSystem();
|
||||
Path tableWALDir = FSUtils.getWALTableDir(env.getMasterConfiguration(), tableName);
|
||||
if (walFs.exists(tableWALDir) && !walFs.delete(tableWALDir, true)) {
|
||||
throw new IOException("Couldn't delete table dir on wal filesystem" + tableWALDir);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
package org.apache.hadoop.hbase.master.procedure;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hbase.HBaseIOException;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
|
@ -31,6 +30,7 @@ import org.apache.hadoop.hbase.client.RegionInfo;
|
|||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.constraint.ConstraintException;
|
||||
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||
import org.apache.hadoop.hbase.master.TableStateManager;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
|
@ -113,13 +113,13 @@ public class DisableTableProcedure
|
|||
case DISABLE_TABLE_ADD_REPLICATION_BARRIER:
|
||||
if (env.getMasterServices().getTableDescriptors().get(tableName)
|
||||
.hasGlobalReplicationScope()) {
|
||||
FileSystem walFS = env.getMasterServices().getMasterWalManager().getFileSystem();
|
||||
MasterFileSystem fs = env.getMasterFileSystem();
|
||||
try (BufferedMutator mutator = env.getMasterServices().getConnection()
|
||||
.getBufferedMutator(TableName.META_TABLE_NAME)) {
|
||||
for (RegionInfo region : env.getAssignmentManager().getRegionStates()
|
||||
.getRegionsOfTable(tableName)) {
|
||||
long maxSequenceId =
|
||||
WALSplitter.getMaxRegionSequenceId(walFS, getWALRegionDir(env, region));
|
||||
long maxSequenceId = WALSplitter.getMaxRegionSequenceId(
|
||||
env.getMasterConfiguration(), region, fs::getFileSystem, fs::getWALFileSystem);
|
||||
long openSeqNum = maxSequenceId > 0 ? maxSequenceId + 1 : HConstants.NO_SEQNUM;
|
||||
mutator.mutate(MetaTableAccessor.makePutForReplicationBarrier(region, openSeqNum,
|
||||
EnvironmentEdgeManager.currentTime()));
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.conf.ConfigurationObserver;
|
||||
import org.apache.hadoop.hbase.ipc.RpcServer;
|
||||
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
|
||||
import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
|
||||
|
@ -119,6 +120,10 @@ public class MasterProcedureEnv implements ConfigurationObserver {
|
|||
return master.getReplicationPeerManager();
|
||||
}
|
||||
|
||||
public MasterFileSystem getMasterFileSystem() {
|
||||
return master.getMasterFileSystem();
|
||||
}
|
||||
|
||||
public boolean isRunning() {
|
||||
if (this.master == null || this.master.getMasterProcedureExecutor() == null) return false;
|
||||
return master.getMasterProcedureExecutor().isRunning();
|
||||
|
|
|
@ -183,6 +183,7 @@ import org.slf4j.LoggerFactory;
|
|||
|
||||
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
|
||||
import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
|
||||
|
@ -990,15 +991,28 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
|
||||
// Use maximum of log sequenceid or that which was found in stores
|
||||
// (particularly if no recovered edits, seqid will be -1).
|
||||
long maxSeqIdFromFile =
|
||||
WALSplitter.getMaxRegionSequenceId(getWalFileSystem(), getWALRegionDirOfDefaultReplica());
|
||||
// always get openSeqNum from the default replica, even if we are secondary replicas
|
||||
long maxSeqIdFromFile = WALSplitter.getMaxRegionSequenceId(conf,
|
||||
RegionReplicaUtil.getRegionInfoForDefaultReplica(getRegionInfo()), this::getFilesystem,
|
||||
this::getWalFileSystem);
|
||||
long nextSeqId = Math.max(maxSeqId, maxSeqIdFromFile) + 1;
|
||||
// The openSeqNum will always be increase even for read only region, as we rely on it to
|
||||
// determine whether a region has been successfully reopend, so here we always need to update
|
||||
// determine whether a region has been successfully reopened, so here we always need to update
|
||||
// the max sequence id file.
|
||||
if (RegionReplicaUtil.isDefaultReplica(getRegionInfo())) {
|
||||
LOG.debug("writing seq id for {}", this.getRegionInfo().getEncodedName());
|
||||
WALSplitter.writeRegionSequenceIdFile(getWalFileSystem(), getWALRegionDir(), nextSeqId - 1);
|
||||
// This means we have replayed all the recovered edits and also written out the max sequence
|
||||
// id file, let's delete the wrong directories introduced in HBASE-20734, see HBASE-22617
|
||||
// for more details.
|
||||
Path wrongRegionWALDir = FSUtils.getWrongWALRegionDir(conf, getRegionInfo().getTable(),
|
||||
getRegionInfo().getEncodedName());
|
||||
FileSystem walFs = getWalFileSystem();
|
||||
if (walFs.exists(wrongRegionWALDir)) {
|
||||
if (!walFs.delete(wrongRegionWALDir, true)) {
|
||||
LOG.debug("Failed to clean up wrong region WAL directory {}", wrongRegionWALDir);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
LOG.info("Opened {}; next sequenceid={}", this.getRegionInfo().getShortNameToLog(), nextSeqId);
|
||||
|
@ -1899,19 +1913,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
return regionDir;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the Region directory under WALRootDirectory; in case of secondary replica return the
|
||||
* region directory corresponding to its default replica
|
||||
* @throws IOException if there is an error getting WALRootDir
|
||||
*/
|
||||
private Path getWALRegionDirOfDefaultReplica() throws IOException {
|
||||
RegionInfo regionInfo = getRegionInfo();
|
||||
if (!RegionReplicaUtil.isDefaultReplica(regionInfo)) {
|
||||
regionInfo = RegionReplicaUtil.getRegionInfoForDefaultReplica(regionInfo);
|
||||
}
|
||||
return FSUtils.getWALRegionDir(conf, regionInfo.getTable(), regionInfo.getEncodedName());
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getEarliestFlushTimeForAllStores() {
|
||||
return Collections.min(lastStoreFlushTimeMap.values());
|
||||
|
@ -4524,63 +4525,65 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
minSeqIdForTheRegion = maxSeqIdInStore;
|
||||
}
|
||||
}
|
||||
long seqid = minSeqIdForTheRegion;
|
||||
long seqId = minSeqIdForTheRegion;
|
||||
|
||||
FileSystem walFS = getWalFileSystem();
|
||||
FileSystem rootFS = getFilesystem();
|
||||
Path regionDir = getWALRegionDir();
|
||||
Path defaultRegionDir = getRegionDir(FSUtils.getRootDir(conf), getRegionInfo());
|
||||
Path wrongRegionWALDir = FSUtils.getWrongWALRegionDir(conf, getRegionInfo().getTable(),
|
||||
getRegionInfo().getEncodedName());
|
||||
Path regionWALDir = getWALRegionDir();
|
||||
Path regionDir = FSUtils.getRegionDirFromRootDir(FSUtils.getRootDir(conf), getRegionInfo());
|
||||
|
||||
// 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 = Collections.emptyNavigableSet();
|
||||
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));
|
||||
NavigableSet<Path> files = WALSplitter.getSplitEditFilesSorted(walFS, regionWALDir);
|
||||
seqId = Math.max(seqId, replayRecoveredEditsForPaths(minSeqIdForTheRegion, walFS,
|
||||
files, reporter, regionWALDir));
|
||||
|
||||
if (seqid > minSeqIdForTheRegion) {
|
||||
if (seqId > minSeqIdForTheRegion) {
|
||||
// Then we added some edits to memory. Flush and cleanup split edit files.
|
||||
internalFlushcache(null, seqid, stores.values(), status, false, FlushLifeCycleTracker.DUMMY);
|
||||
internalFlushcache(null, seqId, stores.values(), status, false, FlushLifeCycleTracker.DUMMY);
|
||||
}
|
||||
// Now delete the content of recovered edits. We're done w/ them.
|
||||
// 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)) {
|
||||
// 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();
|
||||
String fakeFamilyName = WALSplitter.getRegionDirRecoveredEditsDir(regionWALDir).getName();
|
||||
Set<HStoreFile> fakeStoreFiles = new HashSet<>(files.size());
|
||||
for (Path file: files) {
|
||||
fakeStoreFiles.add(
|
||||
new HStoreFile(walFS, file, this.conf, null, null, true));
|
||||
for (Path file : files) {
|
||||
fakeStoreFiles.add(new HStoreFile(walFS, file, this.conf, null, null, true));
|
||||
}
|
||||
getRegionWALFileSystem().removeStoreFiles(fakeFamilyName, fakeStoreFiles);
|
||||
} else {
|
||||
if (filesUnderRootDir != null) {
|
||||
for (Path file : filesUnderRootDir) {
|
||||
if (!rootFS.delete(file, false)) {
|
||||
LOG.error("Failed delete of {} from under the root directory.", file);
|
||||
} else {
|
||||
LOG.debug("Deleted recovered.edits under root directory. file=" + file);
|
||||
}
|
||||
for (Path file : Iterables.concat(files, filesUnderWrongRegionWALDir)) {
|
||||
if (!walFS.delete(file, false)) {
|
||||
LOG.error("Failed delete of {}", file);
|
||||
} else {
|
||||
LOG.debug("Deleted recovered.edits file={}", file);
|
||||
}
|
||||
}
|
||||
for (Path file: files) {
|
||||
if (!walFS.delete(file, false)) {
|
||||
LOG.error("Failed delete of " + file);
|
||||
for (Path file : filesUnderRootDir) {
|
||||
if (!rootFS.delete(file, false)) {
|
||||
LOG.error("Failed delete of {}", file);
|
||||
} else {
|
||||
LOG.debug("Deleted recovered.edits file=" + file);
|
||||
LOG.debug("Deleted recovered.edits file={}", file);
|
||||
}
|
||||
}
|
||||
}
|
||||
return seqid;
|
||||
return seqId;
|
||||
}
|
||||
|
||||
private long replayRecoveredEditsForPaths(long minSeqIdForTheRegion, FileSystem fs,
|
||||
|
@ -7333,21 +7336,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
return new Path(tabledir, name);
|
||||
}
|
||||
|
||||
/**
|
||||
* Computes the Path of the HRegion
|
||||
*
|
||||
* @param rootdir qualified path of HBase root directory
|
||||
* @param info RegionInfo 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 RegionInfo 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 RegionInfo
|
||||
|
|
|
@ -108,7 +108,7 @@ public class HRegionFileSystem {
|
|||
this.tableDir = Objects.requireNonNull(tableDir, "tableDir is null");
|
||||
this.regionInfo = Objects.requireNonNull(regionInfo, "regionInfo is null");
|
||||
this.regionInfoForFs = ServerRegionReplicaUtil.getRegionInfoForFs(regionInfo);
|
||||
this.regionDir = FSUtils.getRegionDir(tableDir, regionInfo);
|
||||
this.regionDir = FSUtils.getRegionDirFromTableDir(tableDir, regionInfo);
|
||||
this.hdfsClientRetriesNumber = conf.getInt("hdfs.client.retries.number",
|
||||
DEFAULT_HDFS_CLIENT_RETRIES_NUMBER);
|
||||
this.baseSleepBeforeRetries = conf.getInt("hdfs.client.sleep.before.retries",
|
||||
|
@ -630,19 +630,26 @@ public class HRegionFileSystem {
|
|||
/**
|
||||
* Create the region splits directory.
|
||||
*/
|
||||
public void createSplitsDir() throws IOException {
|
||||
public void createSplitsDir(RegionInfo daughterA, RegionInfo 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);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -67,11 +67,11 @@ import org.apache.hadoop.hbase.HDFSBlocksDistribution;
|
|||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.fs.HFileSystem;
|
||||
import org.apache.hadoop.hbase.io.HFileLink;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
|
||||
import org.apache.hadoop.hbase.security.AccessDeniedException;
|
||||
import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter;
|
||||
|
@ -708,17 +708,12 @@ public abstract class FSUtils extends CommonFSUtils {
|
|||
|
||||
/**
|
||||
* 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, RegionInfoBuilder.FIRST_META_REGIONINFO);
|
||||
return fs.exists(metaRegionDir);
|
||||
}
|
||||
|
||||
|
@ -1030,7 +1025,11 @@ public abstract class FSUtils extends CommonFSUtils {
|
|||
return regionDirs;
|
||||
}
|
||||
|
||||
public static Path getRegionDir(Path tableDir, RegionInfo region) {
|
||||
public static Path getRegionDirFromRootDir(Path rootDir, RegionInfo region) {
|
||||
return getRegionDirFromTableDir(getTableDir(rootDir, region.getTable()), region);
|
||||
}
|
||||
|
||||
public static Path getRegionDirFromTableDir(Path tableDir, RegionInfo region) {
|
||||
return new Path(tableDir, ServerRegionReplicaUtil.getRegionInfoForFs(region).getEncodedName());
|
||||
}
|
||||
|
||||
|
|
|
@ -77,6 +77,7 @@ import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
|
|||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.CancelableProgressable;
|
||||
import org.apache.hadoop.hbase.util.ClassSize;
|
||||
import org.apache.hadoop.hbase.util.CollectionUtils.IOExceptionSupplier;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
|
@ -549,12 +550,25 @@ public class WALSplitter {
|
|||
if (regionInfo.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
|
||||
return false;
|
||||
}
|
||||
//Only default replica region can reach here, so we can use regioninfo
|
||||
//directly without converting it to default replica's regioninfo.
|
||||
Path regionDir = FSUtils.getWALRegionDir(conf, regionInfo.getTable(),
|
||||
regionInfo.getEncodedName());
|
||||
NavigableSet<Path> files = getSplitEditFilesSorted(FSUtils.getWALFileSystem(conf), regionDir);
|
||||
return files != null && !files.isEmpty();
|
||||
// Only default replica region can reach here, so we can use regioninfo
|
||||
// directly without converting it to default replica's regioninfo.
|
||||
Path regionWALDir =
|
||||
FSUtils.getWALRegionDir(conf, regionInfo.getTable(), regionInfo.getEncodedName());
|
||||
Path regionDir = FSUtils.getRegionDirFromRootDir(FSUtils.getRootDir(conf), regionInfo);
|
||||
Path wrongRegionWALDir =
|
||||
FSUtils.getWrongWALRegionDir(conf, regionInfo.getTable(), regionInfo.getEncodedName());
|
||||
FileSystem walFs = FSUtils.getWALFileSystem(conf);
|
||||
FileSystem rootFs = FSUtils.getRootDirFileSystem(conf);
|
||||
NavigableSet<Path> files = getSplitEditFilesSorted(walFs, regionWALDir);
|
||||
if (!files.isEmpty()) {
|
||||
return true;
|
||||
}
|
||||
files = getSplitEditFilesSorted(rootFs, regionDir);
|
||||
if (!files.isEmpty()) {
|
||||
return true;
|
||||
}
|
||||
files = getSplitEditFilesSorted(walFs, wrongRegionWALDir);
|
||||
return !files.isEmpty();
|
||||
}
|
||||
|
||||
|
||||
|
@ -705,6 +719,33 @@ public class WALSplitter {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This method will check 3 places for finding the max sequence id file. One is the expected
|
||||
* place, another is the old place under the region directory, and the last one is the wrong one
|
||||
* we introduced in HBASE-20734. See HBASE-22617 for more details.
|
||||
* <p/>
|
||||
* Notice that, you should always call this method instead of
|
||||
* {@link #getMaxRegionSequenceId(FileSystem, Path)} until 4.0.0 release.
|
||||
* @deprecated Only for compatibility, will be removed in 4.0.0.
|
||||
*/
|
||||
@Deprecated
|
||||
public static long getMaxRegionSequenceId(Configuration conf, RegionInfo region,
|
||||
IOExceptionSupplier<FileSystem> rootFsSupplier, IOExceptionSupplier<FileSystem> walFsSupplier)
|
||||
throws IOException {
|
||||
FileSystem rootFs = rootFsSupplier.get();
|
||||
FileSystem walFs = walFsSupplier.get();
|
||||
Path regionWALDir = FSUtils.getWALRegionDir(conf, region.getTable(), region.getEncodedName());
|
||||
// This is the old place where we store max sequence id file
|
||||
Path regionDir = FSUtils.getRegionDirFromRootDir(FSUtils.getRootDir(conf), region);
|
||||
// This is for HBASE-20734, where we use a wrong directory, see HBASE-22617 for more details.
|
||||
Path wrongRegionWALDir =
|
||||
FSUtils.getWrongWALRegionDir(conf, region.getTable(), region.getEncodedName());
|
||||
long maxSeqId = getMaxRegionSequenceId(walFs, regionWALDir);
|
||||
maxSeqId = Math.max(maxSeqId, getMaxRegionSequenceId(rootFs, regionDir));
|
||||
maxSeqId = Math.max(maxSeqId, getMaxRegionSequenceId(walFs, wrongRegionWALDir));
|
||||
return maxSeqId;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new {@link Reader} for reading logs to split.
|
||||
*
|
||||
|
|
|
@ -150,7 +150,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());
|
||||
|
||||
|
@ -200,7 +200,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()) {
|
||||
|
@ -267,7 +267,7 @@ public class TestHFileArchiving {
|
|||
Path rootDir = FSUtils.getRootDir(UTIL.getConfiguration());
|
||||
Path tableDir = FSUtils.getTableDir(rootDir, regions.get(0).getRegionInfo().getTable());
|
||||
List<Path> regionDirList = regions.stream()
|
||||
.map(region -> FSUtils.getRegionDir(tableDir, region.getRegionInfo()))
|
||||
.map(region -> FSUtils.getRegionDirFromTableDir(tableDir, region.getRegionInfo()))
|
||||
.collect(Collectors.toList());
|
||||
|
||||
HFileArchiver.archiveRegions(UTIL.getConfiguration(), fs, rootDir, tableDir, regionDirList);
|
||||
|
@ -305,7 +305,7 @@ public class TestHFileArchiving {
|
|||
Path rootDir = FSUtils.getRootDir(UTIL.getConfiguration());
|
||||
Path tableDir = FSUtils.getTableDir(rootDir, regions.get(0).getRegionInfo().getTable());
|
||||
List<Path> regionDirList = regions.stream()
|
||||
.map(region -> FSUtils.getRegionDir(tableDir, region.getRegionInfo()))
|
||||
.map(region -> FSUtils.getRegionDirFromTableDir(tableDir, region.getRegionInfo()))
|
||||
.collect(Collectors.toList());
|
||||
|
||||
// To create a permission denied error, we do archive regions as a non-current user
|
||||
|
|
|
@ -224,7 +224,7 @@ public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
|
|||
MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
|
||||
Path tempdir = mfs.getTempDir();
|
||||
Path tableDir = FSUtils.getTableDir(tempdir, regionInfo.getTable());
|
||||
Path regionDir = FSUtils.getRegionDir(tableDir, regionInfo);
|
||||
Path regionDir = FSUtils.getRegionDirFromTableDir(tableDir, regionInfo);
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
fs.mkdirs(regionDir);
|
||||
|
||||
|
|
|
@ -274,7 +274,7 @@ public class TestTruncateTableProcedure extends TestTableDDLProcedureBase {
|
|||
MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
|
||||
Path tempdir = mfs.getTempDir();
|
||||
Path tableDir = FSUtils.getTableDir(tempdir, regionInfo.getTable());
|
||||
Path regionDir = FSUtils.getRegionDir(tableDir, regionInfo);
|
||||
Path regionDir = FSUtils.getRegionDirFromTableDir(tableDir, regionInfo);
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
fs.mkdirs(regionDir);
|
||||
|
||||
|
|
|
@ -132,7 +132,7 @@ public class TestRecoveredEdits {
|
|||
// There should be no store files.
|
||||
assertTrue(storeFiles.isEmpty());
|
||||
region.close();
|
||||
Path regionDir = region.getRegionDir(hbaseRootDir, hri);
|
||||
Path regionDir = FSUtils.getRegionDirFromRootDir(hbaseRootDir, hri);
|
||||
Path recoveredEditsDir = WALSplitter.getRegionDirRecoveredEditsDir(regionDir);
|
||||
// This is a little fragile getting this path to a file of 10M of edits.
|
||||
Path recoveredEditsFile = new Path(
|
||||
|
|
Loading…
Reference in New Issue