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:
parent
3b0812d3e1
commit
922921ee5f
|
@ -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() {
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -30,7 +30,6 @@ import java.security.PrivilegedExceptionAction;
|
|||
import java.util.Collections;
|
||||
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;
|
||||
|
@ -60,7 +59,7 @@ 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.minikdc.MiniKdc;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.junit.AfterClass;
|
||||
|
@ -104,7 +103,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;
|
||||
}
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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<>();
|
||||
|
|
|
@ -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("***********************************************************************************");
|
||||
|
|
|
@ -22,28 +22,27 @@ import java.io.IOException;
|
|||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
|
||||
import org.apache.hadoop.conf.Configured;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.mapreduce.Import.CellImporter;
|
||||
import org.apache.hadoop.hbase.mapreduce.Import.Importer;
|
||||
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.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.mapreduce.Import.CellImporter;
|
||||
import org.apache.hadoop.hbase.mapreduce.Import.Importer;
|
||||
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
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;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* Tool used to copy a table to another one which can be on a different setup.
|
||||
|
@ -78,7 +77,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);
|
||||
|
@ -423,7 +422,7 @@ public class CopyTable extends Configured implements Tool {
|
|||
if (code == 0) {
|
||||
// bulkloadDir is deleted only LoadIncrementalHFiles was successful so that one can rerun
|
||||
// LoadIncrementalHFiles.
|
||||
FileSystem fs = FSUtils.getCurrentFileSystem(getConf());
|
||||
FileSystem fs = CommonFSUtils.getCurrentFileSystem(getConf());
|
||||
if (!fs.delete(this.bulkloadDir, true)) {
|
||||
LOG.error("Deleting folder " + bulkloadDir + " failed!");
|
||||
code = 1;
|
||||
|
|
|
@ -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;
|
||||
|
||||
/**
|
||||
|
@ -443,7 +445,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);
|
||||
}
|
||||
|
||||
/*
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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,13 +216,13 @@ 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),
|
||||
new Path(peerFSAddress, peerSnapshotTmpDir), peerSnapshotName, scan, true);
|
||||
replicatedScanner = new TableSnapshotScanner(peerConf, CommonFSUtils.getRootDir(peerConf),
|
||||
new Path(peerFSAddress, peerSnapshotTmpDir), peerSnapshotName, scan, true);
|
||||
} else {
|
||||
replicatedScanner = replicatedTable.getScanner(scan);
|
||||
}
|
||||
|
@ -367,13 +367,13 @@ public class VerifyReplication extends Configured implements Tool {
|
|||
}
|
||||
|
||||
private void restoreSnapshotForPeerCluster(Configuration conf, String peerQuorumAddress)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
Configuration peerConf =
|
||||
HBaseConfiguration.createClusterConf(conf, peerQuorumAddress, PEER_CONFIG_PREFIX);
|
||||
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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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[] {};
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -89,6 +89,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests;
|
|||
import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
|
||||
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
|
||||
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;
|
||||
|
@ -1150,7 +1151,7 @@ public class TestCellBasedHFileOutputFormat2 {
|
|||
// 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(htd.getFamilies().size(), families.length);
|
||||
for (FileStatus f : families) {
|
||||
String familyStr = f.getPath().getName();
|
||||
|
@ -1224,7 +1225,7 @@ public class TestCellBasedHFileOutputFormat2 {
|
|||
|
||||
// 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.getTableRegions(TABLE_NAMES[0]).get(0).getEncodedName(),
|
||||
Bytes.toString(FAMILIES[0])));
|
||||
assertEquals(0, fs.listStatus(storePath).length);
|
||||
|
@ -1305,7 +1306,7 @@ public class TestCellBasedHFileOutputFormat2 {
|
|||
|
||||
// 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.getTableRegions(TABLE_NAMES[0]).get(0).getEncodedName(),
|
||||
Bytes.toString(FAMILIES[0])));
|
||||
assertEquals(0, fs.listStatus(storePath).length);
|
||||
|
|
|
@ -49,7 +49,7 @@ import org.apache.hadoop.hbase.mapreduce.WALPlayer.WALCellMapper;
|
|||
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 TestCellBasedWALPlayer2 {
|
|||
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();
|
||||
}
|
||||
|
||||
|
|
|
@ -89,6 +89,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests;
|
|||
import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
|
||||
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
|
||||
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;
|
||||
|
@ -1169,7 +1170,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(htd.getFamilies().size(), families.length);
|
||||
for (FileStatus f : families) {
|
||||
String familyStr = f.getPath().getName();
|
||||
|
@ -1185,8 +1186,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);
|
||||
|
@ -1243,7 +1245,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.getTableRegions(TABLE_NAMES[0]).get(0).getEncodedName(),
|
||||
Bytes.toString(FAMILIES[0])));
|
||||
assertEquals(0, fs.listStatus(storePath).length);
|
||||
|
@ -1324,7 +1326,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.getTableRegions(TABLE_NAMES[0]).get(0).getEncodedName(),
|
||||
Bytes.toString(FAMILIES[0])));
|
||||
assertEquals(0, fs.listStatus(storePath).length);
|
||||
|
|
|
@ -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,17 +53,15 @@ 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()),
|
||||
TEST_UTIL.getTestFileSystem(), true);
|
||||
SnapshotTestingUtils.createSnapshotAndValidate(TEST_UTIL.getAdmin(),
|
||||
TableName.valueOf(tableName), ImmutableList.of(INPUT_FAMILY), null,
|
||||
snapshotNameForTable(tableName), CommonFSUtils.getRootDir(TEST_UTIL.getConfiguration()),
|
||||
TEST_UTIL.getTestFileSystem(), true);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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,13 +77,13 @@ 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(Bytes.toBytes("1"), Bytes.toBytes("2"))), "snapshot2",
|
||||
ImmutableList.of(new Scan(Bytes.toBytes("3"), Bytes.toBytes("4")),
|
||||
new Scan(Bytes.toBytes("5"), Bytes.toBytes("6"))));
|
||||
|
||||
this.restoreDir = new Path(FSUtils.getRootDir(conf), "restore-dir");
|
||||
this.restoreDir = new Path(CommonFSUtils.getRootDir(conf), "restore-dir");
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -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),
|
||||
|
|
|
@ -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.wal.WAL;
|
||||
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||
|
@ -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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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.WAL;
|
||||
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.wal.WALFactory;
|
||||
|
@ -110,7 +110,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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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, new String(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, new String(FAMILY),
|
||||
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
@ -285,7 +287,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)))));
|
||||
}
|
||||
}
|
||||
|
@ -308,7 +310,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()));
|
||||
|
|
|
@ -57,7 +57,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>
|
||||
|
@ -354,7 +354,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>
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -103,8 +103,6 @@ public class FanOutOneBlockAsyncDFSOutput implements AsyncFSOutput {
|
|||
|
||||
private final Configuration conf;
|
||||
|
||||
private final FSUtils fsUtils;
|
||||
|
||||
private final DistributedFileSystem dfs;
|
||||
|
||||
private final DFSClient client;
|
||||
|
@ -325,12 +323,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;
|
||||
|
@ -555,7 +552,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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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,8 +555,8 @@ public final class FanOutOneBlockAsyncDFSOutputHelper {
|
|||
}
|
||||
Encryptor encryptor = createEncryptor(conf, stat, client);
|
||||
FanOutOneBlockAsyncDFSOutput output =
|
||||
new FanOutOneBlockAsyncDFSOutput(conf, fsUtils, dfs, client, namenode, clientName, src,
|
||||
stat.getFileId(), locatedBlock, encryptor, datanodeList, summer, ALLOC);
|
||||
new FanOutOneBlockAsyncDFSOutput(conf, dfs, client, namenode, clientName, src,
|
||||
stat.getFileId(), locatedBlock, encryptor, datanodeList, summer, ALLOC);
|
||||
succ = true;
|
||||
return output;
|
||||
} catch (RemoteException e) {
|
||||
|
|
|
@ -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;
|
||||
|
@ -47,7 +46,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;
|
||||
|
@ -73,11 +71,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;
|
||||
|
@ -210,8 +207,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)
|
||||
|
@ -253,9 +250,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;
|
||||
|
@ -291,7 +289,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
|
||||
|
|
|
@ -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;
|
||||
|
||||
/**
|
||||
|
@ -287,7 +289,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);
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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.procedure2.store.wal.WALProcedureStore;
|
|||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
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"));
|
||||
|
@ -443,7 +443,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
|
||||
|
@ -463,6 +463,6 @@ public class MasterFileSystem {
|
|||
}
|
||||
|
||||
public void logFileSystemState(Logger log) throws IOException {
|
||||
FSUtils.logFileSystemState(fs, rootdir, log);
|
||||
CommonFSUtils.logFileSystemState(fs, rootdir, log);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
||||
/**
|
||||
|
@ -188,7 +189,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;
|
||||
}
|
||||
|
||||
|
@ -215,7 +216,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();
|
||||
|
@ -225,7 +226,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;
|
||||
|
@ -372,13 +373,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);
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
|
|
@ -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)) {
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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="
|
||||
|
|
|
@ -19,8 +19,8 @@
|
|||
package org.apache.hadoop.hbase.master.procedure;
|
||||
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* Base class for all the Namespace procedures that want to use a StateMachineProcedure.
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
@ -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,10 +457,10 @@ 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);
|
||||
TableDescriptorBuilder.newBuilder(tableDescriptor).build(), false);
|
||||
|
||||
// 2. Create Regions
|
||||
newRegions = hdfsRegionHandler.createHdfsRegions(
|
||||
|
|
|
@ -19,19 +19,19 @@
|
|||
package org.apache.hadoop.hbase.master.procedure;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
import org.apache.hadoop.hbase.NamespaceExistException;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||
import org.apache.hadoop.hbase.master.TableNamespaceManager;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
|
||||
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.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateNamespaceState;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
|
||||
/**
|
||||
* The procedure to create a new namespace.
|
||||
|
@ -225,7 +225,7 @@ public class CreateNamespaceProcedure
|
|||
final NamespaceDescriptor nsDescriptor) throws IOException {
|
||||
MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
|
||||
mfs.getFileSystem().mkdirs(
|
||||
FSUtils.getNamespaceDir(mfs.getRootDir(), nsDescriptor.getName()));
|
||||
CommonFSUtils.getNamespaceDir(mfs.getRootDir(), nsDescriptor.getName()));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.master.procedure;
|
|||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
|
@ -35,15 +34,17 @@ import org.apache.hadoop.hbase.client.TableState;
|
|||
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.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;
|
||||
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;
|
||||
|
||||
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;
|
||||
|
@ -308,7 +309,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);
|
||||
|
||||
|
@ -327,7 +328,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);
|
||||
|
|
|
@ -20,24 +20,24 @@ package org.apache.hadoop.hbase.master.procedure;
|
|||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
import org.apache.hadoop.hbase.NamespaceNotFoundException;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.apache.hadoop.hbase.constraint.ConstraintException;
|
||||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||
import org.apache.hadoop.hbase.master.TableNamespaceManager;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
|
||||
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.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteNamespaceState;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
|
||||
/**
|
||||
* The procedure to remove a namespace.
|
||||
|
@ -292,7 +292,7 @@ public class DeleteNamespaceProcedure
|
|||
final String namespaceName) 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)) {
|
||||
|
@ -300,7 +300,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) {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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;
|
||||
|
@ -109,9 +109,9 @@ public class SnapshotFileCache implements Stoppable {
|
|||
* @throws IOException if the {@link FileSystem} or root directory cannot be loaded
|
||||
*/
|
||||
public SnapshotFileCache(Configuration conf, long cacheRefreshPeriod, String refreshThreadName,
|
||||
SnapshotFileInspector inspectSnapshotFiles) throws IOException {
|
||||
this(FSUtils.getCurrentFileSystem(conf), FSUtils.getRootDir(conf), 0, cacheRefreshPeriod,
|
||||
refreshThreadName, inspectSnapshotFiles);
|
||||
SnapshotFileInspector inspectSnapshotFiles) throws IOException {
|
||||
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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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.");
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -70,8 +70,8 @@ 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.hadoop.hbase.util.ReflectionUtils;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
@ -306,7 +306,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);
|
||||
|
@ -396,7 +396,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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -418,7 +418,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());
|
||||
}
|
||||
|
|
|
@ -22,16 +22,15 @@ import java.io.IOException;
|
|||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
|
||||
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.TableName;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.CommonFSUtils;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* A mob compactor to directly compact the mob files.
|
||||
|
@ -55,7 +54,7 @@ public abstract class MobCompactor {
|
|||
this.tableName = tableName;
|
||||
this.column = column;
|
||||
this.pool = pool;
|
||||
mobTableDir = FSUtils.getTableDir(MobUtils.getMobHome(conf), tableName);
|
||||
mobTableDir = CommonFSUtils.getTableDir(MobUtils.getMobHome(conf), tableName);
|
||||
mobFamilyDir = MobUtils.getMobFamilyPath(conf, tableName, column.getNameAsString());
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
@ -1039,8 +1040,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)) {
|
||||
|
@ -1237,11 +1238,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());
|
||||
TableDescriptor tableDescriptor, RegionInfo regionInfo) throws IOException {
|
||||
Path tablePath =
|
||||
CommonFSUtils.getTableDir(CommonFSUtils.getRootDir(conf), tableDescriptor.getTableName());
|
||||
return computeHDFSBlocksDistribution(conf, tableDescriptor, regionInfo, tablePath);
|
||||
}
|
||||
|
||||
|
@ -1962,13 +1963,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;
|
||||
}
|
||||
|
@ -1980,8 +1981,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
@VisibleForTesting
|
||||
public Path getWALRegionDir() throws IOException {
|
||||
if (regionDir == null) {
|
||||
regionDir = FSUtils.getWALRegionDir(conf, getRegionInfo().getTable(),
|
||||
getRegionInfo().getEncodedName());
|
||||
regionDir = CommonFSUtils.getWALRegionDir(conf, getRegionInfo().getTable(),
|
||||
getRegionInfo().getEncodedName());
|
||||
}
|
||||
return regionDir;
|
||||
}
|
||||
|
@ -4414,7 +4415,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(),
|
||||
|
@ -4676,10 +4677,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 =
|
||||
|
@ -7222,7 +7224,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);
|
||||
|
@ -7249,7 +7251,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);
|
||||
|
@ -7302,7 +7304,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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -7392,7 +7394,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);
|
||||
}
|
||||
|
||||
|
@ -7514,8 +7516,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) {
|
||||
|
|
|
@ -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());
|
||||
|
@ -758,10 +758,10 @@ public class HRegionFileSystem {
|
|||
|
||||
static boolean mkdirs(FileSystem fs, Configuration conf, Path dir) throws IOException {
|
||||
if (FSUtils.isDistributedFileSystem(fs) ||
|
||||
!conf.getBoolean(HConstants.ENABLE_DATA_FILE_UMASK, false)) {
|
||||
!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 {
|
||||
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
|
||||
|
|
|
@ -153,6 +153,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;
|
||||
|
@ -717,15 +718,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) {
|
||||
|
|
|
@ -47,7 +47,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;
|
||||
|
@ -124,7 +124,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 "
|
||||
|
@ -382,7 +382,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);
|
||||
|
@ -421,7 +421,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;
|
||||
}
|
||||
|
|
|
@ -23,10 +23,6 @@ import java.io.IOException;
|
|||
import java.io.InterruptedIOException;
|
||||
import java.net.ConnectException;
|
||||
import java.net.SocketTimeoutException;
|
||||
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -35,11 +31,14 @@ import org.apache.hadoop.hbase.Server;
|
|||
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
|
||||
import org.apache.hadoop.hbase.coordination.SplitLogWorkerCoordination;
|
||||
import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor.Status;
|
||||
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.wal.WALFactory;
|
||||
import org.apache.hadoop.hbase.wal.WALSplitter;
|
||||
import org.apache.hadoop.hbase.util.CancelableProgressable;
|
||||
import org.apache.hadoop.hbase.util.ExceptionUtil;
|
||||
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;
|
||||
|
||||
|
@ -89,7 +88,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);
|
||||
|
|
|
@ -26,6 +26,7 @@ import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_K
|
|||
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.TIMERANGE_KEY;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.Collection;
|
||||
|
@ -52,14 +53,16 @@ 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.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
|
||||
/**
|
||||
|
@ -524,7 +527,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);
|
||||
|
|
|
@ -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;
|
||||
|
@ -1200,7 +1199,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());
|
||||
}
|
||||
|
@ -1208,7 +1207,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)) {
|
||||
|
@ -1250,7 +1249,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);
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -38,7 +38,6 @@ import java.util.regex.Matcher;
|
|||
import java.util.regex.Pattern;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -55,7 +54,7 @@ import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
|
|||
import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint;
|
||||
import org.apache.hadoop.hbase.replication.regionserver.ReplicationSinkManager.SinkPeer;
|
||||
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;
|
||||
|
@ -67,7 +66,6 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
|
|||
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
|
||||
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
|
||||
|
||||
/**
|
||||
|
@ -185,7 +183,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));
|
||||
|
|
|
@ -45,12 +45,13 @@ import org.apache.hadoop.hbase.security.token.FsDelegationToken;
|
|||
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
|
||||
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles.LoadQueueItem;
|
||||
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;
|
||||
|
||||
/**
|
||||
|
@ -103,7 +104,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);
|
||||
|
|
|
@ -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.util.Threads;
|
||||
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
@ -85,7 +85,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
|
||||
|
|
|
@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.Server;
|
|||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||
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);
|
||||
|
||||
|
|
|
@ -383,10 +383,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);
|
||||
|
|
|
@ -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) {
|
||||
|
@ -816,7 +818,7 @@ public class RestoreSnapshotHelper {
|
|||
regions.add(hri);
|
||||
}
|
||||
LOG.debug("found " + regions.size() + " regions for table=" +
|
||||
tableDesc.getTableName().getNameAsString());
|
||||
tableDesc.getTableName().getNameAsString());
|
||||
return regions;
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
@ -278,7 +279,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);
|
||||
}
|
||||
|
||||
|
@ -353,11 +354,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 {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -92,7 +92,7 @@ import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
|
|||
import org.apache.hadoop.hbase.security.UserProvider;
|
||||
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.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.FSVisitor;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.util.Tool;
|
||||
|
@ -566,7 +566,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
|
|||
// Check to see if the source and target filesystems are the same
|
||||
// If they are the same filesystem, we will try move the files back
|
||||
// because previously we moved them to the staging directory.
|
||||
if (FSHDFSUtils.isSameHdfs(getConf(), sourceFs, targetFs)) {
|
||||
if (FSUtils.isSameHdfs(getConf(), sourceFs, targetFs)) {
|
||||
for (Pair<byte[], String> el : famPaths) {
|
||||
Path hfileStagingPath = null;
|
||||
Path hfileOrigPath = new Path(el.getSecond());
|
||||
|
|
|
@ -1,332 +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.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(final FileSystem fs, final Path p,
|
||||
Configuration conf, CancelableProgressable reporter)
|
||||
throws IOException {
|
||||
// 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");
|
||||
}
|
||||
}
|
|
@ -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");
|
||||
|
@ -258,7 +258,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);
|
||||
|
@ -284,17 +284,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;
|
||||
}
|
||||
|
@ -383,8 +383,8 @@ 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);
|
||||
throws IOException {
|
||||
FileStatus[] status = CommonFSUtils.listStatus(fs, dir, TABLEINFO_PATHFILTER);
|
||||
if (status == null || status.length < 1) return null;
|
||||
FileStatus mostCurrent = null;
|
||||
for (FileStatus file : status) {
|
||||
|
@ -423,8 +423,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() {
|
||||
|
@ -490,7 +491,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);
|
||||
}
|
||||
|
||||
|
@ -556,12 +557,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 {
|
||||
|
@ -621,7 +622,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;
|
||||
|
|
|
@ -28,14 +28,18 @@ 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.Set;
|
||||
import java.util.Vector;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
|
@ -46,7 +50,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;
|
||||
|
@ -55,6 +58,7 @@ 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;
|
||||
|
@ -62,8 +66,8 @@ 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;
|
||||
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
|
@ -73,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.hadoop.util.StringUtils;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
|
@ -87,6 +92,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;
|
||||
|
@ -96,7 +102,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";
|
||||
|
@ -106,8 +112,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() {
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -160,32 +165,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)
|
||||
throws IOException {
|
||||
Path rootDir = getRootDir(conf);
|
||||
public static boolean deleteRegionDir(final Configuration conf, final RegionInfo hri)
|
||||
throws IOException {
|
||||
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()));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -195,7 +185,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>
|
||||
|
@ -203,38 +193,38 @@ 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 {
|
||||
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());
|
||||
} catch (NoSuchMethodException e) {
|
||||
LOG.debug("DFS Client does not support most favored nodes create; using default create");
|
||||
LOG.trace("Ignoring; use default create", e);
|
||||
} catch (IllegalArgumentException | SecurityException | IllegalAccessException e) {
|
||||
} catch (IllegalArgumentException | SecurityException | IllegalAccessException e) {
|
||||
LOG.debug("Ignoring (most likely Reflection related exception) " + e);
|
||||
}
|
||||
}
|
||||
}
|
||||
return create(fs, path, perm, true);
|
||||
return CommonFSUtils.create(fs, path, perm, true);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -779,10 +769,9 @@ public abstract class FSUtils extends CommonFSUtils {
|
|||
*
|
||||
* @throws IOException When scanning the directory fails.
|
||||
*/
|
||||
public static Map<String, Integer> getTableFragmentation(
|
||||
final HMaster master)
|
||||
throws IOException {
|
||||
Path path = getRootDir(master.getConfiguration());
|
||||
public static Map<String, Integer> getTableFragmentation(final HMaster master)
|
||||
throws IOException {
|
||||
Path path = CommonFSUtils.getRootDir(master.getConfiguration());
|
||||
// since HMaster.getFileSystem() is package private
|
||||
FileSystem fs = path.getFileSystem(master.getConfiguration());
|
||||
return getTableFragmentation(fs, path);
|
||||
|
@ -828,7 +817,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
|
||||
|
@ -933,17 +922,6 @@ public abstract class FSUtils extends CommonFSUtils {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* 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<>();
|
||||
|
@ -1024,7 +1002,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) {
|
||||
|
@ -1267,7 +1245,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);
|
||||
|
@ -1365,8 +1343,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());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1467,8 +1445,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;
|
||||
}
|
||||
|
@ -1607,17 +1585,19 @@ public abstract class FSUtils extends CommonFSUtils {
|
|||
* in case of file system errors or interrupts
|
||||
*/
|
||||
private static void getRegionLocalityMappingFromFS(final Configuration conf,
|
||||
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 String desiredTable, int threadPoolSize,
|
||||
final Map<String, Map<String, Float>> regionDegreeLocalityMapping) throws IOException {
|
||||
final FileSystem fs = FileSystem.get(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
|
||||
|
@ -1743,7 +1723,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.
|
||||
|
@ -1812,4 +1794,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");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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.");
|
||||
|
|
|
@ -401,7 +401,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> {
|
||||
|
@ -424,9 +424,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);
|
||||
|
@ -453,7 +453,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 "
|
||||
|
@ -513,7 +513,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) {
|
||||
|
@ -892,9 +892,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.
|
||||
|
@ -1131,7 +1131,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,
|
||||
|
@ -1184,7 +1184,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
|
||||
|
@ -1313,7 +1313,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) {
|
||||
|
@ -1512,7 +1512,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);
|
||||
|
@ -1549,7 +1549,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;
|
||||
|
@ -1635,7 +1635,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
|
||||
|
@ -1645,7 +1645,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)) {
|
||||
|
@ -1939,7 +1939,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();
|
||||
|
@ -3604,9 +3604,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);
|
||||
}
|
||||
|
@ -3811,13 +3811,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);
|
||||
|
|
|
@ -186,7 +186,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.
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
package org.apache.hadoop.hbase.util;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
|
@ -77,10 +76,9 @@ public final class HFileArchiveUtil {
|
|||
* @return {@link Path} to the directory to archive the given store or <tt>null</tt> if it should
|
||||
* not be archived
|
||||
*/
|
||||
public static Path getStoreArchivePath(Configuration conf,
|
||||
RegionInfo region,
|
||||
byte[] family) throws IOException {
|
||||
Path rootDir = FSUtils.getRootDir(conf);
|
||||
public static Path getStoreArchivePath(Configuration conf, RegionInfo region, byte[] family)
|
||||
throws IOException {
|
||||
Path rootDir = CommonFSUtils.getRootDir(conf);
|
||||
Path tableArchiveDir = getTableArchivePath(rootDir, region.getTable());
|
||||
return HStore.getStoreHomedir(tableArchiveDir, region, family);
|
||||
}
|
||||
|
@ -147,7 +145,7 @@ public final class HFileArchiveUtil {
|
|||
* @return {@link Path} to the archive directory for the table
|
||||
*/
|
||||
public static Path getTableArchivePath(final Path rootdir, final TableName tableName) {
|
||||
return FSUtils.getTableDir(getArchivePath(rootdir), tableName);
|
||||
return CommonFSUtils.getTableDir(getArchivePath(rootdir), tableName);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -158,10 +156,9 @@ public final class HFileArchiveUtil {
|
|||
* @param tableName Name of the table to be archived. Cannot be null.
|
||||
* @return {@link Path} to the archive directory for the table
|
||||
*/
|
||||
public static Path getTableArchivePath(final Configuration conf,
|
||||
final TableName tableName)
|
||||
throws IOException {
|
||||
return FSUtils.getTableDir(getArchivePath(conf), tableName);
|
||||
public static Path getTableArchivePath(final Configuration conf, final TableName tableName)
|
||||
throws IOException {
|
||||
return CommonFSUtils.getTableDir(getArchivePath(conf), tableName);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -173,7 +170,7 @@ public final class HFileArchiveUtil {
|
|||
* @throws IOException if an unexpected error occurs
|
||||
*/
|
||||
public static Path getArchivePath(Configuration conf) throws IOException {
|
||||
return getArchivePath(FSUtils.getRootDir(conf));
|
||||
return getArchivePath(CommonFSUtils.getRootDir(conf));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -21,7 +21,6 @@ import java.io.IOException;
|
|||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -167,7 +166,7 @@ public class HbckRegionInfo implements KeyRange {
|
|||
// we are only guaranteed to have a path and not an HRI for hdfsEntry,
|
||||
// so we get the name from the Path
|
||||
Path tableDir = this.hdfsEntry.regionDir.getParent();
|
||||
return FSUtils.getTableName(tableDir);
|
||||
return CommonFSUtils.getTableName(tableDir);
|
||||
} else {
|
||||
// return the info from the first online/deployed hri
|
||||
for (OnlineEntry e : deployedEntries) {
|
||||
|
|
|
@ -457,8 +457,8 @@ public class RegionSplitter {
|
|||
// Max outstanding splits. default == 50% of servers
|
||||
final int MAX_OUTSTANDING = Math.max(getRegionServerCount(connection) / 2, minOS);
|
||||
|
||||
Path hbDir = FSUtils.getRootDir(conf);
|
||||
Path tableDir = FSUtils.getTableDir(hbDir, tableName);
|
||||
Path hbDir = CommonFSUtils.getRootDir(conf);
|
||||
Path tableDir = CommonFSUtils.getTableDir(hbDir, tableName);
|
||||
Path splitFile = new Path(tableDir, "_balancedSplit");
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
|
||||
|
@ -783,10 +783,9 @@ public class RegionSplitter {
|
|||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
private static Pair<Path, Path> getTableDirAndSplitFile(final Configuration conf,
|
||||
final TableName tableName)
|
||||
throws IOException {
|
||||
Path hbDir = FSUtils.getRootDir(conf);
|
||||
Path tableDir = FSUtils.getTableDir(hbDir, tableName);
|
||||
final TableName tableName) throws IOException {
|
||||
Path hbDir = CommonFSUtils.getRootDir(conf);
|
||||
Path tableDir = CommonFSUtils.getTableDir(hbDir, tableName);
|
||||
Path splitFile = new Path(tableDir, "_balancedSplit");
|
||||
return new Pair<>(tableDir, splitFile);
|
||||
}
|
||||
|
@ -846,8 +845,7 @@ public class RegionSplitter {
|
|||
fs.rename(tmpFile, splitFile);
|
||||
} else {
|
||||
LOG.debug("_balancedSplit file found. Replay log to restore state...");
|
||||
FSUtils.getInstance(fs, connection.getConfiguration())
|
||||
.recoverFileLease(fs, splitFile, connection.getConfiguration(), null);
|
||||
FSUtils.recoverFileLease(fs, splitFile, connection.getConfiguration(), null);
|
||||
|
||||
// parse split file and process remaining splits
|
||||
FSDataInputStream tmpIn = fs.open(splitFile);
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue