You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2021/07/27 20:42:40 UTC

[GitHub] [hbase] saintstack commented on a change in pull request #3359: HBASE-25891 remove dependence storing wal filenames for backup

saintstack commented on a change in pull request #3359:
URL: https://github.com/apache/hbase/pull/3359#discussion_r677780945



##########
File path: hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
##########
@@ -512,11 +512,11 @@ public void addDependentImage(BackupImage image) {
    * Set the incremental timestamp map directly.
    * @param incrTimestampMap timestamp map
    */
-  public void setIncrTimestampMap(HashMap<TableName, HashMap<String, Long>> incrTimestampMap) {
+  public void setIncrTimestampMap(Map<TableName, Map<String, Long>> incrTimestampMap) {
     this.backupImage.setIncrTimeRanges(incrTimestampMap);
   }
 
-  public Map<TableName, HashMap<String, Long>> getIncrTimestampMap() {
+  public Map<TableName, Map<String, Long>> getIncrTimestampMap() {

Review comment:
       Might have been better to do these converstions -- HashMap to Map -- as an issue of their own because they distract from meat of this PR (They are good, just off-the-point I'd say). Next time.

##########
File path: hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
##########
@@ -1011,148 +1006,6 @@ public void deleteIncrementalBackupTableSet(String backupRoot) throws IOExceptio
     }
   }
 
-  /**
-   * Register WAL files as eligible for deletion
-   * @param files files
-   * @param backupId backup id
-   * @param backupRoot root directory path to backup destination
-   * @throws IOException exception
-   */
-  public void addWALFiles(List<String> files, String backupId, String backupRoot)
-      throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("add WAL files to backup system table: " + backupId + " " + backupRoot + " files ["
-          + StringUtils.join(files, ",") + "]");
-    }
-    if (LOG.isDebugEnabled()) {
-      files.forEach(file -> LOG.debug("add :" + file));
-    }
-    try (Table table = connection.getTable(tableName)) {
-      List<Put> puts = createPutsForAddWALFiles(files, backupId, backupRoot);
-      table.put(puts);
-    }
-  }
-
-  /**
-   * Register WAL files as eligible for deletion
-   * @param backupRoot root directory path to backup
-   * @throws IOException exception
-   */
-  public Iterator<WALItem> getWALFilesIterator(String backupRoot) throws IOException {
-    LOG.trace("get WAL files from backup system table");
-
-    final Table table = connection.getTable(tableName);
-    Scan scan = createScanForGetWALs(backupRoot);
-    final ResultScanner scanner = table.getScanner(scan);
-    final Iterator<Result> it = scanner.iterator();
-    return new Iterator<WALItem>() {
-
-      @Override
-      public boolean hasNext() {
-        boolean next = it.hasNext();
-        if (!next) {
-          // close all
-          try {
-            scanner.close();
-            table.close();
-          } catch (IOException e) {
-            LOG.error("Close WAL Iterator", e);
-          }
-        }
-        return next;
-      }
-
-      @Override
-      public WALItem next() {
-        Result next = it.next();
-        List<Cell> cells = next.listCells();
-        byte[] buf = cells.get(0).getValueArray();
-        int len = cells.get(0).getValueLength();
-        int offset = cells.get(0).getValueOffset();
-        String backupId = new String(buf, offset, len);
-        buf = cells.get(1).getValueArray();
-        len = cells.get(1).getValueLength();
-        offset = cells.get(1).getValueOffset();
-        String walFile = new String(buf, offset, len);
-        buf = cells.get(2).getValueArray();
-        len = cells.get(2).getValueLength();
-        offset = cells.get(2).getValueOffset();
-        String backupRoot = new String(buf, offset, len);
-        return new WALItem(backupId, walFile, backupRoot);
-      }
-
-      @Override
-      public void remove() {
-        // not implemented
-        throw new RuntimeException("remove is not supported");
-      }
-    };
-  }
-
-  /**
-   * Check if WAL file is eligible for deletion Future: to support all backup destinations
-   * @param file name of a file to check
-   * @return true, if deletable, false otherwise.
-   * @throws IOException exception
-   */
-  // TODO: multiple backup destination support
-  public boolean isWALFileDeletable(String file) throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Check if WAL file has been already backed up in backup system table " + file);
-    }
-    try (Table table = connection.getTable(tableName)) {
-      Get get = createGetForCheckWALFile(file);
-      Result res = table.get(get);
-      return (!res.isEmpty());
-    }
-  }
-
-  /**
-   * Check if WAL file is eligible for deletion using multi-get
-   * @param files names of a file to check
-   * @return map of results (key: FileStatus object. value: true if the file is deletable, false
-   *         otherwise)
-   * @throws IOException exception
-   */
-  public Map<FileStatus, Boolean> areWALFilesDeletable(Iterable<FileStatus> files)
-      throws IOException {
-    final int BUF_SIZE = 100;
-
-    Map<FileStatus, Boolean> ret = new HashMap<>();
-    try (Table table = connection.getTable(tableName)) {
-      List<Get> getBuffer = new ArrayList<>();
-      List<FileStatus> fileStatuses = new ArrayList<>();
-
-      for (FileStatus file : files) {
-        String fn = file.getPath().getName();
-        if (fn.startsWith(WALProcedureStore.LOG_PREFIX)) {
-          ret.put(file, true);
-          continue;
-        }
-        String wal = file.getPath().toString();
-        Get get = createGetForCheckWALFile(wal);
-        getBuffer.add(get);
-        fileStatuses.add(file);
-        if (getBuffer.size() >= BUF_SIZE) {
-          Result[] results = table.get(getBuffer);
-          for (int i = 0; i < results.length; i++) {
-            ret.put(fileStatuses.get(i), !results[i].isEmpty());
-          }
-          getBuffer.clear();
-          fileStatuses.clear();
-        }
-      }
-
-      if (!getBuffer.isEmpty()) {
-        Result[] results = table.get(getBuffer);
-        for (int i = 0; i < results.length; i++) {
-          ret.put(fileStatuses.get(i), !results[i].isEmpty());
-        }
-      }
-    }
-    return ret;
-  }
-

Review comment:
       These removals of accounting from system table look good.

##########
File path: hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
##########
@@ -514,24 +513,6 @@ public void addIncrementalBackupTableSet(Set<TableName> tables) throws IOExcepti
     systemTable.addIncrementalBackupTableSet(tables, backupInfo.getBackupRootDir());
   }
 
-  /**
-   * Saves list of WAL files after incremental backup operation. These files will be stored until
-   * TTL expiration and are used by Backup Log Cleaner plug-in to determine which WAL files can be
-   * safely purged.
-   */
-  public void recordWALFiles(List<String> files) throws IOException {
-    systemTable.addWALFiles(files, backupInfo.getBackupId(), backupInfo.getBackupRootDir());

Review comment:
       Can we get rid of the 'system' table altogether?

##########
File path: hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
##########
@@ -1653,7 +1506,7 @@ public static void restoreFromSnapshot(Connection conn) throws IOException {
     }
   }
 
-  protected static boolean snapshotExists(Admin admin, String snapshotName) throws IOException {
+  private static boolean snapshotExists(Admin admin, String snapshotName) throws IOException {

Review comment:
       Good.

##########
File path: hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
##########
@@ -178,11 +174,10 @@ public String toString() {
   final static byte[] BL_PREPARE = Bytes.toBytes("R");
   final static byte[] BL_COMMIT = Bytes.toBytes("D");
 
-  private final static String WALS_PREFIX = "wals:";
   private final static String SET_KEY_PREFIX = "backupset:";
 
   // separator between BULK_LOAD_PREFIX and ordinals
-  protected final static String BLK_LD_DELIM = ":";
+  private final static String BLK_LD_DELIM = ":";

Review comment:
       Good.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org