Class WALSplitUtil

java.lang.Object
org.apache.hadoop.hbase.wal.WALSplitUtil

@Private public final class WALSplitUtil extends Object
This class provides static methods to support WAL splitting related works
  • Nested Class Summary

    Nested Classes
    Modifier and Type
    Class
    Description
    static class 
    A struct used by getMutationsFromWALEntry
  • Field Summary

    Fields
    Modifier and Type
    Field
    Description
    private static final Pattern
     
    private static final org.slf4j.Logger
     
    private static final String
     
    private static final String
     
    private static final String
     
    private static final int
     
  • Constructor Summary

    Constructors
    Modifier
    Constructor
    Description
    private
     
  • Method Summary

    Modifier and Type
    Method
    Description
    (package private) static void
    archive(org.apache.hadoop.fs.Path wal, boolean corrupt, org.apache.hadoop.fs.Path oldWALDir, org.apache.hadoop.fs.FileSystem walFS, org.apache.hadoop.conf.Configuration conf)
    Moves processed logs to a oldLogDir after successful processing Moves corrupted logs (any log that couldn't be successfully parsed to corruptDir (.corrupt) for later investigation
    static void
    finishSplitLogFile(String logfile, org.apache.hadoop.conf.Configuration conf)
    Completes the work done by splitLogFile by archiving logs
    (package private) static String
     
    (package private) static org.apache.hadoop.fs.Path
    getCompletedRecoveredEditsFilePath(org.apache.hadoop.fs.Path srcPath, long maximumEditWALSeqNum)
    Get the completed recovered edits file path, renaming it to be by last edit in the file from its first edit.
    static long
    getMaxRegionSequenceId(org.apache.hadoop.conf.Configuration conf, RegionInfo region, IOExceptionSupplier<org.apache.hadoop.fs.FileSystem> rootFsSupplier, IOExceptionSupplier<org.apache.hadoop.fs.FileSystem> walFsSupplier)
    Deprecated.
    Only for compatibility, will be removed in 4.0.0.
    static long
    getMaxRegionSequenceId(org.apache.hadoop.fs.FileSystem walFS, org.apache.hadoop.fs.Path regionDir)
    Get the max sequence id which is stored in the region directory.
    private static long
    getMaxSequenceId(org.apache.hadoop.fs.FileStatus[] files)
     
    getMutationsFromWALEntry(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry entry, ExtendedCellScanner cells, Pair<WALKey,WALEdit> logEntry, Durability durability)
    Deprecated.
    Since 3.0.0, will be removed in 4.0.0.
    static org.apache.hadoop.fs.FileStatus[]
    getRecoveredHFiles(org.apache.hadoop.fs.FileSystem rootFS, org.apache.hadoop.fs.Path regionDir, String familyName)
     
    private static org.apache.hadoop.fs.Path
    getRecoveredHFilesDir(org.apache.hadoop.fs.Path regionDir, String familyName)
     
    static org.apache.hadoop.fs.Path
    getRegionDirRecoveredEditsDir(org.apache.hadoop.fs.Path regionDir)
     
    (package private) static org.apache.hadoop.fs.Path
    getRegionSplitEditsPath(TableName tableName, byte[] encodedRegionName, long seqId, String fileNameBeingSplit, String tmpDirName, org.apache.hadoop.conf.Configuration conf)
    Path to a file under RECOVERED_EDITS_DIR directory of the region found in logEntry named for the sequenceid in the passed logEntry: e.g.
    private static org.apache.hadoop.fs.FileStatus[]
    getSequenceIdFiles(org.apache.hadoop.fs.FileSystem walFS, org.apache.hadoop.fs.Path regionDir)
     
    static NavigableSet<org.apache.hadoop.fs.Path>
    getSplitEditFilesSorted(org.apache.hadoop.fs.FileSystem walFS, org.apache.hadoop.fs.Path regionDir)
    Returns sorted set of edit files made by splitter, excluding files with '.temp' suffix.
    private static String
     
    static boolean
    hasRecoveredEdits(org.apache.hadoop.conf.Configuration conf, RegionInfo regionInfo)
    Check whether there is recovered.edits in the region dir
    static boolean
    isSequenceIdFile(org.apache.hadoop.fs.Path file)
    Is the given file a region open sequence id file.
    private static void
    mkdir(org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path dir)
     
    static org.apache.hadoop.fs.Path
    moveAsideBadEditsFile(org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path edits)
    Move aside a bad edits file.
    static void
    moveWAL(org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path p, org.apache.hadoop.fs.Path targetDir)
    Move WAL.
    (package private) static org.apache.hadoop.fs.Path
    tryCreateRecoveredHFilesDir(org.apache.hadoop.fs.FileSystem rootFS, org.apache.hadoop.conf.Configuration conf, TableName tableName, String encodedRegionName, String familyName)
    Return path to recovered.hfiles directory of the region's column family: e.g.
    static void
    writeRegionSequenceIdFile(org.apache.hadoop.fs.FileSystem walFS, org.apache.hadoop.fs.Path regionDir, long newMaxSeqId)
    Create a file with name as region's max sequence id

    Methods inherited from class java.lang.Object

    clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
  • Field Details

  • Constructor Details

  • Method Details

    • finishSplitLogFile

      public static void finishSplitLogFile(String logfile, org.apache.hadoop.conf.Configuration conf) throws IOException
      Completes the work done by splitLogFile by archiving logs

      It is invoked by SplitLogManager once it knows that one of the SplitLogWorkers have completed the splitLogFile() part. If the master crashes then this function might get called multiple times.

      Throws:
      IOException
    • archive

      static void archive(org.apache.hadoop.fs.Path wal, boolean corrupt, org.apache.hadoop.fs.Path oldWALDir, org.apache.hadoop.fs.FileSystem walFS, org.apache.hadoop.conf.Configuration conf) throws IOException
      Moves processed logs to a oldLogDir after successful processing Moves corrupted logs (any log that couldn't be successfully parsed to corruptDir (.corrupt) for later investigation
      Throws:
      IOException
    • mkdir

      private static void mkdir(org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path dir) throws IOException
      Throws:
      IOException
    • moveWAL

      public static void moveWAL(org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path p, org.apache.hadoop.fs.Path targetDir) throws IOException
      Move WAL. Used to move processed WALs to archive or bad WALs to corrupt WAL dir. WAL may have already been moved; makes allowance.
      Throws:
      IOException
    • getRegionSplitEditsPath

      static org.apache.hadoop.fs.Path getRegionSplitEditsPath(TableName tableName, byte[] encodedRegionName, long seqId, String fileNameBeingSplit, String tmpDirName, org.apache.hadoop.conf.Configuration conf) throws IOException
      Path to a file under RECOVERED_EDITS_DIR directory of the region found in logEntry named for the sequenceid in the passed logEntry: e.g. /hbase/some_table/2323432434/recovered.edits/2332. This method also ensures existence of RECOVERED_EDITS_DIR under the region creating it if necessary. And also set storage policy for RECOVERED_EDITS_DIR if WAL_STORAGE_POLICY is configured.
      Parameters:
      tableName - the table name
      encodedRegionName - the encoded region name
      seqId - the sequence id which used to generate file name
      fileNameBeingSplit - the file being split currently. Used to generate tmp file name.
      tmpDirName - of the directory used to sideline old recovered edits file
      conf - configuration
      Returns:
      Path to file into which to dump split log edits.
      Throws:
      IOException
    • getTmpRecoveredEditsFileName

      private static String getTmpRecoveredEditsFileName(String fileName)
    • getCompletedRecoveredEditsFilePath

      static org.apache.hadoop.fs.Path getCompletedRecoveredEditsFilePath(org.apache.hadoop.fs.Path srcPath, long maximumEditWALSeqNum)
      Get the completed recovered edits file path, renaming it to be by last edit in the file from its first edit. Then we could use the name to skip recovered edits when doing HRegion#replayRecoveredEditsIfAny(Map, CancelableProgressable, MonitoredTask).
      Returns:
      dstPath take file's last edit log seq num as the name
    • formatRecoveredEditsFileName

      static String formatRecoveredEditsFileName(long seqid)
    • getRegionDirRecoveredEditsDir

      public static org.apache.hadoop.fs.Path getRegionDirRecoveredEditsDir(org.apache.hadoop.fs.Path regionDir)
      Parameters:
      regionDir - This regions directory in the filesystem.
      Returns:
      The directory that holds recovered edits files for the region regionDir
    • hasRecoveredEdits

      public static boolean hasRecoveredEdits(org.apache.hadoop.conf.Configuration conf, RegionInfo regionInfo) throws IOException
      Check whether there is recovered.edits in the region dir
      Parameters:
      conf - conf
      regionInfo - the region to check
      Returns:
      true if recovered.edits exist in the region dir
      Throws:
      IOException
    • getMaxRegionSequenceId

      @Deprecated public static long getMaxRegionSequenceId(org.apache.hadoop.conf.Configuration conf, RegionInfo region, IOExceptionSupplier<org.apache.hadoop.fs.FileSystem> rootFsSupplier, IOExceptionSupplier<org.apache.hadoop.fs.FileSystem> walFsSupplier) throws IOException
      Deprecated.
      Only for compatibility, will be removed in 4.0.0.
      This method will check 3 places for finding the max sequence id file. One is the expected place, another is the old place under the region directory, and the last one is the wrong one we introduced in HBASE-20734. See HBASE-22617 for more details.

      Notice that, you should always call this method instead of getMaxRegionSequenceId(FileSystem, Path) until 4.0.0 release.

      Throws:
      IOException
    • getSplitEditFilesSorted

      public static NavigableSet<org.apache.hadoop.fs.Path> getSplitEditFilesSorted(org.apache.hadoop.fs.FileSystem walFS, org.apache.hadoop.fs.Path regionDir) throws IOException
      Returns sorted set of edit files made by splitter, excluding files with '.temp' suffix.
      Parameters:
      walFS - WAL FileSystem used to retrieving split edits files.
      regionDir - WAL region dir to look for recovered edits files under.
      Returns:
      Files in passed regionDir as a sorted set.
      Throws:
      IOException
    • moveAsideBadEditsFile

      public static org.apache.hadoop.fs.Path moveAsideBadEditsFile(org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path edits) throws IOException
      Move aside a bad edits file.
      Parameters:
      fs - the file system used to rename bad edits file.
      edits - Edits file to move aside.
      Returns:
      The name of the moved aside file.
      Throws:
      IOException
    • isSequenceIdFile

      public static boolean isSequenceIdFile(org.apache.hadoop.fs.Path file)
      Is the given file a region open sequence id file.
    • getSequenceIdFiles

      private static org.apache.hadoop.fs.FileStatus[] getSequenceIdFiles(org.apache.hadoop.fs.FileSystem walFS, org.apache.hadoop.fs.Path regionDir) throws IOException
      Throws:
      IOException
    • getMaxSequenceId

      private static long getMaxSequenceId(org.apache.hadoop.fs.FileStatus[] files)
    • getMaxRegionSequenceId

      public static long getMaxRegionSequenceId(org.apache.hadoop.fs.FileSystem walFS, org.apache.hadoop.fs.Path regionDir) throws IOException
      Get the max sequence id which is stored in the region directory. -1 if none.
      Throws:
      IOException
    • writeRegionSequenceIdFile

      public static void writeRegionSequenceIdFile(org.apache.hadoop.fs.FileSystem walFS, org.apache.hadoop.fs.Path regionDir, long newMaxSeqId) throws IOException
      Create a file with name as region's max sequence id
      Throws:
      IOException
    • getMutationsFromWALEntry

      @Deprecated public static List<WALSplitUtil.MutationReplay> getMutationsFromWALEntry(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry entry, ExtendedCellScanner cells, Pair<WALKey,WALEdit> logEntry, Durability durability) throws IOException
      Deprecated.
      Since 3.0.0, will be removed in 4.0.0.
      This function is used to construct mutations from a WALEntry. It also reconstructs WALKey & WALEdit from the passed in WALEntry
      Parameters:
      logEntry - pair of WALKey and WALEdit instance stores WALKey and WALEdit instances extracted from the passed in WALEntry.
      Returns:
      list of Pair<MutationType, Mutation> to be replayed
      Throws:
      IOException
    • tryCreateRecoveredHFilesDir

      static org.apache.hadoop.fs.Path tryCreateRecoveredHFilesDir(org.apache.hadoop.fs.FileSystem rootFS, org.apache.hadoop.conf.Configuration conf, TableName tableName, String encodedRegionName, String familyName) throws IOException
      Return path to recovered.hfiles directory of the region's column family: e.g. /hbase/some_table/2323432434/cf/recovered.hfiles/. This method also ensures existence of recovered.hfiles directory under the region's column family, creating it if necessary.
      Parameters:
      rootFS - the root file system
      conf - configuration
      tableName - the table name
      encodedRegionName - the encoded region name
      familyName - the column family name
      Returns:
      Path to recovered.hfiles directory of the region's column family.
      Throws:
      IOException
    • getRecoveredHFilesDir

      private static org.apache.hadoop.fs.Path getRecoveredHFilesDir(org.apache.hadoop.fs.Path regionDir, String familyName)
      Parameters:
      regionDir - This regions directory in the filesystem
      familyName - The column family name
      Returns:
      The directory that holds recovered hfiles for the region's column family
    • getRecoveredHFiles

      public static org.apache.hadoop.fs.FileStatus[] getRecoveredHFiles(org.apache.hadoop.fs.FileSystem rootFS, org.apache.hadoop.fs.Path regionDir, String familyName) throws IOException
      Throws:
      IOException