Class HRegionFileSystem
java.lang.Object
org.apache.hadoop.hbase.regionserver.HRegionFileSystem
- Direct Known Subclasses:
HRegionWALFileSystem
View to an on-disk Region. Provides the set of methods necessary to interact with the on-disk
region data.
-
Field Summary
FieldsModifier and TypeFieldDescriptionprivate final int(package private) final org.apache.hadoop.conf.Configurationprivate static final intprivate static final int(package private) final org.apache.hadoop.fs.FileSystemprivate final intIn order to handle NN connectivity hiccups, one need to retry non-idempotent operation at the client level.private static final org.slf4j.Loggerstatic final StringName of the region info file that resides just under the region directory.static final StringTemporary subdirectory of the region directory used for merges.static final StringTemporary subdirectory of the region directory used for splits.(package private) static final StringTemporary subdirectory of the region directory used for compaction output.private final org.apache.hadoop.fs.Pathprivate final RegionInfo(package private) final RegionInfoprivate final org.apache.hadoop.fs.Path -
Constructor Summary
ConstructorsConstructorDescriptionHRegionFileSystem(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path tableDir, RegionInfo regionInfo) Create a view to the on-disk region -
Method Summary
Modifier and TypeMethodDescription(package private) Pair<org.apache.hadoop.fs.Path,org.apache.hadoop.fs.Path> bulkLoadStoreFile(String familyName, org.apache.hadoop.fs.Path srcPath, long seqNum) Bulk load: Add a specified store file to the specified family.(package private) voidWrite out an info file under the stored region directory.(package private) voidcleanupDaughterRegion(RegionInfo regionInfo) Remove daughter regionvoidcleanupMergedRegion(RegionInfo mergedRegion) Remove merged region(package private) voidClean up any temp detritus that may have been left around from previous operation attempts.org.apache.hadoop.fs.PathcommitDaughterRegion(RegionInfo regionInfo, List<org.apache.hadoop.fs.Path> allRegionFiles, MasterProcedureEnv env) Commit a daughter region, moving it from the split temporary directory to the proper location in the filesystem.voidcommitMergedRegion(List<org.apache.hadoop.fs.Path> allMergedFiles, MasterProcedureEnv env) Commit a merged region, making it ready for use.org.apache.hadoop.fs.PathcommitStoreFile(String familyName, org.apache.hadoop.fs.Path buildPath) Move the file from a build/temp location to the main family store directory.(package private) org.apache.hadoop.fs.PathcommitStoreFile(org.apache.hadoop.fs.Path buildPath, org.apache.hadoop.fs.Path dstPath) (package private) booleancreateDir(org.apache.hadoop.fs.Path dir) Creates a directory.private static booleancreateDirOnFileSystem(org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.Path dir) Creates a directory for a filesystem and configuration object.static HRegionFileSystemcreateRegionOnFileSystem(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path tableDir, RegionInfo regionInfo) Create a new Region on file-system.voidcreateSplitsDir(RegionInfo daughterA, RegionInfo daughterB) Creates region split daughter directories under the table dir.(package private) org.apache.hadoop.fs.PathcreateStoreDir(String familyName) Create the store directory for the specified family nameorg.apache.hadoop.fs.PathGenerate a unique temporary Path.org.apache.hadoop.fs.PathcreateTempName(String suffix) Generate a unique temporary Path.(package private) booleandeleteDir(org.apache.hadoop.fs.Path dir) Deletes a directory.voiddeleteFamily(String familyName) Remove the region family from disk, archiving the store files.static voiddeleteRegionFromFileSystem(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path tableDir, RegionInfo regionInfo) Remove the region from the table directory, archiving the region's hfiles.private static StringgenerateUniqueName(String suffix) Generate a unique file name, used by createTempName() and commitStoreFile()Returns the set of families present on disk norg.apache.hadoop.fs.FileSystemReturns the underlyingFileSystem(package private) org.apache.hadoop.fs.PathgetMergesDir(RegionInfo hri) org.apache.hadoop.fs.PathReturnsPathto the region directory.Returns theRegionInfothat describe this on-disk region viewprivate static byte[]Returns Content of the file we write out to the filesystem under a regionorg.apache.hadoop.fs.PathgetSplitsDir(RegionInfo hri) getStoragePolicyName(String familyName) Get the storage policy of the directory of CF.org.apache.hadoop.fs.PathgetStoreDir(String familyName) Returns the directory path of the specified family(package private) StoreFileInfogetStoreFileInfo(String familyName, String fileName) Return the store file information of the specified family/file.(package private) org.apache.hadoop.fs.PathgetStoreFilePath(String familyName, String fileName) Return Qualified Path of the specified family/filegetStoreFiles(String familyName) Returns the store files available for the family.getStoreFiles(String familyName, boolean validate) Returns the store files available for the family.static List<org.apache.hadoop.fs.LocatedFileStatus>getStoreFilesLocatedStatus(HRegionFileSystem regionfs, String familyName, boolean validate) Returns the store files' LocatedFileStatus which available for the family.static org.apache.hadoop.fs.PathgetStoreHomedir(org.apache.hadoop.fs.Path tabledir, String encodedName, byte[] family) static org.apache.hadoop.fs.PathgetStoreHomedir(org.apache.hadoop.fs.Path tabledir, RegionInfo hri, byte[] family) org.apache.hadoop.fs.PathReturnsPathto the region's root directory.org.apache.hadoop.fs.PathReturnsPathto the region's temp directory, used for file creationsbooleanhasReferences(String familyName) Returns true if the specified family has reference filesbooleanCheck whether region has Reference fileprivate voidinsertRegionFilesIntoStoreTracker(List<org.apache.hadoop.fs.Path> allFiles, MasterProcedureEnv env, HRegionFileSystem regionFs) static RegionInfoloadRegionInfoFileContent(org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path regionDir) Create aRegionInfofrom the serialized version on-disk.org.apache.hadoop.fs.PathmergeStoreFile(RegionInfo mergingRegion, String familyName, HStoreFile f) Write out a merge reference under the given merges directory.(package private) static booleanmkdirs(org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.Path dir) static HRegionFileSystemopenRegionFromFileSystem(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path tableDir, RegionInfo regionInfo, boolean readOnly) Open Region from file-system.private org.apache.hadoop.fs.PathpreCommitStoreFile(String familyName, org.apache.hadoop.fs.Path buildPath, long seqNum, boolean generateNewName) Generate the filename in the main family store directory for moving the file from a build/temp location.voidremoveStoreFile(String familyName, org.apache.hadoop.fs.Path filePath) Archives the specified store file from the specified family.voidremoveStoreFiles(String familyName, Collection<HStoreFile> storeFiles) Closes and archives the specified store files from the specified family.(package private) booleanrename(org.apache.hadoop.fs.Path srcpath, org.apache.hadoop.fs.Path dstPath) Renames a directory.voidsetStoragePolicy(String policyName) Set storage policy for a whole region.voidsetStoragePolicy(String familyName, String policyName) Set the directory of CF to the specified storage policy.private voidsleepBeforeRetry(String msg, int sleepMultiplier) sleeping logic; handles the interrupt exception.private static voidsleepBeforeRetry(String msg, int sleepMultiplier, int baseSleepBeforeRetries, int hdfsClientRetriesNumber) sleeping logic for static methods; handles the interrupt exception.org.apache.hadoop.fs.PathsplitStoreFile(RegionInfo hri, String familyName, HStoreFile f, byte[] splitRow, boolean top, RegionSplitPolicy splitPolicy) Write out a split reference.private static voidwriteRegionInfoFileContent(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path regionInfoFile, byte[] content) Write the .regioninfo file on-disk.private voidwriteRegionInfoOnFilesystem(boolean useTempDir) Write out an info file under the region directory.private voidwriteRegionInfoOnFilesystem(byte[] regionInfoContent, boolean useTempDir) Write out an info file under the region directory.
-
Field Details
-
LOG
-
REGION_INFO_FILE
Name of the region info file that resides just under the region directory.- See Also:
-
REGION_MERGES_DIR
Temporary subdirectory of the region directory used for merges.- See Also:
-
REGION_SPLITS_DIR
Temporary subdirectory of the region directory used for splits.- See Also:
-
REGION_TEMP_DIR
Temporary subdirectory of the region directory used for compaction output.- See Also:
-
regionInfo
-
regionInfoForFs
-
conf
-
tableDir
-
fs
-
regionDir
-
hdfsClientRetriesNumber
In order to handle NN connectivity hiccups, one need to retry non-idempotent operation at the client level. -
baseSleepBeforeRetries
-
DEFAULT_HDFS_CLIENT_RETRIES_NUMBER
- See Also:
-
DEFAULT_BASE_SLEEP_BEFORE_RETRIES
- See Also:
-
-
Constructor Details
-
HRegionFileSystem
HRegionFileSystem(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path tableDir, RegionInfo regionInfo) Create a view to the on-disk region- Parameters:
conf- theConfigurationto usefs-FileSystemthat contains the regiontableDir-Pathto where the table is being storedregionInfo-RegionInfofor region
-
-
Method Details
-
getFileSystem
Returns the underlyingFileSystem -
getRegionInfo
Returns theRegionInfothat describe this on-disk region view -
getRegionInfoForFS
-
getTableDir
ReturnsPathto the region's root directory. -
getRegionDir
ReturnsPathto the region directory. -
getTempDir
ReturnsPathto the region's temp directory, used for file creations -
cleanupTempDir
Clean up any temp detritus that may have been left around from previous operation attempts.- Throws:
IOException
-
getStoreDir
Returns the directory path of the specified family- Parameters:
familyName- Column Family Name- Returns:
Pathto the directory of the specified family
-
getStoreHomedir
public static org.apache.hadoop.fs.Path getStoreHomedir(org.apache.hadoop.fs.Path tabledir, RegionInfo hri, byte[] family) - Parameters:
tabledir-Pathto where the table is being storedhri-RegionInfofor the region.family-ColumnFamilyDescriptordescribing the column family- Returns:
- Path to family/Store home directory.
-
getStoreHomedir
public static org.apache.hadoop.fs.Path getStoreHomedir(org.apache.hadoop.fs.Path tabledir, String encodedName, byte[] family) - Parameters:
tabledir-Pathto where the table is being storedencodedName- Encoded region name.family-ColumnFamilyDescriptordescribing the column family- Returns:
- Path to family/Store home directory.
-
createStoreDir
Create the store directory for the specified family name- Parameters:
familyName- Column Family Name- Returns:
Pathto the directory of the specified family- Throws:
IOException- if the directory creation fails.
-
setStoragePolicy
Set the directory of CF to the specified storage policy.
"LAZY_PERSIST", "ALL_SSD", "ONE_SSD", "HOT", "WARM", "COLD"
SeeHdfsConstantsfor more details.- Parameters:
familyName- The name of column family.policyName- The name of the storage policy: 'HOT', 'COLD', etc. See hadoop 2.6+ org.apache.hadoop.hdfs.protocol.HdfsConstants for possible list e.g 'COLD', 'WARM', 'HOT', 'ONE_SSD', 'ALL_SSD', 'LAZY_PERSIST'.
-
setStoragePolicy
Set storage policy for a whole region.
"LAZY_PERSIST", "ALL_SSD", "ONE_SSD", "HOT", "WARM", "COLD"
SeeHdfsConstantsfor more details.- Parameters:
policyName- The name of the storage policy: 'HOT', 'COLD', etc. See hadoop 2.6+ org.apache.hadoop.hdfs.protocol.HdfsConstants for possible list e.g 'COLD', 'WARM', 'HOT', 'ONE_SSD', 'ALL_SSD', 'LAZY_PERSIST'.
-
getStoragePolicyName
Get the storage policy of the directory of CF.- Parameters:
familyName- The name of column family.- Returns:
- Storage policy name, or
nullif not usingHFileSystemor exception thrown when trying to get policy
-
getStoreFiles
Returns the store files available for the family. This methods performs the filtering based on the valid store files.- Parameters:
familyName- Column Family Name- Returns:
- a set of
StoreFileInfofor the specified family. - Throws:
IOException
-
getStoreFiles
Returns the store files available for the family. This methods performs the filtering based on the valid store files.- Parameters:
familyName- Column Family Name- Returns:
- a set of
StoreFileInfofor the specified family. - Throws:
IOException
-
getStoreFilesLocatedStatus
public static List<org.apache.hadoop.fs.LocatedFileStatus> getStoreFilesLocatedStatus(HRegionFileSystem regionfs, String familyName, boolean validate) throws IOException Returns the store files' LocatedFileStatus which available for the family. This methods performs the filtering based on the valid store files.- Parameters:
familyName- Column Family Name- Returns:
- a list of store files' LocatedFileStatus for the specified family.
- Throws:
IOException
-
getStoreFilePath
Return Qualified Path of the specified family/file- Parameters:
familyName- Column Family NamefileName- File Name- Returns:
- The qualified Path for the specified family/file
-
getStoreFileInfo
Return the store file information of the specified family/file.- Parameters:
familyName- Column Family NamefileName- File Name- Returns:
- The
StoreFileInfofor the specified family/file - Throws:
IOException
-
hasReferences
Returns true if the specified family has reference files- Parameters:
familyName- Column Family Name- Returns:
- true if family contains reference files
- Throws:
IOException
-
hasReferences
Check whether region has Reference file- Parameters:
htd- table desciptor of the region- Returns:
- true if region has reference file
- Throws:
IOException
-
getFamilies
Returns the set of families present on disk n- Throws:
IOException
-
deleteFamily
Remove the region family from disk, archiving the store files.- Parameters:
familyName- Column Family Name- Throws:
IOException- if an error occours during the archiving
-
generateUniqueName
Generate a unique file name, used by createTempName() and commitStoreFile()- Parameters:
suffix- extra information to append to the generated name- Returns:
- Unique file name
-
createTempName
Generate a unique temporary Path. Used in conjuction with commitStoreFile() to get a safer file creation.Path file = fs.createTempName(); ...StoreFile.Writer(file)... fs.commitStoreFile("family", file);- Returns:
- Unique
Pathof the temporary file
-
createTempName
Generate a unique temporary Path. Used in conjuction with commitStoreFile() to get a safer file creation.Path file = fs.createTempName(); ...StoreFile.Writer(file)... fs.commitStoreFile("family", file);- Parameters:
suffix- extra information to append to the generated name- Returns:
- Unique
Pathof the temporary file
-
commitStoreFile
public org.apache.hadoop.fs.Path commitStoreFile(String familyName, org.apache.hadoop.fs.Path buildPath) throws IOException Move the file from a build/temp location to the main family store directory.- Parameters:
familyName- Family that will gain the filebuildPath-Pathto the file to commit.- Returns:
- The new
Pathof the committed file - Throws:
IOException
-
preCommitStoreFile
private org.apache.hadoop.fs.Path preCommitStoreFile(String familyName, org.apache.hadoop.fs.Path buildPath, long seqNum, boolean generateNewName) throws IOException Generate the filename in the main family store directory for moving the file from a build/temp location.- Parameters:
familyName- Family that will gain the filebuildPath-Pathto the file to commit.seqNum- Sequence Number to append to the file name (less then 0 if no sequence number)generateNewName- False if you want to keep the buildPath name- Returns:
- The new
Pathof the to be committed file - Throws:
IOException
-
commitStoreFile
org.apache.hadoop.fs.Path commitStoreFile(org.apache.hadoop.fs.Path buildPath, org.apache.hadoop.fs.Path dstPath) throws IOException - Throws:
IOException
-
removeStoreFile
public void removeStoreFile(String familyName, org.apache.hadoop.fs.Path filePath) throws IOException Archives the specified store file from the specified family.- Parameters:
familyName- Family that contains the store filesfilePath-Pathto the store file to remove- Throws:
IOException- if the archiving fails
-
removeStoreFiles
public void removeStoreFiles(String familyName, Collection<HStoreFile> storeFiles) throws IOException Closes and archives the specified store files from the specified family.- Parameters:
familyName- Family that contains the store filesstoreFiles- set of store files to remove- Throws:
IOException- if the archiving fails
-
bulkLoadStoreFile
Pair<org.apache.hadoop.fs.Path,org.apache.hadoop.fs.Path> bulkLoadStoreFile(String familyName, org.apache.hadoop.fs.Path srcPath, long seqNum) throws IOException Bulk load: Add a specified store file to the specified family. If the source file is on the same different file-system is moved from the source location to the destination location, otherwise is copied over.- Parameters:
familyName- Family that will gain the filesrcPath-Pathto the file to importseqNum- Bulk Load sequence number- Returns:
- The destination
Pathof the bulk loaded file - Throws:
IOException
-
getSplitsDir
-
cleanupDaughterRegion
Remove daughter region- Parameters:
regionInfo- daughterRegionInfo- Throws:
IOException
-
commitDaughterRegion
public org.apache.hadoop.fs.Path commitDaughterRegion(RegionInfo regionInfo, List<org.apache.hadoop.fs.Path> allRegionFiles, MasterProcedureEnv env) throws IOException Commit a daughter region, moving it from the split temporary directory to the proper location in the filesystem.- Parameters:
regionInfo- daughterRegionInfo- Throws:
IOException
-
insertRegionFilesIntoStoreTracker
private void insertRegionFilesIntoStoreTracker(List<org.apache.hadoop.fs.Path> allFiles, MasterProcedureEnv env, HRegionFileSystem regionFs) throws IOException - Throws:
IOException
-
createSplitsDir
Creates region split daughter directories under the table dir. If the daughter regions already exist, for example, in the case of a recovery from a previous failed split procedure, this method deletes the given region dir recursively, then recreates it again.- Throws:
IOException
-
splitStoreFile
public org.apache.hadoop.fs.Path splitStoreFile(RegionInfo hri, String familyName, HStoreFile f, byte[] splitRow, boolean top, RegionSplitPolicy splitPolicy) throws IOException Write out a split reference. Package local so it doesnt leak out of regionserver.- Parameters:
hri-RegionInfoof the destinationfamilyName- Column Family Namef- File to split.splitRow- Split Rowtop- True if we are referring to the top half of the hfile.splitPolicy- A split policy instance; be careful! May not be full populated; e.g. if this method is invoked on the Master side, then the RegionSplitPolicy will NOT have a reference to a Region.- Returns:
- Path to created reference.
- Throws:
IOException
-
getMergesDir
-
cleanupMergedRegion
Remove merged region- Parameters:
mergedRegion-RegionInfo- Throws:
IOException
-
mkdirs
static boolean mkdirs(org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.Path dir) throws IOException - Throws:
IOException
-
mergeStoreFile
public org.apache.hadoop.fs.Path mergeStoreFile(RegionInfo mergingRegion, String familyName, HStoreFile f) throws IOException Write out a merge reference under the given merges directory.- Parameters:
mergingRegion-RegionInfofor one of the regions being merged.familyName- Column Family Namef- File to create reference.- Returns:
- Path to created reference.
- Throws:
IOException- if the merge write fails.
-
commitMergedRegion
public void commitMergedRegion(List<org.apache.hadoop.fs.Path> allMergedFiles, MasterProcedureEnv env) throws IOException Commit a merged region, making it ready for use.- Throws:
IOException
-
getRegionInfoFileContent
Returns Content of the file we write out to the filesystem under a region- Throws:
IOException
-
loadRegionInfoFileContent
public static RegionInfo loadRegionInfoFileContent(org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path regionDir) throws IOException Create aRegionInfofrom the serialized version on-disk.- Parameters:
fs-FileSystemthat contains the Region Info fileregionDir-Pathto the Region Directory that contains the Info file- Returns:
- An
RegionInfoinstance gotten from the Region Info file. - Throws:
IOException- if an error occurred during file open/read operation.
-
writeRegionInfoFileContent
private static void writeRegionInfoFileContent(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path regionInfoFile, byte[] content) throws IOException Write the .regioninfo file on-disk. Overwrites if exists already.- Throws:
IOException
-
checkRegionInfoOnFilesystem
Write out an info file under the stored region directory. Useful recovering mangled regions. If the regionInfo already exists on-disk, then we fast exit.- Throws:
IOException
-
writeRegionInfoOnFilesystem
Write out an info file under the region directory. Useful recovering mangled regions.- Parameters:
useTempDir- indicate whether or not using the region .tmp dir for a safer file creation.- Throws:
IOException
-
writeRegionInfoOnFilesystem
private void writeRegionInfoOnFilesystem(byte[] regionInfoContent, boolean useTempDir) throws IOException Write out an info file under the region directory. Useful recovering mangled regions.- Parameters:
regionInfoContent- serialized version of theRegionInfouseTempDir- indicate whether or not using the region .tmp dir for a safer file creation.- Throws:
IOException
-
createRegionOnFileSystem
public static HRegionFileSystem createRegionOnFileSystem(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path tableDir, RegionInfo regionInfo) throws IOException Create a new Region on file-system.- Parameters:
conf- theConfigurationto usefs-FileSystemfrom which to add the regiontableDir-Pathto where the table is being storedregionInfo-RegionInfofor region to be added- Throws:
IOException- if the region creation fails due to a FileSystem exception.
-
openRegionFromFileSystem
public static HRegionFileSystem openRegionFromFileSystem(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path tableDir, RegionInfo regionInfo, boolean readOnly) throws IOException Open Region from file-system.- Parameters:
conf- theConfigurationto usefs-FileSystemfrom which to add the regiontableDir-Pathto where the table is being storedregionInfo-RegionInfofor region to be addedreadOnly- True if you don't want to edit the region data- Throws:
IOException- if the region creation fails due to a FileSystem exception.
-
deleteRegionFromFileSystem
public static void deleteRegionFromFileSystem(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path tableDir, RegionInfo regionInfo) throws IOException Remove the region from the table directory, archiving the region's hfiles.- Parameters:
conf- theConfigurationto usefs-FileSystemfrom which to remove the regiontableDir-Pathto where the table is being storedregionInfo-RegionInfofor region to be deleted- Throws:
IOException- if the request cannot be completed
-
createDir
Creates a directory. Assumes the user has already checked for this directory existence.- Returns:
- the result of fs.mkdirs(). In case underlying fs throws an IOException, it checks whether the directory exists or not, and returns true if it exists.
- Throws:
IOException
-
rename
boolean rename(org.apache.hadoop.fs.Path srcpath, org.apache.hadoop.fs.Path dstPath) throws IOException Renames a directory. Assumes the user has already checked for this directory existence.- Returns:
- true if rename is successful.
- Throws:
IOException
-
deleteDir
Deletes a directory. Assumes the user has already checked for this directory existence.- Returns:
- true if the directory is deleted.
- Throws:
IOException
-
sleepBeforeRetry
sleeping logic; handles the interrupt exception.- Throws:
InterruptedException
-
createDirOnFileSystem
private static boolean createDirOnFileSystem(org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.Path dir) throws IOException Creates a directory for a filesystem and configuration object. Assumes the user has already checked for this directory existence.- Returns:
- the result of fs.mkdirs(). In case underlying fs throws an IOException, it checks whether the directory exists or not, and returns true if it exists.
- Throws:
IOException
-
sleepBeforeRetry
private static void sleepBeforeRetry(String msg, int sleepMultiplier, int baseSleepBeforeRetries, int hdfsClientRetriesNumber) throws InterruptedException sleeping logic for static methods; handles the interrupt exception. Keeping a static version for this to avoid re-looking for the integer values.- Throws:
InterruptedException
-