You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2020/10/12 15:02:53 UTC

[hbase] branch branch-2.3 updated: HBASE-25168 Unify WAL name timestamp parsers

This is an automated email from the ASF dual-hosted git repository.

stack pushed a commit to branch branch-2.3
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2.3 by this push:
     new b8a7e0f  HBASE-25168 Unify WAL name timestamp parsers
b8a7e0f is described below

commit b8a7e0fed3e21547d545ab1bca494a35bd93ca3e
Author: stack <st...@apache.org>
AuthorDate: Fri Oct 9 21:00:30 2020 -0700

    HBASE-25168 Unify WAL name timestamp parsers
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
    Signed-off-by: Peter Somogyi <ps...@apache.org>
---
 .../hadoop/hbase/mapreduce/WALInputFormat.java     |  2 +-
 .../hadoop/hbase/wal/AbstractFSWALProvider.java    | 55 ++++++++++++++++------
 .../main/java/org/apache/hadoop/hbase/wal/WAL.java | 29 ------------
 3 files changed, 42 insertions(+), 44 deletions(-)

diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java
index b410fc2..14bfec7 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java
@@ -336,7 +336,7 @@ public class WALInputFormat extends InputFormat<WALKey, WALEdit> {
 
   static void addFile(List<FileStatus> result, LocatedFileStatus lfs, long startTime,
       long endTime) {
-    long timestamp = WAL.getTimestamp(lfs.getPath().getName());
+    long timestamp = AbstractFSWALProvider.getTimestamp(lfs.getPath().getName());
     if (timestamp > 0) {
       // Looks like a valid timestamp.
       if (timestamp <= endTime && timestamp >= startTime) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
index cdbe51e..e632724 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
@@ -27,6 +27,7 @@ import java.util.Objects;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -257,29 +258,38 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
   public static final String SPLITTING_EXT = "-splitting";
 
   /**
-   * It returns the file create timestamp from the file name. For name format see
+   * Pattern used to validate a WAL file name see {@link #validateWALFilename(String)} for
+   * description.
+   */
+  private static final Pattern WAL_FILE_NAME_PATTERN =
+    Pattern.compile("(.+)\\.(\\d+)(\\.[0-9A-Za-z]+)?");
+
+  /**
+   * Define for when no timestamp found.
+   */
+  private static final long NO_TIMESTAMP = -1L;
+
+  /**
+   * It returns the file create timestamp (the 'FileNum') from the file name. For name format see
    * {@link #validateWALFilename(String)} public until remaining tests move to o.a.h.h.wal
    * @param wal must not be null
    * @return the file number that is part of the WAL file name
    */
   @VisibleForTesting
   public static long extractFileNumFromWAL(final WAL wal) {
-    final Path walName = ((AbstractFSWAL<?>) wal).getCurrentFileName();
-    if (walName == null) {
+    final Path walPath = ((AbstractFSWAL<?>) wal).getCurrentFileName();
+    if (walPath == null) {
       throw new IllegalArgumentException("The WAL path couldn't be null");
     }
-    final String[] walPathStrs = walName.toString().split("\\" + WAL_FILE_NAME_DELIMITER);
-    return Long.parseLong(walPathStrs[walPathStrs.length - (isMetaFile(walName) ? 2 : 1)]);
+    String name = walPath.getName();
+    long timestamp = getTimestamp(name);
+    if (timestamp == NO_TIMESTAMP) {
+      throw new IllegalArgumentException(name + " is not a valid wal file name");
+    }
+    return timestamp;
   }
 
   /**
-   * Pattern used to validate a WAL file name see {@link #validateWALFilename(String)} for
-   * description.
-   */
-  private static final Pattern pattern =
-    Pattern.compile(".*\\.\\d*(" + META_WAL_PROVIDER_ID + ")*");
-
-  /**
    * A WAL file name is of the format: &lt;wal-name&gt;{@link #WAL_FILE_NAME_DELIMITER}
    * &lt;file-creation-timestamp&gt;[.meta]. provider-name is usually made up of a server-name and a
    * provider-id
@@ -287,7 +297,24 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
    * @return <tt>true</tt> if the filename matches an WAL, <tt>false</tt> otherwise
    */
   public static boolean validateWALFilename(String filename) {
-    return pattern.matcher(filename).matches();
+    return WAL_FILE_NAME_PATTERN.matcher(filename).matches();
+  }
+
+  /**
+   * Split a WAL filename to get a start time. WALs usually have the time we start writing to them
+   * with as part of their name, usually the suffix. Sometimes there will be an extra suffix as when
+   * it is a WAL for the meta table. For example, WALs might look like this
+   * <code>10.20.20.171%3A60020.1277499063250</code> where <code>1277499063250</code> is the
+   * timestamp. Could also be a meta WAL which adds a '.meta' suffix or a
+   * synchronous replication WAL which adds a '.syncrep' suffix. Check for these. File also may have
+   * no timestamp on it. For example the recovered.edits files are WALs but are named in ascending
+   * order. Here is an example: 0000000000000016310. Allow for this.
+   * @param name Name of the WAL file.
+   * @return Timestamp or {@link #NO_TIMESTAMP}.
+   */
+  public static long getTimestamp(String name) {
+    Matcher matcher = WAL_FILE_NAME_PATTERN.matcher(name);
+    return matcher.matches() ? Long.parseLong(matcher.group(2)): NO_TIMESTAMP;
   }
 
   /**
@@ -437,7 +464,7 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
      * @return start time
      */
     private static long getTS(Path p) {
-      return WAL.getTimestamp(p.getName());
+      return getTimestamp(p.getName());
     }
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
index a507bcc..d01ee8e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
@@ -32,7 +32,6 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-import static org.apache.commons.lang3.StringUtils.isNumeric;
 
 /**
  * A Write Ahead Log (WAL) provides service for reading, writing waledits. This interface provides
@@ -312,32 +311,4 @@ public interface WAL extends Closeable, WALFileLengthProvider {
       return this.key + "=" + this.edit;
     }
   }
-
-  /**
-   * Split a WAL filename to get a start time. WALs usually have the time we start writing to them
-   * as part of their name, usually the suffix. Sometimes there will be an extra suffix as when it
-   * is a WAL for the meta table. For example, WALs might look like this
-   * <code>10.20.20.171%3A60020.1277499063250</code> where <code>1277499063250</code> is the
-   * timestamp. Could also be a meta WAL which adds a '.meta' suffix or a
-   * synchronous replication WAL which adds a '.syncrep' suffix. Check for these. File also may have
-   * no timestamp on it. For example the recovered.edits files are WALs but are named in ascending
-   * order. Here is an example: 0000000000000016310. Allow for this.
-   * @param name Name of the WAL file.
-   * @return Timestamp or -1.
-   */
-  public static long getTimestamp(String name) {
-    String [] splits = name.split("\\.");
-    if (splits.length <= 1) {
-      return -1;
-    }
-    String timestamp = splits[splits.length - 1];
-    if (!isNumeric(timestamp)) {
-      // Its a '.meta' or a '.syncrep' suffix.
-      timestamp = splits[splits.length - 2];
-      if (!isNumeric(timestamp)) {
-        return -1;
-      }
-    }
-    return Long.parseLong(timestamp);
-  }
 }