HBASE-24249 Move code in FSHDFSUtils to FSUtils and mark related clas… (#1586)

Signed-off-by: stack <stack@apache.org>
This commit is contained in:
Duo Zhang 2020-04-29 10:44:34 +08:00 committed by GitHub
parent 6928674eb8
commit 9f52e6b725
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
219 changed files with 1414 additions and 1525 deletions

View File

@ -41,7 +41,6 @@ import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_YARN_
import java.io.IOException;
import java.net.URI;
import java.util.Objects;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
@ -49,7 +48,7 @@ import org.apache.hadoop.hbase.backup.BackupRestoreConstants.BackupCommand;
import org.apache.hadoop.hbase.backup.impl.BackupCommands;
import org.apache.hadoop.hbase.backup.impl.BackupManager;
import org.apache.hadoop.hbase.util.AbstractHBaseTool;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.util.ToolRunner;
import org.apache.log4j.Level;
import org.apache.log4j.LogManager;
@ -179,9 +178,9 @@ public class BackupDriver extends AbstractHBaseTool {
public static void main(String[] args) throws Exception {
Configuration conf = HBaseConfiguration.create();
Path hbasedir = FSUtils.getRootDir(conf);
Path hbasedir = CommonFSUtils.getRootDir(conf);
URI defaultFs = hbasedir.getFileSystem(conf).getUri();
FSUtils.setFsDefault(conf, new Path(defaultFs));
CommonFSUtils.setFsDefault(conf, new Path(defaultFs));
int ret = ToolRunner.run(conf, new BackupDriver(), args);
System.exit(ret);
}

View File

@ -36,7 +36,6 @@ import java.io.IOException;
import java.net.URI;
import java.util.List;
import java.util.Objects;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
@ -49,7 +48,7 @@ import org.apache.hadoop.hbase.backup.util.BackupUtils;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.util.AbstractHBaseTool;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.util.ToolRunner;
import org.apache.log4j.Level;
import org.apache.log4j.LogManager;
@ -224,9 +223,9 @@ public class RestoreDriver extends AbstractHBaseTool {
public static void main(String[] args) throws Exception {
Configuration conf = HBaseConfiguration.create();
Path hbasedir = FSUtils.getRootDir(conf);
Path hbasedir = CommonFSUtils.getRootDir(conf);
URI defaultFs = hbasedir.getFileSystem(conf).getUri();
FSUtils.setFsDefault(conf, new Path(defaultFs));
CommonFSUtils.setFsDefault(conf, new Path(defaultFs));
int ret = ToolRunner.run(conf, new RestoreDriver(), args);
System.exit(ret);
}

View File

@ -29,7 +29,6 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -45,7 +44,7 @@ import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.mapreduce.WALPlayer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
@ -127,7 +126,7 @@ public class IncrementalTableBackupClient extends TableBackupClient {
} catch (URISyntaxException use) {
throw new IOException("Unable to get FileSystem", use);
}
Path rootdir = FSUtils.getRootDir(conf);
Path rootdir = CommonFSUtils.getRootDir(conf);
Path tgtRoot = new Path(new Path(backupInfo.getBackupRootDir()), backupId);
for (Map.Entry<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>> tblEntry :
@ -142,7 +141,7 @@ public class IncrementalTableBackupClient extends TableBackupClient {
if (mapForSrc[srcIdx] == null) {
mapForSrc[srcIdx] = new TreeMap<>(Bytes.BYTES_COMPARATOR);
}
Path tblDir = FSUtils.getTableDir(rootdir, srcTable);
Path tblDir = CommonFSUtils.getTableDir(rootdir, srcTable);
Path tgtTable = new Path(new Path(tgtRoot, srcTable.getNamespaceAsString()),
srcTable.getQualifierAsString());
for (Map.Entry<String,Map<String,List<Pair<String, Boolean>>>> regionEntry :

View File

@ -21,7 +21,6 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@ -37,8 +36,8 @@ import org.apache.hadoop.hbase.backup.HBackupFileSystem;
import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -90,7 +89,7 @@ public abstract class TableBackupClient {
this.tableList = request.getTableList();
this.conn = conn;
this.conf = conn.getConfiguration();
this.fs = FSUtils.getCurrentFileSystem(conf);
this.fs = CommonFSUtils.getCurrentFileSystem(conf);
backupInfo =
backupManager.createBackupInfo(backupId, request.getBackupType(), tableList,
request.getTargetRootDir(), request.getTotalTasks(), request.getBandwidth());
@ -161,18 +160,18 @@ public abstract class TableBackupClient {
* @throws IOException exception
*/
protected static void cleanupExportSnapshotLog(Configuration conf) throws IOException {
FileSystem fs = FSUtils.getCurrentFileSystem(conf);
FileSystem fs = CommonFSUtils.getCurrentFileSystem(conf);
Path stagingDir =
new Path(conf.get(BackupRestoreConstants.CONF_STAGING_ROOT, fs.getWorkingDirectory()
.toString()));
FileStatus[] files = FSUtils.listStatus(fs, stagingDir);
FileStatus[] files = CommonFSUtils.listStatus(fs, stagingDir);
if (files == null) {
return;
}
for (FileStatus file : files) {
if (file.getPath().getName().startsWith("exportSnapshot-")) {
LOG.debug("Delete log files of exporting snapshot: " + file.getPath().getName());
if (FSUtils.delete(fs, file.getPath(), true) == false) {
if (CommonFSUtils.delete(fs, file.getPath(), true) == false) {
LOG.warn("Can not delete " + file.getPath());
}
}
@ -209,7 +208,7 @@ public abstract class TableBackupClient {
}
Path tableDir = targetDirPath.getParent();
FileStatus[] backups = FSUtils.listStatus(outputFs, tableDir);
FileStatus[] backups = CommonFSUtils.listStatus(outputFs, tableDir);
if (backups == null || backups.length == 0) {
outputFs.delete(tableDir, true);
LOG.debug(tableDir.toString() + " is empty, remove it.");
@ -350,14 +349,14 @@ public abstract class TableBackupClient {
*/
protected void cleanupDistCpLog(BackupInfo backupInfo, Configuration conf) throws IOException {
Path rootPath = new Path(backupInfo.getHLogTargetDir()).getParent();
FileStatus[] files = FSUtils.listStatus(fs, rootPath);
FileStatus[] files = CommonFSUtils.listStatus(fs, rootPath);
if (files == null) {
return;
}
for (FileStatus file : files) {
if (file.getPath().getName().startsWith("_distcp_logs")) {
LOG.debug("Delete log files of DistCp: " + file.getPath().getName());
FSUtils.delete(fs, file.getPath(), true);
CommonFSUtils.delete(fs, file.getPath(), true);
}
}
}

View File

@ -123,7 +123,7 @@ public final class BackupUtils {
*/
public static void copyTableRegionInfo(Connection conn, BackupInfo backupInfo, Configuration conf)
throws IOException {
Path rootDir = FSUtils.getRootDir(conf);
Path rootDir = CommonFSUtils.getRootDir(conf);
FileSystem fs = rootDir.getFileSystem(conf);
// for each table in the table set, copy out the table info and region
@ -138,7 +138,8 @@ public final class BackupUtils {
// write a copy of descriptor to the target directory
Path target = new Path(backupInfo.getTableBackupDir(table));
FileSystem targetFs = target.getFileSystem(conf);
FSTableDescriptors descriptors = new FSTableDescriptors(targetFs, FSUtils.getRootDir(conf));
FSTableDescriptors descriptors =
new FSTableDescriptors(targetFs, CommonFSUtils.getRootDir(conf));
descriptors.createTableDescriptorForTableDirectory(target, orig, false);
LOG.debug("Attempting to copy table info for:" + table + " target: " + target
+ " descriptor: " + orig);
@ -164,7 +165,7 @@ public final class BackupUtils {
final byte[] content = RegionInfo.toDelimitedByteArray(regionInfo);
Path regionInfoFile = new Path(regionInfoDir, "." + HConstants.REGIONINFO_QUALIFIER_STR);
// First check to get the permissions
FsPermission perms = FSUtils.getFilePermissions(fs, conf, HConstants.DATA_FILE_UMASK_KEY);
FsPermission perms = CommonFSUtils.getFilePermissions(fs, conf, HConstants.DATA_FILE_UMASK_KEY);
// Write the RegionInfo file content
FSDataOutputStream out = FSUtils.create(conf, fs, regionInfoFile, perms, null);
try {
@ -225,7 +226,7 @@ public final class BackupUtils {
*/
public static long getFilesLength(FileSystem fs, Path dir) throws IOException {
long totalLength = 0;
FileStatus[] files = FSUtils.listStatus(fs, dir);
FileStatus[] files = CommonFSUtils.listStatus(fs, dir);
if (files != null) {
for (FileStatus fileStatus : files) {
if (fileStatus.isDirectory()) {

View File

@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
@ -55,7 +56,7 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
* <a href="https://issues.apache.org/jira/browse/HBASE-20838">HBASE-20838</a> for more details.
*/
@InterfaceAudience.Private
public abstract class CommonFSUtils {
public final class CommonFSUtils {
private static final Logger LOG = LoggerFactory.getLogger(CommonFSUtils.class);
/** Parameter name for HBase WAL directory */
@ -68,8 +69,7 @@ public abstract class CommonFSUtils {
/** Full access permissions (starting point for a umask) */
public static final String FULL_RWX_PERMISSIONS = "777";
protected CommonFSUtils() {
super();
private CommonFSUtils() {
}
/**

View File

@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.HFileTestUtil;
import org.junit.After;
import org.slf4j.Logger;
@ -77,7 +77,7 @@ public class TestRefreshHFilesBase {
protected void addHFilesToRegions() throws IOException {
MasterFileSystem mfs = HTU.getMiniHBaseCluster().getMaster().getMasterFileSystem();
Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), TABLE_NAME);
Path tableDir = CommonFSUtils.getTableDir(mfs.getRootDir(), TABLE_NAME);
for (Region region : cluster.getRegions(TABLE_NAME)) {
Path regionDir = new Path(tableDir, region.getRegionInfo().getEncodedName());
Path familyDir = new Path(regionDir, Bytes.toString(FAMILY));

View File

@ -35,7 +35,6 @@ import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
@ -67,13 +66,12 @@ import org.apache.hadoop.hbase.security.token.TokenProvider;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.SecurityTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.minikdc.MiniKdc;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
@ -85,6 +83,8 @@ import org.junit.rules.TestName;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
@Category({MediumTests.class, SecurityTests.class})
public class TestShadeSaslAuthenticationProvider {
private static final Logger LOG = LoggerFactory.getLogger(TestShadeSaslAuthenticationProvider.class);
@ -119,7 +119,7 @@ public class TestShadeSaslAuthenticationProvider {
USER_DATABASE_FILE.toString());
Path rootdir = new Path(testDir, "hbase-root");
FSUtils.setRootDir(CONF, rootdir);
CommonFSUtils.setRootDir(CONF, rootdir);
LocalHBaseCluster cluster = new LocalHBaseCluster(CONF, 1);
return cluster;
}

View File

@ -19,10 +19,9 @@
package org.apache.hadoop.hbase.chaos.actions;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
@ -55,7 +54,7 @@ public class RestartActiveNameNodeAction extends RestartActionBaseAction {
@Override
public void perform() throws Exception {
LOG.info("Performing action: Restart active namenode");
Configuration conf = FSUtils.getRootDir(getConf()).getFileSystem(getConf()).getConf();
Configuration conf = CommonFSUtils.getRootDir(getConf()).getFileSystem(getConf()).getConf();
String nameServiceID = DFSUtil.getNamenodeNameServiceId(conf);
if (!HAUtil.isHAEnabled(conf, nameServiceID)) {
throw new Exception("HA for namenode is not enabled");

View File

@ -21,10 +21,9 @@ package org.apache.hadoop.hbase.chaos.actions;
import java.io.IOException;
import java.util.LinkedList;
import java.util.List;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@ -50,7 +49,7 @@ public class RestartRandomDataNodeAction extends RestartActionBaseAction {
}
public ServerName[] getDataNodes() throws IOException {
DistributedFileSystem fs = (DistributedFileSystem) FSUtils.getRootDir(getConf())
DistributedFileSystem fs = (DistributedFileSystem) CommonFSUtils.getRootDir(getConf())
.getFileSystem(getConf());
DFSClient dfsClient = fs.getClient();
List<ServerName> hosts = new LinkedList<>();

View File

@ -24,7 +24,6 @@ import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -34,7 +33,7 @@ import org.apache.hadoop.hbase.IntegrationTestingUtility;
import org.apache.hadoop.hbase.security.SecurityConstants;
import org.apache.hadoop.hbase.testclassification.IntegrationTests;
import org.apache.hadoop.hbase.util.AbstractHBaseTool;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
@ -217,7 +216,7 @@ public class IntegrationTestZKAndFSPermissions extends AbstractHBaseTool {
}
private void testFSPerms() throws IOException {
Path rootDir = FSUtils.getRootDir(conf);
Path rootDir = CommonFSUtils.getRootDir(conf);
LOG.info("");
LOG.info("***********************************************************************************");

View File

@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.mapreduce.Import.Importer;
import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
@ -78,7 +78,7 @@ public class CopyTable extends Configured implements Tool {
private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";
private Path generateUniqTempDir(boolean withDirCreated) throws IOException {
FileSystem fs = FSUtils.getCurrentFileSystem(getConf());
FileSystem fs = CommonFSUtils.getCurrentFileSystem(getConf());
Path dir = new Path(fs.getWorkingDirectory(), NAME);
if (!fs.exists(dir)) {
fs.mkdirs(dir);
@ -422,7 +422,7 @@ public class CopyTable extends Configured implements Tool {
.isEmpty()) {
// bulkloadDir is deleted only BulkLoadHFiles was successful so that one can rerun
// BulkLoadHFiles.
FileSystem fs = FSUtils.getCurrentFileSystem(getConf());
FileSystem fs = CommonFSUtils.getCurrentFileSystem(getConf());
if (!fs.delete(this.bulkloadDir, true)) {
LOG.error("Deleting folder " + bulkloadDir + " failed!");
code = 1;

View File

@ -21,6 +21,7 @@ import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TASK_KEY;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TIME_KEY;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.EXCLUDE_FROM_MINOR_COMPACTION_KEY;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_KEY;
import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.net.InetSocketAddress;
@ -71,8 +72,8 @@ import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
import org.apache.hadoop.hbase.util.BloomFilterUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.MapReduceExtendedCell;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.SequenceFile;
@ -88,6 +89,7 @@ import org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
@ -431,7 +433,7 @@ public class HFileOutputFormat2
String policy =
conf.get(STORAGE_POLICY_PROPERTY_CF_PREFIX + Bytes.toString(tableAndFamily),
conf.get(STORAGE_POLICY_PROPERTY));
FSUtils.setStoragePolicy(fs, cfPath, policy);
CommonFSUtils.setStoragePolicy(fs, cfPath, policy);
}
/*

View File

@ -24,7 +24,6 @@ import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -32,8 +31,8 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.ConfigurationUtil;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.slf4j.Logger;
@ -62,16 +61,12 @@ public class MultiTableSnapshotInputFormatImpl {
/**
* Configure conf to read from snapshotScans, with snapshots restored to a subdirectory of
* restoreDir.
* <p/>
* Sets: {@link #RESTORE_DIRS_KEY}, {@link #SNAPSHOT_TO_SCANS_KEY}
*
* @param conf
* @param snapshotScans
* @param restoreDir
* @throws IOException
*/
public void setInput(Configuration conf, Map<String, Collection<Scan>> snapshotScans,
Path restoreDir) throws IOException {
Path rootDir = FSUtils.getRootDir(conf);
Path rootDir = CommonFSUtils.getRootDir(conf);
FileSystem fs = rootDir.getFileSystem(conf);
setSnapshotToScans(conf, snapshotScans);
@ -92,7 +87,7 @@ public class MultiTableSnapshotInputFormatImpl {
*/
public List<TableSnapshotInputFormatImpl.InputSplit> getSplits(Configuration conf)
throws IOException {
Path rootDir = FSUtils.getRootDir(conf);
Path rootDir = CommonFSUtils.getRootDir(conf);
FileSystem fs = rootDir.getFileSystem(conf);
List<TableSnapshotInputFormatImpl.InputSplit> rtn = Lists.newArrayList();
@ -229,12 +224,11 @@ public class MultiTableSnapshotInputFormatImpl {
* @param conf configuration to restore with
* @param snapshotToDir mapping from snapshot names to restore directories
* @param fs filesystem to do snapshot restoration on
* @throws IOException
*/
public void restoreSnapshots(Configuration conf, Map<String, Path> snapshotToDir, FileSystem fs)
throws IOException {
// TODO: restore from record readers to parallelize.
Path rootDir = FSUtils.getRootDir(conf);
Path rootDir = CommonFSUtils.getRootDir(conf);
for (Map.Entry<String, Path> entry : snapshotToDir.entrySet()) {
String snapshotName = entry.getKey();

View File

@ -26,7 +26,6 @@ import java.lang.reflect.InvocationTargetException;
import java.util.ArrayList;
import java.util.List;
import java.util.UUID;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -45,7 +44,7 @@ import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.RegionSplitter;
import org.apache.hadoop.io.Writable;
import org.apache.yetus.audience.InterfaceAudience;
@ -53,6 +52,7 @@ 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.MapReduceProtos.TableSnapshotRegionSplit;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
@ -223,7 +223,7 @@ public class TableSnapshotInputFormatImpl {
this.split = split;
TableDescriptor htd = split.htd;
HRegionInfo hri = this.split.getRegionInfo();
FileSystem fs = FSUtils.getCurrentFileSystem(conf);
FileSystem fs = CommonFSUtils.getCurrentFileSystem(conf);
// region is immutable, this should be fine,
@ -277,7 +277,7 @@ public class TableSnapshotInputFormatImpl {
public static List<InputSplit> getSplits(Configuration conf) throws IOException {
String snapshotName = getSnapshotName(conf);
Path rootDir = FSUtils.getRootDir(conf);
Path rootDir = CommonFSUtils.getRootDir(conf);
FileSystem fs = rootDir.getFileSystem(conf);
SnapshotManifest manifest = getSnapshotManifest(conf, snapshotName, rootDir, fs);
@ -363,7 +363,7 @@ public class TableSnapshotInputFormatImpl {
// load table descriptor
TableDescriptor htd = manifest.getTableDescriptor();
Path tableDir = FSUtils.getTableDir(restoreDir, htd.getTableName());
Path tableDir = CommonFSUtils.getTableDir(restoreDir, htd.getTableName());
boolean localityEnabled = conf.getBoolean(SNAPSHOT_INPUTFORMAT_LOCALITY_ENABLED_KEY,
SNAPSHOT_INPUTFORMAT_LOCALITY_ENABLED_DEFAULT);
@ -533,7 +533,7 @@ public class TableSnapshotInputFormatImpl {
conf.set(SPLIT_ALGO, splitAlgo.getClass().getName());
}
conf.setInt(NUM_SPLITS_PER_REGION, numSplitsPerRegion);
Path rootDir = FSUtils.getRootDir(conf);
Path rootDir = CommonFSUtils.getRootDir(conf);
FileSystem fs = rootDir.getFileSystem(conf);
restoreDir = new Path(restoreDir, UUID.randomUUID().toString());

View File

@ -54,7 +54,7 @@ import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
import org.apache.hadoop.hbase.replication.ReplicationUtils;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.ZKConfig;
@ -216,12 +216,12 @@ public class VerifyReplication extends Configured implements Tool {
String peerFSAddress = conf.get(NAME + ".peerFSAddress", null);
String peerHBaseRootAddress = conf.get(NAME + ".peerHBaseRootAddress", null);
FileSystem.setDefaultUri(peerConf, peerFSAddress);
FSUtils.setRootDir(peerConf, new Path(peerHBaseRootAddress));
LOG.info("Using peer snapshot:" + peerSnapshotName + " with temp dir:"
+ peerSnapshotTmpDir + " peer root uri:" + FSUtils.getRootDir(peerConf)
+ " peerFSAddress:" + peerFSAddress);
CommonFSUtils.setRootDir(peerConf, new Path(peerHBaseRootAddress));
LOG.info("Using peer snapshot:" + peerSnapshotName + " with temp dir:" +
peerSnapshotTmpDir + " peer root uri:" + CommonFSUtils.getRootDir(peerConf) +
" peerFSAddress:" + peerFSAddress);
replicatedScanner = new TableSnapshotScanner(peerConf, FSUtils.getRootDir(peerConf),
replicatedScanner = new TableSnapshotScanner(peerConf, CommonFSUtils.getRootDir(peerConf),
new Path(peerFSAddress, peerSnapshotTmpDir), peerSnapshotName, scan, true);
} else {
replicatedScanner = replicatedTable.getScanner(scan);
@ -371,9 +371,9 @@ public class VerifyReplication extends Configured implements Tool {
Configuration peerConf =
HBaseConfiguration.createClusterConf(conf, peerQuorumAddress, PEER_CONFIG_PREFIX);
FileSystem.setDefaultUri(peerConf, peerFSAddress);
FSUtils.setRootDir(peerConf, new Path(peerFSAddress, peerHBaseRootAddress));
CommonFSUtils.setRootDir(peerConf, new Path(peerFSAddress, peerHBaseRootAddress));
FileSystem fs = FileSystem.get(peerConf);
RestoreSnapshotHelper.copySnapshotForScanner(peerConf, fs, FSUtils.getRootDir(peerConf),
RestoreSnapshotHelper.copySnapshotForScanner(peerConf, fs, CommonFSUtils.getRootDir(peerConf),
new Path(peerFSAddress, peerSnapshotTmpDir), peerSnapshotName);
}

View File

@ -24,7 +24,6 @@ import java.util.Base64;
import java.util.HashSet;
import java.util.Set;
import java.util.UUID;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.FileStatus;
@ -49,8 +48,8 @@ import org.apache.hadoop.hbase.mapreduce.TableMapper;
import org.apache.hadoop.hbase.mob.MobConstants;
import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.io.Text;
@ -61,7 +60,6 @@ import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -267,7 +265,7 @@ public class MobRefReporter extends Configured implements Tool {
Path backRefDir = HFileLink.getBackReferencesDir(archive, file);
try {
FileStatus[] backRefs = FSUtils.listStatus(archive.getFileSystem(conf), backRefDir);
FileStatus[] backRefs = CommonFSUtils.listStatus(archive.getFileSystem(conf), backRefDir);
if (backRefs != null) {
boolean found = false;
for (FileStatus backRef : backRefs) {

View File

@ -26,7 +26,6 @@ import java.util.LinkedList;
import java.util.List;
import java.util.Optional;
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.FSDataOutputStream;
@ -271,7 +270,7 @@ public class CompactionTool extends Configured implements Tool {
*/
private static String[] getStoreDirHosts(final FileSystem fs, final Path path)
throws IOException {
FileStatus[] files = FSUtils.listStatus(fs, path);
FileStatus[] files = CommonFSUtils.listStatus(fs, path);
if (files == null) {
return new String[] {};
}

View File

@ -34,7 +34,6 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.function.BiConsumer;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
@ -54,6 +53,7 @@ import org.apache.hadoop.hbase.io.hadoopbackport.ThrottledInputStream;
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.util.AbstractHBaseTool;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
import org.apache.hadoop.hbase.util.Pair;
@ -75,8 +75,10 @@ import org.apache.hadoop.util.Tool;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
import org.apache.hbase.thirdparty.org.apache.commons.cli.Option;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
@ -255,7 +257,7 @@ public class ExportSnapshot extends AbstractHBaseTool implements Tool {
TableName table =HFileLink.getReferencedTableName(inputPath.getName());
String region = HFileLink.getReferencedRegionName(inputPath.getName());
String hfile = HFileLink.getReferencedHFileName(inputPath.getName());
path = new Path(FSUtils.getTableDir(new Path("./"), table),
path = new Path(CommonFSUtils.getTableDir(new Path("./"), table),
new Path(region, new Path(family, hfile)));
break;
case WAL:
@ -845,8 +847,8 @@ public class ExportSnapshot extends AbstractHBaseTool implements Tool {
final FileSystem fs, final Path rootDir, final Path snapshotDir) throws IOException {
// Update the conf with the current root dir, since may be a different cluster
Configuration conf = new Configuration(baseConf);
FSUtils.setRootDir(conf, rootDir);
FSUtils.setFsDefault(conf, FSUtils.getRootDir(conf));
CommonFSUtils.setRootDir(conf, rootDir);
CommonFSUtils.setFsDefault(conf, CommonFSUtils.getRootDir(conf));
SnapshotDescription snapshotDesc = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
SnapshotReferenceUtil.verifySnapshot(conf, fs, snapshotDir, snapshotDesc);
}
@ -959,9 +961,9 @@ public class ExportSnapshot extends AbstractHBaseTool implements Tool {
targetName = snapshotName;
}
if (inputRoot == null) {
inputRoot = FSUtils.getRootDir(conf);
inputRoot = CommonFSUtils.getRootDir(conf);
} else {
FSUtils.setRootDir(conf, inputRoot);
CommonFSUtils.setRootDir(conf, inputRoot);
}
Configuration srcConf = HBaseConfiguration.createClusterConf(conf, null, CONF_SOURCE_PREFIX);

View File

@ -20,12 +20,10 @@ package org.apache.hadoop.hbase;
import java.io.IOException;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Result;
@ -38,13 +36,14 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
import org.apache.hadoop.hbase.mapreduce.TableMapper;
import org.apache.hadoop.hbase.util.AbstractHBaseTool;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.mapreduce.Counters;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.ToolRunner;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.base.Stopwatch;
import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
@ -70,7 +69,7 @@ public class ScanPerformanceEvaluation extends AbstractHBaseTool {
super.setConf(conf);
Path rootDir;
try {
rootDir = FSUtils.getRootDir(conf);
rootDir = CommonFSUtils.getRootDir(conf);
rootDir.getFileSystem(conf);
} catch (IOException ex) {
throw new RuntimeException(ex);

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
import org.junit.Assert;
@ -126,8 +127,9 @@ public abstract class TableSnapshotInputFormatTestBase {
testRestoreSnapshotDoesNotCreateBackRefLinksInit(tableName, snapshotName,tmpTableDir);
Path rootDir = FSUtils.getRootDir(UTIL.getConfiguration());
for (Path regionDir : FSUtils.getRegionDirs(fs, FSUtils.getTableDir(rootDir, tableName))) {
Path rootDir = CommonFSUtils.getRootDir(UTIL.getConfiguration());
for (Path regionDir : FSUtils.getRegionDirs(fs,
CommonFSUtils.getTableDir(rootDir, tableName))) {
for (Path storeDir : FSUtils.getFamilyDirs(fs, regionDir)) {
for (FileStatus status : fs.listStatus(storeDir)) {
System.out.println(status.getPath());
@ -212,7 +214,7 @@ public abstract class TableSnapshotInputFormatTestBase {
Table table = util.getConnection().getTable(tableName);
util.loadTable(table, FAMILIES);
Path rootDir = FSUtils.getRootDir(util.getConfiguration());
Path rootDir = CommonFSUtils.getRootDir(util.getConfiguration());
FileSystem fs = rootDir.getFileSystem(util.getConfiguration());
LOG.info("snapshot");

View File

@ -94,6 +94,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.hadoop.hdfs.DistributedFileSystem;
@ -1198,7 +1199,7 @@ public class TestHFileOutputFormat2 {
// commit so that the filesystem has one directory per column family
hof.getOutputCommitter(context).commitTask(context);
hof.getOutputCommitter(context).commitJob(context);
FileStatus[] families = FSUtils.listStatus(fs, dir, new FSUtils.FamilyDirFilter(fs));
FileStatus[] families = CommonFSUtils.listStatus(fs, dir, new FSUtils.FamilyDirFilter(fs));
assertEquals(tableDescriptorBuilder.build().getColumnFamilies().length, families.length);
for (FileStatus f : families) {
String familyStr = f.getPath().getName();
@ -1215,8 +1216,9 @@ public class TestHFileOutputFormat2 {
assertEquals("Incorrect bloom filter used for column family " + familyStr +
"(reader: " + reader + ")",
hcd.getBloomFilterType(), BloomType.valueOf(Bytes.toString(bloomFilter)));
assertEquals("Incorrect compression used for column family " + familyStr +
"(reader: " + reader + ")", hcd.getCompressionType(), reader.getFileContext().getCompression());
assertEquals(
"Incorrect compression used for column family " + familyStr + "(reader: " + reader + ")",
hcd.getCompressionType(), reader.getFileContext().getCompression());
}
} finally {
dir.getFileSystem(conf).delete(dir, true);
@ -1273,7 +1275,7 @@ public class TestHFileOutputFormat2 {
// deep inspection: get the StoreFile dir
final Path storePath = new Path(
FSUtils.getTableDir(FSUtils.getRootDir(conf), TABLE_NAMES[0]),
CommonFSUtils.getTableDir(CommonFSUtils.getRootDir(conf), TABLE_NAMES[0]),
new Path(admin.getRegions(TABLE_NAMES[0]).get(0).getEncodedName(),
Bytes.toString(FAMILIES[0])));
assertEquals(0, fs.listStatus(storePath).length);
@ -1354,7 +1356,7 @@ public class TestHFileOutputFormat2 {
// deep inspection: get the StoreFile dir
final Path storePath = new Path(
FSUtils.getTableDir(FSUtils.getRootDir(conf), TABLE_NAMES[0]),
CommonFSUtils.getTableDir(CommonFSUtils.getRootDir(conf), TABLE_NAMES[0]),
new Path(admin.getRegions(TABLE_NAMES[0]).get(0).getEncodedName(),
Bytes.toString(FAMILIES[0])));
assertEquals(0, fs.listStatus(storePath).length);

View File

@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.mapreduce.Job;
import org.junit.Before;
import org.junit.BeforeClass;
@ -53,16 +53,14 @@ public class TestMultiTableSnapshotInputFormat extends MultiTableInputFormatTest
@BeforeClass
public static void setUpSnapshots() throws Exception {
TEST_UTIL.enableDebug(MultiTableSnapshotInputFormat.class);
TEST_UTIL.enableDebug(MultiTableSnapshotInputFormatImpl.class);
// take a snapshot of every table we have.
for (String tableName : TABLES) {
SnapshotTestingUtils
.createSnapshotAndValidate(TEST_UTIL.getAdmin(), TableName.valueOf(tableName),
ImmutableList.of(INPUT_FAMILY), null,
snapshotNameForTable(tableName), FSUtils.getRootDir(TEST_UTIL.getConfiguration()),
SnapshotTestingUtils.createSnapshotAndValidate(TEST_UTIL.getAdmin(),
TableName.valueOf(tableName), ImmutableList.of(INPUT_FAMILY), null,
snapshotNameForTable(tableName), CommonFSUtils.getRootDir(TEST_UTIL.getConfiguration()),
TEST_UTIL.getTestFileSystem(), true);
}
}

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.hbase.mapreduce;
import static org.junit.Assert.assertEquals;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.eq;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.doNothing;
import static org.mockito.Mockito.verify;
@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.junit.Before;
import org.junit.ClassRule;
import org.junit.Test;
@ -77,7 +77,7 @@ public class TestMultiTableSnapshotInputFormatImpl {
this.conf = new Configuration();
this.rootDir = new Path("file:///test-root-dir");
FSUtils.setRootDir(conf, rootDir);
CommonFSUtils.setRootDir(conf, rootDir);
this.snapshotScans = ImmutableMap.<String, Collection<Scan>>of("snapshot1",
ImmutableList.of(new Scan().withStartRow(Bytes.toBytes("1"))
.withStopRow(Bytes.toBytes("2"))), "snapshot2",
@ -85,7 +85,7 @@ public class TestMultiTableSnapshotInputFormatImpl {
.withStopRow(Bytes.toBytes("4")),
new Scan().withStartRow(Bytes.toBytes("5")).withStopRow(Bytes.toBytes("6"))));
this.restoreDir = new Path(FSUtils.getRootDir(conf), "restore-dir");
this.restoreDir = new Path(CommonFSUtils.getRootDir(conf), "restore-dir");
}

View File

@ -44,7 +44,7 @@ import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.RegionSplitter;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.mapreduce.InputSplit;
@ -325,7 +325,7 @@ public class TestTableSnapshotInputFormat extends TableSnapshotInputFormatTestBa
admin.split(tableName, Bytes.toBytes("eee"));
TestTableSnapshotScanner.blockUntilSplitFinished(UTIL, tableName, 2);
Path rootDir = FSUtils.getRootDir(UTIL.getConfiguration());
Path rootDir = CommonFSUtils.getRootDir(UTIL.getConfiguration());
FileSystem fs = rootDir.getFileSystem(UTIL.getConfiguration());
SnapshotTestingUtils.createSnapshotAndValidate(admin, tableName, Arrays.asList(FAMILIES),

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.mapreduce;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.any;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@ -49,7 +49,7 @@ import org.apache.hadoop.hbase.mapreduce.WALPlayer.WALKeyValueMapper;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MapReduceTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.LauncherSecurityManager;
import org.apache.hadoop.hbase.util.MapReduceExtendedCell;
import org.apache.hadoop.hbase.wal.WAL;
@ -94,8 +94,8 @@ public class TestWALPlayer {
conf= TEST_UTIL.getConfiguration();
rootDir = TEST_UTIL.createRootDir();
walRootDir = TEST_UTIL.createWALRootDir();
fs = FSUtils.getRootDirFileSystem(conf);
logFs = FSUtils.getWALFileSystem(conf);
fs = CommonFSUtils.getRootDirFileSystem(conf);
logFs = CommonFSUtils.getWALFileSystem(conf);
cluster = TEST_UTIL.startMiniCluster();
}

View File

@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
import org.apache.hadoop.hbase.testclassification.MapReduceTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALEdit;
@ -111,7 +111,7 @@ public class TestWALRecordReader {
hbaseDir = TEST_UTIL.createRootDir();
walRootDir = TEST_UTIL.createWALRootDir();
walFs = FSUtils.getWALFileSystem(conf);
walFs = CommonFSUtils.getWALFileSystem(conf);
logDir = new Path(walRootDir, HConstants.HREGION_LOGDIR_NAME);
}

View File

@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.util.ToolRunner;
import org.junit.After;
import org.junit.Before;
@ -74,7 +74,7 @@ public class TestCompactionTool {
}
HStore store = region.getStore(HBaseTestingUtility.fam1);
assertEquals(10, store.getStorefilesCount());
Path tableDir = FSUtils.getTableDir(rootDir, region.getRegionInfo().getTable());
Path tableDir = CommonFSUtils.getTableDir(rootDir, region.getRegionInfo().getTable());
FileSystem fs = store.getFileSystem();
String storePath = tableDir + "/" + region.getRegionInfo().getEncodedName() + "/"
+ Bytes.toString(HBaseTestingUtility.fam1);

View File

@ -55,7 +55,7 @@ import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.mapreduce.Job;
import org.junit.AfterClass;
import org.junit.Before;
@ -281,14 +281,14 @@ public class TestVerifyReplication extends TestReplicationBase {
runSmallBatchTest();
// Take source and target tables snapshot
Path rootDir = FSUtils.getRootDir(CONF1);
Path rootDir = CommonFSUtils.getRootDir(CONF1);
FileSystem fs = rootDir.getFileSystem(CONF1);
String sourceSnapshotName = "sourceSnapshot-" + System.currentTimeMillis();
SnapshotTestingUtils.createSnapshotAndValidate(UTIL1.getAdmin(), tableName,
Bytes.toString(famName), sourceSnapshotName, rootDir, fs, true);
// Take target snapshot
Path peerRootDir = FSUtils.getRootDir(CONF2);
Path peerRootDir = CommonFSUtils.getRootDir(CONF2);
FileSystem peerFs = peerRootDir.getFileSystem(CONF2);
String peerSnapshotName = "peerSnapshot-" + System.currentTimeMillis();
SnapshotTestingUtils.createSnapshotAndValidate(UTIL2.getAdmin(), tableName,
@ -301,7 +301,7 @@ public class TestVerifyReplication extends TestReplicationBase {
String[] args = new String[] { "--sourceSnapshotName=" + sourceSnapshotName,
"--sourceSnapshotTmpDir=" + tmpPath1, "--peerSnapshotName=" + peerSnapshotName,
"--peerSnapshotTmpDir=" + tmpPath2, "--peerFSAddress=" + peerFSAddress,
"--peerHBaseRootAddress=" + FSUtils.getRootDir(CONF2), UTIL2.getClusterKey(),
"--peerHBaseRootAddress=" + CommonFSUtils.getRootDir(CONF2), UTIL2.getClusterKey(),
tableName.getNameAsString() };
runVerifyReplication(args, NB_ROWS_IN_BATCH, 0);
checkRestoreTmpDir(CONF1, tmpPath1, 1);
@ -331,7 +331,7 @@ public class TestVerifyReplication extends TestReplicationBase {
args = new String[] { "--sourceSnapshotName=" + sourceSnapshotName,
"--sourceSnapshotTmpDir=" + tmpPath1, "--peerSnapshotName=" + peerSnapshotName,
"--peerSnapshotTmpDir=" + tmpPath2, "--peerFSAddress=" + peerFSAddress,
"--peerHBaseRootAddress=" + FSUtils.getRootDir(CONF2), UTIL2.getClusterKey(),
"--peerHBaseRootAddress=" + CommonFSUtils.getRootDir(CONF2), UTIL2.getClusterKey(),
tableName.getNameAsString() };
runVerifyReplication(args, 0, NB_ROWS_IN_BATCH);
checkRestoreTmpDir(CONF1, tmpPath1, 2);
@ -386,14 +386,14 @@ public class TestVerifyReplication extends TestReplicationBase {
runBatchCopyTest();
// Take source and target tables snapshot
Path rootDir = FSUtils.getRootDir(CONF1);
Path rootDir = CommonFSUtils.getRootDir(CONF1);
FileSystem fs = rootDir.getFileSystem(CONF1);
String sourceSnapshotName = "sourceSnapshot-" + System.currentTimeMillis();
SnapshotTestingUtils.createSnapshotAndValidate(UTIL1.getAdmin(), tableName,
Bytes.toString(noRepfamName), sourceSnapshotName, rootDir, fs, true);
// Take target snapshot
Path peerRootDir = FSUtils.getRootDir(CONF2);
Path peerRootDir = CommonFSUtils.getRootDir(CONF2);
FileSystem peerFs = peerRootDir.getFileSystem(CONF2);
String peerSnapshotName = "peerSnapshot-" + System.currentTimeMillis();
SnapshotTestingUtils.createSnapshotAndValidate(UTIL2.getAdmin(), peerTableName,
@ -407,7 +407,7 @@ public class TestVerifyReplication extends TestReplicationBase {
"--sourceSnapshotName=" + sourceSnapshotName,
"--sourceSnapshotTmpDir=" + tmpPath1, "--peerSnapshotName=" + peerSnapshotName,
"--peerSnapshotTmpDir=" + tmpPath2, "--peerFSAddress=" + peerFSAddress,
"--peerHBaseRootAddress=" + FSUtils.getRootDir(CONF2), UTIL2.getClusterKey(),
"--peerHBaseRootAddress=" + CommonFSUtils.getRootDir(CONF2), UTIL2.getClusterKey(),
tableName.getNameAsString() };
runVerifyReplication(args, NB_ROWS_IN_BATCH, 0);
checkRestoreTmpDir(CONF1, tmpPath1, 1);
@ -438,7 +438,7 @@ public class TestVerifyReplication extends TestReplicationBase {
"--sourceSnapshotName=" + sourceSnapshotName,
"--sourceSnapshotTmpDir=" + tmpPath1, "--peerSnapshotName=" + peerSnapshotName,
"--peerSnapshotTmpDir=" + tmpPath2, "--peerFSAddress=" + peerFSAddress,
"--peerHBaseRootAddress=" + FSUtils.getRootDir(CONF2), UTIL2.getClusterKey(),
"--peerHBaseRootAddress=" + CommonFSUtils.getRootDir(CONF2), UTIL2.getClusterKey(),
tableName.getNameAsString() };
runVerifyReplication(args, 0, NB_ROWS_IN_BATCH);
checkRestoreTmpDir(CONF1, tmpPath1, 2);

View File

@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
@ -45,7 +46,7 @@ import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
@ -56,6 +57,7 @@ import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
/**
@ -282,14 +284,14 @@ public class TestVerifyReplicationAdjunct extends TestReplicationBase {
runSmallBatchTest();
// Take source and target tables snapshot
Path rootDir = FSUtils.getRootDir(CONF1);
Path rootDir = CommonFSUtils.getRootDir(CONF1);
FileSystem fs = rootDir.getFileSystem(CONF1);
String sourceSnapshotName = "sourceSnapshot-" + System.currentTimeMillis();
SnapshotTestingUtils.createSnapshotAndValidate(UTIL1.getAdmin(), tableName,
Bytes.toString(famName), sourceSnapshotName, rootDir, fs, true);
// Take target snapshot
Path peerRootDir = FSUtils.getRootDir(CONF2);
Path peerRootDir = CommonFSUtils.getRootDir(CONF2);
FileSystem peerFs = peerRootDir.getFileSystem(CONF2);
String peerSnapshotName = "peerSnapshot-" + System.currentTimeMillis();
SnapshotTestingUtils.createSnapshotAndValidate(UTIL2.getAdmin(), tableName,
@ -302,7 +304,8 @@ public class TestVerifyReplicationAdjunct extends TestReplicationBase {
String[] args = new String[] { "--sourceSnapshotName=" + sourceSnapshotName,
"--sourceSnapshotTmpDir=" + temPath1, "--peerSnapshotName=" + peerSnapshotName,
"--peerSnapshotTmpDir=" + temPath2, "--peerFSAddress=" + peerFSAddress,
"--peerHBaseRootAddress=" + FSUtils.getRootDir(CONF2), "2", tableName.getNameAsString() };
"--peerHBaseRootAddress=" + CommonFSUtils.getRootDir(CONF2), "2",
tableName.getNameAsString() };
TestVerifyReplication.runVerifyReplication(args, NB_ROWS_IN_BATCH, 0);
TestVerifyReplication.checkRestoreTmpDir(CONF1, temPath1, 1);
TestVerifyReplication.checkRestoreTmpDir(CONF2, temPath2, 1);
@ -331,7 +334,8 @@ public class TestVerifyReplicationAdjunct extends TestReplicationBase {
args = new String[] { "--sourceSnapshotName=" + sourceSnapshotName,
"--sourceSnapshotTmpDir=" + temPath1, "--peerSnapshotName=" + peerSnapshotName,
"--peerSnapshotTmpDir=" + temPath2, "--peerFSAddress=" + peerFSAddress,
"--peerHBaseRootAddress=" + FSUtils.getRootDir(CONF2), "2", tableName.getNameAsString() };
"--peerHBaseRootAddress=" + CommonFSUtils.getRootDir(CONF2), "2",
tableName.getNameAsString() };
TestVerifyReplication.runVerifyReplication(args, 0, NB_ROWS_IN_BATCH);
TestVerifyReplication.checkRestoreTmpDir(CONF1, temPath1, 2);
TestVerifyReplication.checkRestoreTmpDir(CONF2, temPath2, 2);

View File

@ -22,7 +22,6 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -46,7 +45,6 @@ import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
import org.apache.hadoop.mapreduce.Job;
import org.junit.AfterClass;
@ -165,14 +163,14 @@ public class TestVerifyReplicationCrossDiffHdfs {
@Test
public void testVerifyRepBySnapshot() throws Exception {
Path rootDir = FSUtils.getRootDir(conf1);
Path rootDir = CommonFSUtils.getRootDir(conf1);
FileSystem fs = rootDir.getFileSystem(conf1);
String sourceSnapshotName = "sourceSnapshot-" + System.currentTimeMillis();
SnapshotTestingUtils.createSnapshotAndValidate(util1.getAdmin(), TABLE_NAME,
Bytes.toString(FAMILY), sourceSnapshotName, rootDir, fs, true);
// Take target snapshot
Path peerRootDir = FSUtils.getRootDir(conf2);
Path peerRootDir = CommonFSUtils.getRootDir(conf2);
FileSystem peerFs = peerRootDir.getFileSystem(conf2);
String peerSnapshotName = "peerSnapshot-" + System.currentTimeMillis();
SnapshotTestingUtils.createSnapshotAndValidate(util2.getAdmin(), TABLE_NAME,
@ -185,7 +183,7 @@ public class TestVerifyReplicationCrossDiffHdfs {
String[] args = new String[] { "--sourceSnapshotName=" + sourceSnapshotName,
"--sourceSnapshotTmpDir=" + temPath1, "--peerSnapshotName=" + peerSnapshotName,
"--peerSnapshotTmpDir=" + temPath2, "--peerFSAddress=" + peerFSAddress,
"--peerHBaseRootAddress=" + FSUtils.getRootDir(conf2), PEER_ID, TABLE_NAME.toString() };
"--peerHBaseRootAddress=" + CommonFSUtils.getRootDir(conf2), PEER_ID, TABLE_NAME.toString() };
// Use the yarn's config override the source cluster's config.
Configuration newConf = HBaseConfiguration.create(conf1);

View File

@ -21,6 +21,7 @@ import static org.apache.hadoop.util.ToolRunner.run;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashSet;
@ -41,7 +42,7 @@ import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
@ -53,6 +54,7 @@ import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
@ -284,7 +286,7 @@ public class TestExportSnapshot {
snapshotFiles.add(hfile);
if (!storeFile.hasReference()) {
verifyNonEmptyFile(new Path(exportedArchive,
new Path(FSUtils.getTableDir(new Path("./"), tableName),
new Path(CommonFSUtils.getTableDir(new Path("./"), tableName),
new Path(regionInfo.getEncodedName(), new Path(family, hfile)))));
}
}
@ -307,7 +309,7 @@ public class TestExportSnapshot {
Set<String> files = new HashSet<>();
LOG.debug("List files in {} in root {} at {}", fs, root, dir);
int rootPrefix = root.makeQualified(fs.getUri(), fs.getWorkingDirectory()).toString().length();
FileStatus[] list = FSUtils.listStatus(fs, dir);
FileStatus[] list = CommonFSUtils.listStatus(fs, dir);
if (list != null) {
for (FileStatus fstat: list) {
LOG.debug(Objects.toString(fstat.getPath()));

View File

@ -58,7 +58,7 @@ org.apache.hadoop.hbase.security.visibility.VisibilityConstants;
org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
org.apache.hadoop.hbase.tool.CanaryTool;
org.apache.hadoop.hbase.util.Bytes;
org.apache.hadoop.hbase.util.FSUtils;
org.apache.hadoop.hbase.util.CommonFSUtils;
org.apache.hadoop.hbase.util.JvmVersion;
org.apache.hadoop.util.StringUtils;
</%import>
@ -355,7 +355,7 @@ AssignmentManager assignmentManager = master.getAssignmentManager();
</tr>
<tr>
<td>HBase Root Directory</td>
<td><% FSUtils.getRootDir(master.getConfiguration()).toString() %></td>
<td><% CommonFSUtils.getRootDir(master.getConfiguration()).toString() %></td>
<td>Location of HBase home directory</td>
</tr>
<tr>

View File

@ -86,7 +86,7 @@ public class HFileArchiver {
*/
public static boolean exists(Configuration conf, FileSystem fs, RegionInfo info)
throws IOException {
Path rootDir = FSUtils.getRootDir(conf);
Path rootDir = CommonFSUtils.getRootDir(conf);
Path regionDir = FSUtils.getRegionDirFromRootDir(rootDir, info);
return fs.exists(regionDir);
}
@ -99,8 +99,8 @@ public class HFileArchiver {
*/
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()),
Path rootDir = CommonFSUtils.getRootDir(conf);
archiveRegion(fs, rootDir, CommonFSUtils.getTableDir(rootDir, info.getTable()),
FSUtils.getRegionDirFromRootDir(rootDir, info));
}
@ -135,8 +135,7 @@ public class HFileArchiver {
// make sure the regiondir lives under the tabledir
Preconditions.checkArgument(regionDir.toString().startsWith(tableDir.toString()));
Path regionArchiveDir = HFileArchiveUtil.getRegionArchiveDir(rootdir,
FSUtils.getTableName(tableDir),
regionDir.getName());
CommonFSUtils.getTableName(tableDir), regionDir.getName());
FileStatusConverter getAsFile = new FileStatusConverter(fs);
// otherwise, we attempt to archive the store files
@ -150,7 +149,7 @@ public class HFileArchiver {
return dirFilter.accept(file) && !file.getName().startsWith(".");
}
};
FileStatus[] storeDirs = FSUtils.listStatus(fs, regionDir, nonHidden);
FileStatus[] storeDirs = CommonFSUtils.listStatus(fs, regionDir, nonHidden);
// if there no files, we can just delete the directory and return;
if (storeDirs == null) {
LOG.debug("Directory {} empty.", regionDir);
@ -263,7 +262,7 @@ public class HFileArchiver {
*/
public static void archiveFamilyByFamilyDir(FileSystem fs, Configuration conf,
RegionInfo parent, Path familyDir, byte[] family) throws IOException {
FileStatus[] storeFiles = FSUtils.listStatus(fs, familyDir);
FileStatus[] storeFiles = CommonFSUtils.listStatus(fs, familyDir);
if (storeFiles == null) {
LOG.debug("No files to dispose of in {}, family={}", parent.getRegionNameAsString(),
Bytes.toString(family));
@ -694,7 +693,7 @@ public class HFileArchiver {
public boolean moveAndClose(Path dest) throws IOException {
this.close();
Path p = this.getPath();
return FSUtils.renameAndSetModifyTime(fs, p, dest);
return CommonFSUtils.renameAndSetModifyTime(fs, p, dest);
}
/**

View File

@ -18,14 +18,13 @@
package org.apache.hadoop.hbase.backup.example;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
@ -59,7 +58,7 @@ public class LongTermArchivingHFileCleaner extends BaseHFileCleanerDelegate {
Path file = fStat.getPath();
// check to see if
FileStatus[] deleteStatus = FSUtils.listStatus(this.fs, file, null);
FileStatus[] deleteStatus = CommonFSUtils.listStatus(this.fs, file, null);
// if the file doesn't exist, then it can be deleted (but should never
// happen since deleted files shouldn't get passed in)
if (deleteStatus == null) {

View File

@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.client;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -31,7 +30,7 @@ import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
import org.apache.hadoop.hbase.mob.MobFileCache;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.RegionScanner;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -58,8 +57,8 @@ public class ClientSideRegionScanner extends AbstractClientScanner {
htd = TableDescriptorBuilder.newBuilder(htd).setReadOnly(true).build();
// open region from the snapshot directory
region = HRegion.newHRegion(FSUtils.getTableDir(rootDir, htd.getTableName()), null, fs, conf,
hri, htd, null);
region = HRegion.newHRegion(CommonFSUtils.getTableDir(rootDir, htd.getTableName()), null, fs,
conf, hri, htd, null);
region.setRestoredRegion(true);
// we won't initialize the MobFileCache when not running in RS process. so provided an
// initialized cache. Consider the case: an CF was set from an mob to non-mob. if we only

View File

@ -22,7 +22,6 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.UUID;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -31,7 +30,7 @@ import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -96,7 +95,7 @@ public class TableSnapshotScanner extends AbstractClientScanner {
*/
public TableSnapshotScanner(Configuration conf, Path restoreDir, String snapshotName, Scan scan)
throws IOException {
this(conf, FSUtils.getRootDir(conf), restoreDir, snapshotName, scan);
this(conf, CommonFSUtils.getRootDir(conf), restoreDir, snapshotName, scan);
}
public TableSnapshotScanner(Configuration conf, Path rootDir, Path restoreDir,

View File

@ -38,7 +38,7 @@ import org.apache.hadoop.fs.FilterFileSystem;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.hadoop.hdfs.DFSClient;
@ -154,7 +154,7 @@ public class HFileSystem extends FilterFileSystem {
* 'COLD', 'WARM', 'HOT', 'ONE_SSD', 'ALL_SSD', 'LAZY_PERSIST'.
*/
public void setStoragePolicy(Path path, String policyName) {
FSUtils.setStoragePolicy(this.fs, path, policyName);
CommonFSUtils.setStoragePolicy(this.fs, path, policyName);
}
/**

View File

@ -18,24 +18,23 @@
package org.apache.hadoop.hbase.io;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.io.IOException;
import java.io.InputStream;
import java.io.FileNotFoundException;
import java.util.List;
import org.apache.hadoop.fs.CanSetDropBehind;
import org.apache.hadoop.fs.CanSetReadahead;
import org.apache.hadoop.fs.CanUnbuffer;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PositionedReadable;
import org.apache.hadoop.fs.Seekable;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.security.AccessControlException;
import org.apache.yetus.audience.InterfaceAudience;
@ -115,7 +114,7 @@ public class FileLink {
public FileLinkInputStream(final FileSystem fs, final FileLink fileLink)
throws IOException {
this(fs, fileLink, FSUtils.getDefaultBufferSize(fs));
this(fs, fileLink, CommonFSUtils.getDefaultBufferSize(fs));
}
public FileLinkInputStream(final FileSystem fs, final FileLink fileLink, int bufferSize)

View File

@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.io;
import java.io.IOException;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -32,7 +31,7 @@ import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.mob.MobConstants;
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.CommonFSUtils;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.yetus.audience.InterfaceAudience;
@ -120,7 +119,7 @@ public class HFileLink extends FileLink {
*/
public static final HFileLink buildFromHFileLinkPattern(Configuration conf, Path hFileLinkPattern)
throws IOException {
return buildFromHFileLinkPattern(FSUtils.getRootDir(conf),
return buildFromHFileLinkPattern(CommonFSUtils.getRootDir(conf),
HFileArchiveUtil.getArchivePath(conf), hFileLinkPattern);
}
@ -235,7 +234,7 @@ public class HFileLink extends FileLink {
String regionName = m.group(3);
String hfileName = m.group(4);
String familyName = path.getParent().getName();
Path tableDir = FSUtils.getTableDir(new Path("./"), tableName);
Path tableDir = CommonFSUtils.getTableDir(new Path("./"), tableName);
return new Path(tableDir, new Path(regionName, new Path(familyName,
hfileName)));
}
@ -396,7 +395,7 @@ public class HFileLink extends FileLink {
final String hfileName, final boolean createBackRef) throws IOException {
String familyName = dstFamilyPath.getName();
String regionName = dstFamilyPath.getParent().getName();
String tableName = FSUtils.getTableName(dstFamilyPath.getParent().getParent())
String tableName = CommonFSUtils.getTableName(dstFamilyPath.getParent().getParent())
.getNameAsString();
String name = createHFileLinkName(linkedTable, linkedRegion, hfileName);
@ -501,9 +500,9 @@ public class HFileLink extends FileLink {
Path regionPath = familyPath.getParent();
Path tablePath = regionPath.getParent();
String linkName = createHFileLinkName(FSUtils.getTableName(tablePath),
String linkName = createHFileLinkName(CommonFSUtils.getTableName(tablePath),
regionPath.getName(), hfileName);
Path linkTableDir = FSUtils.getTableDir(rootDir, linkTableName);
Path linkTableDir = CommonFSUtils.getTableDir(rootDir, linkTableName);
Path regionDir = HRegion.getRegionDir(linkTableDir, linkRegionName);
return new Path(new Path(regionDir, familyPath.getName()), linkName);
}
@ -527,7 +526,7 @@ public class HFileLink extends FileLink {
*/
public static Path getHFileFromBackReference(final Configuration conf, final Path linkRefPath)
throws IOException {
return getHFileFromBackReference(FSUtils.getRootDir(conf), linkRefPath);
return getHFileFromBackReference(CommonFSUtils.getRootDir(conf), linkRefPath);
}
}

View File

@ -102,8 +102,6 @@ public class FanOutOneBlockAsyncDFSOutput implements AsyncFSOutput {
private final Configuration conf;
private final FSUtils fsUtils;
private final DistributedFileSystem dfs;
private final DFSClient client;
@ -324,12 +322,11 @@ public class FanOutOneBlockAsyncDFSOutput implements AsyncFSOutput {
}
}
FanOutOneBlockAsyncDFSOutput(Configuration conf, FSUtils fsUtils, DistributedFileSystem dfs,
FanOutOneBlockAsyncDFSOutput(Configuration conf,DistributedFileSystem dfs,
DFSClient client, ClientProtocol namenode, String clientName, String src, long fileId,
LocatedBlock locatedBlock, Encryptor encryptor, List<Channel> datanodeList,
DataChecksum summer, ByteBufAllocator alloc) {
this.conf = conf;
this.fsUtils = fsUtils;
this.dfs = dfs;
this.client = client;
this.namenode = namenode;
@ -556,7 +553,7 @@ public class FanOutOneBlockAsyncDFSOutput implements AsyncFSOutput {
datanodeList.forEach(ch -> ch.close());
datanodeList.forEach(ch -> ch.closeFuture().awaitUninterruptibly());
endFileLease(client, fileId);
fsUtils.recoverFileLease(dfs, new Path(src), conf,
FSUtils.recoverFileLease(dfs, new Path(src), conf,
reporter == null ? new CancelOnClose(client) : reporter);
}

View File

@ -48,7 +48,6 @@ import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.client.ConnectionUtils;
import org.apache.hadoop.hbase.util.CancelableProgressable;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSOutputStream;
import org.apache.hadoop.hdfs.DistributedFileSystem;
@ -514,7 +513,6 @@ public final class FanOutOneBlockAsyncDFSOutputHelper {
boolean overwrite, boolean createParent, short replication, long blockSize,
EventLoopGroup eventLoopGroup, Class<? extends Channel> channelClass) throws IOException {
Configuration conf = dfs.getConf();
FSUtils fsUtils = FSUtils.getInstance(dfs, conf);
DFSClient client = dfs.getClient();
String clientName = client.getClientName();
ClientProtocol namenode = client.getNamenode();
@ -557,7 +555,7 @@ public final class FanOutOneBlockAsyncDFSOutputHelper {
}
Encryptor encryptor = createEncryptor(conf, stat, client);
FanOutOneBlockAsyncDFSOutput output =
new FanOutOneBlockAsyncDFSOutput(conf, fsUtils, dfs, client, namenode, clientName, src,
new FanOutOneBlockAsyncDFSOutput(conf, dfs, client, namenode, clientName, src,
stat.getFileId(), locatedBlock, encryptor, datanodeList, summer, ALLOC);
succ = true;
return output;

View File

@ -30,7 +30,6 @@ import com.codahale.metrics.MetricRegistry;
import com.codahale.metrics.ScheduledReporter;
import com.codahale.metrics.Snapshot;
import com.codahale.metrics.Timer;
import java.io.ByteArrayOutputStream;
import java.io.DataInput;
import java.io.IOException;
@ -48,7 +47,6 @@ import java.util.Set;
import java.util.SortedMap;
import java.util.TimeZone;
import java.util.concurrent.TimeUnit;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
@ -74,11 +72,10 @@ import org.apache.hadoop.hbase.util.BloomFilter;
import org.apache.hadoop.hbase.util.BloomFilterFactory;
import org.apache.hadoop.hbase.util.BloomFilterUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.slf4j.Logger;
@ -211,8 +208,8 @@ public class HFilePrettyPrinter extends Configured implements Tool {
String regionName = cmd.getOptionValue("r");
byte[] rn = Bytes.toBytes(regionName);
byte[][] hri = HRegionInfo.parseRegionName(rn);
Path rootDir = FSUtils.getRootDir(getConf());
Path tableDir = FSUtils.getTableDir(rootDir, TableName.valueOf(hri[0]));
Path rootDir = CommonFSUtils.getRootDir(getConf());
Path tableDir = CommonFSUtils.getTableDir(rootDir, TableName.valueOf(hri[0]));
String enc = HRegionInfo.encodeRegionName(rn);
Path regionDir = new Path(tableDir, enc);
if (verbose)
@ -254,9 +251,10 @@ public class HFilePrettyPrinter extends Configured implements Tool {
throw new RuntimeException("A Configuration instance must be provided.");
}
try {
FSUtils.setFsDefault(getConf(), FSUtils.getRootDir(getConf()));
if (!parseOptions(args))
CommonFSUtils.setFsDefault(getConf(), CommonFSUtils.getRootDir(getConf()));
if (!parseOptions(args)) {
return 1;
}
} catch (IOException ex) {
LOG.error("Error parsing command-line options", ex);
return 1;
@ -292,7 +290,7 @@ public class HFilePrettyPrinter extends Configured implements Tool {
}
if (checkRootDir) {
Path rootPath = FSUtils.getRootDir(getConf());
Path rootPath = CommonFSUtils.getRootDir(getConf());
String rootString = rootPath + rootPath.SEPARATOR;
if (!file.toString().startsWith(rootString)) {
// First we see if fully-qualified URI matches the root dir. It might

View File

@ -47,11 +47,13 @@ import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.BloomFilterWriter;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.io.Writable;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
@ -291,7 +293,7 @@ public class HFileWriterImpl implements HFile.Writer {
/** A helper method to create HFile output streams in constructors */
protected static FSDataOutputStream createOutputStream(Configuration conf,
FileSystem fs, Path path, InetSocketAddress[] favoredNodes) throws IOException {
FsPermission perms = FSUtils.getFilePermissions(fs, conf,
FsPermission perms = CommonFSUtils.getFilePermissions(fs, conf,
HConstants.DATA_FILE_UMASK_KEY);
return FSUtils.create(conf, fs, path, perms, favoredNodes);
}

View File

@ -26,10 +26,12 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.ClusterId;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.util.CommonFSUtils;
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.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
@ -65,7 +67,7 @@ public class CachedClusterId {
private AtomicInteger cacheMisses = new AtomicInteger(0);
public CachedClusterId(Server server, Configuration conf) throws IOException {
this.rootDir = FSUtils.getRootDir(conf);
this.rootDir = CommonFSUtils.getRootDir(conf);
this.fs = rootDir.getFileSystem(conf);
this.server = server;
}

View File

@ -28,7 +28,6 @@ import java.util.Properties;
import java.util.TreeMap;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.stream.Collectors;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -57,8 +56,8 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.PairOfSameType;
import org.apache.hadoop.hbase.util.Threads;
@ -248,7 +247,7 @@ public class CatalogJanitor extends ScheduledChore {
throws IOException {
FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
Path rootdir = this.services.getMasterFileSystem().getRootDir();
Path tabledir = FSUtils.getTableDir(rootdir, mergedRegion.getTable());
Path tabledir = CommonFSUtils.getTableDir(rootdir, mergedRegion.getTable());
TableDescriptor htd = getDescriptor(mergedRegion.getTable());
HRegionFileSystem regionFs = null;
try {
@ -373,14 +372,14 @@ public class CatalogJanitor extends ScheduledChore {
FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
Path rootdir = this.services.getMasterFileSystem().getRootDir();
Path tabledir = FSUtils.getTableDir(rootdir, daughter.getTable());
Path tabledir = CommonFSUtils.getTableDir(rootdir, daughter.getTable());
Path daughterRegionDir = new Path(tabledir, daughter.getEncodedName());
HRegionFileSystem regionFs;
try {
if (!FSUtils.isExists(fs, daughterRegionDir)) {
if (!CommonFSUtils.isExists(fs, daughterRegionDir)) {
return new Pair<>(Boolean.FALSE, Boolean.FALSE);
}
} catch (IOException ioe) {

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.hbase.master;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -43,6 +42,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.replication.ReplicationUtils;
import org.apache.hadoop.hbase.security.access.SnapshotScannerHDFSAclHelper;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.ipc.RemoteException;
@ -107,16 +107,16 @@ public class MasterFileSystem {
// mismatched filesystems if hbase.rootdir is hdfs and fs.defaultFS is
// default localfs. Presumption is that rootdir is fully-qualified before
// we get to here with appropriate fs scheme.
this.rootdir = FSUtils.getRootDir(conf);
this.rootdir = CommonFSUtils.getRootDir(conf);
this.tempdir = new Path(this.rootdir, HConstants.HBASE_TEMP_DIRECTORY);
// Cover both bases, the old way of setting default fs and the new.
// We're supposed to run on 0.20 and 0.21 anyways.
this.fs = this.rootdir.getFileSystem(conf);
this.walRootDir = FSUtils.getWALRootDir(conf);
this.walFs = FSUtils.getWALFileSystem(conf);
FSUtils.setFsDefault(conf, new Path(this.walFs.getUri()));
this.walRootDir = CommonFSUtils.getWALRootDir(conf);
this.walFs = CommonFSUtils.getWALFileSystem(conf);
CommonFSUtils.setFsDefault(conf, new Path(this.walFs.getUri()));
walFs.setConf(conf);
FSUtils.setFsDefault(conf, new Path(this.fs.getUri()));
CommonFSUtils.setFsDefault(conf, new Path(this.fs.getUri()));
// make sure the fs has the same conf
fs.setConf(conf);
this.secureRootSubDirPerms = new FsPermission(conf.get("hbase.rootdir.perms", "700"));
@ -440,7 +440,7 @@ public class MasterFileSystem {
public void deleteFamilyFromFS(Path rootDir, RegionInfo region, byte[] familyName)
throws IOException {
// archive family store files
Path tableDir = FSUtils.getTableDir(rootDir, region.getTable());
Path tableDir = CommonFSUtils.getTableDir(rootDir, region.getTable());
HFileArchiver.archiveFamily(fs, conf, region, tableDir, familyName);
// delete the family folder
@ -460,6 +460,6 @@ public class MasterFileSystem {
}
public void logFileSystemState(Logger log) throws IOException {
FSUtils.logFileSystemState(fs, rootdir, log);
CommonFSUtils.logFileSystemState(fs, rootdir, log);
}
}

View File

@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.wal.WALSplitter;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
@ -189,7 +190,7 @@ public class MasterWalManager {
*/
public FileStatus[] getWALDirPaths(final PathFilter filter) throws IOException {
Path walDirPath = new Path(CommonFSUtils.getWALRootDir(conf), HConstants.HREGION_LOGDIR_NAME);
FileStatus[] walDirForServerNames = FSUtils.listStatus(fs, walDirPath, filter);
FileStatus[] walDirForServerNames = CommonFSUtils.listStatus(fs, walDirPath, filter);
return walDirForServerNames == null? new FileStatus[0]: walDirForServerNames;
}
@ -216,7 +217,7 @@ public class MasterWalManager {
}
try {
if (!this.fs.exists(logsDirPath)) return serverNames;
FileStatus[] logFolders = FSUtils.listStatus(this.fs, logsDirPath, null);
FileStatus[] logFolders = CommonFSUtils.listStatus(this.fs, logsDirPath, null);
// Get online servers after getting log folders to avoid log folder deletion of newly
// checked in region servers . see HBASE-5916
Set<ServerName> onlineServers = services.getServerManager().getOnlineServers().keySet();
@ -226,7 +227,7 @@ public class MasterWalManager {
return serverNames;
}
for (FileStatus status : logFolders) {
FileStatus[] curLogFiles = FSUtils.listStatus(this.fs, status.getPath(), null);
FileStatus[] curLogFiles = CommonFSUtils.listStatus(this.fs, status.getPath(), null);
if (curLogFiles == null || curLogFiles.length == 0) {
// Empty log folder. No recovery needed
continue;
@ -373,13 +374,13 @@ public class MasterWalManager {
AbstractFSWALProvider.getWALDirectoryName(serverName.toString()));
Path splitDir = logDir.suffix(AbstractFSWALProvider.SPLITTING_EXT);
if (fs.exists(splitDir)) {
FileStatus[] logfiles = FSUtils.listStatus(fs, splitDir, META_FILTER);
FileStatus[] logfiles = CommonFSUtils.listStatus(fs, splitDir, META_FILTER);
if (logfiles != null) {
for (FileStatus status : logfiles) {
if (!status.isDir()) {
Path newPath = AbstractFSWAL.getWALArchivePath(this.oldLogDir,
status.getPath());
if (!FSUtils.renameAndSetModifyTime(fs, status.getPath(), newPath)) {
if (!CommonFSUtils.renameAndSetModifyTime(fs, status.getPath(), newPath)) {
LOG.warn("Unable to move " + status.getPath() + " to " + newPath);
} else {
LOG.debug("Archived meta log " + status.getPath() + " to " + newPath);

View File

@ -51,13 +51,11 @@ import org.apache.hadoop.hbase.YouAreDeadException;
import org.apache.hadoop.hbase.client.AsyncClusterConnection;
import org.apache.hadoop.hbase.client.AsyncRegionServerAdmin;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.master.assignment.RegionStates;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.FutureUtils;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
@ -72,7 +70,6 @@ import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FlushedRegionSequenceId;
@ -1028,7 +1025,7 @@ public class ServerManager {
isFlushSeqIdPersistInProgress = true;
try {
Configuration conf = master.getConfiguration();
Path rootDir = FSUtils.getRootDir(conf);
Path rootDir = CommonFSUtils.getRootDir(conf);
Path lastFlushedSeqIdPath = new Path(rootDir, LAST_FLUSHED_SEQ_ID_FILE);
FileSystem fs = FileSystem.get(conf);
if (fs.exists(lastFlushedSeqIdPath)) {
@ -1083,7 +1080,7 @@ public class ServerManager {
return;
}
Configuration conf = master.getConfiguration();
Path rootDir = FSUtils.getRootDir(conf);
Path rootDir = CommonFSUtils.getRootDir(conf);
Path lastFlushedSeqIdPath = new Path(rootDir, LAST_FLUSHED_SEQ_ID_FILE);
FileSystem fs = FileSystem.get(conf);
if (!fs.exists(lastFlushedSeqIdPath)) {

View File

@ -35,7 +35,6 @@ import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@ -52,13 +51,14 @@ import org.apache.hadoop.hbase.log.HBaseMarkers;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.procedure2.util.StringUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
@ -173,7 +173,7 @@ public class SplitLogManager {
LOG.warn(logDir + " doesn't exist. Nothing to do!");
continue;
}
FileStatus[] logfiles = FSUtils.listStatus(fs, logDir, filter);
FileStatus[] logfiles = CommonFSUtils.listStatus(fs, logDir, filter);
if (logfiles == null || logfiles.length == 0) {
LOG.info("{} dir is empty, no logs to split.", logDir);
} else {
@ -254,7 +254,7 @@ public class SplitLogManager {
// recover-lease is done. totalSize will be under in most cases and the
// metrics that it drives will also be under-reported.
totalSize += lf.getLen();
String pathToLog = FSUtils.removeWALRootPath(lf.getPath(), conf);
String pathToLog = CommonFSUtils.removeWALRootPath(lf.getPath(), conf);
if (!enqueueSplitTask(pathToLog, batch)) {
throw new IOException("duplicate log split scheduled for " + lf.getPath());
}

View File

@ -31,7 +31,7 @@ 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.hadoop.hbase.util.CommonFSUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -90,14 +90,14 @@ public class GCRegionProcedure extends AbstractStateMachineRegionProcedure<GCReg
}
FileSystem walFs = mfs.getWALFileSystem();
// Cleanup the directories on WAL filesystem also
Path regionWALDir = FSUtils.getWALRegionDir(env.getMasterConfiguration(),
Path regionWALDir = CommonFSUtils.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(),
Path wrongRegionWALDir = CommonFSUtils.getWrongWALRegionDir(env.getMasterConfiguration(),
getRegion().getTable(), getRegion().getEncodedName());
if (walFs.exists(wrongRegionWALDir)) {
if (!walFs.delete(wrongRegionWALDir, true)) {

View File

@ -24,7 +24,6 @@ import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.stream.Stream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.MetaMutationAnnotation;
@ -56,13 +55,14 @@ import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.wal.WALSplitUtil;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
@ -575,7 +575,7 @@ public class MergeTableRegionsProcedure
*/
private void createMergedRegion(final MasterProcedureEnv env) throws IOException {
final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
final Path tabledir = FSUtils.getTableDir(mfs.getRootDir(), regionsToMerge[0].getTable());
final Path tabledir = CommonFSUtils.getTableDir(mfs.getRootDir(), regionsToMerge[0].getTable());
final FileSystem fs = mfs.getFileSystem();
HRegionFileSystem mergeRegionFs = null;
for (RegionInfo ri: this.regionsToMerge) {
@ -624,7 +624,7 @@ public class MergeTableRegionsProcedure
private void cleanupMergedRegion(final MasterProcedureEnv env) throws IOException {
final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
TableName tn = this.regionsToMerge[0].getTable();
final Path tabledir = FSUtils.getTableDir(mfs.getRootDir(), tn);
final Path tabledir = CommonFSUtils.getTableDir(mfs.getRootDir(), tn);
final FileSystem fs = mfs.getFileSystem();
// See createMergedRegion above where we specify the merge dir as being in the
// FIRST merge parent region.

View File

@ -33,7 +33,6 @@ import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.stream.Stream;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -65,6 +64,7 @@ import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.RegionSplitPolicy;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Pair;
@ -597,7 +597,7 @@ public class SplitTableRegionProcedure
@VisibleForTesting
public void createDaughterRegions(final MasterProcedureEnv env) throws IOException {
final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
final Path tabledir = FSUtils.getTableDir(mfs.getRootDir(), getTableName());
final Path tabledir = CommonFSUtils.getTableDir(mfs.getRootDir(), getTableName());
final FileSystem fs = mfs.getFileSystem();
HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(
env.getMasterConfiguration(), fs, tabledir, getParentRegion(), false);

View File

@ -18,19 +18,18 @@
package org.apache.hadoop.hbase.master.cleaner;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.util.FSUtils;
/**
* HFileLink cleaner that determines if a hfile should be deleted.
@ -62,7 +61,7 @@ public class HFileLinkCleaner extends BaseHFileCleanerDelegate {
// Also check if the HFile is in the HBASE_TEMP_DIRECTORY; this is where the referenced
// file gets created when cloning a snapshot.
hfilePath = HFileLink.getHFileFromBackReference(
new Path(FSUtils.getRootDir(getConf()), HConstants.HBASE_TEMP_DIRECTORY), filePath);
new Path(CommonFSUtils.getRootDir(getConf()), HConstants.HBASE_TEMP_DIRECTORY), filePath);
if (fs.exists(hfilePath)) {
return false;
}
@ -71,12 +70,13 @@ public class HFileLinkCleaner extends BaseHFileCleanerDelegate {
if (fs.exists(hfilePath)) {
return false;
}
hfilePath = HFileLink.getHFileFromBackReference(FSUtils.getRootDir(getConf()), filePath);
hfilePath =
HFileLink.getHFileFromBackReference(CommonFSUtils.getRootDir(getConf()), filePath);
return !fs.exists(hfilePath);
} catch (IOException e) {
if (LOG.isDebugEnabled()) {
LOG.debug("Couldn't verify if the referenced file still exists, keep it just in case: "
+ hfilePath);
LOG.debug("Couldn't verify if the referenced file still exists, keep it just in case: " +
hfilePath);
}
return false;
}
@ -86,7 +86,7 @@ public class HFileLinkCleaner extends BaseHFileCleanerDelegate {
Path backRefDir = null;
try {
backRefDir = HFileLink.getBackReferencesDir(parentDir, filePath.getName());
return FSUtils.listStatus(fs, backRefDir) == null;
return CommonFSUtils.listStatus(fs, backRefDir) == null;
} catch (IOException e) {
if (LOG.isDebugEnabled()) {
LOG.debug("Couldn't get the references, not deleting file, just in case. filePath="

View File

@ -24,14 +24,13 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.TableNamespaceManager;
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
import org.apache.hadoop.hbase.util.FSUtils;
/**
* Base class for all the Namespace procedures that want to use a StateMachineProcedure. It provide
* some basic helpers like basic locking and basic toStringClassDetails().
@ -118,7 +117,8 @@ public abstract class AbstractStateMachineNamespaceProcedure<TState>
@VisibleForTesting
public static void createDirectory(MasterFileSystem mfs, NamespaceDescriptor nsDescriptor)
throws IOException {
mfs.getFileSystem().mkdirs(FSUtils.getNamespaceDir(mfs.getRootDir(), nsDescriptor.getName()));
mfs.getFileSystem()
.mkdirs(CommonFSUtils.getNamespaceDir(mfs.getRootDir(), nsDescriptor.getName()));
}
protected void releaseSyncLatch() {

View File

@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.master.TableStateManager;
import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.yetus.audience.InterfaceAudience;
/**
@ -131,7 +131,7 @@ public abstract class AbstractStateMachineTableProcedure<TState>
protected final Path getWALRegionDir(MasterProcedureEnv env, RegionInfo region)
throws IOException {
return FSUtils.getWALRegionDir(env.getMasterConfiguration(),
return CommonFSUtils.getWALRegionDir(env.getMasterConfiguration(),
region.getTable(), region.getEncodedName());
}

View File

@ -24,7 +24,6 @@ import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -51,13 +50,15 @@ import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
@ -456,8 +457,8 @@ public class CloneSnapshotProcedure
// 1. Create Table Descriptor
// using a copy of descriptor, table will be created enabling first
final Path tempTableDir = FSUtils.getTableDir(tempdir, tableDescriptor.getTableName());
((FSTableDescriptors)(env.getMasterServices().getTableDescriptors()))
final Path tempTableDir = CommonFSUtils.getTableDir(tempdir, tableDescriptor.getTableName());
((FSTableDescriptors) (env.getMasterServices().getTableDescriptors()))
.createTableDescriptorForTableDirectory(tempTableDir,
TableDescriptorBuilder.newBuilder(tableDescriptor).build(), false);

View File

@ -37,8 +37,8 @@ import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
import org.apache.yetus.audience.InterfaceAudience;
@ -329,7 +329,7 @@ public class CreateTableProcedure
// 1. Create Table Descriptor
// using a copy of descriptor, table will be created enabling first
final Path tempTableDir = FSUtils.getTableDir(tempdir, tableDescriptor.getTableName());
final Path tempTableDir = CommonFSUtils.getTableDir(tempdir, tableDescriptor.getTableName());
((FSTableDescriptors)(env.getMasterServices().getTableDescriptors()))
.createTableDescriptorForTableDirectory(tempTableDir, tableDescriptor, false);
@ -348,7 +348,8 @@ public class CreateTableProcedure
final TableDescriptor tableDescriptor,
final Path tempTableDir) throws IOException {
final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
final Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableDescriptor.getTableName());
final Path tableDir =
CommonFSUtils.getTableDir(mfs.getRootDir(), tableDescriptor.getTableName());
FileSystem fs = mfs.getFileSystem();
if (!fs.delete(tableDir, true) && fs.exists(tableDir)) {
throw new IOException("Couldn't delete " + tableDir);

View File

@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.NamespaceNotFoundException;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -234,7 +234,7 @@ public class DeleteNamespaceProcedure
throws IOException {
MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
FileSystem fs = mfs.getFileSystem();
Path p = FSUtils.getNamespaceDir(mfs.getRootDir(), namespaceName);
Path p = CommonFSUtils.getNamespaceDir(mfs.getRootDir(), namespaceName);
try {
for (FileStatus status : fs.listStatus(p)) {
@ -242,7 +242,7 @@ public class DeleteNamespaceProcedure
throw new IOException("Namespace directory contains table dir: " + status.getPath());
}
}
if (!fs.delete(FSUtils.getNamespaceDir(mfs.getRootDir(), namespaceName), true)) {
if (!fs.delete(CommonFSUtils.getNamespaceDir(mfs.getRootDir(), namespaceName), true)) {
throw new IOException("Failed to remove namespace: " + namespaceName);
}
} catch (FileNotFoundException e) {

View File

@ -45,10 +45,12 @@ import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.mob.MobConstants;
import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
@ -275,8 +277,8 @@ public class DeleteTableProcedure
final FileSystem fs = mfs.getFileSystem();
final Path tempdir = mfs.getTempDir();
final Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableName);
final Path tempTableDir = FSUtils.getTableDir(tempdir, tableName);
final Path tableDir = CommonFSUtils.getTableDir(mfs.getRootDir(), tableName);
final Path tempTableDir = CommonFSUtils.getTableDir(tempdir, tableName);
if (fs.exists(tableDir)) {
// Ensure temp exists
@ -324,8 +326,8 @@ public class DeleteTableProcedure
}
// Archive mob data
Path mobTableDir = FSUtils.getTableDir(new Path(mfs.getRootDir(), MobConstants.MOB_DIR_NAME),
tableName);
Path mobTableDir =
CommonFSUtils.getTableDir(new Path(mfs.getRootDir(), MobConstants.MOB_DIR_NAME), tableName);
Path regionDir =
new Path(mobTableDir, MobUtils.getMobRegionInfo(tableName).getEncodedName());
if (fs.exists(regionDir)) {
@ -346,7 +348,7 @@ public class DeleteTableProcedure
// Delete the directory on wal filesystem
FileSystem walFs = mfs.getWALFileSystem();
Path tableWALDir = FSUtils.getWALTableDir(env.getMasterConfiguration(), tableName);
Path tableWALDir = CommonFSUtils.getWALTableDir(env.getMasterConfiguration(), tableName);
if (walFs.exists(tableWALDir) && !walFs.delete(tableWALDir, true)) {
throw new IOException("Couldn't delete table dir on wal filesystem" + tableWALDir);
}

View File

@ -57,8 +57,7 @@ public class MasterProcedureEnv implements ConfigurationObserver {
@Override
public void recoverFileLease(final FileSystem fs, final Path path) throws IOException {
final Configuration conf = master.getConfiguration();
final FSUtils fsUtils = FSUtils.getInstance(fs, conf);
fsUtils.recoverFileLease(fs, path, conf, new CancelableProgressable() {
FSUtils.recoverFileLease(fs, path, conf, new CancelableProgressable() {
@Override
public boolean progress() {
LOG.debug("Recover Procedure Store log lease: " + path);

View File

@ -22,7 +22,6 @@ import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.ThreadPoolExecutor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
@ -31,7 +30,7 @@ import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.yetus.audience.InterfaceAudience;
@ -39,6 +38,7 @@ import org.apache.yetus.audience.InterfaceStability;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
/**
@ -104,7 +104,8 @@ public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
ModifyRegionUtils.editRegions(exec, regions, new ModifyRegionUtils.RegionEditTask() {
@Override
public void editRegion(final RegionInfo regionInfo) throws IOException {
snapshotManifest.addRegion(FSUtils.getTableDir(rootDir, snapshotTable), regionInfo);
snapshotManifest.addRegion(CommonFSUtils.getTableDir(rootDir, snapshotTable),
regionInfo);
}
});
} finally {

View File

@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.snapshot.CorruptedSnapshotException;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
@ -209,7 +209,7 @@ public final class MasterSnapshotVerifier {
// Verify Snapshot HFiles
// Requires the root directory file system as HFiles are stored in the root directory
SnapshotReferenceUtil.verifySnapshot(services.getConfiguration(),
FSUtils.getRootDirFileSystem(services.getConfiguration()), manifest);
CommonFSUtils.getRootDirFileSystem(services.getConfiguration()), manifest);
}
/**

View File

@ -34,7 +34,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.slf4j.Logger;
@ -110,8 +110,8 @@ public class SnapshotFileCache implements Stoppable {
*/
public SnapshotFileCache(Configuration conf, long cacheRefreshPeriod, String refreshThreadName,
SnapshotFileInspector inspectSnapshotFiles) throws IOException {
this(FSUtils.getCurrentFileSystem(conf), FSUtils.getRootDir(conf), 0, cacheRefreshPeriod,
refreshThreadName, inspectSnapshotFiles);
this(CommonFSUtils.getCurrentFileSystem(conf), CommonFSUtils.getRootDir(conf), 0,
cacheRefreshPeriod, refreshThreadName, inspectSnapshotFiles);
}
/**
@ -212,7 +212,7 @@ public class SnapshotFileCache implements Stoppable {
// just list the snapshot directory directly, do not check the modification time for the root
// snapshot directory, as some file system implementations do not modify the parent directory's
// modTime when there are new sub items, for example, S3.
FileStatus[] snapshotDirs = FSUtils.listStatus(fs, snapshotDir,
FileStatus[] snapshotDirs = CommonFSUtils.listStatus(fs, snapshotDir,
p -> !p.getName().equals(SnapshotDescriptionUtils.SNAPSHOT_TMP_DIR_NAME));
// clear the cache, as in the below code, either we will also clear the snapshots, or we will

View File

@ -21,11 +21,6 @@ import java.io.IOException;
import java.util.Collection;
import java.util.Collections;
import java.util.Map;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@ -36,7 +31,11 @@ import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate;
import org.apache.hadoop.hbase.snapshot.CorruptedSnapshotException;
import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Implementation of a file cleaner that checks if a hfile is still used by snapshots of HBase
@ -92,8 +91,8 @@ public class SnapshotHFileCleaner extends BaseHFileCleanerDelegate {
try {
long cacheRefreshPeriod = conf.getLong(HFILE_CACHE_REFRESH_PERIOD_CONF_KEY,
DEFAULT_HFILE_CACHE_REFRESH_PERIOD);
final FileSystem fs = FSUtils.getCurrentFileSystem(conf);
Path rootDir = FSUtils.getRootDir(conf);
final FileSystem fs = CommonFSUtils.getCurrentFileSystem(conf);
Path rootDir = CommonFSUtils.getRootDir(conf);
cache = new SnapshotFileCache(fs, rootDir, cacheRefreshPeriod, cacheRefreshPeriod,
"snapshot-hfile-cleaner-cache-refresher", new SnapshotFileCache.SnapshotFileInspector() {
@Override

View File

@ -35,7 +35,6 @@ import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileStatus;
@ -84,8 +83,8 @@ import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil;
import org.apache.hadoop.hbase.snapshot.TablePartiallyOpenException;
import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.NonceKey;
import org.apache.hadoop.hbase.util.TableDescriptorChecker;
import org.apache.yetus.audience.InterfaceAudience;
@ -93,13 +92,15 @@ import org.apache.yetus.audience.InterfaceStability;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription.Type;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
/**
* This class manages the procedure of taking and restoring snapshots. There is only one
@ -1158,7 +1159,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
LOG.info("Snapshot feature is not enabled, missing log and hfile cleaners.");
Path snapshotDir = SnapshotDescriptionUtils.getSnapshotsDir(mfs.getRootDir());
if (fs.exists(snapshotDir)) {
FileStatus[] snapshots = FSUtils.listStatus(fs, snapshotDir,
FileStatus[] snapshots = CommonFSUtils.listStatus(fs, snapshotDir,
new SnapshotDescriptionUtils.CompletedSnaphotDirectoriesFilter(fs));
if (snapshots != null) {
LOG.error("Snapshots are present, but cleaners are not enabled.");

View File

@ -50,7 +50,7 @@ import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.yetus.audience.InterfaceAudience;
@ -59,6 +59,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
@ -314,7 +315,7 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
*/
protected void snapshotDisabledRegion(final RegionInfo regionInfo)
throws IOException {
snapshotManifest.addRegion(FSUtils.getTableDir(rootDir, snapshotTable), regionInfo);
snapshotManifest.addRegion(CommonFSUtils.getTableDir(rootDir, snapshotTable), regionInfo);
monitor.rethrowException();
status.setStatus("Completed referencing HFiles for offline region " + regionInfo.toString() +
" of table: " + snapshotTable);

View File

@ -19,13 +19,12 @@ package org.apache.hadoop.hbase.mob;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -62,7 +61,7 @@ public class ManualMobMaintHFileCleaner extends BaseHFileCleanerDelegate {
Path region = family.getParent();
Path table = region.getParent();
TableName tableName = FSUtils.getTableName(table);
TableName tableName = CommonFSUtils.getTableName(table);
String mobRegion = MOB_REGIONS.get(tableName);
if (mobRegion == null) {

View File

@ -26,7 +26,6 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocatedFileStatus;
@ -46,6 +45,7 @@ import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.yetus.audience.InterfaceAudience;
@ -168,8 +168,8 @@ public class MobFileCleanerChore extends ScheduledChore {
maxCreationTimeToArchive, table);
}
Path rootDir = FSUtils.getRootDir(conf);
Path tableDir = FSUtils.getTableDir(rootDir, table);
Path rootDir = CommonFSUtils.getRootDir(conf);
Path tableDir = CommonFSUtils.getTableDir(rootDir, table);
// How safe is this call?
List<Path> regionDirs = FSUtils.getRegionDirs(FileSystem.get(conf), tableDir);
@ -310,7 +310,7 @@ public class MobFileCleanerChore extends ScheduledChore {
tableName, Bytes.toString(family));
return;
}
Path mobTableDir = FSUtils.getTableDir(MobUtils.getMobHome(conf), tableName);
Path mobTableDir = CommonFSUtils.getTableDir(MobUtils.getMobHome(conf), tableName);
FileSystem fs = storeFiles.get(0).getFileSystem(conf);
for (Path p : storeFiles) {

View File

@ -31,7 +31,6 @@ import java.util.Date;
import java.util.List;
import java.util.Optional;
import java.util.UUID;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@ -62,11 +61,12 @@ import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ChecksumType;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
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.ImmutableSetMultimap;
import org.apache.hbase.thirdparty.com.google.common.collect.SetMultimap;
@ -282,7 +282,7 @@ public final class MobUtils {
LOG.info("MOB HFiles older than " + expireDate.toGMTString() + " will be deleted!");
FileStatus[] stats = null;
Path mobTableDir = FSUtils.getTableDir(getMobHome(conf), tableName);
Path mobTableDir = CommonFSUtils.getTableDir(getMobHome(conf), tableName);
Path path = getMobFamilyPath(conf, tableName, columnDescriptor.getNameAsString());
try {
stats = fs.listStatus(path);
@ -369,7 +369,7 @@ public final class MobUtils {
* @return The table dir of the mob file.
*/
public static Path getMobTableDir(Path rootDir, TableName tableName) {
return FSUtils.getTableDir(getMobHome(rootDir), tableName);
return CommonFSUtils.getTableDir(getMobHome(rootDir), tableName);
}
/**
@ -391,7 +391,7 @@ public final class MobUtils {
* @return The region dir of the mob files.
*/
public static Path getMobRegionPath(Path rootDir, TableName tableName) {
Path tablePath = FSUtils.getTableDir(getMobHome(rootDir), tableName);
Path tablePath = CommonFSUtils.getTableDir(getMobHome(rootDir), tableName);
RegionInfo regionInfo = getMobRegionInfo(tableName);
return new Path(tablePath, regionInfo.getEncodedName());
}

View File

@ -33,7 +33,6 @@ import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
import java.util.function.Predicate;
import java.util.stream.Collectors;
import org.apache.commons.lang3.builder.HashCodeBuilder;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
@ -47,11 +46,11 @@ import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
import org.apache.hadoop.util.StringUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -201,7 +200,7 @@ public class FileArchiverNotifierImpl implements FileArchiverNotifier {
}
Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(
snapshotName, FSUtils.getRootDir(conf));
snapshotName, CommonFSUtils.getRootDir(conf));
SnapshotDescription sd = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
SnapshotManifest manifest = SnapshotManifest.open(conf, fs, snapshotDir, sd);
// For each region referenced by the snapshot
@ -380,7 +379,7 @@ public class FileArchiverNotifierImpl implements FileArchiverNotifier {
*/
List<SnapshotWithSize> computeSnapshotSizes(List<String> snapshots) throws IOException {
final List<SnapshotWithSize> snapshotSizes = new ArrayList<>(snapshots.size());
final Path rootDir = FSUtils.getRootDir(conf);
final Path rootDir = CommonFSUtils.getRootDir(conf);
// Get the map of store file names to store file path for this table
final Set<String> tableReferencedStoreFiles;

View File

@ -164,6 +164,7 @@ import org.apache.hadoop.hbase.trace.TraceUtil;
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.CommonFSUtils;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HashedBytes;
@ -1049,8 +1050,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
// 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());
Path wrongRegionWALDir = CommonFSUtils.getWrongWALRegionDir(conf,
getRegionInfo().getTable(), getRegionInfo().getEncodedName());
FileSystem walFs = getWalFileSystem();
if (walFs.exists(wrongRegionWALDir)) {
if (!walFs.delete(wrongRegionWALDir, true)) {
@ -1248,11 +1249,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* @param tableDescriptor TableDescriptor of the table
* @param regionInfo encoded name of the region
* @return The HDFS blocks distribution for the given region.
* @throws IOException
*/
public static HDFSBlocksDistribution computeHDFSBlocksDistribution(Configuration conf,
TableDescriptor tableDescriptor, RegionInfo regionInfo) throws IOException {
Path tablePath = FSUtils.getTableDir(FSUtils.getRootDir(conf), tableDescriptor.getTableName());
Path tablePath =
CommonFSUtils.getTableDir(CommonFSUtils.getRootDir(conf), tableDescriptor.getTableName());
return computeHDFSBlocksDistribution(conf, tableDescriptor, regionInfo, tablePath);
}
@ -1978,13 +1979,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
/** @return the WAL {@link HRegionFileSystem} used by this region */
HRegionWALFileSystem getRegionWALFileSystem() throws IOException {
return new HRegionWALFileSystem(conf, getWalFileSystem(),
FSUtils.getWALTableDir(conf, htableDescriptor.getTableName()), fs.getRegionInfo());
CommonFSUtils.getWALTableDir(conf, htableDescriptor.getTableName()), fs.getRegionInfo());
}
/** @return the WAL {@link FileSystem} being used by this region */
FileSystem getWalFileSystem() throws IOException {
if (walFS == null) {
walFS = FSUtils.getWALFileSystem(conf);
walFS = CommonFSUtils.getWALFileSystem(conf);
}
return walFS;
}
@ -1996,7 +1997,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
@VisibleForTesting
public Path getWALRegionDir() throws IOException {
if (regionDir == null) {
regionDir = FSUtils.getWALRegionDir(conf, getRegionInfo().getTable(),
regionDir = CommonFSUtils.getWALRegionDir(conf, getRegionInfo().getTable(),
getRegionInfo().getEncodedName());
}
return regionDir;
@ -4446,7 +4447,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
*/
public void addRegionToSnapshot(SnapshotDescription desc,
ForeignExceptionSnare exnSnare) throws IOException {
Path rootDir = FSUtils.getRootDir(conf);
Path rootDir = CommonFSUtils.getRootDir(conf);
Path snapshotDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(desc, rootDir, conf);
SnapshotManifest manifest = SnapshotManifest.create(conf, getFilesystem(),
@ -4722,10 +4723,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
if (org.apache.commons.lang3.StringUtils.isBlank(specialRecoveredEditsDirStr)) {
FileSystem walFS = getWalFileSystem();
FileSystem rootFS = getFilesystem();
Path wrongRegionWALDir = FSUtils.getWrongWALRegionDir(conf, getRegionInfo().getTable(),
Path wrongRegionWALDir = CommonFSUtils.getWrongWALRegionDir(conf, getRegionInfo().getTable(),
getRegionInfo().getEncodedName());
Path regionWALDir = getWALRegionDir();
Path regionDir = FSUtils.getRegionDirFromRootDir(FSUtils.getRootDir(conf), getRegionInfo());
Path regionDir =
FSUtils.getRegionDirFromRootDir(CommonFSUtils.getRootDir(conf), getRegionInfo());
// We made a mistake in HBASE-20734 so we need to do this dirty hack...
NavigableSet<Path> filesUnderWrongRegionWALDir =
@ -7268,7 +7270,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
(hTableDescriptor == null ? "null" : hTableDescriptor) + ", regionDir=" + rootDir);
createRegionDir(conf, info, rootDir);
FileSystem fs = rootDir.getFileSystem(conf);
Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
Path tableDir = CommonFSUtils.getTableDir(rootDir, info.getTable());
HRegion region = HRegion.newHRegion(tableDir, wal, fs, conf, info, hTableDescriptor, null);
if (initialize) {
region.initialize(null);
@ -7295,7 +7297,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
Path rootDir)
throws IOException {
FileSystem fs = rootDir.getFileSystem(configuration);
Path tableDir = FSUtils.getTableDir(rootDir, ri.getTable());
Path tableDir = CommonFSUtils.getTableDir(rootDir, ri.getTable());
// If directory already exists, will log warning and keep going. Will try to create
// .regioninfo. If one exists, will overwrite.
return HRegionFileSystem.createRegionOnFileSystem(configuration, fs, tableDir, ri);
@ -7348,7 +7350,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
final RegionServerServices rsServices,
final CancelableProgressable reporter)
throws IOException {
return openHRegion(FSUtils.getRootDir(conf), info, htd, wal, conf, rsServices, reporter);
return openHRegion(CommonFSUtils.getRootDir(conf), info, htd, wal, conf, rsServices, reporter);
}
/**
@ -7438,7 +7440,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
final Path rootDir, final RegionInfo info, final TableDescriptor htd, final WAL wal,
final RegionServerServices rsServices, final CancelableProgressable reporter)
throws IOException {
Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
Path tableDir = CommonFSUtils.getTableDir(rootDir, info.getTable());
return openHRegionFromTableDir(conf, fs, tableDir, info, htd, wal, rsServices, reporter);
}
@ -7562,8 +7564,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
LOG.debug("HRegion.Warming up region: " + info);
}
Path rootDir = FSUtils.getRootDir(conf);
Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
Path rootDir = CommonFSUtils.getRootDir(conf);
Path tableDir = CommonFSUtils.getTableDir(rootDir, info.getTable());
FileSystem fs = null;
if (rsServices != null) {

View File

@ -17,6 +17,7 @@
*/
package org.apache.hadoop.hbase.regionserver;
import edu.umd.cs.findbugs.annotations.Nullable;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InterruptedIOException;
@ -26,7 +27,6 @@ import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.UUID;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
@ -46,17 +46,16 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.Reference;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSHDFSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
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 edu.umd.cs.findbugs.annotations.Nullable;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
/**
* View to an on-disk Region.
@ -192,7 +191,7 @@ public class HRegionFileSystem {
* 'COLD', 'WARM', 'HOT', 'ONE_SSD', 'ALL_SSD', 'LAZY_PERSIST'.
*/
public void setStoragePolicy(String familyName, String policyName) {
FSUtils.setStoragePolicy(this.fs, getStoreDir(familyName), policyName);
CommonFSUtils.setStoragePolicy(this.fs, getStoreDir(familyName), policyName);
}
/**
@ -234,7 +233,7 @@ public class HRegionFileSystem {
public Collection<StoreFileInfo> getStoreFiles(final String familyName, final boolean validate)
throws IOException {
Path familyDir = getStoreDir(familyName);
FileStatus[] files = FSUtils.listStatus(this.fs, familyDir);
FileStatus[] files = CommonFSUtils.listStatus(this.fs, familyDir);
if (files == null) {
if (LOG.isTraceEnabled()) {
LOG.trace("No StoreFiles for: " + familyDir);
@ -270,7 +269,7 @@ public class HRegionFileSystem {
final HRegionFileSystem regionfs, final String familyName,
final boolean validate) throws IOException {
Path familyDir = regionfs.getStoreDir(familyName);
List<LocatedFileStatus> locatedFileStatuses = FSUtils.listLocatedStatus(
List<LocatedFileStatus> locatedFileStatuses = CommonFSUtils.listLocatedStatus(
regionfs.getFileSystem(), familyDir);
if (locatedFileStatuses == null) {
if (LOG.isTraceEnabled()) {
@ -328,7 +327,7 @@ public class HRegionFileSystem {
*/
public boolean hasReferences(final String familyName) throws IOException {
Path storeDir = getStoreDir(familyName);
FileStatus[] files = FSUtils.listStatus(fs, storeDir);
FileStatus[] files = CommonFSUtils.listStatus(fs, storeDir);
if (files != null) {
for(FileStatus stat: files) {
if(stat.isDirectory()) {
@ -363,11 +362,12 @@ public class HRegionFileSystem {
* @throws IOException
*/
public Collection<String> getFamilies() throws IOException {
FileStatus[] fds = FSUtils.listStatus(fs, getRegionDir(), new FSUtils.FamilyDirFilter(fs));
FileStatus[] fds =
CommonFSUtils.listStatus(fs, getRegionDir(), new FSUtils.FamilyDirFilter(fs));
if (fds == null) return null;
ArrayList<String> families = new ArrayList<>(fds.length);
for (FileStatus status: fds) {
for (FileStatus status : fds) {
families.add(status.getPath().getName());
}
@ -536,7 +536,7 @@ public class HRegionFileSystem {
// We can't compare FileSystem instances as equals() includes UGI instance
// as part of the comparison and won't work when doing SecureBulkLoad
// TODO deal with viewFS
if (!FSHDFSUtils.isSameHdfs(conf, realSrcFs, desFs)) {
if (!FSUtils.isSameHdfs(conf, realSrcFs, desFs)) {
LOG.info("Bulk-load file " + srcPath + " is on different filesystem than " +
"the destination store. Copying file over to destination filesystem.");
Path tmpPath = createTempName();
@ -583,7 +583,7 @@ public class HRegionFileSystem {
// where we successfully created daughter a but regionserver crashed during
// the creation of region b. In this case, there'll be an orphan daughter
// dir in the filesystem. TOOD: Fix.
FileStatus[] daughters = FSUtils.listStatus(fs, splitdir, new FSUtils.DirFilter(fs));
FileStatus[] daughters = CommonFSUtils.listStatus(fs, splitdir, new FSUtils.DirFilter(fs));
if (daughters != null) {
for (FileStatus daughter: daughters) {
Path daughterDir = new Path(getTableDir(), daughter.getPath().getName());
@ -761,7 +761,7 @@ public class HRegionFileSystem {
!conf.getBoolean(HConstants.ENABLE_DATA_FILE_UMASK, false)) {
return fs.mkdirs(dir);
}
FsPermission perms = FSUtils.getFilePermissions(fs, conf, HConstants.DATA_FILE_UMASK_KEY);
FsPermission perms = CommonFSUtils.getFilePermissions(fs, conf, HConstants.DATA_FILE_UMASK_KEY);
return fs.mkdirs(dir, perms);
}
@ -845,7 +845,7 @@ public class HRegionFileSystem {
* @throws IOException if an unexpected exception occurs
*/
void logFileSystemState(final Logger LOG) throws IOException {
FSUtils.logFileSystemState(fs, this.getRegionDir(), LOG);
CommonFSUtils.logFileSystemState(fs, this.getRegionDir(), LOG);
}
/**
@ -876,18 +876,16 @@ public class HRegionFileSystem {
/**
* Write the .regioninfo file on-disk.
* <p/>
* Overwrites if exists already.
*/
private static void writeRegionInfoFileContent(final Configuration conf, final FileSystem fs,
final Path regionInfoFile, final byte[] content) throws IOException {
// First check to get the permissions
FsPermission perms = FSUtils.getFilePermissions(fs, conf, HConstants.DATA_FILE_UMASK_KEY);
FsPermission perms = CommonFSUtils.getFilePermissions(fs, conf, HConstants.DATA_FILE_UMASK_KEY);
// Write the RegionInfo file content
FSDataOutputStream out = FSUtils.create(conf, fs, regionInfoFile, perms, null);
try {
try (FSDataOutputStream out = FSUtils.create(conf, fs, regionInfoFile, perms, null)) {
out.write(content);
} finally {
out.close();
}
}
@ -965,8 +963,8 @@ public class HRegionFileSystem {
// close the created regioninfo file in the .tmp directory then on next
// creation we will be getting AlreadyCreatedException.
// Hence delete and create the file if exists.
if (FSUtils.isExists(fs, tmpPath)) {
FSUtils.delete(fs, tmpPath, true);
if (CommonFSUtils.isExists(fs, tmpPath)) {
CommonFSUtils.delete(fs, tmpPath, true);
}
// Write HRI to a file in case we need to recover hbase:meta
@ -1079,7 +1077,7 @@ public class HRegionFileSystem {
}
// Archive region
Path rootDir = FSUtils.getRootDir(conf);
Path rootDir = CommonFSUtils.getRootDir(conf);
HFileArchiver.archiveRegion(fs, rootDir, tableDir, regionDir);
// Delete empty region dir

View File

@ -154,6 +154,7 @@ import org.apache.hadoop.hbase.trace.SpanReceiverHost;
import org.apache.hadoop.hbase.trace.TraceUtil;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.CompressionTest;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
@ -714,15 +715,15 @@ public class HRegionServer extends HasThread implements
// Get fs instance used by this RS. Do we use checksum verification in the hbase? If hbase
// checksum verification enabled, then automatically switch off hdfs checksum verification.
boolean useHBaseChecksum = conf.getBoolean(HConstants.HBASE_CHECKSUM_VERIFICATION, true);
FSUtils.setFsDefault(this.conf, FSUtils.getWALRootDir(this.conf));
CommonFSUtils.setFsDefault(this.conf, CommonFSUtils.getWALRootDir(this.conf));
this.walFs = new HFileSystem(this.conf, useHBaseChecksum);
this.walRootDir = FSUtils.getWALRootDir(this.conf);
this.walRootDir = CommonFSUtils.getWALRootDir(this.conf);
// Set 'fs.defaultFS' to match the filesystem on hbase.rootdir else
// underlying hadoop hdfs accessors will be going against wrong filesystem
// (unless all is set to defaults).
FSUtils.setFsDefault(this.conf, FSUtils.getRootDir(this.conf));
CommonFSUtils.setFsDefault(this.conf, CommonFSUtils.getRootDir(this.conf));
this.dataFs = new HFileSystem(this.conf, useHBaseChecksum);
this.dataRootDir = FSUtils.getRootDir(this.conf);
this.dataRootDir = CommonFSUtils.getRootDir(this.conf);
this.tableDescriptors =
new FSTableDescriptors(this.dataFs, this.dataRootDir, !canUpdateTableDescriptor(), false);
if (this instanceof HMaster) {

View File

@ -46,7 +46,7 @@ import org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier;
import org.apache.hadoop.hbase.security.token.ClientTokenUtil;
import org.apache.hadoop.hbase.security.token.FsDelegationToken;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSHDFSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Methods;
import org.apache.hadoop.hbase.util.Pair;
@ -125,7 +125,7 @@ public class SecureBulkLoadManager {
userProvider = UserProvider.instantiate(conf);
ugiReferenceCounter = new ConcurrentHashMap<>();
fs = FileSystem.get(conf);
baseStagingDir = new Path(FSUtils.getRootDir(conf), HConstants.BULKLOAD_STAGING_DIR_NAME);
baseStagingDir = new Path(CommonFSUtils.getRootDir(conf), HConstants.BULKLOAD_STAGING_DIR_NAME);
if (conf.get("hbase.bulkload.staging.dir") != null) {
LOG.warn("hbase.bulkload.staging.dir " + " is deprecated. Bulkload staging directory is "
@ -383,7 +383,7 @@ public class SecureBulkLoadManager {
}
// Check to see if the source and target filesystems are the same
if (!FSHDFSUtils.isSameHdfs(conf, srcFs, fs)) {
if (!FSUtils.isSameHdfs(conf, srcFs, fs)) {
LOG.debug("Bulk-load file " + srcPath + " is on different filesystem than " +
"the destination filesystem. Copying file over to destination staging dir.");
FileUtil.copy(srcFs, p, fs, stageP, false, conf);
@ -422,7 +422,7 @@ public class SecureBulkLoadManager {
if (srcFs == null) {
srcFs = FileSystem.newInstance(p.toUri(), conf);
}
if (!FSHDFSUtils.isSameHdfs(conf, srcFs, fs)) {
if (!FSUtils.isSameHdfs(conf, srcFs, fs)) {
// files are copied so no need to move them back
return;
}

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerImpl;
import org.apache.hadoop.hbase.replication.ReplicationUtils;
import org.apache.hadoop.hbase.replication.SyncReplicationState;
import org.apache.hadoop.hbase.util.CancelableProgressable;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.ExceptionUtil;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Pair;
@ -115,10 +116,11 @@ public class SplitLogWorker implements Runnable {
Path remoteWALDirForPeer = ReplicationUtils.getPeerRemoteWALDir(remoteWALDir, peerId);
Path tmpRemoteWAL = new Path(remoteWALDirForPeer, filename + ".tmp");
FileSystem remoteFs = ReplicationUtils.getRemoteWALFileSystem(conf, remoteWALDir);
try (FSDataInputStream in = fs.open(walFile); @SuppressWarnings("deprecation")
try (FSDataInputStream in = fs.open(walFile);
FSDataOutputStream out = remoteFs.createNonRecursive(tmpRemoteWAL, true,
FSUtils.getDefaultBufferSize(remoteFs), remoteFs.getDefaultReplication(tmpRemoteWAL),
remoteFs.getDefaultBlockSize(tmpRemoteWAL), null)) {
CommonFSUtils.getDefaultBufferSize(remoteFs),
remoteFs.getDefaultReplication(tmpRemoteWAL), remoteFs.getDefaultBlockSize(tmpRemoteWAL),
null)) {
IOUtils.copy(in, out);
}
Path toCommitRemoteWAL =
@ -157,7 +159,7 @@ public class SplitLogWorker implements Runnable {
Path walDir;
FileSystem fs;
try {
walDir = FSUtils.getWALRootDir(conf);
walDir = CommonFSUtils.getWALRootDir(conf);
fs = walDir.getFileSystem(conf);
} catch (IOException e) {
LOG.warn("could not find root dir or fs", e);

View File

@ -27,6 +27,7 @@ import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAX_SEQ_ID_KEY;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.MOB_CELLS_COUNT;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.MOB_FILE_REFS;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.TIMERANGE_KEY;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.Collection;
@ -37,7 +38,6 @@ import java.util.UUID;
import java.util.function.Supplier;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -57,15 +57,17 @@ import org.apache.hadoop.hbase.util.BloomFilterFactory;
import org.apache.hadoop.hbase.util.BloomFilterUtil;
import org.apache.hadoop.hbase.util.BloomFilterWriter;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.RowBloomContext;
import org.apache.hadoop.hbase.util.RowColBloomContext;
import org.apache.hadoop.hbase.util.RowPrefixFixedLengthBloomContext;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
import org.apache.hbase.thirdparty.com.google.common.collect.SetMultimap;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
/**
@ -542,7 +544,7 @@ public class StoreFileWriter implements CellSink, ShipperListener {
if (null == policyName) {
policyName = this.conf.get(HStore.BLOCK_STORAGE_POLICY_KEY);
}
FSUtils.setStoragePolicy(this.fs, dir, policyName);
CommonFSUtils.setStoragePolicy(this.fs, dir, policyName);
if (filePath == null) {
filePath = getUniqueFile(fs, dir);

View File

@ -68,7 +68,6 @@ import org.apache.hadoop.hbase.trace.TraceUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.hadoop.hbase.wal.WAL;
@ -1242,7 +1241,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
abstract int getLogReplication();
private static void split(final Configuration conf, final Path p) throws IOException {
FileSystem fs = FSUtils.getWALFileSystem(conf);
FileSystem fs = CommonFSUtils.getWALFileSystem(conf);
if (!fs.exists(p)) {
throw new FileNotFoundException(p.toString());
}
@ -1250,7 +1249,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
throw new IOException(p + " is not a directory");
}
final Path baseDir = FSUtils.getWALRootDir(conf);
final Path baseDir = CommonFSUtils.getWALRootDir(conf);
Path archiveDir = new Path(baseDir, HConstants.HREGION_OLDLOGDIR_NAME);
if (conf.getBoolean(AbstractFSWALProvider.SEPARATE_OLDLOGDIR,
AbstractFSWALProvider.DEFAULT_SEPARATE_OLDLOGDIR)) {
@ -1292,7 +1291,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
for (int i = 1; i < args.length; i++) {
try {
Path logPath = new Path(args[i]);
FSUtils.setFsDefault(conf, logPath);
CommonFSUtils.setFsDefault(conf, logPath);
split(conf, logPath);
} catch (IOException t) {
t.printStackTrace(System.err);

View File

@ -25,17 +25,12 @@ import java.io.OutputStream;
import java.security.Key;
import java.security.SecureRandom;
import java.util.concurrent.atomic.AtomicLong;
import javax.crypto.spec.SecretKeySpec;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.codec.Codec;
import org.apache.hadoop.hbase.io.crypto.Cipher;
import org.apache.hadoop.hbase.io.crypto.Encryption;
@ -43,12 +38,17 @@ import org.apache.hadoop.hbase.io.crypto.Encryptor;
import org.apache.hadoop.hbase.io.util.LRUDictionary;
import org.apache.hadoop.hbase.security.EncryptionUtil;
import org.apache.hadoop.hbase.security.User;
import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils.StreamLacksCapabilityException;
import org.apache.hadoop.hbase.util.EncryptionTest;
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.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer;
/**
* Base class for Protobuf log writer.
@ -144,8 +144,8 @@ public abstract class AbstractProtobufLogWriter {
boolean doCompress = conf.getBoolean(HConstants.ENABLE_WAL_COMPRESSION, false);
if (doCompress) {
try {
this.compressionContext = new CompressionContext(LRUDictionary.class,
FSUtils.isRecoveredEdits(path),
this.compressionContext =
new CompressionContext(LRUDictionary.class, CommonFSUtils.isRecoveredEdits(path),
conf.getBoolean(CompressionContext.ENABLE_WAL_TAGS_COMPRESSION, true));
} catch (Exception e) {
throw new IOException("Failed to initiate CompressionContext", e);
@ -159,9 +159,9 @@ public abstract class AbstractProtobufLogWriter {
this.conf = conf;
boolean doCompress = initializeCompressionContext(conf, path);
this.trailerWarnSize = conf.getInt(WAL_TRAILER_WARN_SIZE, DEFAULT_WAL_TRAILER_WARN_SIZE);
int bufferSize = FSUtils.getDefaultBufferSize(fs);
int bufferSize = CommonFSUtils.getDefaultBufferSize(fs);
short replication = (short) conf.getInt("hbase.regionserver.hlog.replication",
FSUtils.getDefaultReplication(fs, path));
CommonFSUtils.getDefaultReplication(fs, path));
initOutput(fs, path, overwritable, bufferSize, replication, blocksize);

View File

@ -47,7 +47,7 @@ import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.trace.TraceUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.wal.FSHLogProvider;
@ -212,7 +212,7 @@ public class FSHLog extends AbstractFSWAL<Writer> {
final boolean failIfWALExists, final String prefix, final String suffix) throws IOException {
super(fs, rootDir, logDir, archiveDir, conf, listeners, failIfWALExists, prefix, suffix);
this.minTolerableReplication = conf.getInt("hbase.regionserver.hlog.tolerable.lowreplication",
FSUtils.getDefaultReplication(fs, this.walDir));
CommonFSUtils.getDefaultReplication(fs, this.walDir));
this.lowReplicationRollLimit = conf.getInt("hbase.regionserver.hlog.lowreplication.rolllimit",
5);
this.closeErrorsTolerated = conf.getInt("hbase.regionserver.logroll.errors.tolerated", 2);
@ -422,7 +422,7 @@ public class FSHLog extends AbstractFSWAL<Writer> {
}
if (LOG.isDebugEnabled()) {
LOG.debug("Closing WAL writer in " + FSUtils.getPath(walDir));
LOG.debug("Closing WAL writer in " + CommonFSUtils.getPath(walDir));
}
if (this.writer != null) {
this.writer.close();

View File

@ -20,7 +20,6 @@
package org.apache.hadoop.hbase.regionserver.wal;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
@ -28,7 +27,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.io.util.LRUDictionary;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.hadoop.hbase.wal.WAL.Entry;
import org.apache.yetus.audience.InterfaceAudience;
@ -70,7 +69,7 @@ public abstract class ReaderBase implements AbstractFSWALProvider.Reader {
try {
if (compressionContext == null) {
compressionContext = new CompressionContext(LRUDictionary.class,
FSUtils.isRecoveredEdits(path), hasTagCompression());
CommonFSUtils.isRecoveredEdits(path), hasTagCompression());
} else {
compressionContext.clear();
}

View File

@ -57,16 +57,16 @@ import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint;
import org.apache.hadoop.hbase.replication.regionserver.ReplicationSinkManager.SinkPeer;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.wal.WAL.Entry;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
/**
* A {@link org.apache.hadoop.hbase.replication.ReplicationEndpoint}
@ -177,7 +177,7 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
replicationClusterId = this.conf.get(HConstants.REPLICATION_CLUSTER_ID);
}
// Construct base namespace directory and hfile archive directory path
Path rootDir = FSUtils.getRootDir(conf);
Path rootDir = CommonFSUtils.getRootDir(conf);
Path baseNSDir = new Path(HConstants.BASE_NAMESPACE_DIR);
baseNamespaceDir = new Path(rootDir, baseNSDir);
hfileArchiveDir = new Path(rootDir, new Path(HConstants.HFILE_ARCHIVE_DIRECTORY, baseNSDir));

View File

@ -50,12 +50,13 @@ import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
import org.apache.hadoop.hbase.tool.BulkLoadHFiles.LoadQueueItem;
import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
/**
@ -108,7 +109,8 @@ public class HFileReplicator implements Closeable {
userProvider = UserProvider.instantiate(conf);
fsDelegationToken = new FsDelegationToken(userProvider, "renewer");
this.hbaseStagingDir = new Path(FSUtils.getRootDir(conf), HConstants.BULKLOAD_STAGING_DIR_NAME);
this.hbaseStagingDir =
new Path(CommonFSUtils.getRootDir(conf), HConstants.BULKLOAD_STAGING_DIR_NAME);
this.maxCopyThreads =
this.conf.getInt(REPLICATION_BULKLOAD_COPY_MAXTHREADS_KEY,
REPLICATION_BULKLOAD_COPY_MAXTHREADS_DEFAULT);

View File

@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.replication.ReplicationPeer;
import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
@ -84,7 +84,7 @@ public class RecoveredReplicationSource extends ReplicationSource {
// to look at)
List<ServerName> deadRegionServers = this.replicationQueueInfo.getDeadRegionServers();
LOG.info("NB dead servers : " + deadRegionServers.size());
final Path walDir = FSUtils.getWALRootDir(conf);
final Path walDir = CommonFSUtils.getWALRootDir(conf);
for (ServerName curDeadServerName : deadRegionServers) {
final Path deadRsDirectory =
new Path(walDir, AbstractFSWALProvider.getWALDirectoryName(curDeadServerName

View File

@ -140,7 +140,7 @@ public class ReplaySyncReplicationWALCallable implements RSProcedureCallable {
Path path = new Path(rs.getWALRootDir(), wal);
long length = rs.getWALFileSystem().getFileStatus(path).getLen();
try {
FSUtils.getInstance(fs, conf).recoverFileLease(fs, path, conf);
FSUtils.recoverFileLease(fs, path, conf);
return WALFactory.createReader(rs.getWALFileSystem(), path, rs.getConfiguration());
} catch (EOFException e) {
if (length <= 0) {

View File

@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.AsyncClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
@ -75,8 +75,8 @@ public class ReplicationSyncUp extends Configured implements Tool {
Configuration conf = getConf();
try (ZKWatcher zkw =
new ZKWatcher(conf, "syncupReplication" + System.currentTimeMillis(), abortable, true)) {
Path walRootDir = FSUtils.getWALRootDir(conf);
FileSystem fs = FSUtils.getWALFileSystem(conf);
Path walRootDir = CommonFSUtils.getWALRootDir(conf);
FileSystem fs = CommonFSUtils.getWALFileSystem(conf);
Path oldLogDir = new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
Path logDir = new Path(walRootDir, HConstants.HREGION_LOGDIR_NAME);

View File

@ -384,10 +384,8 @@ class WALEntryStream implements Closeable {
// For HBASE-15019
private void recoverLease(final Configuration conf, final Path path) {
try {
final FileSystem dfs = CommonFSUtils.getWALFileSystem(conf);
FSUtils fsUtils = FSUtils.getInstance(dfs, conf);
fsUtils.recoverFileLease(dfs, path, conf, new CancelableProgressable() {
FSUtils.recoverFileLease(dfs, path, conf, new CancelableProgressable() {
@Override
public boolean progress() {
LOG.debug("recover WAL lease: " + path);

View File

@ -33,7 +33,6 @@ import java.util.Map.Entry;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.ThreadPoolExecutor;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@ -60,6 +59,7 @@ import org.apache.hadoop.hbase.security.access.Permission;
import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil;
import org.apache.hadoop.hbase.security.access.TablePermission;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
import org.apache.hadoop.hbase.util.Pair;
@ -67,7 +67,9 @@ import org.apache.hadoop.io.IOUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
@ -146,15 +148,10 @@ public class RestoreSnapshotHelper {
this(conf, fs, manifest, tableDescriptor, rootDir, monitor, status, true);
}
public RestoreSnapshotHelper(final Configuration conf,
final FileSystem fs,
final SnapshotManifest manifest,
final TableDescriptor tableDescriptor,
final Path rootDir,
final ForeignExceptionDispatcher monitor,
final MonitoredTask status,
final boolean createBackRefs)
{
public RestoreSnapshotHelper(final Configuration conf, final FileSystem fs,
final SnapshotManifest manifest, final TableDescriptor tableDescriptor, final Path rootDir,
final ForeignExceptionDispatcher monitor, final MonitoredTask status,
final boolean createBackRefs) {
this.fs = fs;
this.conf = conf;
this.snapshotManifest = manifest;
@ -162,7 +159,7 @@ public class RestoreSnapshotHelper {
this.snapshotTable = TableName.valueOf(snapshotDesc.getTable());
this.tableDesc = tableDescriptor;
this.rootDir = rootDir;
this.tableDir = FSUtils.getTableDir(rootDir, tableDesc.getTableName());
this.tableDir = CommonFSUtils.getTableDir(rootDir, tableDesc.getTableName());
this.monitor = monitor;
this.status = status;
this.createBackRefs = createBackRefs;
@ -565,8 +562,10 @@ public class RestoreSnapshotHelper {
* @return The set of files in the specified family directory.
*/
private Set<String> getTableRegionFamilyFiles(final Path familyDir) throws IOException {
FileStatus[] hfiles = FSUtils.listStatus(fs, familyDir);
if (hfiles == null) return Collections.emptySet();
FileStatus[] hfiles = CommonFSUtils.listStatus(fs, familyDir);
if (hfiles == null) {
return Collections.emptySet();
}
Set<String> familyFiles = new HashSet<>(hfiles.length);
for (int i = 0; i < hfiles.length; ++i) {
@ -807,8 +806,11 @@ public class RestoreSnapshotHelper {
*/
private List<RegionInfo> getTableRegions() throws IOException {
LOG.debug("get table regions: " + tableDir);
FileStatus[] regionDirs = FSUtils.listStatus(fs, tableDir, new FSUtils.RegionDirFilter(fs));
if (regionDirs == null) return null;
FileStatus[] regionDirs =
CommonFSUtils.listStatus(fs, tableDir, new FSUtils.RegionDirFilter(fs));
if (regionDirs == null) {
return null;
}
List<RegionInfo> regions = new ArrayList<>(regionDirs.length);
for (int i = 0; i < regionDirs.length; ++i) {
@ -857,7 +859,7 @@ public class RestoreSnapshotHelper {
if (LOG.isDebugEnabled()) {
LOG.debug("Restored table dir:" + restoreDir);
FSUtils.logFileSystemState(fs, restoreDir, LOG);
CommonFSUtils.logFileSystemState(fs, restoreDir, LOG);
}
return metaChanges;
}

View File

@ -21,7 +21,6 @@ import java.io.IOException;
import java.security.PrivilegedExceptionAction;
import java.util.Collections;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
@ -37,6 +36,7 @@ import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.access.PermissionStorage;
import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil;
import org.apache.hadoop.hbase.security.access.UserPermission;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.yetus.audience.InterfaceAudience;
@ -44,6 +44,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
@ -262,7 +263,7 @@ public final class SnapshotDescriptionUtils {
*/
public static boolean isWithinDefaultWorkingDir(final Path workingDir, Configuration conf)
throws IOException {
Path defaultWorkingDir = getDefaultWorkingSnapshotDir(FSUtils.getRootDir(conf));
Path defaultWorkingDir = getDefaultWorkingSnapshotDir(CommonFSUtils.getRootDir(conf));
return workingDir.equals(defaultWorkingDir) || isSubDirectoryOf(workingDir, defaultWorkingDir);
}
@ -337,11 +338,11 @@ public final class SnapshotDescriptionUtils {
*/
public static void writeSnapshotInfo(SnapshotDescription snapshot, Path workingDir, FileSystem fs)
throws IOException {
FsPermission perms = FSUtils.getFilePermissions(fs, fs.getConf(),
FsPermission perms = CommonFSUtils.getFilePermissions(fs, fs.getConf(),
HConstants.DATA_FILE_UMASK_KEY);
Path snapshotInfo = new Path(workingDir, SnapshotDescriptionUtils.SNAPSHOTINFO_FILE);
try {
FSDataOutputStream out = FSUtils.create(fs, snapshotInfo, perms, true);
FSDataOutputStream out = CommonFSUtils.create(fs, snapshotInfo, perms, true);
try {
snapshot.writeTo(out);
} finally {

View File

@ -31,7 +31,6 @@ import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@ -42,13 +41,15 @@ import org.apache.hadoop.hbase.client.SnapshotDescription;
import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.io.WALLink;
import org.apache.hadoop.hbase.util.AbstractHBaseTool;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.util.StringUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
import org.apache.hbase.thirdparty.org.apache.commons.cli.Option;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
@ -272,7 +273,7 @@ public final class SnapshotInfo extends AbstractHBaseTool {
Path parentDir = filePath.getParent();
Path backRefDir = HFileLink.getBackReferencesDir(parentDir, filePath.getName());
try {
if (FSUtils.listStatus(fs, backRefDir) == null) {
if (CommonFSUtils.listStatus(fs, backRefDir) == null) {
return false;
}
} catch (IOException e) {
@ -367,8 +368,8 @@ public final class SnapshotInfo extends AbstractHBaseTool {
public int doWork() throws IOException, InterruptedException {
if (remoteDir != null) {
URI defaultFs = remoteDir.getFileSystem(conf).getUri();
FSUtils.setFsDefault(conf, new Path(defaultFs));
FSUtils.setRootDir(conf, remoteDir);
CommonFSUtils.setFsDefault(conf, new Path(defaultFs));
CommonFSUtils.setRootDir(conf, remoteDir);
}
// List Available Snapshots
@ -384,7 +385,7 @@ public final class SnapshotInfo extends AbstractHBaseTool {
return 0;
}
rootDir = FSUtils.getRootDir(conf);
rootDir = CommonFSUtils.getRootDir(conf);
fs = FileSystem.get(rootDir.toUri(), conf);
LOG.debug("fs=" + fs.getUri().toString() + " root=" + rootDir);
@ -568,7 +569,7 @@ public final class SnapshotInfo extends AbstractHBaseTool {
public static SnapshotStats getSnapshotStats(final Configuration conf,
final SnapshotProtos.SnapshotDescription snapshotDesc,
final Map<Path, Integer> filesMap) throws IOException {
Path rootDir = FSUtils.getRootDir(conf);
Path rootDir = CommonFSUtils.getRootDir(conf);
FileSystem fs = FileSystem.get(rootDir.toUri(), conf);
Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotDesc, rootDir);
SnapshotManifest manifest = SnapshotManifest.open(conf, fs, snapshotDir, snapshotDesc);
@ -592,7 +593,7 @@ public final class SnapshotInfo extends AbstractHBaseTool {
*/
public static List<SnapshotDescription> getSnapshotList(final Configuration conf)
throws IOException {
Path rootDir = FSUtils.getRootDir(conf);
Path rootDir = CommonFSUtils.getRootDir(conf);
FileSystem fs = FileSystem.get(rootDir.toUri(), conf);
Path snapshotDir = SnapshotDescriptionUtils.getSnapshotsDir(rootDir);
FileStatus[] snapshots = fs.listStatus(snapshotDir,
@ -623,7 +624,7 @@ public final class SnapshotInfo extends AbstractHBaseTool {
final AtomicLong uniqueHFilesMobSize) throws IOException {
SnapshotProtos.SnapshotDescription snapshotDesc =
ProtobufUtil.createHBaseProtosSnapshotDesc(snapshot);
Path rootDir = FSUtils.getRootDir(conf);
Path rootDir = CommonFSUtils.getRootDir(conf);
final FileSystem fs = FileSystem.get(rootDir.toUri(), conf);
Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotDesc, rootDir);

View File

@ -27,7 +27,6 @@ import java.util.List;
import java.util.Map;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
@ -45,15 +44,17 @@ import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.protobuf.CodedInputStream;
import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDataManifest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
@ -287,7 +288,7 @@ public final class SnapshotManifest {
Path baseDir = tableDir;
// Open the RegionFS
if (isMobRegion) {
baseDir = FSUtils.getTableDir(MobUtils.getMobHome(conf), regionInfo.getTable());
baseDir = CommonFSUtils.getTableDir(MobUtils.getMobHome(conf), regionInfo.getTable());
}
HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(conf, rootFs,
baseDir, regionInfo, true);
@ -337,7 +338,7 @@ public final class SnapshotManifest {
}
private List<StoreFileInfo> getStoreFiles(Path storeDir) throws IOException {
FileStatus[] stats = FSUtils.listStatus(rootFs, storeDir);
FileStatus[] stats = CommonFSUtils.listStatus(rootFs, storeDir);
if (stats == null) return null;
ArrayList<StoreFileInfo> storeFiles = new ArrayList<>(stats.length);

View File

@ -27,7 +27,6 @@ import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executor;
import java.util.concurrent.ExecutorCompletionService;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@ -36,11 +35,14 @@ import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CommonFSUtils;
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.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
@ -123,7 +125,8 @@ public final class SnapshotManifestV1 {
static List<SnapshotRegionManifest> loadRegionManifests(final Configuration conf,
final Executor executor,final FileSystem fs, final Path snapshotDir,
final SnapshotDescription desc) throws IOException {
FileStatus[] regions = FSUtils.listStatus(fs, snapshotDir, new FSUtils.RegionDirFilter(fs));
FileStatus[] regions =
CommonFSUtils.listStatus(fs, snapshotDir, new FSUtils.RegionDirFilter(fs));
if (regions == null) {
LOG.debug("No regions under directory:" + snapshotDir);
return null;

View File

@ -26,7 +26,6 @@ import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executor;
import java.util.concurrent.ExecutorCompletionService;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
@ -36,13 +35,15 @@ 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.StoreFileInfo;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.protobuf.CodedInputStream;
import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
@ -141,7 +142,7 @@ public final class SnapshotManifestV2 {
static List<SnapshotRegionManifest> loadRegionManifests(final Configuration conf,
final Executor executor, final FileSystem fs, final Path snapshotDir,
final SnapshotDescription desc, final int manifestSizeLimit) throws IOException {
FileStatus[] manifestFiles = FSUtils.listStatus(fs, snapshotDir, new PathFilter() {
FileStatus[] manifestFiles = CommonFSUtils.listStatus(fs, snapshotDir, new PathFilter() {
@Override
public boolean accept(Path path) {
return path.getName().startsWith(SNAPSHOT_MANIFEST_PREFIX);

View File

@ -24,13 +24,13 @@ import java.util.List;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.util.AbstractHBaseTool;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker;
@ -53,7 +53,7 @@ public class HFileContentValidator extends AbstractHBaseTool {
* @throws IOException if a remote or network exception occurs
*/
private boolean validateHFileContent(Configuration conf) throws IOException {
FileSystem fileSystem = FSUtils.getCurrentFileSystem(conf);
FileSystem fileSystem = CommonFSUtils.getCurrentFileSystem(conf);
ExecutorService threadPool = createThreadPool(conf);
HFileCorruptionChecker checker;
@ -61,7 +61,7 @@ public class HFileContentValidator extends AbstractHBaseTool {
try {
checker = new HFileCorruptionChecker(conf, threadPool, false);
Path rootDir = FSUtils.getRootDir(conf);
Path rootDir = CommonFSUtils.getRootDir(conf);
LOG.info("Validating HFile contents under {}", rootDir);
Collection<Path> tableDirs = FSUtils.getTableDirs(fileSystem, rootDir);

View File

@ -1,336 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.util;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.lang.reflect.Method;
import java.net.InetSocketAddress;
import java.net.URI;
import java.util.Collection;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FilterFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
/**
* Implementation for hdfs
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class FSHDFSUtils extends FSUtils {
private static final Logger LOG = LoggerFactory.getLogger(FSHDFSUtils.class);
private static Class dfsUtilClazz;
private static Method getNNAddressesMethod;
/**
* @param fs
* @param conf
* @return A set containing all namenode addresses of fs
*/
private static Set<InetSocketAddress> getNNAddresses(DistributedFileSystem fs,
Configuration conf) {
Set<InetSocketAddress> addresses = new HashSet<>();
String serviceName = fs.getCanonicalServiceName();
if (serviceName.startsWith("ha-hdfs")) {
try {
if (dfsUtilClazz == null) {
dfsUtilClazz = Class.forName("org.apache.hadoop.hdfs.DFSUtil");
}
if (getNNAddressesMethod == null) {
try {
// getNNServiceRpcAddressesForCluster is available only in version
// equal to or later than Hadoop 2.6
getNNAddressesMethod =
dfsUtilClazz.getMethod("getNNServiceRpcAddressesForCluster", Configuration.class);
} catch (NoSuchMethodException e) {
// If hadoop version is older than hadoop 2.6
getNNAddressesMethod =
dfsUtilClazz.getMethod("getNNServiceRpcAddresses", Configuration.class);
}
}
Map<String, Map<String, InetSocketAddress>> addressMap =
(Map<String, Map<String, InetSocketAddress>>) getNNAddressesMethod
.invoke(null, conf);
String nameService = serviceName.substring(serviceName.indexOf(":") + 1);
if (addressMap.containsKey(nameService)) {
Map<String, InetSocketAddress> nnMap = addressMap.get(nameService);
for (Map.Entry<String, InetSocketAddress> e2 : nnMap.entrySet()) {
InetSocketAddress addr = e2.getValue();
addresses.add(addr);
}
}
} catch (Exception e) {
LOG.warn("DFSUtil.getNNServiceRpcAddresses failed. serviceName=" + serviceName, e);
}
} else {
URI uri = fs.getUri();
int port = uri.getPort();
if (port < 0) {
int idx = serviceName.indexOf(':');
port = Integer.parseInt(serviceName.substring(idx+1));
}
InetSocketAddress addr = new InetSocketAddress(uri.getHost(), port);
addresses.add(addr);
}
return addresses;
}
/**
* @param conf the Configuration of HBase
* @param srcFs
* @param desFs
* @return Whether srcFs and desFs are on same hdfs or not
*/
public static boolean isSameHdfs(Configuration conf, FileSystem srcFs, FileSystem desFs) {
// By getCanonicalServiceName, we could make sure both srcFs and desFs
// show a unified format which contains scheme, host and port.
String srcServiceName = srcFs.getCanonicalServiceName();
String desServiceName = desFs.getCanonicalServiceName();
if (srcServiceName == null || desServiceName == null) {
return false;
}
if (srcServiceName.equals(desServiceName)) {
return true;
}
if (srcServiceName.startsWith("ha-hdfs") && desServiceName.startsWith("ha-hdfs")) {
Collection<String> internalNameServices =
conf.getTrimmedStringCollection("dfs.internal.nameservices");
if (!internalNameServices.isEmpty()) {
if (internalNameServices.contains(srcServiceName.split(":")[1])) {
return true;
} else {
return false;
}
}
}
if (srcFs instanceof DistributedFileSystem && desFs instanceof DistributedFileSystem) {
//If one serviceName is an HA format while the other is a non-HA format,
// maybe they refer to the same FileSystem.
//For example, srcFs is "ha-hdfs://nameservices" and desFs is "hdfs://activeNamenode:port"
Set<InetSocketAddress> srcAddrs = getNNAddresses((DistributedFileSystem) srcFs, conf);
Set<InetSocketAddress> desAddrs = getNNAddresses((DistributedFileSystem) desFs, conf);
if (Sets.intersection(srcAddrs, desAddrs).size() > 0) {
return true;
}
}
return false;
}
/**
* Recover the lease from HDFS, retrying multiple times.
*/
@Override
public void recoverFileLease(FileSystem fs, Path p, Configuration conf,
CancelableProgressable reporter) throws IOException {
if (fs instanceof FilterFileSystem) {
fs = ((FilterFileSystem) fs).getRawFileSystem();
}
// lease recovery not needed for local file system case.
if (!(fs instanceof DistributedFileSystem)) {
return;
}
recoverDFSFileLease((DistributedFileSystem) fs, p, conf, reporter);
}
/*
* Run the dfs recover lease. recoverLease is asynchronous. It returns:
* -false when it starts the lease recovery (i.e. lease recovery not *yet* done)
* - true when the lease recovery has succeeded or the file is closed.
* But, we have to be careful. Each time we call recoverLease, it starts the recover lease
* process over from the beginning. We could put ourselves in a situation where we are
* doing nothing but starting a recovery, interrupting it to start again, and so on.
* The findings over in HBASE-8354 have it that the namenode will try to recover the lease
* on the file's primary node. If all is well, it should return near immediately. But,
* as is common, it is the very primary node that has crashed and so the namenode will be
* stuck waiting on a socket timeout before it will ask another datanode to start the
* recovery. It does not help if we call recoverLease in the meantime and in particular,
* subsequent to the socket timeout, a recoverLease invocation will cause us to start
* over from square one (possibly waiting on socket timeout against primary node). So,
* in the below, we do the following:
* 1. Call recoverLease.
* 2. If it returns true, break.
* 3. If it returns false, wait a few seconds and then call it again.
* 4. If it returns true, break.
* 5. If it returns false, wait for what we think the datanode socket timeout is
* (configurable) and then try again.
* 6. If it returns true, break.
* 7. If it returns false, repeat starting at step 5. above.
*
* If HDFS-4525 is available, call it every second and we might be able to exit early.
*/
boolean recoverDFSFileLease(final DistributedFileSystem dfs, final Path p,
final Configuration conf, final CancelableProgressable reporter)
throws IOException {
LOG.info("Recover lease on dfs file " + p);
long startWaiting = EnvironmentEdgeManager.currentTime();
// Default is 15 minutes. It's huge, but the idea is that if we have a major issue, HDFS
// usually needs 10 minutes before marking the nodes as dead. So we're putting ourselves
// beyond that limit 'to be safe'.
long recoveryTimeout = conf.getInt("hbase.lease.recovery.timeout", 900000) + startWaiting;
// This setting should be a little bit above what the cluster dfs heartbeat is set to.
long firstPause = conf.getInt("hbase.lease.recovery.first.pause", 4000);
// This should be set to how long it'll take for us to timeout against primary datanode if it
// is dead. We set it to 64 seconds, 4 second than the default READ_TIMEOUT in HDFS, the
// default value for DFS_CLIENT_SOCKET_TIMEOUT_KEY. If recovery is still failing after this
// timeout, then further recovery will take liner backoff with this base, to avoid endless
// preemptions when this value is not properly configured.
long subsequentPauseBase = conf.getLong("hbase.lease.recovery.dfs.timeout", 64 * 1000);
Method isFileClosedMeth = null;
// whether we need to look for isFileClosed method
boolean findIsFileClosedMeth = true;
boolean recovered = false;
// We break the loop if we succeed the lease recovery, timeout, or we throw an exception.
for (int nbAttempt = 0; !recovered; nbAttempt++) {
recovered = recoverLease(dfs, nbAttempt, p, startWaiting);
if (recovered) break;
checkIfCancelled(reporter);
if (checkIfTimedout(conf, recoveryTimeout, nbAttempt, p, startWaiting)) break;
try {
// On the first time through wait the short 'firstPause'.
if (nbAttempt == 0) {
Thread.sleep(firstPause);
} else {
// Cycle here until (subsequentPause * nbAttempt) elapses. While spinning, check
// isFileClosed if available (should be in hadoop 2.0.5... not in hadoop 1 though.
long localStartWaiting = EnvironmentEdgeManager.currentTime();
while ((EnvironmentEdgeManager.currentTime() - localStartWaiting) <
subsequentPauseBase * nbAttempt) {
Thread.sleep(conf.getInt("hbase.lease.recovery.pause", 1000));
if (findIsFileClosedMeth) {
try {
isFileClosedMeth = dfs.getClass().getMethod("isFileClosed",
new Class[]{ Path.class });
} catch (NoSuchMethodException nsme) {
LOG.debug("isFileClosed not available");
} finally {
findIsFileClosedMeth = false;
}
}
if (isFileClosedMeth != null && isFileClosed(dfs, isFileClosedMeth, p)) {
recovered = true;
break;
}
checkIfCancelled(reporter);
}
}
} catch (InterruptedException ie) {
InterruptedIOException iioe = new InterruptedIOException();
iioe.initCause(ie);
throw iioe;
}
}
return recovered;
}
boolean checkIfTimedout(final Configuration conf, final long recoveryTimeout,
final int nbAttempt, final Path p, final long startWaiting) {
if (recoveryTimeout < EnvironmentEdgeManager.currentTime()) {
LOG.warn("Cannot recoverLease after trying for " +
conf.getInt("hbase.lease.recovery.timeout", 900000) +
"ms (hbase.lease.recovery.timeout); continuing, but may be DATALOSS!!!; " +
getLogMessageDetail(nbAttempt, p, startWaiting));
return true;
}
return false;
}
/**
* Try to recover the lease.
* @param dfs
* @param nbAttempt
* @param p
* @param startWaiting
* @return True if dfs#recoverLease came by true.
* @throws FileNotFoundException
*/
boolean recoverLease(final DistributedFileSystem dfs, final int nbAttempt, final Path p,
final long startWaiting)
throws FileNotFoundException {
boolean recovered = false;
try {
recovered = dfs.recoverLease(p);
LOG.info((recovered? "Recovered lease, ": "Failed to recover lease, ") +
getLogMessageDetail(nbAttempt, p, startWaiting));
} catch (IOException e) {
if (e instanceof LeaseExpiredException && e.getMessage().contains("File does not exist")) {
// This exception comes out instead of FNFE, fix it
throw new FileNotFoundException("The given WAL wasn't found at " + p);
} else if (e instanceof FileNotFoundException) {
throw (FileNotFoundException)e;
}
LOG.warn(getLogMessageDetail(nbAttempt, p, startWaiting), e);
}
return recovered;
}
/**
* @param nbAttempt
* @param p
* @param startWaiting
* @return Detail to append to any log message around lease recovering.
*/
private String getLogMessageDetail(final int nbAttempt, final Path p, final long startWaiting) {
return "attempt=" + nbAttempt + " on file=" + p + " after " +
(EnvironmentEdgeManager.currentTime() - startWaiting) + "ms";
}
/**
* Call HDFS-4525 isFileClosed if it is available.
* @param dfs
* @param m
* @param p
* @return True if file is closed.
*/
private boolean isFileClosed(final DistributedFileSystem dfs, final Method m, final Path p) {
try {
return (Boolean) m.invoke(dfs, p);
} catch (SecurityException e) {
LOG.warn("No access", e);
} catch (Exception e) {
LOG.warn("Failed invocation for " + p.toString(), e);
}
return false;
}
void checkIfCancelled(final CancelableProgressable reporter)
throws InterruptedIOException {
if (reporter == null) return;
if (!reporter.progress()) throw new InterruptedIOException("Operation cancelled");
}
}

View File

@ -104,7 +104,7 @@ public class FSTableDescriptors implements TableDescriptors {
* filesystem where that root dir lives. This instance can do write operations (is not read only).
*/
public FSTableDescriptors(final Configuration conf) throws IOException {
this(FSUtils.getCurrentFileSystem(conf), FSUtils.getRootDir(conf));
this(CommonFSUtils.getCurrentFileSystem(conf), CommonFSUtils.getRootDir(conf));
}
public FSTableDescriptors(final FileSystem fs, final Path rootdir) {
@ -120,8 +120,8 @@ public class FSTableDescriptors implements TableDescriptors {
}
public static void tryUpdateMetaTableDescriptor(Configuration conf) throws IOException {
tryUpdateMetaTableDescriptor(conf, FSUtils.getCurrentFileSystem(conf), FSUtils.getRootDir(conf),
null);
tryUpdateMetaTableDescriptor(conf, CommonFSUtils.getCurrentFileSystem(conf),
CommonFSUtils.getRootDir(conf), null);
}
public static void tryUpdateMetaTableDescriptor(Configuration conf, FileSystem fs, Path rootdir,
@ -137,7 +137,7 @@ public class FSTableDescriptors implements TableDescriptors {
TableDescriptor td = builder.build();
LOG.info("Creating new hbase:meta table descriptor {}", td);
TableName tableName = td.getTableName();
Path tableDir = FSUtils.getTableDir(rootdir, tableName);
Path tableDir = CommonFSUtils.getTableDir(rootdir, tableName);
Path p = writeTableDescriptor(fs, td, tableDir, getTableInfoPath(fs, tableDir, true));
if (p == null) {
throw new IOException("Failed update hbase:meta table descriptor");
@ -267,7 +267,7 @@ public class FSTableDescriptors implements TableDescriptors {
for (Path d : FSUtils.getTableDirs(fs, rootdir)) {
TableDescriptor htd = null;
try {
htd = get(FSUtils.getTableName(d));
htd = get(CommonFSUtils.getTableName(d));
} catch (FileNotFoundException fnfe) {
// inability of retrieving one HTD shouldn't stop getting the remaining
LOG.warn("Trouble retrieving htd", fnfe);
@ -293,17 +293,17 @@ public class FSTableDescriptors implements TableDescriptors {
throws IOException {
Map<String, TableDescriptor> htds = new TreeMap<>();
List<Path> tableDirs =
FSUtils.getLocalTableDirs(fs, FSUtils.getNamespaceDir(rootdir, name));
FSUtils.getLocalTableDirs(fs, CommonFSUtils.getNamespaceDir(rootdir, name));
for (Path d: tableDirs) {
TableDescriptor htd = null;
try {
htd = get(FSUtils.getTableName(d));
htd = get(CommonFSUtils.getTableName(d));
} catch (FileNotFoundException fnfe) {
// inability of retrieving one HTD shouldn't stop getting the remaining
LOG.warn("Trouble retrieving htd", fnfe);
}
if (htd == null) continue;
htds.put(FSUtils.getTableName(d).getNameAsString(), htd);
htds.put(CommonFSUtils.getTableName(d).getNameAsString(), htd);
}
return htds;
}
@ -393,7 +393,7 @@ public class FSTableDescriptors implements TableDescriptors {
// only visible for FSTableDescriptorMigrationToSubdir, can be removed with that
static FileStatus getCurrentTableInfoStatus(FileSystem fs, Path dir, boolean removeOldFiles)
throws IOException {
FileStatus [] status = FSUtils.listStatus(fs, dir, TABLEINFO_PATHFILTER);
FileStatus[] status = CommonFSUtils.listStatus(fs, dir, TABLEINFO_PATHFILTER);
if (status == null || status.length < 1) return null;
FileStatus mostCurrent = null;
for (FileStatus file : status) {
@ -432,8 +432,9 @@ public class FSTableDescriptors implements TableDescriptors {
/**
* Return the table directory in HDFS
*/
@VisibleForTesting Path getTableDir(final TableName tableName) {
return FSUtils.getTableDir(rootdir, tableName);
@VisibleForTesting
Path getTableDir(final TableName tableName) {
return CommonFSUtils.getTableDir(rootdir, tableName);
}
private static final PathFilter TABLEINFO_PATHFILTER = new PathFilter() {
@ -499,7 +500,7 @@ public class FSTableDescriptors implements TableDescriptors {
*/
public static TableDescriptor getTableDescriptorFromFs(FileSystem fs,
Path hbaseRootDir, TableName tableName) throws IOException {
Path tableDir = FSUtils.getTableDir(hbaseRootDir, tableName);
Path tableDir = CommonFSUtils.getTableDir(hbaseRootDir, tableName);
return getTableDescriptorFromFs(fs, tableDir);
}
@ -565,12 +566,12 @@ public class FSTableDescriptors implements TableDescriptors {
*/
private static void deleteTableDescriptorFiles(FileSystem fs, Path dir, int maxSequenceId)
throws IOException {
FileStatus [] status = FSUtils.listStatus(fs, dir, TABLEINFO_PATHFILTER);
FileStatus [] status = CommonFSUtils.listStatus(fs, dir, TABLEINFO_PATHFILTER);
for (FileStatus file : status) {
Path path = file.getPath();
int sequenceId = getTableInfoSequenceId(path);
if (sequenceId <= maxSequenceId) {
boolean success = FSUtils.delete(fs, path, false);
boolean success = CommonFSUtils.delete(fs, path, false);
if (success) {
LOG.debug("Deleted " + path);
} else {
@ -630,7 +631,7 @@ public class FSTableDescriptors implements TableDescriptors {
} catch (IOException ioe) {
// Presume clash of names or something; go around again.
LOG.debug("Failed write and/or rename; retrying", ioe);
if (!FSUtils.deleteDirectory(fs, tempPath)) {
if (!CommonFSUtils.deleteDirectory(fs, tempPath)) {
LOG.warn("Failed cleanup of " + tempPath);
}
tableInfoDirPath = null;

View File

@ -28,15 +28,19 @@ import java.io.InterruptedIOException;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.net.InetSocketAddress;
import java.net.URI;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.Vector;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutionException;
@ -47,7 +51,6 @@ import java.util.concurrent.FutureTask;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.regex.Pattern;
import org.apache.commons.lang3.ArrayUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BlockLocation;
@ -56,14 +59,13 @@ import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.FilterFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.ClusterId;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.RegionInfo;
@ -75,12 +77,13 @@ import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSHedgedReadMetrics;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -88,6 +91,7 @@ import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
import org.apache.hbase.thirdparty.com.google.common.collect.Iterators;
import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
import org.apache.hbase.thirdparty.com.google.common.primitives.Ints;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@ -97,7 +101,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos;
* Utility methods for interacting with the underlying file system.
*/
@InterfaceAudience.Private
public abstract class FSUtils extends CommonFSUtils {
public final class FSUtils {
private static final Logger LOG = LoggerFactory.getLogger(FSUtils.class);
private static final String THREAD_POOLSIZE = "hbase.client.localityCheck.threadPoolSize";
@ -107,8 +111,7 @@ public abstract class FSUtils extends CommonFSUtils {
@VisibleForTesting // currently only used in testing. TODO refactor into a test class
public static final boolean WINDOWS = System.getProperty("os.name").startsWith("Windows");
protected FSUtils() {
super();
private FSUtils() {
}
/**
@ -161,32 +164,17 @@ public abstract class FSUtils extends CommonFSUtils {
return result;
}
public static FSUtils getInstance(FileSystem fs, Configuration conf) {
String scheme = fs.getUri().getScheme();
if (scheme == null) {
LOG.warn("Could not find scheme for uri " +
fs.getUri() + ", default to hdfs");
scheme = "hdfs";
}
Class<?> fsUtilsClass = conf.getClass("hbase.fsutil." +
scheme + ".impl", FSHDFSUtils.class); // Default to HDFS impl
FSUtils fsUtils = (FSUtils)ReflectionUtils.newInstance(fsUtilsClass, conf);
return fsUtils;
}
/**
* Delete the region directory if exists.
* @param conf
* @param hri
* @return True if deleted the region directory.
* @throws IOException
*/
public static boolean deleteRegionDir(final Configuration conf, final HRegionInfo hri)
public static boolean deleteRegionDir(final Configuration conf, final RegionInfo hri)
throws IOException {
Path rootDir = getRootDir(conf);
Path rootDir = CommonFSUtils.getRootDir(conf);
FileSystem fs = rootDir.getFileSystem(conf);
return deleteDirectory(fs,
new Path(getTableDir(rootDir, hri.getTable()), hri.getEncodedName()));
return CommonFSUtils.deleteDirectory(fs,
new Path(CommonFSUtils.getTableDir(rootDir, hri.getTable()), hri.getEncodedName()));
}
/**
@ -196,7 +184,7 @@ public abstract class FSUtils extends CommonFSUtils {
* <li>apply the umask in the configuration (if it is enabled)</li>
* <li>use the fs configured buffer size (or 4096 if not set)</li>
* <li>use the configured column family replication or default replication if
* {@link HColumnDescriptor#DEFAULT_DFS_REPLICATION}</li>
* {@link ColumnFamilyDescriptorBuilder#DEFAULT_DFS_REPLICATION}</li>
* <li>use the default block size</li>
* <li>not track progress</li>
* </ol>
@ -204,26 +192,26 @@ public abstract class FSUtils extends CommonFSUtils {
* @param fs {@link FileSystem} on which to write the file
* @param path {@link Path} to the file to write
* @param perm permissions
* @param favoredNodes
* @param favoredNodes favored data nodes
* @return output stream to the created file
* @throws IOException if the file cannot be created
*/
public static FSDataOutputStream create(Configuration conf, FileSystem fs, Path path,
FsPermission perm, InetSocketAddress[] favoredNodes) throws IOException {
if (fs instanceof HFileSystem) {
FileSystem backingFs = ((HFileSystem)fs).getBackingFs();
FileSystem backingFs = ((HFileSystem) fs).getBackingFs();
if (backingFs instanceof DistributedFileSystem) {
// Try to use the favoredNodes version via reflection to allow backwards-
// compatibility.
short replication = Short.parseShort(conf.get(HColumnDescriptor.DFS_REPLICATION,
String.valueOf(HColumnDescriptor.DEFAULT_DFS_REPLICATION)));
short replication = Short.parseShort(conf.get(ColumnFamilyDescriptorBuilder.DFS_REPLICATION,
String.valueOf(ColumnFamilyDescriptorBuilder.DEFAULT_DFS_REPLICATION)));
try {
return (FSDataOutputStream) (DistributedFileSystem.class.getDeclaredMethod("create",
Path.class, FsPermission.class, boolean.class, int.class, short.class, long.class,
Progressable.class, InetSocketAddress[].class).invoke(backingFs, path, perm, true,
getDefaultBufferSize(backingFs),
replication > 0 ? replication : getDefaultReplication(backingFs, path),
getDefaultBlockSize(backingFs, path), null, favoredNodes));
return (FSDataOutputStream) (DistributedFileSystem.class
.getDeclaredMethod("create", Path.class, FsPermission.class, boolean.class, int.class,
short.class, long.class, Progressable.class, InetSocketAddress[].class)
.invoke(backingFs, path, perm, true, CommonFSUtils.getDefaultBufferSize(backingFs),
replication > 0 ? replication : CommonFSUtils.getDefaultReplication(backingFs, path),
CommonFSUtils.getDefaultBlockSize(backingFs, path), null, favoredNodes));
} catch (InvocationTargetException ite) {
// Function was properly called, but threw it's own exception.
throw new IOException(ite.getCause());
@ -235,7 +223,7 @@ public abstract class FSUtils extends CommonFSUtils {
}
}
}
return create(fs, path, perm, true);
return CommonFSUtils.create(fs, path, perm, true);
}
/**
@ -792,10 +780,9 @@ public abstract class FSUtils extends CommonFSUtils {
*
* @throws IOException When scanning the directory fails.
*/
public static Map<String, Integer> getTableFragmentation(
final HMaster master)
public static Map<String, Integer> getTableFragmentation(final HMaster master)
throws IOException {
Path path = getRootDir(master.getConfiguration());
Path path = CommonFSUtils.getRootDir(master.getConfiguration());
// since HMaster.getFileSystem() is package private
FileSystem fs = path.getFileSystem(master.getConfiguration());
return getTableFragmentation(fs, path);
@ -841,7 +828,7 @@ public abstract class FSUtils extends CommonFSUtils {
}
}
// compute percentage per table and store in result list
frags.put(FSUtils.getTableName(d).getNameAsString(),
frags.put(CommonFSUtils.getTableName(d).getNameAsString(),
cfCount == 0? 0: Math.round((float) cfFrag / cfCount * 100));
}
// set overall percentage for all tables
@ -955,22 +942,6 @@ public abstract class FSUtils extends CommonFSUtils {
}
}
public void recoverFileLease(final FileSystem fs, final Path p, Configuration conf)
throws IOException {
recoverFileLease(fs, p, conf, null);
}
/**
* Recover file lease. Used when a file might be suspect
* to be had been left open by another process.
* @param fs FileSystem handle
* @param p Path of file to recover lease
* @param conf Configuration handle
* @throws IOException
*/
public abstract void recoverFileLease(final FileSystem fs, final Path p,
Configuration conf, CancelableProgressable reporter) throws IOException;
public static List<Path> getTableDirs(final FileSystem fs, final Path rootdir)
throws IOException {
List<Path> tableDirs = new ArrayList<>();
@ -1051,7 +1022,7 @@ public abstract class FSUtils extends CommonFSUtils {
}
public static Path getRegionDirFromRootDir(Path rootDir, RegionInfo region) {
return getRegionDirFromTableDir(getTableDir(rootDir, region.getTable()), region);
return getRegionDirFromTableDir(CommonFSUtils.getTableDir(rootDir, region.getTable()), region);
}
public static Path getRegionDirFromTableDir(Path tableDir, RegionInfo region) {
@ -1294,7 +1265,7 @@ public abstract class FSUtils extends CommonFSUtils {
resultMap == null ? new ConcurrentHashMap<>(128, 0.75f, 32) : resultMap;
// only include the directory paths to tables
Path tableDir = FSUtils.getTableDir(hbaseRootDir, tableName);
Path tableDir = CommonFSUtils.getTableDir(hbaseRootDir, tableName);
// Inside a table, there are compaction.dir directories to skip. Otherwise, all else
// should be regions.
final FamilyDirFilter familyFilter = new FamilyDirFilter(fs);
@ -1392,8 +1363,8 @@ public abstract class FSUtils extends CommonFSUtils {
if (!exceptions.isEmpty()) {
// Just throw the first exception as an indication something bad happened
// Don't need to propagate all the exceptions, we already logged them all anyway
Throwables.propagateIfInstanceOf(exceptions.firstElement(), IOException.class);
throw Throwables.propagate(exceptions.firstElement());
Throwables.propagateIfPossible(exceptions.firstElement(), IOException.class);
throw new IOException(exceptions.firstElement());
}
}
@ -1494,8 +1465,8 @@ public abstract class FSUtils extends CommonFSUtils {
// only include the directory paths to tables
for (Path tableDir : FSUtils.getTableDirs(fs, hbaseRootDir)) {
getTableStoreFilePathMap(map, fs, hbaseRootDir,
FSUtils.getTableName(tableDir), sfFilter, executor, progressReporter);
getTableStoreFilePathMap(map, fs, hbaseRootDir, CommonFSUtils.getTableName(tableDir),
sfFilter, executor, progressReporter);
}
return map;
}
@ -1637,14 +1608,16 @@ public abstract class FSUtils extends CommonFSUtils {
final String desiredTable, int threadPoolSize,
final Map<String, Map<String, Float>> regionDegreeLocalityMapping) throws IOException {
final FileSystem fs = FileSystem.get(conf);
final Path rootPath = FSUtils.getRootDir(conf);
final Path rootPath = CommonFSUtils.getRootDir(conf);
final long startTime = EnvironmentEdgeManager.currentTime();
final Path queryPath;
// The table files are in ${hbase.rootdir}/data/<namespace>/<table>/*
if (null == desiredTable) {
queryPath = new Path(new Path(rootPath, HConstants.BASE_NAMESPACE_DIR).toString() + "/*/*/*/");
queryPath =
new Path(new Path(rootPath, HConstants.BASE_NAMESPACE_DIR).toString() + "/*/*/*/");
} else {
queryPath = new Path(FSUtils.getTableDir(rootPath, TableName.valueOf(desiredTable)).toString() + "/*/");
queryPath = new Path(
CommonFSUtils.getTableDir(rootPath, TableName.valueOf(desiredTable)).toString() + "/*/");
}
// reject all paths that are not appropriate
@ -1770,7 +1743,9 @@ public abstract class FSUtils extends CommonFSUtils {
*/
public static DFSHedgedReadMetrics getDFSHedgedReadMetrics(final Configuration c)
throws IOException {
if (!isHDFS(c)) return null;
if (!CommonFSUtils.isHDFS(c)) {
return null;
}
// getHedgedReadMetrics is package private. Get the DFSClient instance that is internal
// to the DFS FS instance and make the method getHedgedReadMetrics accessible, then invoke it
// to get the singleton instance of DFSHedgedReadMetrics shared by DFSClients.
@ -1839,4 +1814,259 @@ public abstract class FSUtils extends CommonFSUtils {
}
return traversedPaths;
}
/**
* @return A set containing all namenode addresses of fs
*/
private static Set<InetSocketAddress> getNNAddresses(DistributedFileSystem fs,
Configuration conf) {
Set<InetSocketAddress> addresses = new HashSet<>();
String serviceName = fs.getCanonicalServiceName();
if (serviceName.startsWith("ha-hdfs")) {
try {
Map<String, Map<String, InetSocketAddress>> addressMap =
DFSUtil.getNNServiceRpcAddressesForCluster(conf);
String nameService = serviceName.substring(serviceName.indexOf(":") + 1);
if (addressMap.containsKey(nameService)) {
Map<String, InetSocketAddress> nnMap = addressMap.get(nameService);
for (Map.Entry<String, InetSocketAddress> e2 : nnMap.entrySet()) {
InetSocketAddress addr = e2.getValue();
addresses.add(addr);
}
}
} catch (Exception e) {
LOG.warn("DFSUtil.getNNServiceRpcAddresses failed. serviceName=" + serviceName, e);
}
} else {
URI uri = fs.getUri();
int port = uri.getPort();
if (port < 0) {
int idx = serviceName.indexOf(':');
port = Integer.parseInt(serviceName.substring(idx + 1));
}
InetSocketAddress addr = new InetSocketAddress(uri.getHost(), port);
addresses.add(addr);
}
return addresses;
}
/**
* @param conf the Configuration of HBase
* @return Whether srcFs and desFs are on same hdfs or not
*/
public static boolean isSameHdfs(Configuration conf, FileSystem srcFs, FileSystem desFs) {
// By getCanonicalServiceName, we could make sure both srcFs and desFs
// show a unified format which contains scheme, host and port.
String srcServiceName = srcFs.getCanonicalServiceName();
String desServiceName = desFs.getCanonicalServiceName();
if (srcServiceName == null || desServiceName == null) {
return false;
}
if (srcServiceName.equals(desServiceName)) {
return true;
}
if (srcServiceName.startsWith("ha-hdfs") && desServiceName.startsWith("ha-hdfs")) {
Collection<String> internalNameServices =
conf.getTrimmedStringCollection("dfs.internal.nameservices");
if (!internalNameServices.isEmpty()) {
if (internalNameServices.contains(srcServiceName.split(":")[1])) {
return true;
} else {
return false;
}
}
}
if (srcFs instanceof DistributedFileSystem && desFs instanceof DistributedFileSystem) {
// If one serviceName is an HA format while the other is a non-HA format,
// maybe they refer to the same FileSystem.
// For example, srcFs is "ha-hdfs://nameservices" and desFs is "hdfs://activeNamenode:port"
Set<InetSocketAddress> srcAddrs = getNNAddresses((DistributedFileSystem) srcFs, conf);
Set<InetSocketAddress> desAddrs = getNNAddresses((DistributedFileSystem) desFs, conf);
if (Sets.intersection(srcAddrs, desAddrs).size() > 0) {
return true;
}
}
return false;
}
public static void recoverFileLease(FileSystem fs, Path p, Configuration conf)
throws IOException {
recoverFileLease(fs, p, conf, null);
}
/**
* Recover the lease from HDFS, retrying multiple times.
*/
public static void recoverFileLease(FileSystem fs, Path p, Configuration conf,
CancelableProgressable reporter) throws IOException {
if (fs instanceof FilterFileSystem) {
fs = ((FilterFileSystem) fs).getRawFileSystem();
}
// lease recovery not needed for local file system case.
if (!(fs instanceof DistributedFileSystem)) {
return;
}
recoverDFSFileLease((DistributedFileSystem) fs, p, conf, reporter);
}
/*
* Run the dfs recover lease. recoverLease is asynchronous. It returns: -false when it starts the
* lease recovery (i.e. lease recovery not *yet* done) - true when the lease recovery has
* succeeded or the file is closed. But, we have to be careful. Each time we call recoverLease, it
* starts the recover lease process over from the beginning. We could put ourselves in a situation
* where we are doing nothing but starting a recovery, interrupting it to start again, and so on.
* The findings over in HBASE-8354 have it that the namenode will try to recover the lease on the
* file's primary node. If all is well, it should return near immediately. But, as is common, it
* is the very primary node that has crashed and so the namenode will be stuck waiting on a socket
* timeout before it will ask another datanode to start the recovery. It does not help if we call
* recoverLease in the meantime and in particular, subsequent to the socket timeout, a
* recoverLease invocation will cause us to start over from square one (possibly waiting on socket
* timeout against primary node). So, in the below, we do the following: 1. Call recoverLease. 2.
* If it returns true, break. 3. If it returns false, wait a few seconds and then call it again.
* 4. If it returns true, break. 5. If it returns false, wait for what we think the datanode
* socket timeout is (configurable) and then try again. 6. If it returns true, break. 7. If it
* returns false, repeat starting at step 5. above. If HDFS-4525 is available, call it every
* second and we might be able to exit early.
*/
private static boolean recoverDFSFileLease(final DistributedFileSystem dfs, final Path p,
final Configuration conf, final CancelableProgressable reporter) throws IOException {
LOG.info("Recover lease on dfs file " + p);
long startWaiting = EnvironmentEdgeManager.currentTime();
// Default is 15 minutes. It's huge, but the idea is that if we have a major issue, HDFS
// usually needs 10 minutes before marking the nodes as dead. So we're putting ourselves
// beyond that limit 'to be safe'.
long recoveryTimeout = conf.getInt("hbase.lease.recovery.timeout", 900000) + startWaiting;
// This setting should be a little bit above what the cluster dfs heartbeat is set to.
long firstPause = conf.getInt("hbase.lease.recovery.first.pause", 4000);
// This should be set to how long it'll take for us to timeout against primary datanode if it
// is dead. We set it to 64 seconds, 4 second than the default READ_TIMEOUT in HDFS, the
// default value for DFS_CLIENT_SOCKET_TIMEOUT_KEY. If recovery is still failing after this
// timeout, then further recovery will take liner backoff with this base, to avoid endless
// preemptions when this value is not properly configured.
long subsequentPauseBase = conf.getLong("hbase.lease.recovery.dfs.timeout", 64 * 1000);
Method isFileClosedMeth = null;
// whether we need to look for isFileClosed method
boolean findIsFileClosedMeth = true;
boolean recovered = false;
// We break the loop if we succeed the lease recovery, timeout, or we throw an exception.
for (int nbAttempt = 0; !recovered; nbAttempt++) {
recovered = recoverLease(dfs, nbAttempt, p, startWaiting);
if (recovered) {
break;
}
checkIfCancelled(reporter);
if (checkIfTimedout(conf, recoveryTimeout, nbAttempt, p, startWaiting)) {
break;
}
try {
// On the first time through wait the short 'firstPause'.
if (nbAttempt == 0) {
Thread.sleep(firstPause);
} else {
// Cycle here until (subsequentPause * nbAttempt) elapses. While spinning, check
// isFileClosed if available (should be in hadoop 2.0.5... not in hadoop 1 though.
long localStartWaiting = EnvironmentEdgeManager.currentTime();
while ((EnvironmentEdgeManager.currentTime() - localStartWaiting) < subsequentPauseBase *
nbAttempt) {
Thread.sleep(conf.getInt("hbase.lease.recovery.pause", 1000));
if (findIsFileClosedMeth) {
try {
isFileClosedMeth =
dfs.getClass().getMethod("isFileClosed", new Class[] { Path.class });
} catch (NoSuchMethodException nsme) {
LOG.debug("isFileClosed not available");
} finally {
findIsFileClosedMeth = false;
}
}
if (isFileClosedMeth != null && isFileClosed(dfs, isFileClosedMeth, p)) {
recovered = true;
break;
}
checkIfCancelled(reporter);
}
}
} catch (InterruptedException ie) {
InterruptedIOException iioe = new InterruptedIOException();
iioe.initCause(ie);
throw iioe;
}
}
return recovered;
}
private static boolean checkIfTimedout(final Configuration conf, final long recoveryTimeout,
final int nbAttempt, final Path p, final long startWaiting) {
if (recoveryTimeout < EnvironmentEdgeManager.currentTime()) {
LOG.warn("Cannot recoverLease after trying for " +
conf.getInt("hbase.lease.recovery.timeout", 900000) +
"ms (hbase.lease.recovery.timeout); continuing, but may be DATALOSS!!!; " +
getLogMessageDetail(nbAttempt, p, startWaiting));
return true;
}
return false;
}
/**
* Try to recover the lease.
* @return True if dfs#recoverLease came by true.
*/
private static boolean recoverLease(final DistributedFileSystem dfs, final int nbAttempt,
final Path p, final long startWaiting) throws FileNotFoundException {
boolean recovered = false;
try {
recovered = dfs.recoverLease(p);
LOG.info((recovered ? "Recovered lease, " : "Failed to recover lease, ") +
getLogMessageDetail(nbAttempt, p, startWaiting));
} catch (IOException e) {
if (e instanceof LeaseExpiredException && e.getMessage().contains("File does not exist")) {
// This exception comes out instead of FNFE, fix it
throw new FileNotFoundException("The given WAL wasn't found at " + p);
} else if (e instanceof FileNotFoundException) {
throw (FileNotFoundException) e;
}
LOG.warn(getLogMessageDetail(nbAttempt, p, startWaiting), e);
}
return recovered;
}
/**
* @return Detail to append to any log message around lease recovering.
*/
private static String getLogMessageDetail(final int nbAttempt, final Path p,
final long startWaiting) {
return "attempt=" + nbAttempt + " on file=" + p + " after " +
(EnvironmentEdgeManager.currentTime() - startWaiting) + "ms";
}
/**
* Call HDFS-4525 isFileClosed if it is available.
* @return True if file is closed.
*/
private static boolean isFileClosed(final DistributedFileSystem dfs, final Method m,
final Path p) {
try {
return (Boolean) m.invoke(dfs, p);
} catch (SecurityException e) {
LOG.warn("No access", e);
} catch (Exception e) {
LOG.warn("Failed invocation for " + p.toString(), e);
}
return false;
}
private static void checkIfCancelled(final CancelableProgressable reporter)
throws InterruptedIOException {
if (reporter == null) {
return;
}
if (!reporter.progress()) {
throw new InterruptedIOException("Operation cancelled");
}
}
}

View File

@ -20,14 +20,13 @@ package org.apache.hadoop.hbase.util;
import java.io.IOException;
import java.util.List;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
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.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Utility methods for interacting with the hbase.root file system.
@ -92,7 +91,7 @@ public final class FSVisitor {
String familyName = familyDir.getName();
// get all the storeFiles in the family
FileStatus[] storeFiles = FSUtils.listStatus(fs, familyDir, fileFilter);
FileStatus[] storeFiles = CommonFSUtils.listStatus(fs, familyDir, fileFilter);
if (storeFiles == null) {
if (LOG.isTraceEnabled()) {
LOG.trace("No hfiles found for family: " + familyDir + ", skipping.");

View File

@ -398,7 +398,7 @@ public class HBaseFsck extends Configured implements Closeable {
*/
@VisibleForTesting
public static Path getTmpDir(Configuration conf) throws IOException {
return new Path(FSUtils.getRootDir(conf), HConstants.HBASE_TEMP_DIRECTORY);
return new Path(CommonFSUtils.getRootDir(conf), HConstants.HBASE_TEMP_DIRECTORY);
}
private static class FileLockCallable implements Callable<FSDataOutputStream> {
@ -421,9 +421,9 @@ public class HBaseFsck extends Configured implements Closeable {
@Override
public FSDataOutputStream call() throws IOException {
try {
FileSystem fs = FSUtils.getCurrentFileSystem(this.conf);
FsPermission defaultPerms = FSUtils.getFilePermissions(fs, this.conf,
HConstants.DATA_FILE_UMASK_KEY);
FileSystem fs = CommonFSUtils.getCurrentFileSystem(this.conf);
FsPermission defaultPerms =
CommonFSUtils.getFilePermissions(fs, this.conf, HConstants.DATA_FILE_UMASK_KEY);
Path tmpDir = getTmpDir(conf);
this.hbckLockPath = new Path(tmpDir, HBCK_LOCK_FILE);
fs.mkdirs(tmpDir);
@ -450,7 +450,7 @@ public class HBaseFsck extends Configured implements Closeable {
IOException exception = null;
do {
try {
return FSUtils.create(fs, hbckLockFilePath, defaultPerms, false);
return CommonFSUtils.create(fs, hbckLockFilePath, defaultPerms, false);
} catch (IOException ioe) {
LOG.info("Failed to create lock file " + hbckLockFilePath.getName()
+ ", try=" + (retryCounter.getAttemptTimes() + 1) + " of "
@ -510,7 +510,7 @@ public class HBaseFsck extends Configured implements Closeable {
do {
try {
IOUtils.closeQuietly(hbckOutFd);
FSUtils.delete(FSUtils.getCurrentFileSystem(getConf()), HBCK_LOCK_PATH, true);
CommonFSUtils.delete(CommonFSUtils.getCurrentFileSystem(getConf()), HBCK_LOCK_PATH, true);
LOG.info("Finishing hbck");
return;
} catch (IOException ioe) {
@ -889,9 +889,9 @@ public class HBaseFsck extends Configured implements Closeable {
List<RegionInfo> regions = MetaTableAccessor.getAllRegions(connection, true);
final RegionBoundariesInformation currentRegionBoundariesInformation =
new RegionBoundariesInformation();
Path hbaseRoot = FSUtils.getRootDir(getConf());
Path hbaseRoot = CommonFSUtils.getRootDir(getConf());
for (RegionInfo regionInfo : regions) {
Path tableDir = FSUtils.getTableDir(hbaseRoot, regionInfo.getTable());
Path tableDir = CommonFSUtils.getTableDir(hbaseRoot, regionInfo.getTable());
currentRegionBoundariesInformation.regionName = regionInfo.getRegionName();
// For each region, get the start and stop key from the META and compare them to the
// same information from the Stores.
@ -1128,7 +1128,7 @@ public class HBaseFsck extends Configured implements Closeable {
private void offlineReferenceFileRepair() throws IOException, InterruptedException {
clearState();
Configuration conf = getConf();
Path hbaseRoot = FSUtils.getRootDir(conf);
Path hbaseRoot = CommonFSUtils.getRootDir(conf);
FileSystem fs = hbaseRoot.getFileSystem(conf);
LOG.info("Computing mapping of all store files");
Map<String, Path> allFiles = FSUtils.getTableStoreFilePathMap(fs, hbaseRoot,
@ -1181,7 +1181,7 @@ public class HBaseFsck extends Configured implements Closeable {
*/
private void offlineHLinkFileRepair() throws IOException, InterruptedException {
Configuration conf = getConf();
Path hbaseRoot = FSUtils.getRootDir(conf);
Path hbaseRoot = CommonFSUtils.getRootDir(conf);
FileSystem fs = hbaseRoot.getFileSystem(conf);
LOG.info("Computing mapping of all link files");
Map<String, Path> allFiles = FSUtils
@ -1310,7 +1310,7 @@ public class HBaseFsck extends Configured implements Closeable {
}
}
Path hbaseRoot = FSUtils.getRootDir(getConf());
Path hbaseRoot = CommonFSUtils.getRootDir(getConf());
FileSystem fs = hbaseRoot.getFileSystem(getConf());
// serialized table info gathering.
for (HbckRegionInfo hbi: hbckRegionInfos) {
@ -1509,7 +1509,7 @@ public class HBaseFsck extends Configured implements Closeable {
Path getSidelineDir() throws IOException {
if (sidelineDir == null) {
Path hbaseDir = FSUtils.getRootDir(getConf());
Path hbaseDir = CommonFSUtils.getRootDir(getConf());
Path hbckDir = new Path(hbaseDir, HConstants.HBCK_SIDELINEDIR_NAME);
sidelineDir = new Path(hbckDir, hbaseDir.getName() + "-"
+ startMillis);
@ -1546,7 +1546,7 @@ public class HBaseFsck extends Configured implements Closeable {
if (parentDir != null) {
rootDir = new Path(rootDir, parentDir);
}
Path sidelineTableDir= FSUtils.getTableDir(rootDir, tableName);
Path sidelineTableDir= CommonFSUtils.getTableDir(rootDir, tableName);
Path sidelineRegionDir = new Path(sidelineTableDir, regionDir.getName());
fs.mkdirs(sidelineRegionDir);
boolean success = false;
@ -1632,7 +1632,7 @@ public class HBaseFsck extends Configured implements Closeable {
* regionInfoMap
*/
public void loadHdfsRegionDirs() throws IOException, InterruptedException {
Path rootDir = FSUtils.getRootDir(getConf());
Path rootDir = CommonFSUtils.getRootDir(getConf());
FileSystem fs = rootDir.getFileSystem(getConf());
// list all tables from HDFS
@ -1642,7 +1642,7 @@ public class HBaseFsck extends Configured implements Closeable {
List<Path> paths = FSUtils.getTableDirs(fs, rootDir);
for (Path path : paths) {
TableName tableName = FSUtils.getTableName(path);
TableName tableName = CommonFSUtils.getTableName(path);
if ((!checkMetaOnly &&
isTableIncluded(tableName)) ||
tableName.equals(TableName.META_TABLE_NAME)) {
@ -1934,7 +1934,7 @@ public class HBaseFsck extends Configured implements Closeable {
return;
}
Path hbaseDir = FSUtils.getRootDir(getConf());
Path hbaseDir = CommonFSUtils.getRootDir(getConf());
FileSystem fs = hbaseDir.getFileSystem(getConf());
UserProvider userProvider = UserProvider.instantiate(getConf());
UserGroupInformation ugi = userProvider.getCurrent().getUGI();
@ -3595,9 +3595,9 @@ public class HBaseFsck extends Configured implements Closeable {
public static void main(String[] args) throws Exception {
// create a fsck object
Configuration conf = HBaseConfiguration.create();
Path hbasedir = FSUtils.getRootDir(conf);
Path hbasedir = CommonFSUtils.getRootDir(conf);
URI defaultFs = hbasedir.getFileSystem(conf).getUri();
FSUtils.setFsDefault(conf, new Path(defaultFs));
CommonFSUtils.setFsDefault(conf, new Path(defaultFs));
int ret = ToolRunner.run(new HBaseFsckTool(conf), args);
System.exit(ret);
}
@ -3801,13 +3801,13 @@ public class HBaseFsck extends Configured implements Closeable {
setHFileCorruptionChecker(hfcc); // so we can get result
Collection<TableName> tables = getIncludedTables();
Collection<Path> tableDirs = new ArrayList<>();
Path rootdir = FSUtils.getRootDir(getConf());
Path rootdir = CommonFSUtils.getRootDir(getConf());
if (tables.size() > 0) {
for (TableName t : tables) {
tableDirs.add(FSUtils.getTableDir(rootdir, t));
tableDirs.add(CommonFSUtils.getTableDir(rootdir, t));
}
} else {
tableDirs = FSUtils.getTableDirs(FSUtils.getCurrentFileSystem(getConf()), rootdir);
tableDirs = FSUtils.getTableDirs(CommonFSUtils.getCurrentFileSystem(getConf()), rootdir);
}
hfcc.checkTables(tableDirs);
hfcc.report(errors);

View File

@ -189,7 +189,7 @@ public class HBaseFsckRepair {
public static HRegion createHDFSRegionDir(Configuration conf,
RegionInfo hri, TableDescriptor htd) throws IOException {
// Create HRegion
Path root = FSUtils.getRootDir(conf);
Path root = CommonFSUtils.getRootDir(conf);
HRegion region = HRegion.createHRegion(hri, root, conf, htd, null);
// Close the new region to flush to disk. Close log file too.

Some files were not shown because too many files have changed in this diff Show More