You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@zookeeper.apache.org by GitBox <gi...@apache.org> on 2022/11/07 14:55:28 UTC

[GitHub] [zookeeper] narendly opened a new pull request, #1883: Add backup and restore with timetable

narendly opened a new pull request, #1883:
URL: https://github.com/apache/zookeeper/pull/1883

   This is a merge-able public version of https://github.com/apache/zookeeper/pull/1492 contributed by @hanm .
   
   Some notable features are
   
   - uses a file system based storage (NFS implementation included)
   - uses generic, public libraries
   - adds a restore feature using `Timetable`
   
   Also available as in LinkedIn's public fork: It is currently publicly available here in LinkedIn's public fork of ZooKeeper: https://github.com/linkedin/zookeeper/commits/li-dev/release-3.6.2-1-backup (a few changes made to make it mergeable)


-- 
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: notifications-unsubscribe@zookeeper.apache.org

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


[GitHub] [zookeeper] sonatype-lift[bot] commented on a diff in pull request #1883: Add backup and restore with timetable

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on code in PR #1883:
URL: https://github.com/apache/zookeeper/pull/1883#discussion_r878545787


##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/RestoreFromBackupTool.java:
##########
@@ -0,0 +1,716 @@
+/*
+ * 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.zookeeper.server.backup;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.nio.file.Files;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Range;
+import org.apache.commons.cli.CommandLine;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.cli.RestoreCommand;
+import org.apache.zookeeper.common.ConfigException;
+import org.apache.zookeeper.server.backup.BackupUtil.BackupFileType;
+import org.apache.zookeeper.server.backup.BackupUtil.IntervalEndpoint;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.storage.BackupStorageUtil;
+import org.apache.zookeeper.server.backup.timetable.TimetableBackup;
+import org.apache.zookeeper.server.backup.timetable.TimetableUtil;
+import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
+import org.apache.zookeeper.server.persistence.Util;
+import org.apache.zookeeper.server.util.ZxidUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ *  This is a tool to restore, from backup storage, a snapshot and set of transaction log files
+ *  that combine to represent a transactionally consistent image of a ZooKeeper ensemble at
+ *  some zxid.
+ */
+public class RestoreFromBackupTool {
+  private static final Logger LOG = LoggerFactory.getLogger(RestoreFromBackupTool.class);
+
+  private static final int MAX_RETRIES = 10;
+  private static final String HEX_PREFIX = "0x";
+  private static final int CONNECTION_TIMEOUT = 300000;
+
+  BackupStorageProvider storage;
+  FileTxnSnapLog snapLog;
+  long zxidToRestore;
+  boolean dryRun;
+  File restoreTempDir;
+  boolean overwrite = false;
+
+  // Spot restoration
+  boolean isSpotRestoration = false;
+  String znodePathToRestore;
+  String zkServerConnectionStr;
+  boolean restoreRecursively = false;
+  ZooKeeper zk;
+  SpotRestorationTool spotRestorationTool;
+
+  List<BackupFileInfo> logs;
+  List<BackupFileInfo> snaps;
+  List<BackupFileInfo> filesToCopy;
+
+  int mostRecentLogNeededIndex;
+  int snapNeededIndex;
+  int oldestLogNeededIndex;
+
+  public enum BackupStorageOption {
+    GPFS("org.apache.zookeeper.server.backup.storage.impl.FileSystemBackupStorage"),
+    NFS("org.apache.zookeeper.server.backup.storage.impl.FileSystemBackupStorage");
+
+    private String storageProviderClassName;
+
+    BackupStorageOption(String className) {
+      this.storageProviderClassName = className;
+    }
+
+    public String getStorageProviderClassName() {
+      return storageProviderClassName;
+    }
+  }
+
+  /**
+   * Default constructor; requires using parseArgs to setup state.
+   */
+  public RestoreFromBackupTool() {
+    this(null, null, -1L, false, null);
+  }
+
+  /**
+   * Constructor
+   * @param storageProvider the backup provider from which to restore the backups
+   * @param snapLog the snap and log provider to use
+   * @param zxidToRestore the zxid upto which to restore, or Long.MAX to restore to the latest
+   *                      available transactionally consistent zxid.
+   * @param dryRun whether this is a dryrun in which case no files are actually copied
+   * @param restoreTempDir a temporary, local (not remote) directory to stage the backup files from
+   *                       remote backup storage for the processing stage of restoration. Note that
+   *                       this directory and the files in it will be removed after restoration.
+   */
+  RestoreFromBackupTool(BackupStorageProvider storageProvider, FileTxnSnapLog snapLog,
+      long zxidToRestore, boolean dryRun, File restoreTempDir) {
+
+    filesToCopy = new ArrayList<>();
+    snapNeededIndex = -1;
+    mostRecentLogNeededIndex = -1;
+    oldestLogNeededIndex = -1;
+
+    this.storage = storageProvider;
+    this.zxidToRestore = zxidToRestore;
+    this.snapLog = snapLog;
+    this.dryRun = dryRun;
+    this.restoreTempDir = restoreTempDir;
+  }
+
+  /**
+   * Parse and validate arguments to the tool
+   * @param cl the command line object with user inputs
+   * @return true if the arguments parse correctly; false in all other cases.
+   * @throws IOException if the backup provider cannot be instantiated correctly.
+   */
+  public void parseArgs(CommandLine cl) {
+    String backupStoragePath = cl.getOptionValue(RestoreCommand.OptionShortForm.BACKUP_STORE);
+    createBackupStorageProvider(backupStoragePath);
+
+    // Read the restore point
+    if (cl.hasOption(RestoreCommand.OptionShortForm.RESTORE_ZXID)) {
+      parseRestoreZxid(cl);
+    } else if (cl.hasOption(RestoreCommand.OptionShortForm.RESTORE_TIMESTAMP)) {
+      parseRestoreTimestamp(cl, backupStoragePath);
+    }
+
+    parseAndValidateSpotRestorationArgs(cl);
+
+    parseAndValidateOfflineRestoreDestination(cl);
+
+    parseRestoreTempDir(cl);
+
+    // Check if overwriting the destination directories is allowed
+    if (cl.hasOption(RestoreCommand.OptionShortForm.OVERWRITE)) {
+      overwrite = true;
+    }
+
+    // Check if this is a dry-run
+    if (cl.hasOption(RestoreCommand.OptionShortForm.DRY_RUN)) {
+      dryRun = true;
+    }
+
+    System.out.println("parseArgs successful.");
+  }
+
+  private void createBackupStorageProvider(String backupStoragePath) {
+    String[] backupStorageParams = backupStoragePath.split(":");
+    if (backupStorageParams.length != 4) {
+      System.err.println(
+          "Failed to parse backup storage connection information from the backup storage path provided, please check the input.");
+      System.err.println(
+          "For example: the format for a gpfs backup storage path should be \"gpfs:<config_path>:<backup_path>:<namespace>\".");
+      System.exit(1);
+    }
+
+    String userProvidedStorageName = backupStorageParams[0].toUpperCase();
+    try {
+      BackupStorageOption storageOption = BackupStorageOption.valueOf(userProvidedStorageName);
+      String backupStorageProviderClassName = storageOption.getStorageProviderClassName();
+
+      BackupConfig.RestorationConfigBuilder configBuilder =
+          new BackupConfig.RestorationConfigBuilder()
+              .setStorageProviderClassName(backupStorageProviderClassName)
+              .setBackupStoragePath(backupStorageParams[2]).setNamespace(backupStorageParams[3]);
+      if (!backupStorageParams[1].isEmpty()) {
+        configBuilder = configBuilder.setStorageConfig(new File(backupStorageParams[1]));

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/io/File.<init>(Ljava/lang/String;)V) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621056&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621056&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621056&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621056&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621056&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/RestoreFromBackupTool.java:
##########
@@ -0,0 +1,716 @@
+/*
+ * 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.zookeeper.server.backup;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.nio.file.Files;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Range;
+import org.apache.commons.cli.CommandLine;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.cli.RestoreCommand;
+import org.apache.zookeeper.common.ConfigException;
+import org.apache.zookeeper.server.backup.BackupUtil.BackupFileType;
+import org.apache.zookeeper.server.backup.BackupUtil.IntervalEndpoint;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.storage.BackupStorageUtil;
+import org.apache.zookeeper.server.backup.timetable.TimetableBackup;
+import org.apache.zookeeper.server.backup.timetable.TimetableUtil;
+import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
+import org.apache.zookeeper.server.persistence.Util;
+import org.apache.zookeeper.server.util.ZxidUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ *  This is a tool to restore, from backup storage, a snapshot and set of transaction log files
+ *  that combine to represent a transactionally consistent image of a ZooKeeper ensemble at
+ *  some zxid.
+ */
+public class RestoreFromBackupTool {
+  private static final Logger LOG = LoggerFactory.getLogger(RestoreFromBackupTool.class);
+
+  private static final int MAX_RETRIES = 10;
+  private static final String HEX_PREFIX = "0x";
+  private static final int CONNECTION_TIMEOUT = 300000;
+
+  BackupStorageProvider storage;
+  FileTxnSnapLog snapLog;
+  long zxidToRestore;
+  boolean dryRun;
+  File restoreTempDir;
+  boolean overwrite = false;
+
+  // Spot restoration
+  boolean isSpotRestoration = false;
+  String znodePathToRestore;
+  String zkServerConnectionStr;
+  boolean restoreRecursively = false;
+  ZooKeeper zk;
+  SpotRestorationTool spotRestorationTool;
+
+  List<BackupFileInfo> logs;
+  List<BackupFileInfo> snaps;
+  List<BackupFileInfo> filesToCopy;
+
+  int mostRecentLogNeededIndex;
+  int snapNeededIndex;
+  int oldestLogNeededIndex;
+
+  public enum BackupStorageOption {
+    GPFS("org.apache.zookeeper.server.backup.storage.impl.FileSystemBackupStorage"),
+    NFS("org.apache.zookeeper.server.backup.storage.impl.FileSystemBackupStorage");
+
+    private String storageProviderClassName;
+
+    BackupStorageOption(String className) {
+      this.storageProviderClassName = className;
+    }
+
+    public String getStorageProviderClassName() {
+      return storageProviderClassName;
+    }
+  }
+
+  /**
+   * Default constructor; requires using parseArgs to setup state.
+   */
+  public RestoreFromBackupTool() {
+    this(null, null, -1L, false, null);
+  }
+
+  /**
+   * Constructor
+   * @param storageProvider the backup provider from which to restore the backups
+   * @param snapLog the snap and log provider to use
+   * @param zxidToRestore the zxid upto which to restore, or Long.MAX to restore to the latest
+   *                      available transactionally consistent zxid.
+   * @param dryRun whether this is a dryrun in which case no files are actually copied
+   * @param restoreTempDir a temporary, local (not remote) directory to stage the backup files from
+   *                       remote backup storage for the processing stage of restoration. Note that
+   *                       this directory and the files in it will be removed after restoration.
+   */
+  RestoreFromBackupTool(BackupStorageProvider storageProvider, FileTxnSnapLog snapLog,
+      long zxidToRestore, boolean dryRun, File restoreTempDir) {
+
+    filesToCopy = new ArrayList<>();
+    snapNeededIndex = -1;
+    mostRecentLogNeededIndex = -1;
+    oldestLogNeededIndex = -1;
+
+    this.storage = storageProvider;
+    this.zxidToRestore = zxidToRestore;
+    this.snapLog = snapLog;
+    this.dryRun = dryRun;
+    this.restoreTempDir = restoreTempDir;
+  }
+
+  /**
+   * Parse and validate arguments to the tool
+   * @param cl the command line object with user inputs
+   * @return true if the arguments parse correctly; false in all other cases.
+   * @throws IOException if the backup provider cannot be instantiated correctly.
+   */
+  public void parseArgs(CommandLine cl) {
+    String backupStoragePath = cl.getOptionValue(RestoreCommand.OptionShortForm.BACKUP_STORE);
+    createBackupStorageProvider(backupStoragePath);
+
+    // Read the restore point
+    if (cl.hasOption(RestoreCommand.OptionShortForm.RESTORE_ZXID)) {
+      parseRestoreZxid(cl);
+    } else if (cl.hasOption(RestoreCommand.OptionShortForm.RESTORE_TIMESTAMP)) {
+      parseRestoreTimestamp(cl, backupStoragePath);
+    }
+
+    parseAndValidateSpotRestorationArgs(cl);
+
+    parseAndValidateOfflineRestoreDestination(cl);
+
+    parseRestoreTempDir(cl);
+
+    // Check if overwriting the destination directories is allowed
+    if (cl.hasOption(RestoreCommand.OptionShortForm.OVERWRITE)) {
+      overwrite = true;
+    }
+
+    // Check if this is a dry-run
+    if (cl.hasOption(RestoreCommand.OptionShortForm.DRY_RUN)) {
+      dryRun = true;
+    }
+
+    System.out.println("parseArgs successful.");
+  }
+
+  private void createBackupStorageProvider(String backupStoragePath) {
+    String[] backupStorageParams = backupStoragePath.split(":");
+    if (backupStorageParams.length != 4) {
+      System.err.println(
+          "Failed to parse backup storage connection information from the backup storage path provided, please check the input.");
+      System.err.println(
+          "For example: the format for a gpfs backup storage path should be \"gpfs:<config_path>:<backup_path>:<namespace>\".");
+      System.exit(1);
+    }
+
+    String userProvidedStorageName = backupStorageParams[0].toUpperCase();
+    try {
+      BackupStorageOption storageOption = BackupStorageOption.valueOf(userProvidedStorageName);
+      String backupStorageProviderClassName = storageOption.getStorageProviderClassName();
+
+      BackupConfig.RestorationConfigBuilder configBuilder =
+          new BackupConfig.RestorationConfigBuilder()
+              .setStorageProviderClassName(backupStorageProviderClassName)
+              .setBackupStoragePath(backupStorageParams[2]).setNamespace(backupStorageParams[3]);
+      if (!backupStorageParams[1].isEmpty()) {
+        configBuilder = configBuilder.setStorageConfig(new File(backupStorageParams[1]));
+      }
+      storage = BackupUtil.createStorageProviderImpl(configBuilder.build().get());
+    } catch (IllegalArgumentException e) {
+      System.err.println("Could not find a valid backup storage option based on the input: "
+          + userProvidedStorageName + ". Error message: " + e.getMessage());
+      e.printStackTrace();
+      System.exit(1);
+    } catch (ConfigException e) {
+      System.err.println(
+          "Could not generate a backup config based on the input, error message: " + e
+              .getMessage());
+      e.getStackTrace();
+      System.exit(1);
+    } catch (InstantiationException | InvocationTargetException | NoSuchMethodException | IllegalAccessException | ClassNotFoundException e) {
+      System.err.println(
+          "Could not generate a backup storage provider based on the input, error message: " + e
+              .getMessage());
+      e.printStackTrace();
+      System.exit(1);
+    }
+  }
+
+  private void parseRestoreZxid(CommandLine cl) {
+    String zxidToRestoreStr = cl.getOptionValue(RestoreCommand.OptionShortForm.RESTORE_ZXID);
+    if (zxidToRestoreStr.equalsIgnoreCase(BackupUtil.LATEST)) {
+      zxidToRestore = Long.MAX_VALUE;
+    } else {
+      int base = 10;
+      String numStr = zxidToRestoreStr;
+
+      if (zxidToRestoreStr.startsWith(HEX_PREFIX)) {
+        numStr = zxidToRestoreStr.substring(2);
+        base = 16;
+      }
+      try {
+        zxidToRestore = Long.parseLong(numStr, base);
+      } catch (NumberFormatException nfe) {
+        System.err
+            .println("Invalid number specified for restore zxid point, the input is: " + numStr);
+        System.exit(2);
+      }
+    }
+  }
+
+  private void parseRestoreTimestamp(CommandLine cl, String backupStoragePath) {
+    String timestampStr = cl.getOptionValue(RestoreCommand.OptionShortForm.RESTORE_TIMESTAMP);
+    String timetableStoragePath = backupStoragePath;
+    if (cl.hasOption(RestoreCommand.OptionShortForm.TIMETABLE_STORAGE_PATH)) {
+      timetableStoragePath =
+          cl.getOptionValue(RestoreCommand.OptionShortForm.TIMETABLE_STORAGE_PATH);
+    }
+    File[] timetableFiles = new File(timetableStoragePath)
+        .listFiles(file -> file.getName().startsWith(TimetableBackup.TIMETABLE_PREFIX));
+    if (timetableFiles == null || timetableFiles.length == 0) {
+      System.err.println("Could not find timetable files at the path: " + timetableStoragePath);
+      System.exit(2);
+    }
+    Map.Entry<Long, String> restorePoint;
+    String message;
+    try {
+      restorePoint = TimetableUtil.findLastZxidFromTimestamp(timetableFiles, timestampStr);
+      zxidToRestore = Long.parseLong(restorePoint.getValue(), 16);
+      String timeToRestore = new SimpleDateFormat("MM/dd/yyyy HH:mm:ss")
+          .format(new java.util.Date(restorePoint.getKey()));
+      if (timestampStr.equalsIgnoreCase(BackupUtil.LATEST)) {
+        message = "Restoring to " + timeToRestore + ", original request was to restore to latest.";
+      } else {
+        String requestedTimeToRestore = new SimpleDateFormat("MM/dd/yyyy HH:mm:ss")
+            .format(new java.util.Date(Long.decode(timestampStr)));
+        message = "Restoring to " + timeToRestore + ", original request was to restore to "
+            + requestedTimeToRestore + ".";
+      }
+      System.out.println(message);
+      LOG.info(message);
+    } catch (IllegalArgumentException | BackupException e) {
+      System.err.println(
+          "Could not find a valid zxid from timetable using the timestamp provided: " + timestampStr
+              + ". The error message is: " + e.getMessage());
+      e.printStackTrace();
+      System.exit(2);
+    }
+  }
+
+  private void parseAndValidateOfflineRestoreDestination(CommandLine cl) {
+    if (isSpotRestoration) {
+      return;
+    }
+    // Read restore destination: dataDir and logDir
+    try {
+      String snapDirPath = cl.getOptionValue(RestoreCommand.OptionShortForm.SNAP_DESTINATION);
+      String logDirPath = cl.getOptionValue(RestoreCommand.OptionShortForm.LOG_DESTINATION);
+
+      if (snapDirPath == null || logDirPath == null) {
+        throw new BackupException(
+            "Snap destination path and log destination path are not defined for offline restoration. SnapDirPath: "
+                + snapDirPath + ", logDirPath: " + logDirPath);
+      }
+
+      File snapDir = new File(snapDirPath);
+      File logDir = new File(logDirPath);
+      snapLog = new FileTxnSnapLog(logDir, snapDir);
+      checkSnapDataDirFileExistence();
+    } catch (IOException ioe) {
+      System.err.println("Could not setup transaction log utility." + ioe);
+      System.exit(3);
+    }
+  }
+
+  private void parseRestoreTempDir(CommandLine cl) {
+    if (cl.hasOption(RestoreCommand.OptionShortForm.LOCAL_RESTORE_TEMP_DIR_PATH)) {
+      String localRestoreTempDirPath =
+          cl.getOptionValue(RestoreCommand.OptionShortForm.LOCAL_RESTORE_TEMP_DIR_PATH);
+      restoreTempDir = new File(localRestoreTempDirPath);

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/io/File.<init>(Ljava/lang/String;)V) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621080&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621080&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621080&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621080&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621080&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/storage/BackupStorageUtil.java:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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.zookeeper.server.backup.storage;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Comparator;
+import java.util.stream.Stream;
+
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.persistence.Util;
+
+/**
+ * Util methods for backup storage
+ */
+public class BackupStorageUtil {
+  public static final String TMP_FILE_PREFIX = "TMP_";
+  private static final File[] NO_FILE = new File[0];
+
+  /**
+   * Parse the prefix from a file name, also works for temporary file names in backup storage
+   * @param fileName The file name to be parsed
+   * @return "log" for ZK transaction log files or "snapshot" for ZK snapshots
+   */
+  public static String getFileTypePrefix(String fileName) {
+    String backupFileName = fileName;
+
+    //Remove the temporary file name prefix in order to determine file type
+    if (fileName.startsWith(TMP_FILE_PREFIX)) {
+      backupFileName = fileName.substring(TMP_FILE_PREFIX.length());
+    }
+
+    String fileTypePrefix;
+    if (backupFileName.startsWith(Util.SNAP_PREFIX)) {
+      fileTypePrefix = Util.SNAP_PREFIX;
+    } else if (backupFileName.startsWith(Util.TXLOG_PREFIX)) {
+      fileTypePrefix = Util.TXLOG_PREFIX;
+    } else {
+      throw new BackupException("No matching base file type found for file " + fileName);
+    }
+
+    return fileTypePrefix;
+  }
+
+  /**
+   * Construct the path of a backup file in the backup storage
+   * @param fileName The name of the file
+   * @param parentDir The path to the parent directory of the backup file.
+   * @return The path of the backup file in the format of:
+   * 1. parentDir path is not supplied: {fileName} or {fileName}
+   * 2. parentDir path is provided: {parentDir}/{fileName} or {parentDir}/{fileName}
+   */
+  public static String constructBackupFilePath(String fileName, String parentDir) {
+    //TODO: store snapshots and Txlogs in different subfolders for better organization
+    if (parentDir != null) {
+      return String.valueOf(Paths.get(parentDir, fileName));
+    }
+    return fileName;
+  }
+
+  /**
+   * Construct temporary file name using backup file name
+   * @param fileName A backup file name: log.lowzxid-highzxid, snapshot.lowzxid-highzxid
+   * @return A temporary backup file name: TMP_log.lowzxid-highzxid, TMP_snapshot.lowzxid-highzxid
+   */
+  public static String constructTempFileName(String fileName) {
+    return TMP_FILE_PREFIX + fileName;
+  }
+
+  /**
+   * A basic method for streaming data from an input stream to an output stream
+   * @param inputStream The stream to read from
+   * @param outputStream The stream to write to
+   * @throws IOException
+   */
+  public static void streamData(InputStream inputStream, OutputStream outputStream)
+      throws IOException {
+    byte[] buffer = new byte[1024];
+    int lengthRead;
+    while ((lengthRead = inputStream.read(buffer)) > 0) {
+      outputStream.write(buffer, 0, lengthRead);
+      outputStream.flush();
+    }
+  }
+
+  /**
+   * Create a new file in a specified path, create the parent directories if they do not exist.
+   * @param file The path to create the file.
+   * @param overwriteIfExist If a file already exists in the location,
+   *                         1. true: delete the existing file and retry the creation of the new file,
+   *                         or 2. false: keep the existing file.
+   * @throws IOException
+   */
+  public static void createFile(File file, boolean overwriteIfExist) throws IOException {
+    file.getParentFile().mkdirs();
+    if (!file.getParentFile().exists()) {
+      throw new BackupException("Failed to create parent directories for file " + file.getName());
+    }
+
+    boolean retry = true;
+    while (retry) {
+      retry = overwriteIfExist;
+      if (!file.createNewFile()) {
+        if (file.exists()) {
+          if (retry && !file.delete()) {
+            throw new BackupException("A file with the file path " + file.getPath()
+                + " already exists, and failed to be overwritten.");
+          }
+        } else {
+          throw new BackupException("Failed to create a file at path: " + file.getPath());
+        }
+      }
+      retry = false;
+    }
+  }
+
+  /**
+   * Get a list of all files whose file name starts with a certain prefix under a directory
+   * @param directory The directory to search for the files
+   * @param prefix The prefix of file name
+   * @return
+   */
+  public static File[] getFilesWithPrefix(File directory, String prefix) {
+    if (directory == null) {
+      return NO_FILE;
+    }
+    FilenameFilter fileFilter = (dir, name) -> name.startsWith(prefix);
+    File[] files = directory.listFiles(fileFilter);
+    return files == null ? NO_FILE : files;
+  }
+
+  /**
+   * Delete all the files whose file names starts with temporary file name prefix
+   * @param directory The directory to search for temporary files
+   * @throws IOException
+   */
+  public static void cleanUpTempFiles(File directory) throws IOException {
+    File[] tempFiles = getFilesWithPrefix(directory, TMP_FILE_PREFIX);
+    for (File tempFile : tempFiles) {
+      Files.delete(Paths.get(tempFile.getPath()));

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/nio/file/Paths.get(Ljava/lang/String;[Ljava/lang/String;)Ljava/nio/file/Path;) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621054&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621054&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621054&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621054&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621054&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/BackupManager.java:
##########
@@ -0,0 +1,661 @@
+/*
+ * 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.zookeeper.server.backup;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import javax.management.JMException;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.zookeeper.jmx.MBeanRegistry;
+import org.apache.zookeeper.server.backup.BackupUtil.BackupFileType;
+import org.apache.zookeeper.server.backup.BackupUtil.IntervalEndpoint;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.monitoring.BackupBean;
+import org.apache.zookeeper.server.backup.monitoring.BackupStats;
+import org.apache.zookeeper.server.backup.monitoring.TimetableBackupBean;
+import org.apache.zookeeper.server.backup.monitoring.TimetableBackupStats;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.timetable.TimetableBackup;
+import org.apache.zookeeper.server.persistence.FileTxnLog;
+import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
+import org.apache.zookeeper.server.persistence.TxnLog;
+import org.apache.zookeeper.server.persistence.Util;
+import org.apache.zookeeper.server.persistence.ZxidRange;
+import org.apache.zookeeper.server.util.ZxidUtils;
+import org.apache.zookeeper.txn.TxnHeader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class manages the backing up of txnlog and snap files to remote
+ * storage for longer term and durable retention than is possible on
+ * an ensemble server
+ */
+public class BackupManager {
+  private static final Logger LOG = LoggerFactory.getLogger(BackupManager.class);
+
+  private final Logger logger;
+  private final File snapDir;
+  private final File dataLogDir;
+  private final File tmpDir;
+  private final BackupConfig backupConfig;
+  private final long backupIntervalInMilliseconds;
+  private BackupProcess logBackup = null;
+  private BackupProcess snapBackup = null;
+
+  // backupStatus, backupLogZxid and backedupSnapZxid need to be access while synchronized
+  // on backupStatus.
+  private final BackupStatus backupStatus;
+  private BackupPoint backupPoint;
+
+  private final BackupStorageProvider backupStorage;
+  private final long serverId;
+  private final String namespace;
+
+  @VisibleForTesting
+  protected BackupBean backupBean = null;
+  protected TimetableBackupBean timetableBackupBean = null;
+
+  private BackupStats backupStats = null;
+
+  // Optional timetable backup
+  private BackupProcess timetableBackup = null;
+
+  /**
+   * Tracks a file that needs to be backed up, including temporary copies of the file
+   */
+  public static class BackupFile {
+    private final File file;
+    private final boolean isTemporary;
+    private final ZxidRange zxidRange;
+
+    /**
+     * Create an instance of a BackupFile for the given initial file and zxid range
+     * @param backupFile the initial/original file
+     * @param isTemporaryFile whether the file is a temporary file
+     * @param fileMinZxid the min zxid associated with this file
+     * @param fileMaxZxid the max zxid associated with this file
+     */
+    public BackupFile(File backupFile, boolean isTemporaryFile, long fileMinZxid, long fileMaxZxid) {
+      this(backupFile, isTemporaryFile, new ZxidRange(fileMinZxid, fileMaxZxid));
+    }
+
+    /**
+     * Create an instance of a BackupFile for the given initial file and zxid range
+     * @param backupFile the initial/original file
+     * @param isTemporaryFile whether the file is a temporary file
+     * @param zxidRange the zxid range associated with this file
+     */
+    public BackupFile(File backupFile, boolean isTemporaryFile, ZxidRange zxidRange) {
+      Preconditions.checkNotNull(zxidRange);
+
+      if (!zxidRange.isHighPresent()) {
+        throw new IllegalArgumentException("ZxidRange must have a high value");
+      }
+
+      this.file = backupFile;
+      this.isTemporary = isTemporaryFile;
+      this.zxidRange = zxidRange;
+    }
+
+    /**
+     * Perform cleanup including deleting temporary files.
+     */
+    public void cleanup() {
+      if (isTemporary && exists()) {
+        file.delete();
+      }
+    }
+
+    /**
+     * Whether the file representing the zxids exists
+     * @return whether the file represented exists
+     */
+    public boolean exists() {
+      return file != null && file.exists();
+    }
+
+    /**
+     * Get the current file (topmost on the stack)
+     * @return the current file
+     */
+    public File getFile() { return file; }
+
+    /**
+     * Get the zxid range associated with this file
+     * @return the zxid range
+     */
+    public ZxidRange getZxidRange() {
+      return zxidRange;
+    }
+
+    /**
+     * Get the min zxid associated with this file
+     * @return the min associated zxid
+     */
+    public long getMinZxid() { return zxidRange.getLow(); }
+
+    /**
+     * Get the max zxid associated with this file
+     * @return the max associated zxid
+     */
+    public long getMaxZxid() { return zxidRange.getHigh(); }
+
+    @Override
+    public String toString() {
+      return String.format("%s : %s : %d - %d",
+          file == null ? "[empty]" : file.getPath(),
+          isTemporary ? "temp" : "perm",
+          zxidRange.getLow(),
+          zxidRange.getHigh());
+    }
+  }
+
+  /**
+   * Implements txnlog specific logic for BackupProcess
+   */
+  protected class TxnLogBackup extends BackupProcess {
+    private long iterationEndPoint;
+    private final FileTxnSnapLog snapLog;
+
+    /**
+     * Constructor for TxnLogBackup
+     * @param snapLog the FileTxnSnapLog object to use
+     */
+    public TxnLogBackup(FileTxnSnapLog snapLog) {
+      super(LoggerFactory.getLogger(TxnLogBackup.class), BackupManager.this.backupStorage,
+          backupIntervalInMilliseconds);
+      this.snapLog = snapLog;
+    }
+
+    protected void initialize() throws IOException {
+      backupStorage.cleanupInvalidFiles(null);
+
+      BackupFileInfo latest =
+          BackupUtil.getLatest(backupStorage, BackupFileType.TXNLOG, IntervalEndpoint.START);
+
+      long rZxid = latest == null
+          ? BackupUtil.INVALID_LOG_ZXID
+          : latest.getIntervalEndpoint(IntervalEndpoint.END);
+
+      logger.info("Latest Zxid from storage: {}  from status: {}",
+          ZxidUtils.zxidToString(rZxid), ZxidUtils.zxidToString(backupPoint.getLogZxid()));
+
+      if (rZxid != backupPoint.getLogZxid()) {
+        synchronized (backupStatus) {
+          backupPoint.setLogZxid(rZxid);
+          backupStatus.update(backupPoint);
+        }
+      }
+    }
+
+    protected void startIteration() {
+      // Store the current last logged zxid.  This becomes the stopping point
+      // for the current iteration so we don't keep chasing our own tail as
+      // new transactions get written.
+      iterationEndPoint = snapLog.getLastLoggedZxid();
+      backupStats.setTxnLogBackupIterationStart();
+    }
+
+    protected void endIteration(boolean errorFree) {
+      backupStats.setTxnLogBackupIterationDone(errorFree);
+      iterationEndPoint = 0L;
+    }
+
+    /**
+     * Gets the next txnlog file to backup.  This is a temporary file created by copying
+     * all transaction from the previous backup point until the end zxid for this iteration, or
+     * a file indicating that some log records were lost.
+     * @return the file that needs to be backed-up.  The minZxid is the first
+     *      zxid contained in the file.  The maxZxid is the last zxid that is contained in the
+     *      file.
+     * @throws IOException
+     */
+    protected BackupFile getNextFileToBackup() throws IOException {
+      long startingZxid = backupStatus.read().getLogZxid() + 1;
+
+      // Don't keep chasing the tail so stop if past the last zxid at the time
+      // this iteration started.
+      if (startingZxid > iterationEndPoint) {
+        return null;
+      }
+
+      TxnLog.TxnIterator iter = null;
+      FileTxnLog newFile = null;
+      long lastZxid = -1;
+      int txnCopied = 0;
+      BackupFile ret;
+
+      logger.info("Creating backup file from zxid {}.", ZxidUtils.zxidToString(startingZxid));
+
+      try {
+        iter = snapLog.readTxnLog(startingZxid, true);
+
+        // Use a temp directory to avoid conflicts with live txnlog files
+        newFile = new FileTxnLog(tmpDir);
+
+        // TODO: Ideally, we should have have a way to prevent lost TxLogs
+        // Check for lost txnlogs; <=1 indicates that no backups have been done before so
+        // nothing can be considered lost.
+        // If a lost sequence is found then return a file whose name encodes the lost
+        // sequence and back that up so the backup store has a record of the lost sequence
+        if (startingZxid > 1 &&
+            iter.getHeader() != null &&
+            iter.getHeader().getZxid() > startingZxid) {
+
+          logger.error("TxnLog backups lost.  Required starting zxid={}  First available zxid={}",
+              ZxidUtils.zxidToString(startingZxid),
+              ZxidUtils.zxidToString(iter.getHeader().getZxid()));
+
+          String fileName = String.format("%s.%s",
+              BackupUtil.LOST_LOG_PREFIX,
+              Long.toHexString(startingZxid));
+          File lostZxidFile = new File(tmpDir, fileName);
+          lostZxidFile.createNewFile();
+
+          return new BackupFile(lostZxidFile, true, startingZxid, iter.getHeader().getZxid() - 1);
+        }
+
+        while (iter.getHeader() != null) {
+          TxnHeader hdr = iter.getHeader();
+
+          if (hdr.getZxid() > iterationEndPoint) {
+            break;
+          }
+
+          newFile.append(hdr, iter.getTxn());
+
+          // update position and count only AFTER the record has been successfully
+          // copied
+          lastZxid = hdr.getZxid();
+          txnCopied++;
+
+          iter.next();
+        }
+
+        ret = makeBackupFileFromCopiedLog(newFile, lastZxid);
+
+        if (ret != null) {
+          logger.info("Copied {} records starting at {} and ending at zxid {}.",
+              txnCopied,
+              ZxidUtils.zxidToString(ret.getMinZxid()),
+              ZxidUtils.zxidToString(ret.getMaxZxid()));
+        }
+
+      } catch (IOException e) {
+        logger.warn("Hit exception after {} records.  Exception: {} ", txnCopied, e.fillInStackTrace());
+
+        // If any records were copied return those and ignore the error.  Otherwise
+        // rethrow the error to be handled by the caller as a failed backup iteration.
+        if (txnCopied <= 0) {
+          throw e;
+        }
+
+        ret = makeBackupFileFromCopiedLog(newFile, lastZxid);
+      } finally {
+        if (iter != null) {
+          iter.close();
+        }
+
+        if (newFile != null) {
+          newFile.close();
+        }
+      }
+
+      return ret;
+    }
+
+    private BackupFile makeBackupFileFromCopiedLog(FileTxnLog backupTxnLog, long lastZxid) {
+
+      if (backupTxnLog == null) {
+        return null;
+      }
+
+      // The logFile gets initialized with the first transaction's zxid
+      File logFile = backupTxnLog.getCurrentFile();
+
+      if (logFile == null) {
+        return null;
+      }
+
+      long firstZxid = Util.getZxidFromName(logFile.getName(), Util.TXLOG_PREFIX);
+
+      if (lastZxid == -1) {
+        lastZxid = firstZxid;
+      }
+
+      return new BackupFile(logFile, true, new ZxidRange(firstZxid, lastZxid));
+    }
+
+    protected void backupComplete(BackupFile file) throws IOException {
+      synchronized (backupStatus) {
+        backupPoint.setLogZxid(file.getMaxZxid());
+        backupStatus.update(backupPoint);
+      }
+
+      logger.info("Updated backedup tnxlog zxid to {}",
+          ZxidUtils.zxidToString(backupPoint.getLogZxid()));
+    }
+  }
+
+  /**
+   * Implements snapshot specific logic for BackupProcess
+   */
+  protected class SnapBackup extends BackupProcess {
+    private final FileTxnSnapLog snapLog;
+    private final List<BackupFile> filesToBackup = new ArrayList<>();
+
+    /**
+     * Constructor for SnapBackup
+     * @param snapLog the FileTxnSnapLog object to use
+     */
+    public SnapBackup(FileTxnSnapLog snapLog) {
+      super(LoggerFactory.getLogger(SnapBackup.class), BackupManager.this.backupStorage,
+          backupIntervalInMilliseconds);
+      this.snapLog = snapLog;
+    }
+
+    protected void initialize() throws IOException {
+      backupStorage.cleanupInvalidFiles(null);
+
+      BackupFileInfo latest =
+          BackupUtil.getLatest(backupStorage, BackupFileType.SNAPSHOT, IntervalEndpoint.START);
+
+      long rZxid = latest == null
+          ? BackupUtil.INVALID_SNAP_ZXID
+          : latest.getIntervalEndpoint(IntervalEndpoint.START);
+
+      logger.info("Latest Zxid from storage: {}  from status: {}",
+          ZxidUtils.zxidToString(rZxid), ZxidUtils.zxidToString(backupPoint.getSnapZxid()));
+
+      if (rZxid != backupPoint.getSnapZxid()) {
+        synchronized (backupStatus) {
+          backupPoint.setSnapZxid(rZxid);
+          backupStatus.update(backupPoint);
+        }
+      }
+    }
+
+    /**
+     * Prepares snapshot files to back up and populate filesToBackup.
+     * Note that this implementation persists all snapshots instead of only persisting the
+     * latest snapshot.
+     * @throws IOException
+     */
+    protected void startIteration() throws IOException {
+      backupStats.setSnapshotBackupIterationStart();
+      filesToBackup.clear();
+
+      // Get all available snapshots excluding the ones whose lastProcessedZxid falls into the
+      // zxid range [0, backedupSnapZxid]
+      List<File> candidateSnapshots = snapLog.findValidSnapshots(0, backupPoint.getSnapZxid());
+      // Sort candidateSnapshots from oldest to newest
+      Collections.reverse(candidateSnapshots);
+
+      if (candidateSnapshots.size() == 0) {
+        // Either no snapshots or no newer snapshots to back up, so return
+        return;
+      }
+
+      for (int i = 0; i < candidateSnapshots.size(); i++) {
+        File f = candidateSnapshots.get(i);
+        ZxidRange zxidRange = Util.getZxidRangeFromName(f.getName(), Util.SNAP_PREFIX);
+
+        if (i == candidateSnapshots.size() - 1) {
+          // This is the most recent snapshot
+
+          // Handle backwards compatibility for snapshots that use old style naming where
+          // only the starting zxid is included.
+          // TODO: Can be removed after all snapshots being produced have ending zxid
+          if (!zxidRange.isHighPresent()) {
+            // Use the last logged zxid for the zxidRange for the latest snapshot as a best effort
+            // approach
+            // TODO: Because this is the best effort approach, the zxidRange will not be accurate
+            // TODO: Consider rewriting these latest snapshots to backup storage if necessary
+            // TODO: when we know the high zxid when we get a newer snapshot
+            long latestZxid = snapLog.getLastLoggedZxid();
+            long consistentAt = latestZxid == -1 ? zxidRange.getLow() : latestZxid;
+            zxidRange = new ZxidRange(zxidRange.getLow(), consistentAt);
+          }
+        } else {
+          // All newer snapshots that are not the most recent snapshot
+
+          // Handle backwards compatibility for snapshots that use old style naming where
+          // only the starting zxid is included.
+          // TODO: Can be removed after all snapshots being produced have ending zxid
+          if (!zxidRange.isHighPresent()) {
+            // ZxidRange will be [low, high] where high will be the zxid right before the next
+            // snapshot's lastProcessedZxid
+            long nextSnapshotStartZxid =
+                Util.getZxidFromName(candidateSnapshots.get(i + 1).getName(), Util.SNAP_PREFIX);
+            zxidRange = new ZxidRange(zxidRange.getLow(), nextSnapshotStartZxid - 1);
+          }
+        }
+
+        filesToBackup.add(new BackupFile(f, false, zxidRange));
+      }
+    }
+
+    protected void endIteration(boolean errorFree) {
+      backupStats.setSnapshotBackupIterationDone(errorFree);
+      filesToBackup.clear();
+    }
+
+    protected BackupFile getNextFileToBackup() {
+      if (filesToBackup.isEmpty()) {
+        return null;
+      }
+
+      return filesToBackup.remove(0);
+    }
+
+    protected void backupComplete(BackupFile file) throws IOException {
+      synchronized (backupStatus) {
+        backupPoint.setSnapZxid(file.getMinZxid());
+        backupStatus.update(backupPoint);
+      }
+      backupStats.incrementNumberOfSnapshotFilesBackedUpThisIteration();
+
+      logger.info("Updated backedup snap zxid to {}",
+          ZxidUtils.zxidToString(backupPoint.getSnapZxid()));
+    }
+  }
+
+  /**
+   * Constructor for the BackupManager.
+   * @param snapDir the snapshot directory
+   * @param dataLogDir the txnlog directory
+   * @param serverId the id of the zk server
+   * @param backupConfig the backup config object
+   * @throws IOException
+   */
+  public BackupManager(File snapDir, File dataLogDir, long serverId, BackupConfig backupConfig)
+      throws IOException {
+    logger = LoggerFactory.getLogger(BackupManager.class);
+    logger.info("snapDir={}", snapDir.getPath());
+    logger.info("dataLogDir={}", dataLogDir.getPath());
+    logger.info("backupStatusDir={}", backupConfig.getStatusDir().getPath());
+    logger.info("tmpDir={}", backupConfig.getTmpDir().getPath());
+    logger.info("backupIntervalInMinutes={}", backupConfig.getBackupIntervalInMinutes());
+    logger.info("serverId={}", serverId);
+    logger.info("namespace={}", backupConfig.getNamespace());
+
+    this.snapDir = snapDir;
+    this.dataLogDir = dataLogDir;
+    this.backupConfig = backupConfig;
+    // Note: tmpDir is namespaced
+    this.tmpDir = new File(String.join(File.separator, backupConfig.getTmpDir().getAbsolutePath(),

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/io/File.<init>(Ljava/lang/String;)V) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621058&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621058&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621058&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621058&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621058&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/storage/impl/FileSystemBackupStorage.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.zookeeper.server.backup.storage.impl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.zookeeper.server.backup.BackupConfig;
+import org.apache.zookeeper.server.backup.BackupFileInfo;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.storage.BackupStorageUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation for backup storage provider for file systems that store files in a tree (hierarchical) structure
+ * To use this class for different file systems, use appropriate address for backupStoragePath in BackupConfig
+ * For example:
+ * 1. hard disk drive & solid-state drive: /mountPoint/relativePathToMountPoint
+ * 2. NFS: /nfsClientMountPoint/relativePathToMountPoint
+ * 3. local disk: an absolute path to a directory
+ */
+public class FileSystemBackupStorage implements BackupStorageProvider {
+  private static final Logger LOG = LoggerFactory.getLogger(FileSystemBackupStorage.class);
+  private final String fileRootPath;
+  private final ReadWriteLock rwLock;
+  private final Lock sharedLock;
+  private final Lock exclusiveLock;
+
+  /**
+   * Constructor using BackupConfig to get backup storage info
+   * @param backupConfig The information and settings about backup storage, to be set as a part of ZooKeeper server config
+   */
+  public FileSystemBackupStorage(BackupConfig backupConfig) {
+    if (!new File(backupConfig.getBackupStoragePath()).exists()) {
+      throw new BackupException(
+          "The backup storage is not ready, please check the path: " + backupConfig
+              .getBackupStoragePath());
+    }
+    fileRootPath = String
+        .join(File.separator, backupConfig.getBackupStoragePath(), backupConfig.getNamespace());
+    rwLock = new ReentrantReadWriteLock();
+    sharedLock = rwLock.readLock();
+    exclusiveLock = rwLock.writeLock();
+  }
+
+  @Override
+  public BackupFileInfo getBackupFileInfo(File file) throws IOException {
+    String backupFilePath = BackupStorageUtil.constructBackupFilePath(file.getName(), fileRootPath);
+    File backupFile = new File(backupFilePath);
+
+    if (!backupFile.exists()) {
+      return new BackupFileInfo(backupFile, BackupFileInfo.NOT_SET, BackupFileInfo.NOT_SET);
+    }
+
+    BasicFileAttributes fileAttributes =
+        Files.readAttributes(Paths.get(backupFilePath), BasicFileAttributes.class);
+    return new BackupFileInfo(backupFile, fileAttributes.lastModifiedTime().toMillis(),
+        fileAttributes.size());
+  }
+
+  @Override
+  public List<BackupFileInfo> getBackupFileInfos(File path, String prefix) throws IOException {
+    String filePath = path == null ? "" : path.getPath();
+    String backupDirPath = Paths.get(fileRootPath, filePath).toString();
+    File backupDir = new File(backupDirPath);
+
+    if (!backupDir.exists()) {
+      return new ArrayList<>();
+    }
+
+    File[] files = BackupStorageUtil.getFilesWithPrefix(backupDir, prefix);
+
+    // Read the file info and add to the list. If an exception is thrown, the entire operation will fail
+    List<BackupFileInfo> backupFileInfos = new ArrayList<>();
+    for (File file : files) {
+      backupFileInfos.add(getBackupFileInfo(file));
+    }
+    return backupFileInfos;
+  }
+
+  @Override
+  public List<File> getDirectories(File path) {
+    String filePath = path == null ? "" : path.getPath();
+    String backupDirPath = BackupStorageUtil.constructBackupFilePath(filePath, fileRootPath);
+    File backupDir = new File(backupDirPath);

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/io/File.<init>(Ljava/lang/String;)V) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621066&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621066&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621066&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621066&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621066&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/BackupManager.java:
##########
@@ -0,0 +1,661 @@
+/*
+ * 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.zookeeper.server.backup;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import javax.management.JMException;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.zookeeper.jmx.MBeanRegistry;
+import org.apache.zookeeper.server.backup.BackupUtil.BackupFileType;
+import org.apache.zookeeper.server.backup.BackupUtil.IntervalEndpoint;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.monitoring.BackupBean;
+import org.apache.zookeeper.server.backup.monitoring.BackupStats;
+import org.apache.zookeeper.server.backup.monitoring.TimetableBackupBean;
+import org.apache.zookeeper.server.backup.monitoring.TimetableBackupStats;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.timetable.TimetableBackup;
+import org.apache.zookeeper.server.persistence.FileTxnLog;
+import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
+import org.apache.zookeeper.server.persistence.TxnLog;
+import org.apache.zookeeper.server.persistence.Util;
+import org.apache.zookeeper.server.persistence.ZxidRange;
+import org.apache.zookeeper.server.util.ZxidUtils;
+import org.apache.zookeeper.txn.TxnHeader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class manages the backing up of txnlog and snap files to remote
+ * storage for longer term and durable retention than is possible on
+ * an ensemble server
+ */
+public class BackupManager {
+  private static final Logger LOG = LoggerFactory.getLogger(BackupManager.class);
+
+  private final Logger logger;
+  private final File snapDir;
+  private final File dataLogDir;
+  private final File tmpDir;
+  private final BackupConfig backupConfig;
+  private final long backupIntervalInMilliseconds;
+  private BackupProcess logBackup = null;
+  private BackupProcess snapBackup = null;
+
+  // backupStatus, backupLogZxid and backedupSnapZxid need to be access while synchronized
+  // on backupStatus.
+  private final BackupStatus backupStatus;
+  private BackupPoint backupPoint;
+
+  private final BackupStorageProvider backupStorage;
+  private final long serverId;
+  private final String namespace;
+
+  @VisibleForTesting
+  protected BackupBean backupBean = null;
+  protected TimetableBackupBean timetableBackupBean = null;
+
+  private BackupStats backupStats = null;
+
+  // Optional timetable backup
+  private BackupProcess timetableBackup = null;
+
+  /**
+   * Tracks a file that needs to be backed up, including temporary copies of the file
+   */
+  public static class BackupFile {
+    private final File file;
+    private final boolean isTemporary;
+    private final ZxidRange zxidRange;
+
+    /**
+     * Create an instance of a BackupFile for the given initial file and zxid range
+     * @param backupFile the initial/original file
+     * @param isTemporaryFile whether the file is a temporary file
+     * @param fileMinZxid the min zxid associated with this file
+     * @param fileMaxZxid the max zxid associated with this file
+     */
+    public BackupFile(File backupFile, boolean isTemporaryFile, long fileMinZxid, long fileMaxZxid) {
+      this(backupFile, isTemporaryFile, new ZxidRange(fileMinZxid, fileMaxZxid));
+    }
+
+    /**
+     * Create an instance of a BackupFile for the given initial file and zxid range
+     * @param backupFile the initial/original file
+     * @param isTemporaryFile whether the file is a temporary file
+     * @param zxidRange the zxid range associated with this file
+     */
+    public BackupFile(File backupFile, boolean isTemporaryFile, ZxidRange zxidRange) {
+      Preconditions.checkNotNull(zxidRange);
+
+      if (!zxidRange.isHighPresent()) {
+        throw new IllegalArgumentException("ZxidRange must have a high value");
+      }
+
+      this.file = backupFile;
+      this.isTemporary = isTemporaryFile;
+      this.zxidRange = zxidRange;
+    }
+
+    /**
+     * Perform cleanup including deleting temporary files.
+     */
+    public void cleanup() {
+      if (isTemporary && exists()) {
+        file.delete();
+      }
+    }
+
+    /**
+     * Whether the file representing the zxids exists
+     * @return whether the file represented exists
+     */
+    public boolean exists() {
+      return file != null && file.exists();
+    }
+
+    /**
+     * Get the current file (topmost on the stack)
+     * @return the current file
+     */
+    public File getFile() { return file; }
+
+    /**
+     * Get the zxid range associated with this file
+     * @return the zxid range
+     */
+    public ZxidRange getZxidRange() {
+      return zxidRange;
+    }
+
+    /**
+     * Get the min zxid associated with this file
+     * @return the min associated zxid
+     */
+    public long getMinZxid() { return zxidRange.getLow(); }
+
+    /**
+     * Get the max zxid associated with this file
+     * @return the max associated zxid
+     */
+    public long getMaxZxid() { return zxidRange.getHigh(); }
+
+    @Override
+    public String toString() {
+      return String.format("%s : %s : %d - %d",
+          file == null ? "[empty]" : file.getPath(),
+          isTemporary ? "temp" : "perm",
+          zxidRange.getLow(),
+          zxidRange.getHigh());
+    }
+  }
+
+  /**
+   * Implements txnlog specific logic for BackupProcess
+   */
+  protected class TxnLogBackup extends BackupProcess {
+    private long iterationEndPoint;
+    private final FileTxnSnapLog snapLog;
+
+    /**
+     * Constructor for TxnLogBackup
+     * @param snapLog the FileTxnSnapLog object to use
+     */
+    public TxnLogBackup(FileTxnSnapLog snapLog) {
+      super(LoggerFactory.getLogger(TxnLogBackup.class), BackupManager.this.backupStorage,
+          backupIntervalInMilliseconds);
+      this.snapLog = snapLog;
+    }
+
+    protected void initialize() throws IOException {
+      backupStorage.cleanupInvalidFiles(null);
+
+      BackupFileInfo latest =
+          BackupUtil.getLatest(backupStorage, BackupFileType.TXNLOG, IntervalEndpoint.START);
+
+      long rZxid = latest == null
+          ? BackupUtil.INVALID_LOG_ZXID
+          : latest.getIntervalEndpoint(IntervalEndpoint.END);
+
+      logger.info("Latest Zxid from storage: {}  from status: {}",
+          ZxidUtils.zxidToString(rZxid), ZxidUtils.zxidToString(backupPoint.getLogZxid()));
+
+      if (rZxid != backupPoint.getLogZxid()) {
+        synchronized (backupStatus) {
+          backupPoint.setLogZxid(rZxid);
+          backupStatus.update(backupPoint);
+        }
+      }
+    }
+
+    protected void startIteration() {
+      // Store the current last logged zxid.  This becomes the stopping point
+      // for the current iteration so we don't keep chasing our own tail as
+      // new transactions get written.
+      iterationEndPoint = snapLog.getLastLoggedZxid();
+      backupStats.setTxnLogBackupIterationStart();
+    }
+
+    protected void endIteration(boolean errorFree) {
+      backupStats.setTxnLogBackupIterationDone(errorFree);
+      iterationEndPoint = 0L;
+    }
+
+    /**
+     * Gets the next txnlog file to backup.  This is a temporary file created by copying
+     * all transaction from the previous backup point until the end zxid for this iteration, or
+     * a file indicating that some log records were lost.
+     * @return the file that needs to be backed-up.  The minZxid is the first
+     *      zxid contained in the file.  The maxZxid is the last zxid that is contained in the
+     *      file.
+     * @throws IOException
+     */
+    protected BackupFile getNextFileToBackup() throws IOException {
+      long startingZxid = backupStatus.read().getLogZxid() + 1;
+
+      // Don't keep chasing the tail so stop if past the last zxid at the time
+      // this iteration started.
+      if (startingZxid > iterationEndPoint) {
+        return null;
+      }
+
+      TxnLog.TxnIterator iter = null;
+      FileTxnLog newFile = null;
+      long lastZxid = -1;
+      int txnCopied = 0;
+      BackupFile ret;
+
+      logger.info("Creating backup file from zxid {}.", ZxidUtils.zxidToString(startingZxid));
+
+      try {
+        iter = snapLog.readTxnLog(startingZxid, true);
+
+        // Use a temp directory to avoid conflicts with live txnlog files
+        newFile = new FileTxnLog(tmpDir);
+
+        // TODO: Ideally, we should have have a way to prevent lost TxLogs
+        // Check for lost txnlogs; <=1 indicates that no backups have been done before so
+        // nothing can be considered lost.
+        // If a lost sequence is found then return a file whose name encodes the lost
+        // sequence and back that up so the backup store has a record of the lost sequence
+        if (startingZxid > 1 &&
+            iter.getHeader() != null &&
+            iter.getHeader().getZxid() > startingZxid) {
+
+          logger.error("TxnLog backups lost.  Required starting zxid={}  First available zxid={}",
+              ZxidUtils.zxidToString(startingZxid),
+              ZxidUtils.zxidToString(iter.getHeader().getZxid()));
+
+          String fileName = String.format("%s.%s",
+              BackupUtil.LOST_LOG_PREFIX,
+              Long.toHexString(startingZxid));
+          File lostZxidFile = new File(tmpDir, fileName);
+          lostZxidFile.createNewFile();
+
+          return new BackupFile(lostZxidFile, true, startingZxid, iter.getHeader().getZxid() - 1);
+        }
+
+        while (iter.getHeader() != null) {
+          TxnHeader hdr = iter.getHeader();
+
+          if (hdr.getZxid() > iterationEndPoint) {
+            break;
+          }
+
+          newFile.append(hdr, iter.getTxn());
+
+          // update position and count only AFTER the record has been successfully
+          // copied
+          lastZxid = hdr.getZxid();
+          txnCopied++;
+
+          iter.next();
+        }
+
+        ret = makeBackupFileFromCopiedLog(newFile, lastZxid);
+
+        if (ret != null) {
+          logger.info("Copied {} records starting at {} and ending at zxid {}.",
+              txnCopied,
+              ZxidUtils.zxidToString(ret.getMinZxid()),
+              ZxidUtils.zxidToString(ret.getMaxZxid()));
+        }
+
+      } catch (IOException e) {
+        logger.warn("Hit exception after {} records.  Exception: {} ", txnCopied, e.fillInStackTrace());
+
+        // If any records were copied return those and ignore the error.  Otherwise
+        // rethrow the error to be handled by the caller as a failed backup iteration.
+        if (txnCopied <= 0) {
+          throw e;
+        }
+
+        ret = makeBackupFileFromCopiedLog(newFile, lastZxid);
+      } finally {
+        if (iter != null) {
+          iter.close();
+        }
+
+        if (newFile != null) {
+          newFile.close();
+        }
+      }
+
+      return ret;
+    }
+
+    private BackupFile makeBackupFileFromCopiedLog(FileTxnLog backupTxnLog, long lastZxid) {
+
+      if (backupTxnLog == null) {
+        return null;
+      }
+
+      // The logFile gets initialized with the first transaction's zxid
+      File logFile = backupTxnLog.getCurrentFile();
+
+      if (logFile == null) {
+        return null;
+      }
+
+      long firstZxid = Util.getZxidFromName(logFile.getName(), Util.TXLOG_PREFIX);
+
+      if (lastZxid == -1) {
+        lastZxid = firstZxid;
+      }
+
+      return new BackupFile(logFile, true, new ZxidRange(firstZxid, lastZxid));
+    }
+
+    protected void backupComplete(BackupFile file) throws IOException {
+      synchronized (backupStatus) {
+        backupPoint.setLogZxid(file.getMaxZxid());
+        backupStatus.update(backupPoint);
+      }
+
+      logger.info("Updated backedup tnxlog zxid to {}",
+          ZxidUtils.zxidToString(backupPoint.getLogZxid()));
+    }
+  }
+
+  /**
+   * Implements snapshot specific logic for BackupProcess
+   */
+  protected class SnapBackup extends BackupProcess {
+    private final FileTxnSnapLog snapLog;
+    private final List<BackupFile> filesToBackup = new ArrayList<>();
+
+    /**
+     * Constructor for SnapBackup
+     * @param snapLog the FileTxnSnapLog object to use
+     */
+    public SnapBackup(FileTxnSnapLog snapLog) {
+      super(LoggerFactory.getLogger(SnapBackup.class), BackupManager.this.backupStorage,
+          backupIntervalInMilliseconds);
+      this.snapLog = snapLog;
+    }
+
+    protected void initialize() throws IOException {
+      backupStorage.cleanupInvalidFiles(null);
+
+      BackupFileInfo latest =
+          BackupUtil.getLatest(backupStorage, BackupFileType.SNAPSHOT, IntervalEndpoint.START);
+
+      long rZxid = latest == null
+          ? BackupUtil.INVALID_SNAP_ZXID
+          : latest.getIntervalEndpoint(IntervalEndpoint.START);
+
+      logger.info("Latest Zxid from storage: {}  from status: {}",
+          ZxidUtils.zxidToString(rZxid), ZxidUtils.zxidToString(backupPoint.getSnapZxid()));
+
+      if (rZxid != backupPoint.getSnapZxid()) {
+        synchronized (backupStatus) {
+          backupPoint.setSnapZxid(rZxid);
+          backupStatus.update(backupPoint);
+        }
+      }
+    }
+
+    /**
+     * Prepares snapshot files to back up and populate filesToBackup.
+     * Note that this implementation persists all snapshots instead of only persisting the
+     * latest snapshot.
+     * @throws IOException
+     */
+    protected void startIteration() throws IOException {
+      backupStats.setSnapshotBackupIterationStart();
+      filesToBackup.clear();
+
+      // Get all available snapshots excluding the ones whose lastProcessedZxid falls into the
+      // zxid range [0, backedupSnapZxid]
+      List<File> candidateSnapshots = snapLog.findValidSnapshots(0, backupPoint.getSnapZxid());
+      // Sort candidateSnapshots from oldest to newest
+      Collections.reverse(candidateSnapshots);
+
+      if (candidateSnapshots.size() == 0) {
+        // Either no snapshots or no newer snapshots to back up, so return
+        return;
+      }
+
+      for (int i = 0; i < candidateSnapshots.size(); i++) {
+        File f = candidateSnapshots.get(i);
+        ZxidRange zxidRange = Util.getZxidRangeFromName(f.getName(), Util.SNAP_PREFIX);
+
+        if (i == candidateSnapshots.size() - 1) {
+          // This is the most recent snapshot
+
+          // Handle backwards compatibility for snapshots that use old style naming where
+          // only the starting zxid is included.
+          // TODO: Can be removed after all snapshots being produced have ending zxid
+          if (!zxidRange.isHighPresent()) {
+            // Use the last logged zxid for the zxidRange for the latest snapshot as a best effort
+            // approach
+            // TODO: Because this is the best effort approach, the zxidRange will not be accurate
+            // TODO: Consider rewriting these latest snapshots to backup storage if necessary
+            // TODO: when we know the high zxid when we get a newer snapshot
+            long latestZxid = snapLog.getLastLoggedZxid();
+            long consistentAt = latestZxid == -1 ? zxidRange.getLow() : latestZxid;
+            zxidRange = new ZxidRange(zxidRange.getLow(), consistentAt);
+          }
+        } else {
+          // All newer snapshots that are not the most recent snapshot
+
+          // Handle backwards compatibility for snapshots that use old style naming where
+          // only the starting zxid is included.
+          // TODO: Can be removed after all snapshots being produced have ending zxid
+          if (!zxidRange.isHighPresent()) {
+            // ZxidRange will be [low, high] where high will be the zxid right before the next
+            // snapshot's lastProcessedZxid
+            long nextSnapshotStartZxid =
+                Util.getZxidFromName(candidateSnapshots.get(i + 1).getName(), Util.SNAP_PREFIX);
+            zxidRange = new ZxidRange(zxidRange.getLow(), nextSnapshotStartZxid - 1);
+          }
+        }
+
+        filesToBackup.add(new BackupFile(f, false, zxidRange));
+      }
+    }
+
+    protected void endIteration(boolean errorFree) {
+      backupStats.setSnapshotBackupIterationDone(errorFree);
+      filesToBackup.clear();
+    }
+
+    protected BackupFile getNextFileToBackup() {
+      if (filesToBackup.isEmpty()) {
+        return null;
+      }
+
+      return filesToBackup.remove(0);
+    }
+
+    protected void backupComplete(BackupFile file) throws IOException {
+      synchronized (backupStatus) {
+        backupPoint.setSnapZxid(file.getMinZxid());
+        backupStatus.update(backupPoint);
+      }
+      backupStats.incrementNumberOfSnapshotFilesBackedUpThisIteration();
+
+      logger.info("Updated backedup snap zxid to {}",
+          ZxidUtils.zxidToString(backupPoint.getSnapZxid()));
+    }
+  }
+
+  /**
+   * Constructor for the BackupManager.
+   * @param snapDir the snapshot directory
+   * @param dataLogDir the txnlog directory
+   * @param serverId the id of the zk server
+   * @param backupConfig the backup config object
+   * @throws IOException
+   */
+  public BackupManager(File snapDir, File dataLogDir, long serverId, BackupConfig backupConfig)
+      throws IOException {
+    logger = LoggerFactory.getLogger(BackupManager.class);
+    logger.info("snapDir={}", snapDir.getPath());
+    logger.info("dataLogDir={}", dataLogDir.getPath());
+    logger.info("backupStatusDir={}", backupConfig.getStatusDir().getPath());
+    logger.info("tmpDir={}", backupConfig.getTmpDir().getPath());
+    logger.info("backupIntervalInMinutes={}", backupConfig.getBackupIntervalInMinutes());
+    logger.info("serverId={}", serverId);
+    logger.info("namespace={}", backupConfig.getNamespace());
+
+    this.snapDir = snapDir;
+    this.dataLogDir = dataLogDir;
+    this.backupConfig = backupConfig;
+    // Note: tmpDir is namespaced
+    this.tmpDir = new File(String.join(File.separator, backupConfig.getTmpDir().getAbsolutePath(),
+        backupConfig.getNamespace()));
+    // Note: statusDir is namespaced
+    this.backupStatus = new BackupStatus(new File(String
+        .join(File.separator, backupConfig.getStatusDir().getAbsolutePath(),

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/io/File.<init>(Ljava/lang/String;)V) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621077&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621077&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621077&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621077&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621077&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/BackupFileInfo.java:
##########
@@ -0,0 +1,164 @@
+/**
+ * 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.zookeeper.server.backup;
+
+import java.io.File;
+
+import com.google.common.collect.Range;
+import org.apache.zookeeper.server.backup.BackupUtil.BackupFileType;
+import org.apache.zookeeper.server.backup.BackupUtil.IntervalEndpoint;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.timetable.TimetableBackup;
+import org.apache.zookeeper.server.persistence.SnapStream;
+import org.apache.zookeeper.server.persistence.Util;
+
+/**
+ * Metadata for a file that has been backed-up
+ * Assumes that the name of the backed up file uses the format:
+ * prefix.lowzxid-highzxid where prefix is one of the standard snap or log file prefixes, or
+ * "lostLog".
+ * In the case of timetable backup file, it takes a format of timetable.lowTimestamp-highTimestamp.
+ */
+public class BackupFileInfo {
+  public static final long NOT_SET = -1L;
+  private static final String DASH_DELIMITER = "-";
+
+  private final File backupFile;
+  private final File standardFile;
+  private final Range<Long> range;
+  private final BackupFileType fileType;
+  private final long modificationTime;
+  private final long size;
+
+  /**
+   * Constructor that pulls backup metadata based on the backed-up filename
+   * @param backedupFile the backed-up file with the name in the form prefix.lowzxid-highzxid
+   *                     for example snapshot.9a0000a344-9a0000b012.
+   *                     if timetable backup, ranges are decimal longs (posix timestamps)
+   * @param modificationTime the file modification time
+   * @param size the size of the file in bytes
+   */
+  public BackupFileInfo(File backedupFile, long modificationTime, long size) {
+    this.backupFile = backedupFile;
+    this.modificationTime = modificationTime;
+    this.size = size;
+
+    String backedupFilename = this.backupFile.getName();
+
+    if (backedupFilename.startsWith(BackupUtil.LOST_LOG_PREFIX)) {
+      this.fileType = BackupFileType.LOSTLOG;
+      this.standardFile = this.backupFile;
+    } else if (backedupFilename.startsWith(Util.SNAP_PREFIX)) {
+      this.fileType = BackupFileType.SNAPSHOT;
+      // For snapshot backup files, we need to consider the case of snapshot compression
+      String streamMode = SnapStream.getStreamMode(backedupFilename).getName();
+      String standardFileName;
+      if (streamMode.isEmpty()) {
+        // No compression was used, so simply drop the end part
+        standardFileName = backedupFilename.split(DASH_DELIMITER)[0];
+      } else {
+        // Snapshot compression is enabled; standardName looks like "snapshot.<zxid>.<streamMode>"
+        // Need to remove the ending zxid
+        String[] nameParts = backedupFilename.split("\\.");
+        if (nameParts.length != 3) {
+          throw new BackupException(
+              "BackupFileInfo: unable to create standardFile reference! backedupFilename: "
+                  + backedupFilename + " StreamMode: " + streamMode);
+        }
+        String zxidPartWithoutEnd = nameParts[1].split(DASH_DELIMITER)[0];
+        // Combine all parts to generate a backup name
+        standardFileName = nameParts[0] + "." + zxidPartWithoutEnd + "." + nameParts[2];
+      }
+      this.standardFile = new File(this.backupFile.getParentFile(), standardFileName);
+    } else if (backedupFilename.startsWith(Util.TXLOG_PREFIX)) {
+      this.fileType = BackupFileType.TXNLOG;
+      this.standardFile =
+          new File(this.backupFile.getParentFile(), backedupFilename.split(DASH_DELIMITER)[0]);

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/io/File.<init>(Ljava/io/File;Ljava/lang/String;)V) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621094&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621094&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621094&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621094&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621094&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/storage/impl/FileSystemBackupStorage.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.zookeeper.server.backup.storage.impl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.zookeeper.server.backup.BackupConfig;
+import org.apache.zookeeper.server.backup.BackupFileInfo;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.storage.BackupStorageUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation for backup storage provider for file systems that store files in a tree (hierarchical) structure
+ * To use this class for different file systems, use appropriate address for backupStoragePath in BackupConfig
+ * For example:
+ * 1. hard disk drive & solid-state drive: /mountPoint/relativePathToMountPoint
+ * 2. NFS: /nfsClientMountPoint/relativePathToMountPoint
+ * 3. local disk: an absolute path to a directory
+ */
+public class FileSystemBackupStorage implements BackupStorageProvider {
+  private static final Logger LOG = LoggerFactory.getLogger(FileSystemBackupStorage.class);
+  private final String fileRootPath;
+  private final ReadWriteLock rwLock;
+  private final Lock sharedLock;
+  private final Lock exclusiveLock;
+
+  /**
+   * Constructor using BackupConfig to get backup storage info
+   * @param backupConfig The information and settings about backup storage, to be set as a part of ZooKeeper server config
+   */
+  public FileSystemBackupStorage(BackupConfig backupConfig) {
+    if (!new File(backupConfig.getBackupStoragePath()).exists()) {
+      throw new BackupException(
+          "The backup storage is not ready, please check the path: " + backupConfig
+              .getBackupStoragePath());
+    }
+    fileRootPath = String
+        .join(File.separator, backupConfig.getBackupStoragePath(), backupConfig.getNamespace());
+    rwLock = new ReentrantReadWriteLock();
+    sharedLock = rwLock.readLock();
+    exclusiveLock = rwLock.writeLock();
+  }
+
+  @Override
+  public BackupFileInfo getBackupFileInfo(File file) throws IOException {
+    String backupFilePath = BackupStorageUtil.constructBackupFilePath(file.getName(), fileRootPath);
+    File backupFile = new File(backupFilePath);
+
+    if (!backupFile.exists()) {
+      return new BackupFileInfo(backupFile, BackupFileInfo.NOT_SET, BackupFileInfo.NOT_SET);
+    }
+
+    BasicFileAttributes fileAttributes =
+        Files.readAttributes(Paths.get(backupFilePath), BasicFileAttributes.class);

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/nio/file/Paths.get(Ljava/lang/String;[Ljava/lang/String;)Ljava/nio/file/Path;) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621099&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621099&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621099&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621099&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621099&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/BackupProcess.java:
##########
@@ -0,0 +1,175 @@
+package org.apache.zookeeper.server.backup;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.slf4j.Logger;
+
+/**
+ * Base class for the txnlog and snap backup, and timetable backup processes.
+ * Provides the main backup loop and copying to remote storage (via HDFS, NFS, etc. APIs)
+ */
+public abstract class BackupProcess implements Runnable {
+  protected final Logger logger;
+  protected final BackupStorageProvider backupStorage;
+  private final long backupIntervalInMilliseconds;
+  protected volatile boolean isRunning = true;
+
+  /**
+   * Initialize starting backup point based on remote storage and backupStatus file
+   */
+  protected abstract void initialize() throws IOException;
+
+  /**
+   * Marks the start of a backup iteration.  A backup iteration is run every
+   * backup.interval.  This is called at the start of the iteration and before
+   * any calls to getNextFileToBackup
+   * @throws IOException
+   */
+  protected abstract void startIteration() throws IOException;
+
+  /**
+   * Marks the end of a backup iteration.  After this call there will be no more
+   * calls to getNextFileToBackup or backupComplete until startIteration is
+   * called again.
+   * @param errorFree whether the iteration was error free
+   * @throws IOException
+   */
+  protected abstract void endIteration(boolean errorFree);
+
+  /**
+   * Get the next file to backup
+   * @return the next file to copy to backup storage.
+   * @throws IOException
+   */
+  protected abstract BackupManager.BackupFile getNextFileToBackup() throws IOException;
+
+  /**
+   * Marks that the copy of the specified file to backup storage has completed
+   * @param file the file to backup
+   * @throws IOException
+   */
+  protected abstract void backupComplete(BackupManager.BackupFile file) throws IOException;
+
+  /**
+   * Create an instance of the backup process
+   * @param logger the logger to use for this process.
+   */
+  public BackupProcess(Logger logger, BackupStorageProvider backupStorage,
+      long backupIntervalInMilliseconds) {
+    if (logger == null) {
+      throw new IllegalArgumentException("BackupProcess: logger is null!");
+    }
+
+    this.logger = logger;
+    this.backupStorage = backupStorage;
+    this.backupIntervalInMilliseconds = backupIntervalInMilliseconds;
+  }
+
+  /**
+   * Runs the main file based backup loop indefinitely.
+   */
+  public void run() {
+    run(0);
+  }
+
+  /**
+   * Runs the main file based backup loop the specified number of time.
+   * Calls methods implemented by derived classes to get the next file to copy.
+   */
+  public void run(int iterations) {
+    try {
+      boolean errorFree = true;
+      logger.debug("Thread starting.");
+
+      while (isRunning) {
+        BackupManager.BackupFile fileToCopy;
+        long startTime = System.currentTimeMillis();
+
+        try {
+          if (logger.isDebugEnabled()) {
+            logger.debug("Starting iteration");
+          }
+
+          // Cleanup any invalid backups that may have been left behind by the
+          // previous failed iteration.
+          // NOTE: Not done on first iteration (errorFree initialized to true) since
+          //       initialize already does this.
+          if (!errorFree) {
+            backupStorage.cleanupInvalidFiles(null);
+          }
+
+          startIteration();
+
+          while ((fileToCopy = getNextFileToBackup()) != null) {
+            // Consider: compress file before sending to remote storage
+            copyToRemoteStorage(fileToCopy);
+            backupComplete(fileToCopy);
+            fileToCopy.cleanup();
+          }
+
+          errorFree = true;
+        } catch (IOException e) {
+          errorFree = false;
+          logger.warn("Exception hit during backup", e);
+        }
+
+        endIteration(errorFree);
+        long elapsedTime = System.currentTimeMillis() - startTime;
+        logger.info("Completed backup iteration in {} milliseconds.  ErrorFree: {}.",
+            elapsedTime, errorFree);
+
+        if (iterations != 0) {
+          iterations--;
+
+          if (iterations < 1) {
+            break;
+          }
+        }
+
+        // Count elapsed time towards the backup interval
+        long waitTime = backupIntervalInMilliseconds - elapsedTime;
+
+        synchronized (this) {  // synchronized to get notification of termination
+          if (waitTime > 0) {
+            wait(waitTime);
+          }
+        }
+      }
+    } catch (InterruptedException e) {
+      logger.warn("Interrupted exception while waiting for backup interval.", e.fillInStackTrace());
+    } catch (Exception e) {
+      logger.error("Hit unexpected exception", e.fillInStackTrace());
+    }
+
+    logger.warn("BackupProcess thread exited loop!");
+  }
+
+  /**
+   * Copy given file to remote storage via Backup Storage (HDFS, NFS, etc.) APIs.
+   * @param fileToCopy the file to copy
+   * @throws IOException
+   */
+  private void copyToRemoteStorage(BackupManager.BackupFile fileToCopy) throws IOException {
+    if (fileToCopy.getFile() == null) {
+      return;
+    }
+
+    // Use the file name to encode the max included zxid
+    String backupName = BackupUtil.makeBackupName(
+        fileToCopy.getFile().getName(), fileToCopy.getMaxZxid());
+
+    backupStorage.copyToBackupStorage(fileToCopy.getFile(), new File(backupName));

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/io/File.<init>(Ljava/lang/String;)V) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621126&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621126&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621126&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621126&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621126&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/timetable/TimetableBackup.java:
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.zookeeper.server.backup.timetable;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.ObjectOutputStream;
+import java.util.Comparator;
+import java.util.SortedSet;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.zookeeper.server.backup.BackupFileInfo;
+import org.apache.zookeeper.server.backup.BackupManager;
+import org.apache.zookeeper.server.backup.BackupPoint;
+import org.apache.zookeeper.server.backup.BackupProcess;
+import org.apache.zookeeper.server.backup.BackupStatus;
+import org.apache.zookeeper.server.backup.BackupUtil;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.monitoring.TimetableBackupStats;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
+import org.apache.zookeeper.txn.TxnHeader;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implements Timetable-specific logic for BackupProcess.
+ *
+ * Backup timetable encodes data of the format <timestamp>:<zxid>. This is used to locate the
+ * closest zxid backup point given the timestamp. The use case is for users who wish to restore
+ * from backup at a specific time recorded in the backup timetable.
+ */
+public class TimetableBackup extends BackupProcess {
+  public static final String TIMETABLE_PREFIX = "timetable";
+  // Use an ordered map of <timestamp>:<zxid>
+  private final TreeMap<Long, String> timetableRecordMap = new TreeMap<>();
+  private final File tmpDir;
+  // Lock is used to keep access to timetableRecordMap exclusive
+  private final Lock lock = new ReentrantLock(true);
+
+  // Candidate files to be backed up each iteration sorted by name (using SortedSet)
+  private final SortedSet<BackupManager.BackupFile> candidateTimetableBackupFiles =
+      new TreeSet<>(Comparator.comparing(o -> o.getFile().getName()));
+  // BackupStatus is used here to keep track of timetable backup status in case of a crash/restart
+  // BackupStatus is written to a file. After a crash (e.g. JVM crash) or a restart, the
+  // locally-stored zkBackupStatus file will be read back to restore a BackupPoint
+  private final BackupStatus backupStatus;
+  private final BackupPoint backupPoint;
+  private final TimetableBackupStats backupStats; // Metrics
+
+  /**
+   * Create an instance of TimetableBackup.
+   * @param snapLog
+   * @param tmpDir
+   * @param backupStorageProvider
+   * @param backupIntervalInMilliseconds
+   * @param timetableBackupIntervalInMs
+   * @param backupStatus
+   */
+  public TimetableBackup(FileTxnSnapLog snapLog, File tmpDir,
+      BackupStorageProvider backupStorageProvider, long backupIntervalInMilliseconds,
+      long timetableBackupIntervalInMs, BackupStatus backupStatus, BackupPoint backupPoint,
+      TimetableBackupStats backupStats) {
+    super(LoggerFactory.getLogger(TimetableBackup.class), backupStorageProvider,
+        backupIntervalInMilliseconds);
+    this.tmpDir = tmpDir;
+    this.backupStatus = backupStatus;
+    this.backupPoint = backupPoint;
+    this.backupStats = backupStats;
+    // Start creating records
+    (new Thread(new TimetableRecorder(snapLog, timetableBackupIntervalInMs))).start();
+    logger.info("TimetableBackup::Starting TimetableBackup Process with backup interval: "
+        + backupIntervalInMilliseconds + " ms and timetable backup interval: "
+        + timetableBackupIntervalInMs + " ms.");
+  }
+
+  @Override
+  protected void initialize() throws IOException {
+    // Get the latest timetable backup file from backup storage
+    BackupFileInfo latest = BackupUtil.getLatest(backupStorage, BackupUtil.BackupFileType.TIMETABLE,
+        BackupUtil.IntervalEndpoint.END);
+
+    long latestTimestampBackedUp = latest == null ? BackupUtil.INVALID_TIMESTAMP
+        : latest.getIntervalEndpoint(BackupUtil.IntervalEndpoint.END);
+
+    logger.info(
+        "TimetableBackup::initialize(): latest timestamp from storage: {}, from BackupStatus: {}",
+        latestTimestampBackedUp, backupPoint.getTimestamp());
+
+    if (latestTimestampBackedUp != backupPoint.getTimestamp()) {
+      synchronized (backupStatus) {
+        backupPoint.setTimestamp(latestTimestampBackedUp);
+        backupStatus.update(backupPoint);
+      }
+    }
+  }
+
+  @Override
+  protected void startIteration() throws IOException {
+    backupStats.setTimetableBackupIterationStart();
+
+    // It's possible that the last iteration failed and left some tmp backup files behind - add
+    // them back to candidate backup files so the I/O write will happen again
+    getAllTmpBackupFiles();
+
+    // Create a timetable backup file from the cached records (timetableRecordMap)
+    lock.lock();
+    try {
+      if (!timetableRecordMap.isEmpty()) {
+        // Create a temp timetable backup file
+        // Make the backup file temporary so that it will be deleted after the iteration provided
+        // the I/O write to the backup storage succeeds. Otherwise, this temporary backup file will
+        // continue to exist in tmpDir locally until it's successfully written to the backup storage
+        BackupManager.BackupFile timetableBackupFromThisIteration =
+            new BackupManager.BackupFile(createTimetableBackupFile(), true, 0, 0);
+        candidateTimetableBackupFiles.add(timetableBackupFromThisIteration);
+        timetableRecordMap.clear(); // Clear the map
+      }
+    } catch (Exception e) {
+      logger.error("TimetableBackup::startIteration(): failed to create timetable file to back up!",
+          e);
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  protected void endIteration(boolean errorFree) {
+    // timetableRecordMap is cleared in startIteration() already, so we do not clear it here
+    backupStats.setTimetableBackupIterationDone(errorFree);
+  }
+
+  @Override
+  protected BackupManager.BackupFile getNextFileToBackup() throws IOException {
+    BackupManager.BackupFile nextFile = null;
+    if (!candidateTimetableBackupFiles.isEmpty()) {
+      nextFile = candidateTimetableBackupFiles.first();
+    }
+    return nextFile;
+  }
+
+  @Override
+  protected void backupComplete(BackupManager.BackupFile file) throws IOException {
+    // Remove from the candidate set because it has been copied to the backup storage successfully
+    candidateTimetableBackupFiles.remove(file);
+    synchronized (backupStatus) {
+      // Update the latest timestamp to which the timetable backup was successful
+      backupPoint.setTimestamp(Long.parseLong(file.getFile().getName().split("-")[1]));
+      backupStatus.update(backupPoint);
+    }
+    logger.info(
+        "TimetableBackup::backupComplete(): backup complete for file: {}. Updated backed up "
+            + "timestamp to {}", file.getFile().getName(), backupPoint.getTimestamp());
+  }
+
+  /**
+   * Create a file name for timetable backup.
+   * E.g.) timetable.<lowest timestamp>-<highest timestamp>
+   * @return
+   */
+  private String makeTimetableBackupFileName() {
+    return String.format("%s.%d-%d", TIMETABLE_PREFIX, timetableRecordMap.firstKey(),
+        timetableRecordMap.lastKey());
+  }
+
+  /**
+   * Write the content of timetableRecordMap to a file using a tmpDir.
+   * @return
+   * @throws IOException
+   */
+  private File createTimetableBackupFile() throws IOException {
+    File tempTimetableBackupFile = new File(tmpDir, makeTimetableBackupFileName());

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/io/File.<init>(Ljava/io/File;Ljava/lang/String;)V) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621124&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621124&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621124&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621124&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621124&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/BackupFileInfo.java:
##########
@@ -0,0 +1,164 @@
+/**
+ * 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.zookeeper.server.backup;
+
+import java.io.File;
+
+import com.google.common.collect.Range;
+import org.apache.zookeeper.server.backup.BackupUtil.BackupFileType;
+import org.apache.zookeeper.server.backup.BackupUtil.IntervalEndpoint;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.timetable.TimetableBackup;
+import org.apache.zookeeper.server.persistence.SnapStream;
+import org.apache.zookeeper.server.persistence.Util;
+
+/**
+ * Metadata for a file that has been backed-up
+ * Assumes that the name of the backed up file uses the format:
+ * prefix.lowzxid-highzxid where prefix is one of the standard snap or log file prefixes, or
+ * "lostLog".
+ * In the case of timetable backup file, it takes a format of timetable.lowTimestamp-highTimestamp.
+ */
+public class BackupFileInfo {
+  public static final long NOT_SET = -1L;
+  private static final String DASH_DELIMITER = "-";
+
+  private final File backupFile;
+  private final File standardFile;
+  private final Range<Long> range;
+  private final BackupFileType fileType;
+  private final long modificationTime;
+  private final long size;
+
+  /**
+   * Constructor that pulls backup metadata based on the backed-up filename
+   * @param backedupFile the backed-up file with the name in the form prefix.lowzxid-highzxid
+   *                     for example snapshot.9a0000a344-9a0000b012.
+   *                     if timetable backup, ranges are decimal longs (posix timestamps)
+   * @param modificationTime the file modification time
+   * @param size the size of the file in bytes
+   */
+  public BackupFileInfo(File backedupFile, long modificationTime, long size) {
+    this.backupFile = backedupFile;
+    this.modificationTime = modificationTime;
+    this.size = size;
+
+    String backedupFilename = this.backupFile.getName();
+
+    if (backedupFilename.startsWith(BackupUtil.LOST_LOG_PREFIX)) {
+      this.fileType = BackupFileType.LOSTLOG;
+      this.standardFile = this.backupFile;
+    } else if (backedupFilename.startsWith(Util.SNAP_PREFIX)) {
+      this.fileType = BackupFileType.SNAPSHOT;
+      // For snapshot backup files, we need to consider the case of snapshot compression
+      String streamMode = SnapStream.getStreamMode(backedupFilename).getName();
+      String standardFileName;
+      if (streamMode.isEmpty()) {
+        // No compression was used, so simply drop the end part
+        standardFileName = backedupFilename.split(DASH_DELIMITER)[0];
+      } else {
+        // Snapshot compression is enabled; standardName looks like "snapshot.<zxid>.<streamMode>"
+        // Need to remove the ending zxid
+        String[] nameParts = backedupFilename.split("\\.");
+        if (nameParts.length != 3) {
+          throw new BackupException(
+              "BackupFileInfo: unable to create standardFile reference! backedupFilename: "
+                  + backedupFilename + " StreamMode: " + streamMode);
+        }
+        String zxidPartWithoutEnd = nameParts[1].split(DASH_DELIMITER)[0];
+        // Combine all parts to generate a backup name
+        standardFileName = nameParts[0] + "." + zxidPartWithoutEnd + "." + nameParts[2];
+      }
+      this.standardFile = new File(this.backupFile.getParentFile(), standardFileName);

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/io/File.<init>(Ljava/io/File;Ljava/lang/String;)V) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621128&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621128&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621128&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621128&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621128&lift_comment_rating=5) ]



-- 
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: notifications-unsubscribe@zookeeper.apache.org

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


[GitHub] [zookeeper] sonatype-lift[bot] commented on a diff in pull request #1883: Add backup and restore with timetable

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on code in PR #1883:
URL: https://github.com/apache/zookeeper/pull/1883#discussion_r878545647


##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/storage/impl/FileSystemBackupStorage.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.zookeeper.server.backup.storage.impl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.zookeeper.server.backup.BackupConfig;
+import org.apache.zookeeper.server.backup.BackupFileInfo;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.storage.BackupStorageUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation for backup storage provider for file systems that store files in a tree (hierarchical) structure
+ * To use this class for different file systems, use appropriate address for backupStoragePath in BackupConfig
+ * For example:
+ * 1. hard disk drive & solid-state drive: /mountPoint/relativePathToMountPoint
+ * 2. NFS: /nfsClientMountPoint/relativePathToMountPoint
+ * 3. local disk: an absolute path to a directory
+ */
+public class FileSystemBackupStorage implements BackupStorageProvider {
+  private static final Logger LOG = LoggerFactory.getLogger(FileSystemBackupStorage.class);
+  private final String fileRootPath;
+  private final ReadWriteLock rwLock;
+  private final Lock sharedLock;
+  private final Lock exclusiveLock;
+
+  /**
+   * Constructor using BackupConfig to get backup storage info
+   * @param backupConfig The information and settings about backup storage, to be set as a part of ZooKeeper server config
+   */
+  public FileSystemBackupStorage(BackupConfig backupConfig) {
+    if (!new File(backupConfig.getBackupStoragePath()).exists()) {
+      throw new BackupException(
+          "The backup storage is not ready, please check the path: " + backupConfig
+              .getBackupStoragePath());
+    }
+    fileRootPath = String
+        .join(File.separator, backupConfig.getBackupStoragePath(), backupConfig.getNamespace());
+    rwLock = new ReentrantReadWriteLock();
+    sharedLock = rwLock.readLock();
+    exclusiveLock = rwLock.writeLock();
+  }
+
+  @Override
+  public BackupFileInfo getBackupFileInfo(File file) throws IOException {
+    String backupFilePath = BackupStorageUtil.constructBackupFilePath(file.getName(), fileRootPath);
+    File backupFile = new File(backupFilePath);
+
+    if (!backupFile.exists()) {
+      return new BackupFileInfo(backupFile, BackupFileInfo.NOT_SET, BackupFileInfo.NOT_SET);
+    }
+
+    BasicFileAttributes fileAttributes =
+        Files.readAttributes(Paths.get(backupFilePath), BasicFileAttributes.class);
+    return new BackupFileInfo(backupFile, fileAttributes.lastModifiedTime().toMillis(),
+        fileAttributes.size());
+  }
+
+  @Override
+  public List<BackupFileInfo> getBackupFileInfos(File path, String prefix) throws IOException {
+    String filePath = path == null ? "" : path.getPath();
+    String backupDirPath = Paths.get(fileRootPath, filePath).toString();
+    File backupDir = new File(backupDirPath);
+
+    if (!backupDir.exists()) {
+      return new ArrayList<>();
+    }
+
+    File[] files = BackupStorageUtil.getFilesWithPrefix(backupDir, prefix);
+
+    // Read the file info and add to the list. If an exception is thrown, the entire operation will fail
+    List<BackupFileInfo> backupFileInfos = new ArrayList<>();
+    for (File file : files) {
+      backupFileInfos.add(getBackupFileInfo(file));
+    }
+    return backupFileInfos;
+  }
+
+  @Override
+  public List<File> getDirectories(File path) {
+    String filePath = path == null ? "" : path.getPath();
+    String backupDirPath = BackupStorageUtil.constructBackupFilePath(filePath, fileRootPath);
+    File backupDir = new File(backupDirPath);
+
+    if (!backupDir.exists()) {
+      throw new BackupException(
+          "Backup directory " + filePath + " does not exist, could not get directory list.");
+    }
+
+    // Filter out all the files which are directories
+    FilenameFilter fileFilter = (dir, name) -> new File(dir, name).isDirectory();
+    File[] dirs = backupDir.listFiles(fileFilter);
+
+    if (dirs == null) {
+      return new ArrayList<>();
+    }
+    return Arrays.asList(dirs);
+  }
+
+  @Override
+  public InputStream open(File path) throws IOException {
+    if (!path.exists() || path.isDirectory()) {
+      throw new BackupException("The file with the file path " + path
+          + " does not exist or is a directory, could not open the file.");
+    }
+    return new FileInputStream(path);
+  }
+
+  @Override
+  public void copyToBackupStorage(File srcFile, File destName) throws IOException {
+    InputStream inputStream = null;
+    OutputStream outputStream = null;
+    String backupTempFilePath;
+    File backupTempFile;
+
+    sharedLock.lock();
+    try {
+      inputStream = open(srcFile);
+
+      backupTempFilePath = BackupStorageUtil
+          .constructBackupFilePath(BackupStorageUtil.constructTempFileName(destName.getName()),
+              fileRootPath);
+      backupTempFile = new File(backupTempFilePath);

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/io/File.<init>(Ljava/lang/String;)V) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226620986&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226620986&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226620986&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226620986&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226620986&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/RestoreFromBackupTool.java:
##########
@@ -0,0 +1,716 @@
+/*
+ * 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.zookeeper.server.backup;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.nio.file.Files;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Range;
+import org.apache.commons.cli.CommandLine;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.cli.RestoreCommand;
+import org.apache.zookeeper.common.ConfigException;
+import org.apache.zookeeper.server.backup.BackupUtil.BackupFileType;
+import org.apache.zookeeper.server.backup.BackupUtil.IntervalEndpoint;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.storage.BackupStorageUtil;
+import org.apache.zookeeper.server.backup.timetable.TimetableBackup;
+import org.apache.zookeeper.server.backup.timetable.TimetableUtil;
+import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
+import org.apache.zookeeper.server.persistence.Util;
+import org.apache.zookeeper.server.util.ZxidUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ *  This is a tool to restore, from backup storage, a snapshot and set of transaction log files
+ *  that combine to represent a transactionally consistent image of a ZooKeeper ensemble at
+ *  some zxid.
+ */
+public class RestoreFromBackupTool {
+  private static final Logger LOG = LoggerFactory.getLogger(RestoreFromBackupTool.class);
+
+  private static final int MAX_RETRIES = 10;
+  private static final String HEX_PREFIX = "0x";
+  private static final int CONNECTION_TIMEOUT = 300000;
+
+  BackupStorageProvider storage;
+  FileTxnSnapLog snapLog;
+  long zxidToRestore;
+  boolean dryRun;
+  File restoreTempDir;
+  boolean overwrite = false;
+
+  // Spot restoration
+  boolean isSpotRestoration = false;
+  String znodePathToRestore;
+  String zkServerConnectionStr;
+  boolean restoreRecursively = false;
+  ZooKeeper zk;
+  SpotRestorationTool spotRestorationTool;
+
+  List<BackupFileInfo> logs;
+  List<BackupFileInfo> snaps;
+  List<BackupFileInfo> filesToCopy;
+
+  int mostRecentLogNeededIndex;
+  int snapNeededIndex;
+  int oldestLogNeededIndex;
+
+  public enum BackupStorageOption {
+    GPFS("org.apache.zookeeper.server.backup.storage.impl.FileSystemBackupStorage"),
+    NFS("org.apache.zookeeper.server.backup.storage.impl.FileSystemBackupStorage");
+
+    private String storageProviderClassName;
+
+    BackupStorageOption(String className) {
+      this.storageProviderClassName = className;
+    }
+
+    public String getStorageProviderClassName() {
+      return storageProviderClassName;
+    }
+  }
+
+  /**
+   * Default constructor; requires using parseArgs to setup state.
+   */
+  public RestoreFromBackupTool() {
+    this(null, null, -1L, false, null);
+  }
+
+  /**
+   * Constructor
+   * @param storageProvider the backup provider from which to restore the backups
+   * @param snapLog the snap and log provider to use
+   * @param zxidToRestore the zxid upto which to restore, or Long.MAX to restore to the latest
+   *                      available transactionally consistent zxid.
+   * @param dryRun whether this is a dryrun in which case no files are actually copied
+   * @param restoreTempDir a temporary, local (not remote) directory to stage the backup files from
+   *                       remote backup storage for the processing stage of restoration. Note that
+   *                       this directory and the files in it will be removed after restoration.
+   */
+  RestoreFromBackupTool(BackupStorageProvider storageProvider, FileTxnSnapLog snapLog,
+      long zxidToRestore, boolean dryRun, File restoreTempDir) {
+
+    filesToCopy = new ArrayList<>();
+    snapNeededIndex = -1;
+    mostRecentLogNeededIndex = -1;
+    oldestLogNeededIndex = -1;
+
+    this.storage = storageProvider;
+    this.zxidToRestore = zxidToRestore;
+    this.snapLog = snapLog;
+    this.dryRun = dryRun;
+    this.restoreTempDir = restoreTempDir;
+  }
+
+  /**
+   * Parse and validate arguments to the tool
+   * @param cl the command line object with user inputs
+   * @return true if the arguments parse correctly; false in all other cases.
+   * @throws IOException if the backup provider cannot be instantiated correctly.
+   */
+  public void parseArgs(CommandLine cl) {
+    String backupStoragePath = cl.getOptionValue(RestoreCommand.OptionShortForm.BACKUP_STORE);
+    createBackupStorageProvider(backupStoragePath);
+
+    // Read the restore point
+    if (cl.hasOption(RestoreCommand.OptionShortForm.RESTORE_ZXID)) {
+      parseRestoreZxid(cl);
+    } else if (cl.hasOption(RestoreCommand.OptionShortForm.RESTORE_TIMESTAMP)) {
+      parseRestoreTimestamp(cl, backupStoragePath);
+    }
+
+    parseAndValidateSpotRestorationArgs(cl);
+
+    parseAndValidateOfflineRestoreDestination(cl);
+
+    parseRestoreTempDir(cl);
+
+    // Check if overwriting the destination directories is allowed
+    if (cl.hasOption(RestoreCommand.OptionShortForm.OVERWRITE)) {
+      overwrite = true;
+    }
+
+    // Check if this is a dry-run
+    if (cl.hasOption(RestoreCommand.OptionShortForm.DRY_RUN)) {
+      dryRun = true;
+    }
+
+    System.out.println("parseArgs successful.");
+  }
+
+  private void createBackupStorageProvider(String backupStoragePath) {
+    String[] backupStorageParams = backupStoragePath.split(":");
+    if (backupStorageParams.length != 4) {
+      System.err.println(
+          "Failed to parse backup storage connection information from the backup storage path provided, please check the input.");
+      System.err.println(
+          "For example: the format for a gpfs backup storage path should be \"gpfs:<config_path>:<backup_path>:<namespace>\".");
+      System.exit(1);
+    }
+
+    String userProvidedStorageName = backupStorageParams[0].toUpperCase();
+    try {
+      BackupStorageOption storageOption = BackupStorageOption.valueOf(userProvidedStorageName);
+      String backupStorageProviderClassName = storageOption.getStorageProviderClassName();
+
+      BackupConfig.RestorationConfigBuilder configBuilder =
+          new BackupConfig.RestorationConfigBuilder()
+              .setStorageProviderClassName(backupStorageProviderClassName)
+              .setBackupStoragePath(backupStorageParams[2]).setNamespace(backupStorageParams[3]);
+      if (!backupStorageParams[1].isEmpty()) {
+        configBuilder = configBuilder.setStorageConfig(new File(backupStorageParams[1]));
+      }
+      storage = BackupUtil.createStorageProviderImpl(configBuilder.build().get());
+    } catch (IllegalArgumentException e) {
+      System.err.println("Could not find a valid backup storage option based on the input: "
+          + userProvidedStorageName + ". Error message: " + e.getMessage());
+      e.printStackTrace();
+      System.exit(1);
+    } catch (ConfigException e) {
+      System.err.println(
+          "Could not generate a backup config based on the input, error message: " + e
+              .getMessage());
+      e.getStackTrace();
+      System.exit(1);
+    } catch (InstantiationException | InvocationTargetException | NoSuchMethodException | IllegalAccessException | ClassNotFoundException e) {
+      System.err.println(
+          "Could not generate a backup storage provider based on the input, error message: " + e
+              .getMessage());
+      e.printStackTrace();
+      System.exit(1);
+    }
+  }
+
+  private void parseRestoreZxid(CommandLine cl) {
+    String zxidToRestoreStr = cl.getOptionValue(RestoreCommand.OptionShortForm.RESTORE_ZXID);
+    if (zxidToRestoreStr.equalsIgnoreCase(BackupUtil.LATEST)) {
+      zxidToRestore = Long.MAX_VALUE;
+    } else {
+      int base = 10;
+      String numStr = zxidToRestoreStr;
+
+      if (zxidToRestoreStr.startsWith(HEX_PREFIX)) {
+        numStr = zxidToRestoreStr.substring(2);
+        base = 16;
+      }
+      try {
+        zxidToRestore = Long.parseLong(numStr, base);
+      } catch (NumberFormatException nfe) {
+        System.err
+            .println("Invalid number specified for restore zxid point, the input is: " + numStr);
+        System.exit(2);
+      }
+    }
+  }
+
+  private void parseRestoreTimestamp(CommandLine cl, String backupStoragePath) {
+    String timestampStr = cl.getOptionValue(RestoreCommand.OptionShortForm.RESTORE_TIMESTAMP);
+    String timetableStoragePath = backupStoragePath;
+    if (cl.hasOption(RestoreCommand.OptionShortForm.TIMETABLE_STORAGE_PATH)) {
+      timetableStoragePath =
+          cl.getOptionValue(RestoreCommand.OptionShortForm.TIMETABLE_STORAGE_PATH);
+    }
+    File[] timetableFiles = new File(timetableStoragePath)
+        .listFiles(file -> file.getName().startsWith(TimetableBackup.TIMETABLE_PREFIX));
+    if (timetableFiles == null || timetableFiles.length == 0) {
+      System.err.println("Could not find timetable files at the path: " + timetableStoragePath);
+      System.exit(2);
+    }
+    Map.Entry<Long, String> restorePoint;
+    String message;
+    try {
+      restorePoint = TimetableUtil.findLastZxidFromTimestamp(timetableFiles, timestampStr);
+      zxidToRestore = Long.parseLong(restorePoint.getValue(), 16);
+      String timeToRestore = new SimpleDateFormat("MM/dd/yyyy HH:mm:ss")
+          .format(new java.util.Date(restorePoint.getKey()));
+      if (timestampStr.equalsIgnoreCase(BackupUtil.LATEST)) {
+        message = "Restoring to " + timeToRestore + ", original request was to restore to latest.";
+      } else {
+        String requestedTimeToRestore = new SimpleDateFormat("MM/dd/yyyy HH:mm:ss")
+            .format(new java.util.Date(Long.decode(timestampStr)));
+        message = "Restoring to " + timeToRestore + ", original request was to restore to "
+            + requestedTimeToRestore + ".";
+      }
+      System.out.println(message);
+      LOG.info(message);
+    } catch (IllegalArgumentException | BackupException e) {
+      System.err.println(
+          "Could not find a valid zxid from timetable using the timestamp provided: " + timestampStr
+              + ". The error message is: " + e.getMessage());
+      e.printStackTrace();
+      System.exit(2);
+    }
+  }
+
+  private void parseAndValidateOfflineRestoreDestination(CommandLine cl) {
+    if (isSpotRestoration) {
+      return;
+    }
+    // Read restore destination: dataDir and logDir
+    try {
+      String snapDirPath = cl.getOptionValue(RestoreCommand.OptionShortForm.SNAP_DESTINATION);
+      String logDirPath = cl.getOptionValue(RestoreCommand.OptionShortForm.LOG_DESTINATION);
+
+      if (snapDirPath == null || logDirPath == null) {
+        throw new BackupException(
+            "Snap destination path and log destination path are not defined for offline restoration. SnapDirPath: "
+                + snapDirPath + ", logDirPath: " + logDirPath);
+      }
+
+      File snapDir = new File(snapDirPath);

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/io/File.<init>(Ljava/lang/String;)V) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226620998&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226620998&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226620998&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226620998&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226620998&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/RestoreFromBackupTool.java:
##########
@@ -0,0 +1,716 @@
+/*
+ * 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.zookeeper.server.backup;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.nio.file.Files;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Range;
+import org.apache.commons.cli.CommandLine;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.cli.RestoreCommand;
+import org.apache.zookeeper.common.ConfigException;
+import org.apache.zookeeper.server.backup.BackupUtil.BackupFileType;
+import org.apache.zookeeper.server.backup.BackupUtil.IntervalEndpoint;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.storage.BackupStorageUtil;
+import org.apache.zookeeper.server.backup.timetable.TimetableBackup;
+import org.apache.zookeeper.server.backup.timetable.TimetableUtil;
+import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
+import org.apache.zookeeper.server.persistence.Util;
+import org.apache.zookeeper.server.util.ZxidUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ *  This is a tool to restore, from backup storage, a snapshot and set of transaction log files
+ *  that combine to represent a transactionally consistent image of a ZooKeeper ensemble at
+ *  some zxid.
+ */
+public class RestoreFromBackupTool {
+  private static final Logger LOG = LoggerFactory.getLogger(RestoreFromBackupTool.class);
+
+  private static final int MAX_RETRIES = 10;
+  private static final String HEX_PREFIX = "0x";
+  private static final int CONNECTION_TIMEOUT = 300000;
+
+  BackupStorageProvider storage;
+  FileTxnSnapLog snapLog;
+  long zxidToRestore;
+  boolean dryRun;
+  File restoreTempDir;
+  boolean overwrite = false;
+
+  // Spot restoration
+  boolean isSpotRestoration = false;
+  String znodePathToRestore;
+  String zkServerConnectionStr;
+  boolean restoreRecursively = false;
+  ZooKeeper zk;
+  SpotRestorationTool spotRestorationTool;
+
+  List<BackupFileInfo> logs;
+  List<BackupFileInfo> snaps;
+  List<BackupFileInfo> filesToCopy;
+
+  int mostRecentLogNeededIndex;
+  int snapNeededIndex;
+  int oldestLogNeededIndex;
+
+  public enum BackupStorageOption {
+    GPFS("org.apache.zookeeper.server.backup.storage.impl.FileSystemBackupStorage"),
+    NFS("org.apache.zookeeper.server.backup.storage.impl.FileSystemBackupStorage");
+
+    private String storageProviderClassName;
+
+    BackupStorageOption(String className) {
+      this.storageProviderClassName = className;
+    }
+
+    public String getStorageProviderClassName() {
+      return storageProviderClassName;
+    }
+  }
+
+  /**
+   * Default constructor; requires using parseArgs to setup state.
+   */
+  public RestoreFromBackupTool() {
+    this(null, null, -1L, false, null);
+  }
+
+  /**
+   * Constructor
+   * @param storageProvider the backup provider from which to restore the backups
+   * @param snapLog the snap and log provider to use
+   * @param zxidToRestore the zxid upto which to restore, or Long.MAX to restore to the latest
+   *                      available transactionally consistent zxid.
+   * @param dryRun whether this is a dryrun in which case no files are actually copied
+   * @param restoreTempDir a temporary, local (not remote) directory to stage the backup files from
+   *                       remote backup storage for the processing stage of restoration. Note that
+   *                       this directory and the files in it will be removed after restoration.
+   */
+  RestoreFromBackupTool(BackupStorageProvider storageProvider, FileTxnSnapLog snapLog,
+      long zxidToRestore, boolean dryRun, File restoreTempDir) {
+
+    filesToCopy = new ArrayList<>();
+    snapNeededIndex = -1;
+    mostRecentLogNeededIndex = -1;
+    oldestLogNeededIndex = -1;
+
+    this.storage = storageProvider;
+    this.zxidToRestore = zxidToRestore;
+    this.snapLog = snapLog;
+    this.dryRun = dryRun;
+    this.restoreTempDir = restoreTempDir;
+  }
+
+  /**
+   * Parse and validate arguments to the tool
+   * @param cl the command line object with user inputs
+   * @return true if the arguments parse correctly; false in all other cases.
+   * @throws IOException if the backup provider cannot be instantiated correctly.
+   */
+  public void parseArgs(CommandLine cl) {
+    String backupStoragePath = cl.getOptionValue(RestoreCommand.OptionShortForm.BACKUP_STORE);
+    createBackupStorageProvider(backupStoragePath);
+
+    // Read the restore point
+    if (cl.hasOption(RestoreCommand.OptionShortForm.RESTORE_ZXID)) {
+      parseRestoreZxid(cl);
+    } else if (cl.hasOption(RestoreCommand.OptionShortForm.RESTORE_TIMESTAMP)) {
+      parseRestoreTimestamp(cl, backupStoragePath);
+    }
+
+    parseAndValidateSpotRestorationArgs(cl);
+
+    parseAndValidateOfflineRestoreDestination(cl);
+
+    parseRestoreTempDir(cl);
+
+    // Check if overwriting the destination directories is allowed
+    if (cl.hasOption(RestoreCommand.OptionShortForm.OVERWRITE)) {
+      overwrite = true;
+    }
+
+    // Check if this is a dry-run
+    if (cl.hasOption(RestoreCommand.OptionShortForm.DRY_RUN)) {
+      dryRun = true;
+    }
+
+    System.out.println("parseArgs successful.");
+  }
+
+  private void createBackupStorageProvider(String backupStoragePath) {
+    String[] backupStorageParams = backupStoragePath.split(":");
+    if (backupStorageParams.length != 4) {
+      System.err.println(
+          "Failed to parse backup storage connection information from the backup storage path provided, please check the input.");
+      System.err.println(
+          "For example: the format for a gpfs backup storage path should be \"gpfs:<config_path>:<backup_path>:<namespace>\".");
+      System.exit(1);
+    }
+
+    String userProvidedStorageName = backupStorageParams[0].toUpperCase();
+    try {
+      BackupStorageOption storageOption = BackupStorageOption.valueOf(userProvidedStorageName);
+      String backupStorageProviderClassName = storageOption.getStorageProviderClassName();
+
+      BackupConfig.RestorationConfigBuilder configBuilder =
+          new BackupConfig.RestorationConfigBuilder()
+              .setStorageProviderClassName(backupStorageProviderClassName)
+              .setBackupStoragePath(backupStorageParams[2]).setNamespace(backupStorageParams[3]);
+      if (!backupStorageParams[1].isEmpty()) {
+        configBuilder = configBuilder.setStorageConfig(new File(backupStorageParams[1]));
+      }
+      storage = BackupUtil.createStorageProviderImpl(configBuilder.build().get());
+    } catch (IllegalArgumentException e) {
+      System.err.println("Could not find a valid backup storage option based on the input: "
+          + userProvidedStorageName + ". Error message: " + e.getMessage());
+      e.printStackTrace();
+      System.exit(1);
+    } catch (ConfigException e) {
+      System.err.println(
+          "Could not generate a backup config based on the input, error message: " + e
+              .getMessage());
+      e.getStackTrace();
+      System.exit(1);
+    } catch (InstantiationException | InvocationTargetException | NoSuchMethodException | IllegalAccessException | ClassNotFoundException e) {
+      System.err.println(
+          "Could not generate a backup storage provider based on the input, error message: " + e
+              .getMessage());
+      e.printStackTrace();
+      System.exit(1);
+    }
+  }
+
+  private void parseRestoreZxid(CommandLine cl) {
+    String zxidToRestoreStr = cl.getOptionValue(RestoreCommand.OptionShortForm.RESTORE_ZXID);
+    if (zxidToRestoreStr.equalsIgnoreCase(BackupUtil.LATEST)) {
+      zxidToRestore = Long.MAX_VALUE;
+    } else {
+      int base = 10;
+      String numStr = zxidToRestoreStr;
+
+      if (zxidToRestoreStr.startsWith(HEX_PREFIX)) {
+        numStr = zxidToRestoreStr.substring(2);
+        base = 16;
+      }
+      try {
+        zxidToRestore = Long.parseLong(numStr, base);
+      } catch (NumberFormatException nfe) {
+        System.err
+            .println("Invalid number specified for restore zxid point, the input is: " + numStr);
+        System.exit(2);
+      }
+    }
+  }
+
+  private void parseRestoreTimestamp(CommandLine cl, String backupStoragePath) {
+    String timestampStr = cl.getOptionValue(RestoreCommand.OptionShortForm.RESTORE_TIMESTAMP);
+    String timetableStoragePath = backupStoragePath;
+    if (cl.hasOption(RestoreCommand.OptionShortForm.TIMETABLE_STORAGE_PATH)) {
+      timetableStoragePath =
+          cl.getOptionValue(RestoreCommand.OptionShortForm.TIMETABLE_STORAGE_PATH);
+    }
+    File[] timetableFiles = new File(timetableStoragePath)
+        .listFiles(file -> file.getName().startsWith(TimetableBackup.TIMETABLE_PREFIX));
+    if (timetableFiles == null || timetableFiles.length == 0) {
+      System.err.println("Could not find timetable files at the path: " + timetableStoragePath);
+      System.exit(2);
+    }
+    Map.Entry<Long, String> restorePoint;
+    String message;
+    try {
+      restorePoint = TimetableUtil.findLastZxidFromTimestamp(timetableFiles, timestampStr);
+      zxidToRestore = Long.parseLong(restorePoint.getValue(), 16);
+      String timeToRestore = new SimpleDateFormat("MM/dd/yyyy HH:mm:ss")
+          .format(new java.util.Date(restorePoint.getKey()));
+      if (timestampStr.equalsIgnoreCase(BackupUtil.LATEST)) {
+        message = "Restoring to " + timeToRestore + ", original request was to restore to latest.";
+      } else {
+        String requestedTimeToRestore = new SimpleDateFormat("MM/dd/yyyy HH:mm:ss")
+            .format(new java.util.Date(Long.decode(timestampStr)));
+        message = "Restoring to " + timeToRestore + ", original request was to restore to "
+            + requestedTimeToRestore + ".";
+      }
+      System.out.println(message);
+      LOG.info(message);
+    } catch (IllegalArgumentException | BackupException e) {
+      System.err.println(
+          "Could not find a valid zxid from timetable using the timestamp provided: " + timestampStr
+              + ". The error message is: " + e.getMessage());
+      e.printStackTrace();
+      System.exit(2);
+    }
+  }
+
+  private void parseAndValidateOfflineRestoreDestination(CommandLine cl) {
+    if (isSpotRestoration) {
+      return;
+    }
+    // Read restore destination: dataDir and logDir
+    try {
+      String snapDirPath = cl.getOptionValue(RestoreCommand.OptionShortForm.SNAP_DESTINATION);
+      String logDirPath = cl.getOptionValue(RestoreCommand.OptionShortForm.LOG_DESTINATION);
+
+      if (snapDirPath == null || logDirPath == null) {
+        throw new BackupException(
+            "Snap destination path and log destination path are not defined for offline restoration. SnapDirPath: "
+                + snapDirPath + ", logDirPath: " + logDirPath);
+      }
+
+      File snapDir = new File(snapDirPath);
+      File logDir = new File(logDirPath);
+      snapLog = new FileTxnSnapLog(logDir, snapDir);
+      checkSnapDataDirFileExistence();
+    } catch (IOException ioe) {
+      System.err.println("Could not setup transaction log utility." + ioe);
+      System.exit(3);
+    }
+  }
+
+  private void parseRestoreTempDir(CommandLine cl) {
+    if (cl.hasOption(RestoreCommand.OptionShortForm.LOCAL_RESTORE_TEMP_DIR_PATH)) {
+      String localRestoreTempDirPath =
+          cl.getOptionValue(RestoreCommand.OptionShortForm.LOCAL_RESTORE_TEMP_DIR_PATH);
+      restoreTempDir = new File(localRestoreTempDirPath);
+    }
+
+    if (restoreTempDir == null) {
+      if (isSpotRestoration) {
+        throw new BackupException(
+            "Local restore temp dir path is not defined for spot restoration.");
+      } else {
+        // This is an offline restoration
+        // If the user hasn't provided the restore temp dir parameter,
+        //then the tool will just create a temporary folder inside snapLog and delete it afterwards.
+        this.restoreTempDir = new File(snapLog.getDataDir(), "RestoreTempDir_" + zxidToRestore);
+      }
+    }
+  }
+
+  private void parseAndValidateSpotRestorationArgs(CommandLine cl) {
+    if (cl.hasOption(RestoreCommand.OptionShortForm.ZNODE_PATH_TO_RESTORE)) {
+      znodePathToRestore = cl.getOptionValue(RestoreCommand.OptionShortForm.ZNODE_PATH_TO_RESTORE);
+    }
+    if (cl.hasOption(RestoreCommand.OptionShortForm.ZK_SERVER_CONNECTION_STRING)) {
+      zkServerConnectionStr =
+          cl.getOptionValue(RestoreCommand.OptionShortForm.ZK_SERVER_CONNECTION_STRING);
+    }
+    if (cl.hasOption(RestoreCommand.OptionShortForm.RECURSIVE_SPOT_RESTORE)) {
+      restoreRecursively = true;
+    }
+    if (znodePathToRestore != null && zkServerConnectionStr != null) {
+      isSpotRestoration = true;
+    } else if (znodePathToRestore == null && zkServerConnectionStr == null) {
+      isSpotRestoration = false;
+    } else {
+      throw new BackupException(
+          "Znode path and zk server connection string must be provided in order to do spot restoration. Provided znode path: "
+              + znodePathToRestore + ", provided zk server connection string: "
+              + zkServerConnectionStr);
+    }
+  }
+
+  /**
+   * Attempts to perform a restore with up to MAX_RETRIES retries.
+   * @return true if the restore completed successfully, false in all other cases.
+   */
+  public boolean runWithRetries(CommandLine cl) {
+    parseArgs(cl);
+
+    int tries = 0;
+
+    if (dryRun) {
+      System.out.println("This is a DRYRUN, no files will actually be copied.");
+    }
+
+    while (tries < MAX_RETRIES) {
+      try {
+        run();
+        return true;
+      } catch (IllegalArgumentException re) {
+        System.err.println(
+            "Restore attempt failed, could not find all the required backup files to restore. "
+                + "Error message: " + re.getMessage());
+        re.printStackTrace();
+        return false;
+      } catch (BackupException be) {
+        System.err.println(
+            "Restoration attempt failed due to a backup exception, it's usually caused by required "
+                + "directories not existing or failure of creating directories, etc. Please check the message. "
+                + "Error message: " + be.getMessage());
+        be.printStackTrace();
+        return false;
+      } catch (Exception e) {
+        tries++;
+        System.err.println("Restore attempt failed; attempting again. " + tries + "/" + MAX_RETRIES
+            + ". Error message: " + e.getMessage());
+        e.printStackTrace();
+      }
+    }
+
+    System.err.println("Failed to restore after " + (tries + 1) + " attempts.");
+    return false;
+  }
+
+  /**
+   * Attempts to perform a restore.
+   */
+  public void run() throws IOException, InterruptedException {
+    try {
+      if (!findFilesToRestore()) {
+        throw new IllegalArgumentException("Failed to find valid snapshot and logs to restore.");
+      }
+
+      if (restoreTempDir == null || storage == null) {
+        throw new BackupException("The RestoreTempDir and BackupStorageProvider cannot be null.");
+      }
+
+      if (!restoreTempDir.exists() && !restoreTempDir.mkdirs()) {
+        throw new BackupException(
+            "Failed to create a temporary directory at path: " + restoreTempDir.getPath()
+                + " to store copied backup files.");
+      }
+
+      if (!dryRun) {
+        // This step will create a "version-2" directory inside restoreTempDir,
+        // all the selected backup files will be copied to version-2 directory
+        FileTxnSnapLog restoreTempSnapLog =
+            new FileTxnSnapLog(this.restoreTempDir, this.restoreTempDir);
+
+        copyBackupFilesToLocalTempDir(restoreTempSnapLog);
+        processCopiedBackupFiles(restoreTempSnapLog, zxidToRestore);
+        if (isSpotRestoration) {
+          performSpotRestoration(restoreTempDir);
+        } else {
+          // It is an offline restoration
+          copyProcessedRestoredFilesToDestination(restoreTempSnapLog);
+        }
+      }
+    } finally {
+      if (restoreTempDir != null && restoreTempDir.exists()) {
+        // Using recursive delete here because a "version-2" directory is created under restoreTempDir with FileTxnSnapLog
+        BackupStorageUtil.deleteDirectoryRecursively(restoreTempDir);
+      }
+    }
+  }
+
+  /**
+   * Finds the set of files (snapshot and txlog) that are required to restore to a transactionally
+   * consistent point for the requested zxid.
+   * Note that when restoring to the latest zxid, the transactionally consistent point may NOT
+   * be the latest backed up zxid if logs have been lost in between; or if there is no
+   * transactionally consistent point in which nothing will be restored but the restored will
+   * be considered successful.
+   * @return true if a transactionally consistent set of files could be found for the requested
+   *         restore point; false in all other cases.
+   * @throws IOException
+   */
+  private boolean findFilesToRestore() throws IOException {
+    snaps = BackupUtil.getBackupFiles(storage, BackupFileType.SNAPSHOT, IntervalEndpoint.START,
+        BackupUtil.SortOrder.DESCENDING);
+    logs = BackupUtil.getBackupFiles(storage,
+        new BackupFileType[]{BackupFileType.TXNLOG, BackupFileType.LOSTLOG}, IntervalEndpoint.START,
+        BackupUtil.SortOrder.DESCENDING);
+    filesToCopy = new ArrayList<>();
+
+    snapNeededIndex = -1;
+
+    while (findNextPossibleSnapshot()) {
+      if (findLogRange()) {
+        filesToCopy.add(snaps.get(snapNeededIndex));
+        filesToCopy.addAll(logs.subList(mostRecentLogNeededIndex, oldestLogNeededIndex + 1));
+        return true;
+      }
+
+      if (zxidToRestore != Long.MAX_VALUE) {
+        break;
+      }
+    }
+
+    // Restoring to an empty data tree (i.e. no backup files) is valid for restoring to
+    // latest
+    if (zxidToRestore == Long.MAX_VALUE) {
+      return true;
+    }
+
+    return false;
+  }
+
+  /**
+   * Find the next snapshot whose range is below the requested restore point.
+   * Note: in practice this only gets called once when zxidToRestore != Long.MAX
+   * @return true if a snapshot is found; false in all other cases.
+   */
+  private boolean findNextPossibleSnapshot() {
+    for (snapNeededIndex++; snapNeededIndex < snaps.size(); snapNeededIndex++) {
+      if (snaps.get(snapNeededIndex).getRange().upperEndpoint() <= zxidToRestore) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+
+  /**
+   * Find the range of log files needed to make the current proposed snapshot transactionally
+   * consistent to the requested zxid.
+   * When zxidToRestore == Long.MAX the transaction log range terminates either on the most
+   * recent backedup txnlog OR the last txnlog prior to a lost log range (assuming that log
+   * still makes the snapshot transactionally consistent).
+   * @return true if a valid log range is found; false in all other cases.
+   */
+  private boolean findLogRange() {
+    Preconditions.checkState(snapNeededIndex >= 0 && snapNeededIndex < snaps.size());
+
+    if (logs.size() == 0) {
+      return false;
+    }
+
+    BackupFileInfo snap = snaps.get(snapNeededIndex);
+    Range<Long> snapRange = snap.getRange();
+    oldestLogNeededIndex = logs.size() - 1;
+    mostRecentLogNeededIndex = 0;
+
+    // Find the first txlog that might make the snapshot valid, OR is a lost log
+    for (int i = 0; i < logs.size() - 1; i++) {
+      BackupFileInfo log = logs.get(i);
+      Range<Long> logRange = log.getRange();
+
+      if (logRange.lowerEndpoint() <= snapRange.lowerEndpoint()) {
+        oldestLogNeededIndex = i;
+        break;
+      }
+    }
+
+    // Starting if the oldest log that might allow the snapshot to be valid, find the txnlog
+    // that includes the restore point, OR is a lost log
+    for (int i = oldestLogNeededIndex; i > 0; i--) {
+      BackupFileInfo log = logs.get(i);
+      Range<Long> logRange = log.getRange();
+
+      if (log.getFileType() == BackupFileType.LOSTLOG
+          || logRange.upperEndpoint() >= zxidToRestore) {
+
+        mostRecentLogNeededIndex = i;
+        break;
+      }
+    }
+
+    return validateLogRange();
+  }
+
+  private boolean validateLogRange() {
+    Preconditions.checkState(oldestLogNeededIndex >= 0);
+    Preconditions.checkState(oldestLogNeededIndex < logs.size());
+    Preconditions.checkState(mostRecentLogNeededIndex >= 0);
+    Preconditions.checkState(mostRecentLogNeededIndex < logs.size());
+    Preconditions.checkState(oldestLogNeededIndex >= mostRecentLogNeededIndex);
+    Preconditions.checkState(snapNeededIndex >= 0);
+    Preconditions.checkState(snapNeededIndex < snaps.size());
+
+    BackupFileInfo snap = snaps.get(snapNeededIndex);
+    BackupFileInfo oldestLog = logs.get(oldestLogNeededIndex);
+    BackupFileInfo newestLog = logs.get(mostRecentLogNeededIndex);
+
+    if (oldestLog.getFileType() == BackupFileType.LOSTLOG) {
+      LOG.error("Could not find logs to make the snapshot '" + snap.getBackedUpFile()
+          + "' valid. Lost logs at " + logs.get(oldestLogNeededIndex).getRange());
+      return false;
+    }
+
+    if (newestLog.getFileType() == BackupFileType.LOSTLOG) {
+      if (zxidToRestore == Long.MAX_VALUE && oldestLogNeededIndex != mostRecentLogNeededIndex) {
+        // When restoring to the latest, we can use the last valid log prior to lost log
+        // range.
+        mostRecentLogNeededIndex++;
+      } else {
+        LOG.error("Could not find logs to make the snapshot '" + snap.getBackedUpFile()
+            + "' valid. Lost logs at " + logs.get(mostRecentLogNeededIndex).getRange() + ".");
+        return false;
+      }
+    }
+
+    Range<Long> fullRange = oldestLog.getRange().span(newestLog.getRange());
+
+    if (fullRange.lowerEndpoint() > snap.getRange().lowerEndpoint()) {
+      LOG.error("Could not find logs to make snap '" + snap.getBackedUpFile()
+          + "' valid. Logs start at zxid " + ZxidUtils.zxidToString(fullRange.lowerEndpoint())
+          + ".");
+      return false;
+    }
+
+    if (fullRange.upperEndpoint() < snap.getRange().upperEndpoint()) {
+      LOG.error("Could not find logs to make snap '" + snap.getBackedUpFile()
+          + "' valid. Logs end at zxid " + ZxidUtils.zxidToString(fullRange.upperEndpoint()) + ".");
+      return false;
+    }
+
+    if (zxidToRestore != Long.MAX_VALUE && fullRange.upperEndpoint() < zxidToRestore) {
+      LOG.error("Could not find logs to restore to zxid " + zxidToRestore + ". Logs end at zxid "
+          + ZxidUtils.zxidToString(fullRange.upperEndpoint()) + ".");
+      return false;
+    }
+
+    return true;
+  }
+
+  /**
+   * Check if the specified snap dir and data dir already have files inside.
+   * If so, ask user to confirm if they want to overwrite these two directories with restored files,
+   * which means to wipe out all existing files and the directories be populated with restored files.
+   */
+  private void checkSnapDataDirFileExistence() {
+    File dataDir = snapLog.getDataDir();
+    File snapDir = snapLog.getSnapDir();
+    if (!dataDir.exists() && !dataDir.mkdirs()) {
+      throw new BackupException("Failed to create a data directory at path: " + dataDir.getPath()
+          + " to store restored txn logs.");
+    }
+    if (!snapDir.exists() && !snapDir.mkdirs()) {
+      throw new BackupException("Failed to create a snap directory at path: " + snapDir.getPath()
+          + " to store restored snapshot files.");
+    }
+    String[] dataDirFiles = dataDir.list();
+    String[] snapDirFiles = snapDir.list();
+    if (Objects.requireNonNull(dataDirFiles).length > 0
+        || Objects.requireNonNull(snapDirFiles).length > 0) {
+      if (overwrite) {
+        LOG.warn(
+            "Overwriting the destination directories for restoration, deleting all existing files. "
+                + "The files under dataDir: " + dataDir.getPath() + " are: " + Arrays
+                .toString(dataDirFiles) + "; and files under snapDir: " + snapDir.getPath()
+                + " are: " + Arrays.toString(snapDirFiles) + ".");
+        Arrays.stream(Objects.requireNonNull(dataDir.listFiles())).forEach(File::delete);
+        Arrays.stream(Objects.requireNonNull(snapDir.listFiles())).forEach(File::delete);
+      } else {
+        throw new BackupException(
+            "The destination directories are not empty, user chose not to overwrite the entire directory, "
+                + "exiting restoration. Please check the destination directory dataDir path: "
+                + dataDir.getPath() + ", and snapDir path" + snapDir.getPath());
+      }
+    }
+  }
+
+  /**
+   * Copy selected backup files from backup storage to a local restore temporary directory for further processing later
+   * @param restoreTempSnapLog A FileTxnSnapLog instance created on the specified local temporary directory path
+   * @throws IOException
+   */
+  private void copyBackupFilesToLocalTempDir(FileTxnSnapLog restoreTempSnapLog) throws IOException {
+    // Copy backup files to local temp directory
+    for (BackupFileInfo backedupFile : filesToCopy) {
+      String standardFilename = backedupFile.getStandardFile().getName();
+      // Does not matter if we use dataDir or logDir since we make the paths of these two directories
+      // same in restoreTempSnapLog object for restoration
+      File localTempDest = new File(restoreTempSnapLog.getDataDir(), standardFilename);

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/io/File.<init>(Ljava/io/File;Ljava/lang/String;)V) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621002&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621002&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621002&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621002&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621002&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/storage/BackupStorageUtil.java:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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.zookeeper.server.backup.storage;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Comparator;
+import java.util.stream.Stream;
+
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.persistence.Util;
+
+/**
+ * Util methods for backup storage
+ */
+public class BackupStorageUtil {
+  public static final String TMP_FILE_PREFIX = "TMP_";
+  private static final File[] NO_FILE = new File[0];
+
+  /**
+   * Parse the prefix from a file name, also works for temporary file names in backup storage
+   * @param fileName The file name to be parsed
+   * @return "log" for ZK transaction log files or "snapshot" for ZK snapshots
+   */
+  public static String getFileTypePrefix(String fileName) {
+    String backupFileName = fileName;
+
+    //Remove the temporary file name prefix in order to determine file type
+    if (fileName.startsWith(TMP_FILE_PREFIX)) {
+      backupFileName = fileName.substring(TMP_FILE_PREFIX.length());
+    }
+
+    String fileTypePrefix;
+    if (backupFileName.startsWith(Util.SNAP_PREFIX)) {
+      fileTypePrefix = Util.SNAP_PREFIX;
+    } else if (backupFileName.startsWith(Util.TXLOG_PREFIX)) {
+      fileTypePrefix = Util.TXLOG_PREFIX;
+    } else {
+      throw new BackupException("No matching base file type found for file " + fileName);
+    }
+
+    return fileTypePrefix;
+  }
+
+  /**
+   * Construct the path of a backup file in the backup storage
+   * @param fileName The name of the file
+   * @param parentDir The path to the parent directory of the backup file.
+   * @return The path of the backup file in the format of:
+   * 1. parentDir path is not supplied: {fileName} or {fileName}
+   * 2. parentDir path is provided: {parentDir}/{fileName} or {parentDir}/{fileName}
+   */
+  public static String constructBackupFilePath(String fileName, String parentDir) {
+    //TODO: store snapshots and Txlogs in different subfolders for better organization
+    if (parentDir != null) {
+      return String.valueOf(Paths.get(parentDir, fileName));

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/nio/file/Paths.get(Ljava/lang/String;[Ljava/lang/String;)Ljava/nio/file/Path;) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621008&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621008&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621008&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621008&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621008&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/storage/BackupStorageUtil.java:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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.zookeeper.server.backup.storage;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Comparator;
+import java.util.stream.Stream;
+
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.persistence.Util;
+
+/**
+ * Util methods for backup storage
+ */
+public class BackupStorageUtil {
+  public static final String TMP_FILE_PREFIX = "TMP_";
+  private static final File[] NO_FILE = new File[0];
+
+  /**
+   * Parse the prefix from a file name, also works for temporary file names in backup storage
+   * @param fileName The file name to be parsed
+   * @return "log" for ZK transaction log files or "snapshot" for ZK snapshots
+   */
+  public static String getFileTypePrefix(String fileName) {
+    String backupFileName = fileName;
+
+    //Remove the temporary file name prefix in order to determine file type
+    if (fileName.startsWith(TMP_FILE_PREFIX)) {
+      backupFileName = fileName.substring(TMP_FILE_PREFIX.length());
+    }
+
+    String fileTypePrefix;
+    if (backupFileName.startsWith(Util.SNAP_PREFIX)) {
+      fileTypePrefix = Util.SNAP_PREFIX;
+    } else if (backupFileName.startsWith(Util.TXLOG_PREFIX)) {
+      fileTypePrefix = Util.TXLOG_PREFIX;
+    } else {
+      throw new BackupException("No matching base file type found for file " + fileName);
+    }
+
+    return fileTypePrefix;
+  }
+
+  /**
+   * Construct the path of a backup file in the backup storage
+   * @param fileName The name of the file
+   * @param parentDir The path to the parent directory of the backup file.
+   * @return The path of the backup file in the format of:
+   * 1. parentDir path is not supplied: {fileName} or {fileName}
+   * 2. parentDir path is provided: {parentDir}/{fileName} or {parentDir}/{fileName}
+   */
+  public static String constructBackupFilePath(String fileName, String parentDir) {
+    //TODO: store snapshots and Txlogs in different subfolders for better organization
+    if (parentDir != null) {
+      return String.valueOf(Paths.get(parentDir, fileName));
+    }
+    return fileName;
+  }
+
+  /**
+   * Construct temporary file name using backup file name
+   * @param fileName A backup file name: log.lowzxid-highzxid, snapshot.lowzxid-highzxid
+   * @return A temporary backup file name: TMP_log.lowzxid-highzxid, TMP_snapshot.lowzxid-highzxid
+   */
+  public static String constructTempFileName(String fileName) {
+    return TMP_FILE_PREFIX + fileName;
+  }
+
+  /**
+   * A basic method for streaming data from an input stream to an output stream
+   * @param inputStream The stream to read from
+   * @param outputStream The stream to write to
+   * @throws IOException
+   */
+  public static void streamData(InputStream inputStream, OutputStream outputStream)
+      throws IOException {
+    byte[] buffer = new byte[1024];
+    int lengthRead;
+    while ((lengthRead = inputStream.read(buffer)) > 0) {
+      outputStream.write(buffer, 0, lengthRead);
+      outputStream.flush();
+    }
+  }
+
+  /**
+   * Create a new file in a specified path, create the parent directories if they do not exist.
+   * @param file The path to create the file.
+   * @param overwriteIfExist If a file already exists in the location,
+   *                         1. true: delete the existing file and retry the creation of the new file,
+   *                         or 2. false: keep the existing file.
+   * @throws IOException
+   */
+  public static void createFile(File file, boolean overwriteIfExist) throws IOException {
+    file.getParentFile().mkdirs();
+    if (!file.getParentFile().exists()) {
+      throw new BackupException("Failed to create parent directories for file " + file.getName());
+    }
+
+    boolean retry = true;
+    while (retry) {
+      retry = overwriteIfExist;
+      if (!file.createNewFile()) {
+        if (file.exists()) {
+          if (retry && !file.delete()) {
+            throw new BackupException("A file with the file path " + file.getPath()
+                + " already exists, and failed to be overwritten.");
+          }
+        } else {
+          throw new BackupException("Failed to create a file at path: " + file.getPath());
+        }
+      }
+      retry = false;
+    }
+  }
+
+  /**
+   * Get a list of all files whose file name starts with a certain prefix under a directory
+   * @param directory The directory to search for the files
+   * @param prefix The prefix of file name
+   * @return
+   */
+  public static File[] getFilesWithPrefix(File directory, String prefix) {
+    if (directory == null) {
+      return NO_FILE;
+    }
+    FilenameFilter fileFilter = (dir, name) -> name.startsWith(prefix);
+    File[] files = directory.listFiles(fileFilter);
+    return files == null ? NO_FILE : files;
+  }
+
+  /**
+   * Delete all the files whose file names starts with temporary file name prefix
+   * @param directory The directory to search for temporary files
+   * @throws IOException
+   */
+  public static void cleanUpTempFiles(File directory) throws IOException {
+    File[] tempFiles = getFilesWithPrefix(directory, TMP_FILE_PREFIX);
+    for (File tempFile : tempFiles) {
+      Files.delete(Paths.get(tempFile.getPath()));
+    }
+  }
+
+  /**
+   * Delete a directory and all files inside it
+   * @param directory The path to the directory
+   * @throws IOException
+   */
+  public static void deleteDirectoryRecursively(File directory) throws IOException {
+    Stream<Path> files = Files.walk(Paths.get(directory.getPath()));

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/nio/file/Paths.get(Ljava/lang/String;[Ljava/lang/String;)Ljava/nio/file/Path;) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621027&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621027&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621027&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621027&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621027&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/storage/impl/FileSystemBackupStorage.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.zookeeper.server.backup.storage.impl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.zookeeper.server.backup.BackupConfig;
+import org.apache.zookeeper.server.backup.BackupFileInfo;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.storage.BackupStorageUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation for backup storage provider for file systems that store files in a tree (hierarchical) structure
+ * To use this class for different file systems, use appropriate address for backupStoragePath in BackupConfig
+ * For example:
+ * 1. hard disk drive & solid-state drive: /mountPoint/relativePathToMountPoint
+ * 2. NFS: /nfsClientMountPoint/relativePathToMountPoint
+ * 3. local disk: an absolute path to a directory
+ */
+public class FileSystemBackupStorage implements BackupStorageProvider {
+  private static final Logger LOG = LoggerFactory.getLogger(FileSystemBackupStorage.class);
+  private final String fileRootPath;
+  private final ReadWriteLock rwLock;
+  private final Lock sharedLock;
+  private final Lock exclusiveLock;
+
+  /**
+   * Constructor using BackupConfig to get backup storage info
+   * @param backupConfig The information and settings about backup storage, to be set as a part of ZooKeeper server config
+   */
+  public FileSystemBackupStorage(BackupConfig backupConfig) {
+    if (!new File(backupConfig.getBackupStoragePath()).exists()) {
+      throw new BackupException(
+          "The backup storage is not ready, please check the path: " + backupConfig
+              .getBackupStoragePath());
+    }
+    fileRootPath = String
+        .join(File.separator, backupConfig.getBackupStoragePath(), backupConfig.getNamespace());
+    rwLock = new ReentrantReadWriteLock();
+    sharedLock = rwLock.readLock();
+    exclusiveLock = rwLock.writeLock();
+  }
+
+  @Override
+  public BackupFileInfo getBackupFileInfo(File file) throws IOException {
+    String backupFilePath = BackupStorageUtil.constructBackupFilePath(file.getName(), fileRootPath);
+    File backupFile = new File(backupFilePath);
+
+    if (!backupFile.exists()) {
+      return new BackupFileInfo(backupFile, BackupFileInfo.NOT_SET, BackupFileInfo.NOT_SET);
+    }
+
+    BasicFileAttributes fileAttributes =
+        Files.readAttributes(Paths.get(backupFilePath), BasicFileAttributes.class);
+    return new BackupFileInfo(backupFile, fileAttributes.lastModifiedTime().toMillis(),
+        fileAttributes.size());
+  }
+
+  @Override
+  public List<BackupFileInfo> getBackupFileInfos(File path, String prefix) throws IOException {
+    String filePath = path == null ? "" : path.getPath();
+    String backupDirPath = Paths.get(fileRootPath, filePath).toString();
+    File backupDir = new File(backupDirPath);
+
+    if (!backupDir.exists()) {
+      return new ArrayList<>();
+    }
+
+    File[] files = BackupStorageUtil.getFilesWithPrefix(backupDir, prefix);
+
+    // Read the file info and add to the list. If an exception is thrown, the entire operation will fail
+    List<BackupFileInfo> backupFileInfos = new ArrayList<>();
+    for (File file : files) {
+      backupFileInfos.add(getBackupFileInfo(file));
+    }
+    return backupFileInfos;
+  }
+
+  @Override
+  public List<File> getDirectories(File path) {
+    String filePath = path == null ? "" : path.getPath();
+    String backupDirPath = BackupStorageUtil.constructBackupFilePath(filePath, fileRootPath);
+    File backupDir = new File(backupDirPath);
+
+    if (!backupDir.exists()) {
+      throw new BackupException(
+          "Backup directory " + filePath + " does not exist, could not get directory list.");
+    }
+
+    // Filter out all the files which are directories
+    FilenameFilter fileFilter = (dir, name) -> new File(dir, name).isDirectory();
+    File[] dirs = backupDir.listFiles(fileFilter);
+
+    if (dirs == null) {
+      return new ArrayList<>();
+    }
+    return Arrays.asList(dirs);
+  }
+
+  @Override
+  public InputStream open(File path) throws IOException {
+    if (!path.exists() || path.isDirectory()) {
+      throw new BackupException("The file with the file path " + path
+          + " does not exist or is a directory, could not open the file.");
+    }
+    return new FileInputStream(path);
+  }
+
+  @Override
+  public void copyToBackupStorage(File srcFile, File destName) throws IOException {
+    InputStream inputStream = null;
+    OutputStream outputStream = null;
+    String backupTempFilePath;
+    File backupTempFile;
+
+    sharedLock.lock();
+    try {
+      inputStream = open(srcFile);
+
+      backupTempFilePath = BackupStorageUtil
+          .constructBackupFilePath(BackupStorageUtil.constructTempFileName(destName.getName()),
+              fileRootPath);
+      backupTempFile = new File(backupTempFilePath);
+
+      BackupStorageUtil.createFile(backupTempFile, true);
+      outputStream = new FileOutputStream(backupTempFile);
+
+      BackupStorageUtil.streamData(inputStream, outputStream);
+
+      Files.move(Paths.get(backupTempFilePath),
+          Paths.get(BackupStorageUtil.constructBackupFilePath(destName.getName(), fileRootPath)),

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/nio/file/Paths.get(Ljava/lang/String;[Ljava/lang/String;)Ljava/nio/file/Path;) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226620992&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226620992&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226620992&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226620992&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226620992&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/storage/impl/FileSystemBackupStorage.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.zookeeper.server.backup.storage.impl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.zookeeper.server.backup.BackupConfig;
+import org.apache.zookeeper.server.backup.BackupFileInfo;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.storage.BackupStorageUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation for backup storage provider for file systems that store files in a tree (hierarchical) structure
+ * To use this class for different file systems, use appropriate address for backupStoragePath in BackupConfig
+ * For example:
+ * 1. hard disk drive & solid-state drive: /mountPoint/relativePathToMountPoint
+ * 2. NFS: /nfsClientMountPoint/relativePathToMountPoint
+ * 3. local disk: an absolute path to a directory
+ */
+public class FileSystemBackupStorage implements BackupStorageProvider {
+  private static final Logger LOG = LoggerFactory.getLogger(FileSystemBackupStorage.class);
+  private final String fileRootPath;
+  private final ReadWriteLock rwLock;
+  private final Lock sharedLock;
+  private final Lock exclusiveLock;
+
+  /**
+   * Constructor using BackupConfig to get backup storage info
+   * @param backupConfig The information and settings about backup storage, to be set as a part of ZooKeeper server config
+   */
+  public FileSystemBackupStorage(BackupConfig backupConfig) {
+    if (!new File(backupConfig.getBackupStoragePath()).exists()) {
+      throw new BackupException(
+          "The backup storage is not ready, please check the path: " + backupConfig
+              .getBackupStoragePath());
+    }
+    fileRootPath = String
+        .join(File.separator, backupConfig.getBackupStoragePath(), backupConfig.getNamespace());
+    rwLock = new ReentrantReadWriteLock();
+    sharedLock = rwLock.readLock();
+    exclusiveLock = rwLock.writeLock();
+  }
+
+  @Override
+  public BackupFileInfo getBackupFileInfo(File file) throws IOException {
+    String backupFilePath = BackupStorageUtil.constructBackupFilePath(file.getName(), fileRootPath);
+    File backupFile = new File(backupFilePath);
+
+    if (!backupFile.exists()) {
+      return new BackupFileInfo(backupFile, BackupFileInfo.NOT_SET, BackupFileInfo.NOT_SET);
+    }
+
+    BasicFileAttributes fileAttributes =
+        Files.readAttributes(Paths.get(backupFilePath), BasicFileAttributes.class);
+    return new BackupFileInfo(backupFile, fileAttributes.lastModifiedTime().toMillis(),
+        fileAttributes.size());
+  }
+
+  @Override
+  public List<BackupFileInfo> getBackupFileInfos(File path, String prefix) throws IOException {
+    String filePath = path == null ? "" : path.getPath();
+    String backupDirPath = Paths.get(fileRootPath, filePath).toString();
+    File backupDir = new File(backupDirPath);
+
+    if (!backupDir.exists()) {
+      return new ArrayList<>();
+    }
+
+    File[] files = BackupStorageUtil.getFilesWithPrefix(backupDir, prefix);
+
+    // Read the file info and add to the list. If an exception is thrown, the entire operation will fail
+    List<BackupFileInfo> backupFileInfos = new ArrayList<>();
+    for (File file : files) {
+      backupFileInfos.add(getBackupFileInfo(file));
+    }
+    return backupFileInfos;
+  }
+
+  @Override
+  public List<File> getDirectories(File path) {
+    String filePath = path == null ? "" : path.getPath();
+    String backupDirPath = BackupStorageUtil.constructBackupFilePath(filePath, fileRootPath);
+    File backupDir = new File(backupDirPath);
+
+    if (!backupDir.exists()) {
+      throw new BackupException(
+          "Backup directory " + filePath + " does not exist, could not get directory list.");
+    }
+
+    // Filter out all the files which are directories
+    FilenameFilter fileFilter = (dir, name) -> new File(dir, name).isDirectory();

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/io/File.<init>(Ljava/io/File;Ljava/lang/String;)V) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621000&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621000&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621000&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621000&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621000&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/RestoreFromBackupTool.java:
##########
@@ -0,0 +1,716 @@
+/*
+ * 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.zookeeper.server.backup;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.nio.file.Files;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Range;
+import org.apache.commons.cli.CommandLine;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.cli.RestoreCommand;
+import org.apache.zookeeper.common.ConfigException;
+import org.apache.zookeeper.server.backup.BackupUtil.BackupFileType;
+import org.apache.zookeeper.server.backup.BackupUtil.IntervalEndpoint;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.storage.BackupStorageUtil;
+import org.apache.zookeeper.server.backup.timetable.TimetableBackup;
+import org.apache.zookeeper.server.backup.timetable.TimetableUtil;
+import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
+import org.apache.zookeeper.server.persistence.Util;
+import org.apache.zookeeper.server.util.ZxidUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ *  This is a tool to restore, from backup storage, a snapshot and set of transaction log files
+ *  that combine to represent a transactionally consistent image of a ZooKeeper ensemble at
+ *  some zxid.
+ */
+public class RestoreFromBackupTool {
+  private static final Logger LOG = LoggerFactory.getLogger(RestoreFromBackupTool.class);
+
+  private static final int MAX_RETRIES = 10;
+  private static final String HEX_PREFIX = "0x";
+  private static final int CONNECTION_TIMEOUT = 300000;
+
+  BackupStorageProvider storage;
+  FileTxnSnapLog snapLog;
+  long zxidToRestore;
+  boolean dryRun;
+  File restoreTempDir;
+  boolean overwrite = false;
+
+  // Spot restoration
+  boolean isSpotRestoration = false;
+  String znodePathToRestore;
+  String zkServerConnectionStr;
+  boolean restoreRecursively = false;
+  ZooKeeper zk;
+  SpotRestorationTool spotRestorationTool;
+
+  List<BackupFileInfo> logs;
+  List<BackupFileInfo> snaps;
+  List<BackupFileInfo> filesToCopy;
+
+  int mostRecentLogNeededIndex;
+  int snapNeededIndex;
+  int oldestLogNeededIndex;
+
+  public enum BackupStorageOption {
+    GPFS("org.apache.zookeeper.server.backup.storage.impl.FileSystemBackupStorage"),
+    NFS("org.apache.zookeeper.server.backup.storage.impl.FileSystemBackupStorage");
+
+    private String storageProviderClassName;
+
+    BackupStorageOption(String className) {
+      this.storageProviderClassName = className;
+    }
+
+    public String getStorageProviderClassName() {
+      return storageProviderClassName;
+    }
+  }
+
+  /**
+   * Default constructor; requires using parseArgs to setup state.
+   */
+  public RestoreFromBackupTool() {
+    this(null, null, -1L, false, null);
+  }
+
+  /**
+   * Constructor
+   * @param storageProvider the backup provider from which to restore the backups
+   * @param snapLog the snap and log provider to use
+   * @param zxidToRestore the zxid upto which to restore, or Long.MAX to restore to the latest
+   *                      available transactionally consistent zxid.
+   * @param dryRun whether this is a dryrun in which case no files are actually copied
+   * @param restoreTempDir a temporary, local (not remote) directory to stage the backup files from
+   *                       remote backup storage for the processing stage of restoration. Note that
+   *                       this directory and the files in it will be removed after restoration.
+   */
+  RestoreFromBackupTool(BackupStorageProvider storageProvider, FileTxnSnapLog snapLog,
+      long zxidToRestore, boolean dryRun, File restoreTempDir) {
+
+    filesToCopy = new ArrayList<>();
+    snapNeededIndex = -1;
+    mostRecentLogNeededIndex = -1;
+    oldestLogNeededIndex = -1;
+
+    this.storage = storageProvider;
+    this.zxidToRestore = zxidToRestore;
+    this.snapLog = snapLog;
+    this.dryRun = dryRun;
+    this.restoreTempDir = restoreTempDir;
+  }
+
+  /**
+   * Parse and validate arguments to the tool
+   * @param cl the command line object with user inputs
+   * @return true if the arguments parse correctly; false in all other cases.
+   * @throws IOException if the backup provider cannot be instantiated correctly.
+   */
+  public void parseArgs(CommandLine cl) {
+    String backupStoragePath = cl.getOptionValue(RestoreCommand.OptionShortForm.BACKUP_STORE);
+    createBackupStorageProvider(backupStoragePath);
+
+    // Read the restore point
+    if (cl.hasOption(RestoreCommand.OptionShortForm.RESTORE_ZXID)) {
+      parseRestoreZxid(cl);
+    } else if (cl.hasOption(RestoreCommand.OptionShortForm.RESTORE_TIMESTAMP)) {
+      parseRestoreTimestamp(cl, backupStoragePath);
+    }
+
+    parseAndValidateSpotRestorationArgs(cl);
+
+    parseAndValidateOfflineRestoreDestination(cl);
+
+    parseRestoreTempDir(cl);
+
+    // Check if overwriting the destination directories is allowed
+    if (cl.hasOption(RestoreCommand.OptionShortForm.OVERWRITE)) {
+      overwrite = true;
+    }
+
+    // Check if this is a dry-run
+    if (cl.hasOption(RestoreCommand.OptionShortForm.DRY_RUN)) {
+      dryRun = true;
+    }
+
+    System.out.println("parseArgs successful.");
+  }
+
+  private void createBackupStorageProvider(String backupStoragePath) {
+    String[] backupStorageParams = backupStoragePath.split(":");
+    if (backupStorageParams.length != 4) {
+      System.err.println(
+          "Failed to parse backup storage connection information from the backup storage path provided, please check the input.");
+      System.err.println(
+          "For example: the format for a gpfs backup storage path should be \"gpfs:<config_path>:<backup_path>:<namespace>\".");
+      System.exit(1);
+    }
+
+    String userProvidedStorageName = backupStorageParams[0].toUpperCase();
+    try {
+      BackupStorageOption storageOption = BackupStorageOption.valueOf(userProvidedStorageName);
+      String backupStorageProviderClassName = storageOption.getStorageProviderClassName();
+
+      BackupConfig.RestorationConfigBuilder configBuilder =
+          new BackupConfig.RestorationConfigBuilder()
+              .setStorageProviderClassName(backupStorageProviderClassName)
+              .setBackupStoragePath(backupStorageParams[2]).setNamespace(backupStorageParams[3]);
+      if (!backupStorageParams[1].isEmpty()) {
+        configBuilder = configBuilder.setStorageConfig(new File(backupStorageParams[1]));
+      }
+      storage = BackupUtil.createStorageProviderImpl(configBuilder.build().get());
+    } catch (IllegalArgumentException e) {
+      System.err.println("Could not find a valid backup storage option based on the input: "
+          + userProvidedStorageName + ". Error message: " + e.getMessage());
+      e.printStackTrace();
+      System.exit(1);
+    } catch (ConfigException e) {
+      System.err.println(
+          "Could not generate a backup config based on the input, error message: " + e
+              .getMessage());
+      e.getStackTrace();
+      System.exit(1);
+    } catch (InstantiationException | InvocationTargetException | NoSuchMethodException | IllegalAccessException | ClassNotFoundException e) {
+      System.err.println(
+          "Could not generate a backup storage provider based on the input, error message: " + e
+              .getMessage());
+      e.printStackTrace();
+      System.exit(1);
+    }
+  }
+
+  private void parseRestoreZxid(CommandLine cl) {
+    String zxidToRestoreStr = cl.getOptionValue(RestoreCommand.OptionShortForm.RESTORE_ZXID);
+    if (zxidToRestoreStr.equalsIgnoreCase(BackupUtil.LATEST)) {
+      zxidToRestore = Long.MAX_VALUE;
+    } else {
+      int base = 10;
+      String numStr = zxidToRestoreStr;
+
+      if (zxidToRestoreStr.startsWith(HEX_PREFIX)) {
+        numStr = zxidToRestoreStr.substring(2);
+        base = 16;
+      }
+      try {
+        zxidToRestore = Long.parseLong(numStr, base);
+      } catch (NumberFormatException nfe) {
+        System.err
+            .println("Invalid number specified for restore zxid point, the input is: " + numStr);
+        System.exit(2);
+      }
+    }
+  }
+
+  private void parseRestoreTimestamp(CommandLine cl, String backupStoragePath) {
+    String timestampStr = cl.getOptionValue(RestoreCommand.OptionShortForm.RESTORE_TIMESTAMP);
+    String timetableStoragePath = backupStoragePath;
+    if (cl.hasOption(RestoreCommand.OptionShortForm.TIMETABLE_STORAGE_PATH)) {
+      timetableStoragePath =
+          cl.getOptionValue(RestoreCommand.OptionShortForm.TIMETABLE_STORAGE_PATH);
+    }
+    File[] timetableFiles = new File(timetableStoragePath)
+        .listFiles(file -> file.getName().startsWith(TimetableBackup.TIMETABLE_PREFIX));
+    if (timetableFiles == null || timetableFiles.length == 0) {
+      System.err.println("Could not find timetable files at the path: " + timetableStoragePath);
+      System.exit(2);
+    }
+    Map.Entry<Long, String> restorePoint;
+    String message;
+    try {
+      restorePoint = TimetableUtil.findLastZxidFromTimestamp(timetableFiles, timestampStr);
+      zxidToRestore = Long.parseLong(restorePoint.getValue(), 16);
+      String timeToRestore = new SimpleDateFormat("MM/dd/yyyy HH:mm:ss")
+          .format(new java.util.Date(restorePoint.getKey()));
+      if (timestampStr.equalsIgnoreCase(BackupUtil.LATEST)) {
+        message = "Restoring to " + timeToRestore + ", original request was to restore to latest.";
+      } else {
+        String requestedTimeToRestore = new SimpleDateFormat("MM/dd/yyyy HH:mm:ss")
+            .format(new java.util.Date(Long.decode(timestampStr)));
+        message = "Restoring to " + timeToRestore + ", original request was to restore to "
+            + requestedTimeToRestore + ".";
+      }
+      System.out.println(message);
+      LOG.info(message);
+    } catch (IllegalArgumentException | BackupException e) {
+      System.err.println(
+          "Could not find a valid zxid from timetable using the timestamp provided: " + timestampStr
+              + ". The error message is: " + e.getMessage());
+      e.printStackTrace();
+      System.exit(2);
+    }
+  }
+
+  private void parseAndValidateOfflineRestoreDestination(CommandLine cl) {
+    if (isSpotRestoration) {
+      return;
+    }
+    // Read restore destination: dataDir and logDir
+    try {
+      String snapDirPath = cl.getOptionValue(RestoreCommand.OptionShortForm.SNAP_DESTINATION);
+      String logDirPath = cl.getOptionValue(RestoreCommand.OptionShortForm.LOG_DESTINATION);
+
+      if (snapDirPath == null || logDirPath == null) {
+        throw new BackupException(
+            "Snap destination path and log destination path are not defined for offline restoration. SnapDirPath: "
+                + snapDirPath + ", logDirPath: " + logDirPath);
+      }
+
+      File snapDir = new File(snapDirPath);
+      File logDir = new File(logDirPath);

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/io/File.<init>(Ljava/lang/String;)V) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621006&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621006&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621006&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621006&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621006&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/storage/impl/FileSystemBackupStorage.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.zookeeper.server.backup.storage.impl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.zookeeper.server.backup.BackupConfig;
+import org.apache.zookeeper.server.backup.BackupFileInfo;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.storage.BackupStorageUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation for backup storage provider for file systems that store files in a tree (hierarchical) structure
+ * To use this class for different file systems, use appropriate address for backupStoragePath in BackupConfig
+ * For example:
+ * 1. hard disk drive & solid-state drive: /mountPoint/relativePathToMountPoint
+ * 2. NFS: /nfsClientMountPoint/relativePathToMountPoint
+ * 3. local disk: an absolute path to a directory
+ */
+public class FileSystemBackupStorage implements BackupStorageProvider {
+  private static final Logger LOG = LoggerFactory.getLogger(FileSystemBackupStorage.class);
+  private final String fileRootPath;
+  private final ReadWriteLock rwLock;
+  private final Lock sharedLock;
+  private final Lock exclusiveLock;
+
+  /**
+   * Constructor using BackupConfig to get backup storage info
+   * @param backupConfig The information and settings about backup storage, to be set as a part of ZooKeeper server config
+   */
+  public FileSystemBackupStorage(BackupConfig backupConfig) {
+    if (!new File(backupConfig.getBackupStoragePath()).exists()) {
+      throw new BackupException(
+          "The backup storage is not ready, please check the path: " + backupConfig
+              .getBackupStoragePath());
+    }
+    fileRootPath = String
+        .join(File.separator, backupConfig.getBackupStoragePath(), backupConfig.getNamespace());
+    rwLock = new ReentrantReadWriteLock();
+    sharedLock = rwLock.readLock();
+    exclusiveLock = rwLock.writeLock();
+  }
+
+  @Override
+  public BackupFileInfo getBackupFileInfo(File file) throws IOException {
+    String backupFilePath = BackupStorageUtil.constructBackupFilePath(file.getName(), fileRootPath);
+    File backupFile = new File(backupFilePath);
+
+    if (!backupFile.exists()) {
+      return new BackupFileInfo(backupFile, BackupFileInfo.NOT_SET, BackupFileInfo.NOT_SET);
+    }
+
+    BasicFileAttributes fileAttributes =
+        Files.readAttributes(Paths.get(backupFilePath), BasicFileAttributes.class);
+    return new BackupFileInfo(backupFile, fileAttributes.lastModifiedTime().toMillis(),
+        fileAttributes.size());
+  }
+
+  @Override
+  public List<BackupFileInfo> getBackupFileInfos(File path, String prefix) throws IOException {
+    String filePath = path == null ? "" : path.getPath();
+    String backupDirPath = Paths.get(fileRootPath, filePath).toString();
+    File backupDir = new File(backupDirPath);
+
+    if (!backupDir.exists()) {
+      return new ArrayList<>();
+    }
+
+    File[] files = BackupStorageUtil.getFilesWithPrefix(backupDir, prefix);
+
+    // Read the file info and add to the list. If an exception is thrown, the entire operation will fail
+    List<BackupFileInfo> backupFileInfos = new ArrayList<>();
+    for (File file : files) {
+      backupFileInfos.add(getBackupFileInfo(file));
+    }
+    return backupFileInfos;
+  }
+
+  @Override
+  public List<File> getDirectories(File path) {
+    String filePath = path == null ? "" : path.getPath();
+    String backupDirPath = BackupStorageUtil.constructBackupFilePath(filePath, fileRootPath);
+    File backupDir = new File(backupDirPath);
+
+    if (!backupDir.exists()) {
+      throw new BackupException(
+          "Backup directory " + filePath + " does not exist, could not get directory list.");
+    }
+
+    // Filter out all the files which are directories
+    FilenameFilter fileFilter = (dir, name) -> new File(dir, name).isDirectory();
+    File[] dirs = backupDir.listFiles(fileFilter);
+
+    if (dirs == null) {
+      return new ArrayList<>();
+    }
+    return Arrays.asList(dirs);
+  }
+
+  @Override
+  public InputStream open(File path) throws IOException {
+    if (!path.exists() || path.isDirectory()) {
+      throw new BackupException("The file with the file path " + path
+          + " does not exist or is a directory, could not open the file.");
+    }
+    return new FileInputStream(path);
+  }
+
+  @Override
+  public void copyToBackupStorage(File srcFile, File destName) throws IOException {
+    InputStream inputStream = null;
+    OutputStream outputStream = null;
+    String backupTempFilePath;
+    File backupTempFile;
+
+    sharedLock.lock();
+    try {
+      inputStream = open(srcFile);
+
+      backupTempFilePath = BackupStorageUtil
+          .constructBackupFilePath(BackupStorageUtil.constructTempFileName(destName.getName()),
+              fileRootPath);
+      backupTempFile = new File(backupTempFilePath);
+
+      BackupStorageUtil.createFile(backupTempFile, true);
+      outputStream = new FileOutputStream(backupTempFile);
+
+      BackupStorageUtil.streamData(inputStream, outputStream);
+
+      Files.move(Paths.get(backupTempFilePath),
+          Paths.get(BackupStorageUtil.constructBackupFilePath(destName.getName(), fileRootPath)),
+          StandardCopyOption.REPLACE_EXISTING);
+    } finally {
+      if (inputStream != null) {
+        inputStream.close();
+      }
+      if (outputStream != null) {
+        outputStream.close();
+      }
+      sharedLock.unlock();
+    }
+  }
+
+  @Override
+  public void copyToLocalStorage(File srcName, File destFile) throws IOException {
+    InputStream inputStream = null;
+    OutputStream outputStream = null;
+
+    // Create input stream from the source file in backup storage
+    String backupFilePath =
+        BackupStorageUtil.constructBackupFilePath(srcName.getName(), fileRootPath);
+    File backupFile = new File(backupFilePath);

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/io/File.<init>(Ljava/lang/String;)V) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621018&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621018&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621018&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621018&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621018&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/storage/impl/FileSystemBackupStorage.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.zookeeper.server.backup.storage.impl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.zookeeper.server.backup.BackupConfig;
+import org.apache.zookeeper.server.backup.BackupFileInfo;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.storage.BackupStorageUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation for backup storage provider for file systems that store files in a tree (hierarchical) structure
+ * To use this class for different file systems, use appropriate address for backupStoragePath in BackupConfig
+ * For example:
+ * 1. hard disk drive & solid-state drive: /mountPoint/relativePathToMountPoint
+ * 2. NFS: /nfsClientMountPoint/relativePathToMountPoint
+ * 3. local disk: an absolute path to a directory
+ */
+public class FileSystemBackupStorage implements BackupStorageProvider {
+  private static final Logger LOG = LoggerFactory.getLogger(FileSystemBackupStorage.class);
+  private final String fileRootPath;
+  private final ReadWriteLock rwLock;
+  private final Lock sharedLock;
+  private final Lock exclusiveLock;
+
+  /**
+   * Constructor using BackupConfig to get backup storage info
+   * @param backupConfig The information and settings about backup storage, to be set as a part of ZooKeeper server config
+   */
+  public FileSystemBackupStorage(BackupConfig backupConfig) {
+    if (!new File(backupConfig.getBackupStoragePath()).exists()) {
+      throw new BackupException(
+          "The backup storage is not ready, please check the path: " + backupConfig
+              .getBackupStoragePath());
+    }
+    fileRootPath = String
+        .join(File.separator, backupConfig.getBackupStoragePath(), backupConfig.getNamespace());
+    rwLock = new ReentrantReadWriteLock();
+    sharedLock = rwLock.readLock();
+    exclusiveLock = rwLock.writeLock();
+  }
+
+  @Override
+  public BackupFileInfo getBackupFileInfo(File file) throws IOException {
+    String backupFilePath = BackupStorageUtil.constructBackupFilePath(file.getName(), fileRootPath);
+    File backupFile = new File(backupFilePath);
+
+    if (!backupFile.exists()) {
+      return new BackupFileInfo(backupFile, BackupFileInfo.NOT_SET, BackupFileInfo.NOT_SET);
+    }
+
+    BasicFileAttributes fileAttributes =
+        Files.readAttributes(Paths.get(backupFilePath), BasicFileAttributes.class);
+    return new BackupFileInfo(backupFile, fileAttributes.lastModifiedTime().toMillis(),
+        fileAttributes.size());
+  }
+
+  @Override
+  public List<BackupFileInfo> getBackupFileInfos(File path, String prefix) throws IOException {
+    String filePath = path == null ? "" : path.getPath();
+    String backupDirPath = Paths.get(fileRootPath, filePath).toString();

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/nio/file/Paths.get(Ljava/lang/String;[Ljava/lang/String;)Ljava/nio/file/Path;) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621042&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621042&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621042&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621042&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621042&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/storage/impl/FileSystemBackupStorage.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.zookeeper.server.backup.storage.impl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.zookeeper.server.backup.BackupConfig;
+import org.apache.zookeeper.server.backup.BackupFileInfo;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.storage.BackupStorageUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation for backup storage provider for file systems that store files in a tree (hierarchical) structure
+ * To use this class for different file systems, use appropriate address for backupStoragePath in BackupConfig
+ * For example:
+ * 1. hard disk drive & solid-state drive: /mountPoint/relativePathToMountPoint
+ * 2. NFS: /nfsClientMountPoint/relativePathToMountPoint
+ * 3. local disk: an absolute path to a directory
+ */
+public class FileSystemBackupStorage implements BackupStorageProvider {
+  private static final Logger LOG = LoggerFactory.getLogger(FileSystemBackupStorage.class);
+  private final String fileRootPath;
+  private final ReadWriteLock rwLock;
+  private final Lock sharedLock;
+  private final Lock exclusiveLock;
+
+  /**
+   * Constructor using BackupConfig to get backup storage info
+   * @param backupConfig The information and settings about backup storage, to be set as a part of ZooKeeper server config
+   */
+  public FileSystemBackupStorage(BackupConfig backupConfig) {
+    if (!new File(backupConfig.getBackupStoragePath()).exists()) {
+      throw new BackupException(
+          "The backup storage is not ready, please check the path: " + backupConfig
+              .getBackupStoragePath());
+    }
+    fileRootPath = String
+        .join(File.separator, backupConfig.getBackupStoragePath(), backupConfig.getNamespace());
+    rwLock = new ReentrantReadWriteLock();
+    sharedLock = rwLock.readLock();
+    exclusiveLock = rwLock.writeLock();
+  }
+
+  @Override
+  public BackupFileInfo getBackupFileInfo(File file) throws IOException {
+    String backupFilePath = BackupStorageUtil.constructBackupFilePath(file.getName(), fileRootPath);
+    File backupFile = new File(backupFilePath);

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/io/File.<init>(Ljava/lang/String;)V) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621035&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621035&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621035&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621035&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621035&lift_comment_rating=5) ]



-- 
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: notifications-unsubscribe@zookeeper.apache.org

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


[GitHub] [zookeeper] eolivelli commented on pull request #1883: Add backup and restore with timetable

Posted by GitBox <gi...@apache.org>.
eolivelli commented on PR #1883:
URL: https://github.com/apache/zookeeper/pull/1883#issuecomment-1305733757

   closed/reopened in order to trigger CI


-- 
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: notifications-unsubscribe@zookeeper.apache.org

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


[GitHub] [zookeeper] sonatype-lift[bot] commented on a diff in pull request #1883: Add backup and restore with timetable

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on code in PR #1883:
URL: https://github.com/apache/zookeeper/pull/1883#discussion_r878546062


##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/storage/impl/FileSystemBackupStorage.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.zookeeper.server.backup.storage.impl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.zookeeper.server.backup.BackupConfig;
+import org.apache.zookeeper.server.backup.BackupFileInfo;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.storage.BackupStorageUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation for backup storage provider for file systems that store files in a tree (hierarchical) structure
+ * To use this class for different file systems, use appropriate address for backupStoragePath in BackupConfig
+ * For example:
+ * 1. hard disk drive & solid-state drive: /mountPoint/relativePathToMountPoint
+ * 2. NFS: /nfsClientMountPoint/relativePathToMountPoint
+ * 3. local disk: an absolute path to a directory
+ */
+public class FileSystemBackupStorage implements BackupStorageProvider {
+  private static final Logger LOG = LoggerFactory.getLogger(FileSystemBackupStorage.class);
+  private final String fileRootPath;
+  private final ReadWriteLock rwLock;
+  private final Lock sharedLock;
+  private final Lock exclusiveLock;
+
+  /**
+   * Constructor using BackupConfig to get backup storage info
+   * @param backupConfig The information and settings about backup storage, to be set as a part of ZooKeeper server config
+   */
+  public FileSystemBackupStorage(BackupConfig backupConfig) {
+    if (!new File(backupConfig.getBackupStoragePath()).exists()) {
+      throw new BackupException(
+          "The backup storage is not ready, please check the path: " + backupConfig
+              .getBackupStoragePath());
+    }
+    fileRootPath = String
+        .join(File.separator, backupConfig.getBackupStoragePath(), backupConfig.getNamespace());
+    rwLock = new ReentrantReadWriteLock();
+    sharedLock = rwLock.readLock();
+    exclusiveLock = rwLock.writeLock();
+  }
+
+  @Override
+  public BackupFileInfo getBackupFileInfo(File file) throws IOException {
+    String backupFilePath = BackupStorageUtil.constructBackupFilePath(file.getName(), fileRootPath);
+    File backupFile = new File(backupFilePath);
+
+    if (!backupFile.exists()) {
+      return new BackupFileInfo(backupFile, BackupFileInfo.NOT_SET, BackupFileInfo.NOT_SET);
+    }
+
+    BasicFileAttributes fileAttributes =
+        Files.readAttributes(Paths.get(backupFilePath), BasicFileAttributes.class);
+    return new BackupFileInfo(backupFile, fileAttributes.lastModifiedTime().toMillis(),
+        fileAttributes.size());
+  }
+
+  @Override
+  public List<BackupFileInfo> getBackupFileInfos(File path, String prefix) throws IOException {
+    String filePath = path == null ? "" : path.getPath();
+    String backupDirPath = Paths.get(fileRootPath, filePath).toString();
+    File backupDir = new File(backupDirPath);
+
+    if (!backupDir.exists()) {
+      return new ArrayList<>();
+    }
+
+    File[] files = BackupStorageUtil.getFilesWithPrefix(backupDir, prefix);
+
+    // Read the file info and add to the list. If an exception is thrown, the entire operation will fail
+    List<BackupFileInfo> backupFileInfos = new ArrayList<>();
+    for (File file : files) {
+      backupFileInfos.add(getBackupFileInfo(file));
+    }
+    return backupFileInfos;
+  }
+
+  @Override
+  public List<File> getDirectories(File path) {
+    String filePath = path == null ? "" : path.getPath();
+    String backupDirPath = BackupStorageUtil.constructBackupFilePath(filePath, fileRootPath);
+    File backupDir = new File(backupDirPath);
+
+    if (!backupDir.exists()) {
+      throw new BackupException(
+          "Backup directory " + filePath + " does not exist, could not get directory list.");
+    }
+
+    // Filter out all the files which are directories
+    FilenameFilter fileFilter = (dir, name) -> new File(dir, name).isDirectory();
+    File[] dirs = backupDir.listFiles(fileFilter);
+
+    if (dirs == null) {
+      return new ArrayList<>();
+    }
+    return Arrays.asList(dirs);
+  }
+
+  @Override
+  public InputStream open(File path) throws IOException {
+    if (!path.exists() || path.isDirectory()) {
+      throw new BackupException("The file with the file path " + path
+          + " does not exist or is a directory, could not open the file.");
+    }
+    return new FileInputStream(path);
+  }
+
+  @Override
+  public void copyToBackupStorage(File srcFile, File destName) throws IOException {
+    InputStream inputStream = null;
+    OutputStream outputStream = null;
+    String backupTempFilePath;
+    File backupTempFile;
+
+    sharedLock.lock();
+    try {
+      inputStream = open(srcFile);
+
+      backupTempFilePath = BackupStorageUtil
+          .constructBackupFilePath(BackupStorageUtil.constructTempFileName(destName.getName()),
+              fileRootPath);
+      backupTempFile = new File(backupTempFilePath);
+
+      BackupStorageUtil.createFile(backupTempFile, true);
+      outputStream = new FileOutputStream(backupTempFile);
+
+      BackupStorageUtil.streamData(inputStream, outputStream);
+
+      Files.move(Paths.get(backupTempFilePath),
+          Paths.get(BackupStorageUtil.constructBackupFilePath(destName.getName(), fileRootPath)),
+          StandardCopyOption.REPLACE_EXISTING);
+    } finally {
+      if (inputStream != null) {
+        inputStream.close();
+      }
+      if (outputStream != null) {
+        outputStream.close();
+      }
+      sharedLock.unlock();
+    }
+  }
+
+  @Override
+  public void copyToLocalStorage(File srcName, File destFile) throws IOException {
+    InputStream inputStream = null;
+    OutputStream outputStream = null;
+
+    // Create input stream from the source file in backup storage
+    String backupFilePath =
+        BackupStorageUtil.constructBackupFilePath(srcName.getName(), fileRootPath);
+    File backupFile = new File(backupFilePath);
+
+    try {
+      inputStream = open(backupFile);
+
+      BackupStorageUtil.createFile(destFile, true);
+      outputStream = new FileOutputStream(destFile);
+
+      BackupStorageUtil.streamData(inputStream, outputStream);
+    } finally {
+      if (inputStream != null) {
+        inputStream.close();
+      }
+      if (outputStream != null) {
+        outputStream.close();
+      }
+    }
+  }

Review Comment:
   *RESOURCE_LEAK:*  resource of type `java.io.FileOutputStream` acquired by call to `FileOutputStream(...)` at line 193 is not released after line 204.
   **Note**: potential exception at line 198
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621613&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621613&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621613&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621613&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621613&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/timetable/TimetableBackup.java:
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.zookeeper.server.backup.timetable;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.ObjectOutputStream;
+import java.util.Comparator;
+import java.util.SortedSet;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.zookeeper.server.backup.BackupFileInfo;
+import org.apache.zookeeper.server.backup.BackupManager;
+import org.apache.zookeeper.server.backup.BackupPoint;
+import org.apache.zookeeper.server.backup.BackupProcess;
+import org.apache.zookeeper.server.backup.BackupStatus;
+import org.apache.zookeeper.server.backup.BackupUtil;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.monitoring.TimetableBackupStats;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
+import org.apache.zookeeper.txn.TxnHeader;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implements Timetable-specific logic for BackupProcess.
+ *
+ * Backup timetable encodes data of the format <timestamp>:<zxid>. This is used to locate the
+ * closest zxid backup point given the timestamp. The use case is for users who wish to restore
+ * from backup at a specific time recorded in the backup timetable.
+ */
+public class TimetableBackup extends BackupProcess {
+  public static final String TIMETABLE_PREFIX = "timetable";
+  // Use an ordered map of <timestamp>:<zxid>
+  private final TreeMap<Long, String> timetableRecordMap = new TreeMap<>();
+  private final File tmpDir;
+  // Lock is used to keep access to timetableRecordMap exclusive
+  private final Lock lock = new ReentrantLock(true);
+
+  // Candidate files to be backed up each iteration sorted by name (using SortedSet)
+  private final SortedSet<BackupManager.BackupFile> candidateTimetableBackupFiles =
+      new TreeSet<>(Comparator.comparing(o -> o.getFile().getName()));
+  // BackupStatus is used here to keep track of timetable backup status in case of a crash/restart
+  // BackupStatus is written to a file. After a crash (e.g. JVM crash) or a restart, the
+  // locally-stored zkBackupStatus file will be read back to restore a BackupPoint
+  private final BackupStatus backupStatus;
+  private final BackupPoint backupPoint;
+  private final TimetableBackupStats backupStats; // Metrics
+
+  /**
+   * Create an instance of TimetableBackup.
+   * @param snapLog
+   * @param tmpDir
+   * @param backupStorageProvider
+   * @param backupIntervalInMilliseconds
+   * @param timetableBackupIntervalInMs
+   * @param backupStatus
+   */
+  public TimetableBackup(FileTxnSnapLog snapLog, File tmpDir,
+      BackupStorageProvider backupStorageProvider, long backupIntervalInMilliseconds,
+      long timetableBackupIntervalInMs, BackupStatus backupStatus, BackupPoint backupPoint,
+      TimetableBackupStats backupStats) {
+    super(LoggerFactory.getLogger(TimetableBackup.class), backupStorageProvider,
+        backupIntervalInMilliseconds);
+    this.tmpDir = tmpDir;
+    this.backupStatus = backupStatus;
+    this.backupPoint = backupPoint;
+    this.backupStats = backupStats;
+    // Start creating records
+    (new Thread(new TimetableRecorder(snapLog, timetableBackupIntervalInMs))).start();
+    logger.info("TimetableBackup::Starting TimetableBackup Process with backup interval: "
+        + backupIntervalInMilliseconds + " ms and timetable backup interval: "
+        + timetableBackupIntervalInMs + " ms.");
+  }
+
+  @Override
+  protected void initialize() throws IOException {
+    // Get the latest timetable backup file from backup storage
+    BackupFileInfo latest = BackupUtil.getLatest(backupStorage, BackupUtil.BackupFileType.TIMETABLE,
+        BackupUtil.IntervalEndpoint.END);
+
+    long latestTimestampBackedUp = latest == null ? BackupUtil.INVALID_TIMESTAMP
+        : latest.getIntervalEndpoint(BackupUtil.IntervalEndpoint.END);
+
+    logger.info(
+        "TimetableBackup::initialize(): latest timestamp from storage: {}, from BackupStatus: {}",
+        latestTimestampBackedUp, backupPoint.getTimestamp());
+
+    if (latestTimestampBackedUp != backupPoint.getTimestamp()) {
+      synchronized (backupStatus) {
+        backupPoint.setTimestamp(latestTimestampBackedUp);
+        backupStatus.update(backupPoint);
+      }
+    }
+  }
+
+  @Override
+  protected void startIteration() throws IOException {
+    backupStats.setTimetableBackupIterationStart();
+
+    // It's possible that the last iteration failed and left some tmp backup files behind - add
+    // them back to candidate backup files so the I/O write will happen again
+    getAllTmpBackupFiles();
+
+    // Create a timetable backup file from the cached records (timetableRecordMap)
+    lock.lock();
+    try {
+      if (!timetableRecordMap.isEmpty()) {
+        // Create a temp timetable backup file
+        // Make the backup file temporary so that it will be deleted after the iteration provided
+        // the I/O write to the backup storage succeeds. Otherwise, this temporary backup file will
+        // continue to exist in tmpDir locally until it's successfully written to the backup storage
+        BackupManager.BackupFile timetableBackupFromThisIteration =
+            new BackupManager.BackupFile(createTimetableBackupFile(), true, 0, 0);
+        candidateTimetableBackupFiles.add(timetableBackupFromThisIteration);
+        timetableRecordMap.clear(); // Clear the map
+      }
+    } catch (Exception e) {
+      logger.error("TimetableBackup::startIteration(): failed to create timetable file to back up!",
+          e);
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  protected void endIteration(boolean errorFree) {
+    // timetableRecordMap is cleared in startIteration() already, so we do not clear it here
+    backupStats.setTimetableBackupIterationDone(errorFree);
+  }
+
+  @Override
+  protected BackupManager.BackupFile getNextFileToBackup() throws IOException {
+    BackupManager.BackupFile nextFile = null;
+    if (!candidateTimetableBackupFiles.isEmpty()) {

Review Comment:
   *THREAD_SAFETY_VIOLATION:*  Read/Write race. Non-private method `TimetableBackup.getNextFileToBackup()` reads without synchronization from container `this.candidateTimetableBackupFiles` via call to `SortedSet.isEmpty()`. Potentially races with write in method `TimetableBackup.backupComplete(...)`.
    Reporting because another access to the same memory occurs on a background thread, although this access may not.
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621617&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621617&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621617&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621617&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621617&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/BackupManager.java:
##########
@@ -0,0 +1,661 @@
+/*
+ * 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.zookeeper.server.backup;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import javax.management.JMException;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.zookeeper.jmx.MBeanRegistry;
+import org.apache.zookeeper.server.backup.BackupUtil.BackupFileType;
+import org.apache.zookeeper.server.backup.BackupUtil.IntervalEndpoint;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.monitoring.BackupBean;
+import org.apache.zookeeper.server.backup.monitoring.BackupStats;
+import org.apache.zookeeper.server.backup.monitoring.TimetableBackupBean;
+import org.apache.zookeeper.server.backup.monitoring.TimetableBackupStats;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.timetable.TimetableBackup;
+import org.apache.zookeeper.server.persistence.FileTxnLog;
+import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
+import org.apache.zookeeper.server.persistence.TxnLog;
+import org.apache.zookeeper.server.persistence.Util;
+import org.apache.zookeeper.server.persistence.ZxidRange;
+import org.apache.zookeeper.server.util.ZxidUtils;
+import org.apache.zookeeper.txn.TxnHeader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class manages the backing up of txnlog and snap files to remote
+ * storage for longer term and durable retention than is possible on
+ * an ensemble server
+ */
+public class BackupManager {
+  private static final Logger LOG = LoggerFactory.getLogger(BackupManager.class);
+
+  private final Logger logger;
+  private final File snapDir;
+  private final File dataLogDir;
+  private final File tmpDir;
+  private final BackupConfig backupConfig;
+  private final long backupIntervalInMilliseconds;
+  private BackupProcess logBackup = null;
+  private BackupProcess snapBackup = null;
+
+  // backupStatus, backupLogZxid and backedupSnapZxid need to be access while synchronized
+  // on backupStatus.
+  private final BackupStatus backupStatus;
+  private BackupPoint backupPoint;
+
+  private final BackupStorageProvider backupStorage;
+  private final long serverId;
+  private final String namespace;
+
+  @VisibleForTesting
+  protected BackupBean backupBean = null;
+  protected TimetableBackupBean timetableBackupBean = null;
+
+  private BackupStats backupStats = null;
+
+  // Optional timetable backup
+  private BackupProcess timetableBackup = null;
+
+  /**
+   * Tracks a file that needs to be backed up, including temporary copies of the file
+   */
+  public static class BackupFile {
+    private final File file;
+    private final boolean isTemporary;
+    private final ZxidRange zxidRange;
+
+    /**
+     * Create an instance of a BackupFile for the given initial file and zxid range
+     * @param backupFile the initial/original file
+     * @param isTemporaryFile whether the file is a temporary file
+     * @param fileMinZxid the min zxid associated with this file
+     * @param fileMaxZxid the max zxid associated with this file
+     */
+    public BackupFile(File backupFile, boolean isTemporaryFile, long fileMinZxid, long fileMaxZxid) {
+      this(backupFile, isTemporaryFile, new ZxidRange(fileMinZxid, fileMaxZxid));
+    }
+
+    /**
+     * Create an instance of a BackupFile for the given initial file and zxid range
+     * @param backupFile the initial/original file
+     * @param isTemporaryFile whether the file is a temporary file
+     * @param zxidRange the zxid range associated with this file
+     */
+    public BackupFile(File backupFile, boolean isTemporaryFile, ZxidRange zxidRange) {
+      Preconditions.checkNotNull(zxidRange);
+
+      if (!zxidRange.isHighPresent()) {
+        throw new IllegalArgumentException("ZxidRange must have a high value");
+      }
+
+      this.file = backupFile;
+      this.isTemporary = isTemporaryFile;
+      this.zxidRange = zxidRange;
+    }
+
+    /**
+     * Perform cleanup including deleting temporary files.
+     */
+    public void cleanup() {
+      if (isTemporary && exists()) {
+        file.delete();
+      }
+    }
+
+    /**
+     * Whether the file representing the zxids exists
+     * @return whether the file represented exists
+     */
+    public boolean exists() {
+      return file != null && file.exists();
+    }
+
+    /**
+     * Get the current file (topmost on the stack)
+     * @return the current file
+     */
+    public File getFile() { return file; }
+
+    /**
+     * Get the zxid range associated with this file
+     * @return the zxid range
+     */
+    public ZxidRange getZxidRange() {
+      return zxidRange;
+    }
+
+    /**
+     * Get the min zxid associated with this file
+     * @return the min associated zxid
+     */
+    public long getMinZxid() { return zxidRange.getLow(); }
+
+    /**
+     * Get the max zxid associated with this file
+     * @return the max associated zxid
+     */
+    public long getMaxZxid() { return zxidRange.getHigh(); }
+
+    @Override
+    public String toString() {
+      return String.format("%s : %s : %d - %d",
+          file == null ? "[empty]" : file.getPath(),
+          isTemporary ? "temp" : "perm",
+          zxidRange.getLow(),
+          zxidRange.getHigh());
+    }
+  }
+
+  /**
+   * Implements txnlog specific logic for BackupProcess
+   */
+  protected class TxnLogBackup extends BackupProcess {
+    private long iterationEndPoint;
+    private final FileTxnSnapLog snapLog;
+
+    /**
+     * Constructor for TxnLogBackup
+     * @param snapLog the FileTxnSnapLog object to use
+     */
+    public TxnLogBackup(FileTxnSnapLog snapLog) {
+      super(LoggerFactory.getLogger(TxnLogBackup.class), BackupManager.this.backupStorage,
+          backupIntervalInMilliseconds);
+      this.snapLog = snapLog;
+    }
+
+    protected void initialize() throws IOException {
+      backupStorage.cleanupInvalidFiles(null);
+
+      BackupFileInfo latest =
+          BackupUtil.getLatest(backupStorage, BackupFileType.TXNLOG, IntervalEndpoint.START);
+
+      long rZxid = latest == null
+          ? BackupUtil.INVALID_LOG_ZXID
+          : latest.getIntervalEndpoint(IntervalEndpoint.END);
+
+      logger.info("Latest Zxid from storage: {}  from status: {}",
+          ZxidUtils.zxidToString(rZxid), ZxidUtils.zxidToString(backupPoint.getLogZxid()));
+
+      if (rZxid != backupPoint.getLogZxid()) {
+        synchronized (backupStatus) {
+          backupPoint.setLogZxid(rZxid);
+          backupStatus.update(backupPoint);
+        }
+      }
+    }
+
+    protected void startIteration() {
+      // Store the current last logged zxid.  This becomes the stopping point
+      // for the current iteration so we don't keep chasing our own tail as
+      // new transactions get written.
+      iterationEndPoint = snapLog.getLastLoggedZxid();
+      backupStats.setTxnLogBackupIterationStart();
+    }
+
+    protected void endIteration(boolean errorFree) {
+      backupStats.setTxnLogBackupIterationDone(errorFree);
+      iterationEndPoint = 0L;
+    }
+
+    /**
+     * Gets the next txnlog file to backup.  This is a temporary file created by copying
+     * all transaction from the previous backup point until the end zxid for this iteration, or
+     * a file indicating that some log records were lost.
+     * @return the file that needs to be backed-up.  The minZxid is the first
+     *      zxid contained in the file.  The maxZxid is the last zxid that is contained in the
+     *      file.
+     * @throws IOException
+     */
+    protected BackupFile getNextFileToBackup() throws IOException {
+      long startingZxid = backupStatus.read().getLogZxid() + 1;
+
+      // Don't keep chasing the tail so stop if past the last zxid at the time
+      // this iteration started.
+      if (startingZxid > iterationEndPoint) {
+        return null;
+      }
+
+      TxnLog.TxnIterator iter = null;
+      FileTxnLog newFile = null;
+      long lastZxid = -1;
+      int txnCopied = 0;
+      BackupFile ret;
+
+      logger.info("Creating backup file from zxid {}.", ZxidUtils.zxidToString(startingZxid));
+
+      try {
+        iter = snapLog.readTxnLog(startingZxid, true);
+
+        // Use a temp directory to avoid conflicts with live txnlog files
+        newFile = new FileTxnLog(tmpDir);
+
+        // TODO: Ideally, we should have have a way to prevent lost TxLogs
+        // Check for lost txnlogs; <=1 indicates that no backups have been done before so
+        // nothing can be considered lost.
+        // If a lost sequence is found then return a file whose name encodes the lost
+        // sequence and back that up so the backup store has a record of the lost sequence
+        if (startingZxid > 1 &&
+            iter.getHeader() != null &&
+            iter.getHeader().getZxid() > startingZxid) {
+
+          logger.error("TxnLog backups lost.  Required starting zxid={}  First available zxid={}",
+              ZxidUtils.zxidToString(startingZxid),
+              ZxidUtils.zxidToString(iter.getHeader().getZxid()));
+
+          String fileName = String.format("%s.%s",
+              BackupUtil.LOST_LOG_PREFIX,
+              Long.toHexString(startingZxid));
+          File lostZxidFile = new File(tmpDir, fileName);
+          lostZxidFile.createNewFile();
+
+          return new BackupFile(lostZxidFile, true, startingZxid, iter.getHeader().getZxid() - 1);
+        }
+
+        while (iter.getHeader() != null) {
+          TxnHeader hdr = iter.getHeader();
+
+          if (hdr.getZxid() > iterationEndPoint) {
+            break;
+          }
+
+          newFile.append(hdr, iter.getTxn());
+
+          // update position and count only AFTER the record has been successfully
+          // copied
+          lastZxid = hdr.getZxid();
+          txnCopied++;
+
+          iter.next();
+        }
+
+        ret = makeBackupFileFromCopiedLog(newFile, lastZxid);
+
+        if (ret != null) {
+          logger.info("Copied {} records starting at {} and ending at zxid {}.",
+              txnCopied,
+              ZxidUtils.zxidToString(ret.getMinZxid()),
+              ZxidUtils.zxidToString(ret.getMaxZxid()));
+        }
+
+      } catch (IOException e) {
+        logger.warn("Hit exception after {} records.  Exception: {} ", txnCopied, e.fillInStackTrace());
+
+        // If any records were copied return those and ignore the error.  Otherwise
+        // rethrow the error to be handled by the caller as a failed backup iteration.
+        if (txnCopied <= 0) {
+          throw e;
+        }
+
+        ret = makeBackupFileFromCopiedLog(newFile, lastZxid);
+      } finally {
+        if (iter != null) {
+          iter.close();
+        }
+
+        if (newFile != null) {
+          newFile.close();
+        }
+      }
+
+      return ret;
+    }
+
+    private BackupFile makeBackupFileFromCopiedLog(FileTxnLog backupTxnLog, long lastZxid) {
+
+      if (backupTxnLog == null) {
+        return null;
+      }
+
+      // The logFile gets initialized with the first transaction's zxid
+      File logFile = backupTxnLog.getCurrentFile();
+
+      if (logFile == null) {
+        return null;
+      }
+
+      long firstZxid = Util.getZxidFromName(logFile.getName(), Util.TXLOG_PREFIX);
+
+      if (lastZxid == -1) {
+        lastZxid = firstZxid;
+      }
+
+      return new BackupFile(logFile, true, new ZxidRange(firstZxid, lastZxid));
+    }
+
+    protected void backupComplete(BackupFile file) throws IOException {
+      synchronized (backupStatus) {
+        backupPoint.setLogZxid(file.getMaxZxid());
+        backupStatus.update(backupPoint);
+      }
+
+      logger.info("Updated backedup tnxlog zxid to {}",
+          ZxidUtils.zxidToString(backupPoint.getLogZxid()));
+    }
+  }
+
+  /**
+   * Implements snapshot specific logic for BackupProcess
+   */
+  protected class SnapBackup extends BackupProcess {
+    private final FileTxnSnapLog snapLog;
+    private final List<BackupFile> filesToBackup = new ArrayList<>();
+
+    /**
+     * Constructor for SnapBackup
+     * @param snapLog the FileTxnSnapLog object to use
+     */
+    public SnapBackup(FileTxnSnapLog snapLog) {
+      super(LoggerFactory.getLogger(SnapBackup.class), BackupManager.this.backupStorage,
+          backupIntervalInMilliseconds);
+      this.snapLog = snapLog;
+    }
+
+    protected void initialize() throws IOException {
+      backupStorage.cleanupInvalidFiles(null);
+
+      BackupFileInfo latest =
+          BackupUtil.getLatest(backupStorage, BackupFileType.SNAPSHOT, IntervalEndpoint.START);
+
+      long rZxid = latest == null
+          ? BackupUtil.INVALID_SNAP_ZXID
+          : latest.getIntervalEndpoint(IntervalEndpoint.START);
+
+      logger.info("Latest Zxid from storage: {}  from status: {}",
+          ZxidUtils.zxidToString(rZxid), ZxidUtils.zxidToString(backupPoint.getSnapZxid()));

Review Comment:
   *THREAD_SAFETY_VIOLATION:*  Read/Write race. Non-private method `BackupManager$SnapBackup.initialize()` indirectly reads without synchronization from `this.this$0.backupPoint.snapZxid`. Potentially races with write in method `BackupManager$SnapBackup.backupComplete(...)`.
    Reporting because this access may occur on a background thread.
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621718&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621718&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621718&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621718&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621718&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/timetable/TimetableBackup.java:
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.zookeeper.server.backup.timetable;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.ObjectOutputStream;
+import java.util.Comparator;
+import java.util.SortedSet;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.zookeeper.server.backup.BackupFileInfo;
+import org.apache.zookeeper.server.backup.BackupManager;
+import org.apache.zookeeper.server.backup.BackupPoint;
+import org.apache.zookeeper.server.backup.BackupProcess;
+import org.apache.zookeeper.server.backup.BackupStatus;
+import org.apache.zookeeper.server.backup.BackupUtil;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.monitoring.TimetableBackupStats;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
+import org.apache.zookeeper.txn.TxnHeader;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implements Timetable-specific logic for BackupProcess.
+ *
+ * Backup timetable encodes data of the format <timestamp>:<zxid>. This is used to locate the
+ * closest zxid backup point given the timestamp. The use case is for users who wish to restore
+ * from backup at a specific time recorded in the backup timetable.
+ */
+public class TimetableBackup extends BackupProcess {
+  public static final String TIMETABLE_PREFIX = "timetable";
+  // Use an ordered map of <timestamp>:<zxid>
+  private final TreeMap<Long, String> timetableRecordMap = new TreeMap<>();
+  private final File tmpDir;
+  // Lock is used to keep access to timetableRecordMap exclusive
+  private final Lock lock = new ReentrantLock(true);
+
+  // Candidate files to be backed up each iteration sorted by name (using SortedSet)
+  private final SortedSet<BackupManager.BackupFile> candidateTimetableBackupFiles =
+      new TreeSet<>(Comparator.comparing(o -> o.getFile().getName()));
+  // BackupStatus is used here to keep track of timetable backup status in case of a crash/restart
+  // BackupStatus is written to a file. After a crash (e.g. JVM crash) or a restart, the
+  // locally-stored zkBackupStatus file will be read back to restore a BackupPoint
+  private final BackupStatus backupStatus;
+  private final BackupPoint backupPoint;
+  private final TimetableBackupStats backupStats; // Metrics
+
+  /**
+   * Create an instance of TimetableBackup.
+   * @param snapLog
+   * @param tmpDir
+   * @param backupStorageProvider
+   * @param backupIntervalInMilliseconds
+   * @param timetableBackupIntervalInMs
+   * @param backupStatus
+   */
+  public TimetableBackup(FileTxnSnapLog snapLog, File tmpDir,
+      BackupStorageProvider backupStorageProvider, long backupIntervalInMilliseconds,
+      long timetableBackupIntervalInMs, BackupStatus backupStatus, BackupPoint backupPoint,
+      TimetableBackupStats backupStats) {
+    super(LoggerFactory.getLogger(TimetableBackup.class), backupStorageProvider,
+        backupIntervalInMilliseconds);
+    this.tmpDir = tmpDir;
+    this.backupStatus = backupStatus;
+    this.backupPoint = backupPoint;
+    this.backupStats = backupStats;
+    // Start creating records
+    (new Thread(new TimetableRecorder(snapLog, timetableBackupIntervalInMs))).start();
+    logger.info("TimetableBackup::Starting TimetableBackup Process with backup interval: "
+        + backupIntervalInMilliseconds + " ms and timetable backup interval: "
+        + timetableBackupIntervalInMs + " ms.");
+  }
+
+  @Override
+  protected void initialize() throws IOException {
+    // Get the latest timetable backup file from backup storage
+    BackupFileInfo latest = BackupUtil.getLatest(backupStorage, BackupUtil.BackupFileType.TIMETABLE,
+        BackupUtil.IntervalEndpoint.END);
+
+    long latestTimestampBackedUp = latest == null ? BackupUtil.INVALID_TIMESTAMP
+        : latest.getIntervalEndpoint(BackupUtil.IntervalEndpoint.END);
+
+    logger.info(
+        "TimetableBackup::initialize(): latest timestamp from storage: {}, from BackupStatus: {}",
+        latestTimestampBackedUp, backupPoint.getTimestamp());
+
+    if (latestTimestampBackedUp != backupPoint.getTimestamp()) {
+      synchronized (backupStatus) {
+        backupPoint.setTimestamp(latestTimestampBackedUp);
+        backupStatus.update(backupPoint);
+      }
+    }
+  }
+
+  @Override
+  protected void startIteration() throws IOException {
+    backupStats.setTimetableBackupIterationStart();
+
+    // It's possible that the last iteration failed and left some tmp backup files behind - add
+    // them back to candidate backup files so the I/O write will happen again
+    getAllTmpBackupFiles();
+
+    // Create a timetable backup file from the cached records (timetableRecordMap)
+    lock.lock();
+    try {
+      if (!timetableRecordMap.isEmpty()) {
+        // Create a temp timetable backup file
+        // Make the backup file temporary so that it will be deleted after the iteration provided
+        // the I/O write to the backup storage succeeds. Otherwise, this temporary backup file will
+        // continue to exist in tmpDir locally until it's successfully written to the backup storage
+        BackupManager.BackupFile timetableBackupFromThisIteration =
+            new BackupManager.BackupFile(createTimetableBackupFile(), true, 0, 0);
+        candidateTimetableBackupFiles.add(timetableBackupFromThisIteration);
+        timetableRecordMap.clear(); // Clear the map
+      }
+    } catch (Exception e) {
+      logger.error("TimetableBackup::startIteration(): failed to create timetable file to back up!",
+          e);
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  protected void endIteration(boolean errorFree) {
+    // timetableRecordMap is cleared in startIteration() already, so we do not clear it here
+    backupStats.setTimetableBackupIterationDone(errorFree);
+  }
+
+  @Override
+  protected BackupManager.BackupFile getNextFileToBackup() throws IOException {
+    BackupManager.BackupFile nextFile = null;
+    if (!candidateTimetableBackupFiles.isEmpty()) {
+      nextFile = candidateTimetableBackupFiles.first();
+    }
+    return nextFile;
+  }
+
+  @Override
+  protected void backupComplete(BackupManager.BackupFile file) throws IOException {
+    // Remove from the candidate set because it has been copied to the backup storage successfully
+    candidateTimetableBackupFiles.remove(file);
+    synchronized (backupStatus) {
+      // Update the latest timestamp to which the timetable backup was successful
+      backupPoint.setTimestamp(Long.parseLong(file.getFile().getName().split("-")[1]));
+      backupStatus.update(backupPoint);
+    }
+    logger.info(
+        "TimetableBackup::backupComplete(): backup complete for file: {}. Updated backed up "
+            + "timestamp to {}", file.getFile().getName(), backupPoint.getTimestamp());
+  }
+
+  /**
+   * Create a file name for timetable backup.
+   * E.g.) timetable.<lowest timestamp>-<highest timestamp>
+   * @return
+   */
+  private String makeTimetableBackupFileName() {
+    return String.format("%s.%d-%d", TIMETABLE_PREFIX, timetableRecordMap.firstKey(),
+        timetableRecordMap.lastKey());
+  }
+
+  /**
+   * Write the content of timetableRecordMap to a file using a tmpDir.
+   * @return
+   * @throws IOException
+   */
+  private File createTimetableBackupFile() throws IOException {
+    File tempTimetableBackupFile = new File(tmpDir, makeTimetableBackupFileName());
+    logger.info(
+        "TimetableBackup::createTimetableBackupFile(): created temporary timetable backup file with"
+            + " name: " + tempTimetableBackupFile.getName());
+    FileOutputStream fos = new FileOutputStream(tempTimetableBackupFile);
+    ObjectOutputStream oos = new ObjectOutputStream(fos);
+    oos.writeObject(timetableRecordMap);
+    oos.flush();
+    oos.close();
+    fos.close();
+    logger.info(
+        "TimetableBackup::createTimetableBackupFile(): successfully wrote cached timetable data to "
+            + "temporary timetable backup file with name: " + tempTimetableBackupFile.getName());
+    return tempTimetableBackupFile;

Review Comment:
   *RESOURCE_LEAK:*  resource of type `java.io.FileOutputStream` acquired to `fos` by call to `FileOutputStream(...)` at line 195 is not released after line 204.
   **Note**: potential exception at line 196
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621738&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621738&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621738&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621738&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621738&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/BackupManager.java:
##########
@@ -0,0 +1,661 @@
+/*
+ * 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.zookeeper.server.backup;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import javax.management.JMException;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.zookeeper.jmx.MBeanRegistry;
+import org.apache.zookeeper.server.backup.BackupUtil.BackupFileType;
+import org.apache.zookeeper.server.backup.BackupUtil.IntervalEndpoint;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.monitoring.BackupBean;
+import org.apache.zookeeper.server.backup.monitoring.BackupStats;
+import org.apache.zookeeper.server.backup.monitoring.TimetableBackupBean;
+import org.apache.zookeeper.server.backup.monitoring.TimetableBackupStats;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.timetable.TimetableBackup;
+import org.apache.zookeeper.server.persistence.FileTxnLog;
+import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
+import org.apache.zookeeper.server.persistence.TxnLog;
+import org.apache.zookeeper.server.persistence.Util;
+import org.apache.zookeeper.server.persistence.ZxidRange;
+import org.apache.zookeeper.server.util.ZxidUtils;
+import org.apache.zookeeper.txn.TxnHeader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class manages the backing up of txnlog and snap files to remote
+ * storage for longer term and durable retention than is possible on
+ * an ensemble server
+ */
+public class BackupManager {
+  private static final Logger LOG = LoggerFactory.getLogger(BackupManager.class);
+
+  private final Logger logger;
+  private final File snapDir;
+  private final File dataLogDir;
+  private final File tmpDir;
+  private final BackupConfig backupConfig;
+  private final long backupIntervalInMilliseconds;
+  private BackupProcess logBackup = null;
+  private BackupProcess snapBackup = null;
+
+  // backupStatus, backupLogZxid and backedupSnapZxid need to be access while synchronized
+  // on backupStatus.
+  private final BackupStatus backupStatus;
+  private BackupPoint backupPoint;
+
+  private final BackupStorageProvider backupStorage;
+  private final long serverId;
+  private final String namespace;
+
+  @VisibleForTesting
+  protected BackupBean backupBean = null;
+  protected TimetableBackupBean timetableBackupBean = null;
+
+  private BackupStats backupStats = null;
+
+  // Optional timetable backup
+  private BackupProcess timetableBackup = null;
+
+  /**
+   * Tracks a file that needs to be backed up, including temporary copies of the file
+   */
+  public static class BackupFile {
+    private final File file;
+    private final boolean isTemporary;
+    private final ZxidRange zxidRange;
+
+    /**
+     * Create an instance of a BackupFile for the given initial file and zxid range
+     * @param backupFile the initial/original file
+     * @param isTemporaryFile whether the file is a temporary file
+     * @param fileMinZxid the min zxid associated with this file
+     * @param fileMaxZxid the max zxid associated with this file
+     */
+    public BackupFile(File backupFile, boolean isTemporaryFile, long fileMinZxid, long fileMaxZxid) {
+      this(backupFile, isTemporaryFile, new ZxidRange(fileMinZxid, fileMaxZxid));
+    }
+
+    /**
+     * Create an instance of a BackupFile for the given initial file and zxid range
+     * @param backupFile the initial/original file
+     * @param isTemporaryFile whether the file is a temporary file
+     * @param zxidRange the zxid range associated with this file
+     */
+    public BackupFile(File backupFile, boolean isTemporaryFile, ZxidRange zxidRange) {
+      Preconditions.checkNotNull(zxidRange);
+
+      if (!zxidRange.isHighPresent()) {
+        throw new IllegalArgumentException("ZxidRange must have a high value");
+      }
+
+      this.file = backupFile;
+      this.isTemporary = isTemporaryFile;
+      this.zxidRange = zxidRange;
+    }
+
+    /**
+     * Perform cleanup including deleting temporary files.
+     */
+    public void cleanup() {
+      if (isTemporary && exists()) {
+        file.delete();
+      }
+    }
+
+    /**
+     * Whether the file representing the zxids exists
+     * @return whether the file represented exists
+     */
+    public boolean exists() {
+      return file != null && file.exists();
+    }
+
+    /**
+     * Get the current file (topmost on the stack)
+     * @return the current file
+     */
+    public File getFile() { return file; }
+
+    /**
+     * Get the zxid range associated with this file
+     * @return the zxid range
+     */
+    public ZxidRange getZxidRange() {
+      return zxidRange;
+    }
+
+    /**
+     * Get the min zxid associated with this file
+     * @return the min associated zxid
+     */
+    public long getMinZxid() { return zxidRange.getLow(); }
+
+    /**
+     * Get the max zxid associated with this file
+     * @return the max associated zxid
+     */
+    public long getMaxZxid() { return zxidRange.getHigh(); }
+
+    @Override
+    public String toString() {
+      return String.format("%s : %s : %d - %d",
+          file == null ? "[empty]" : file.getPath(),
+          isTemporary ? "temp" : "perm",
+          zxidRange.getLow(),
+          zxidRange.getHigh());
+    }
+  }
+
+  /**
+   * Implements txnlog specific logic for BackupProcess
+   */
+  protected class TxnLogBackup extends BackupProcess {
+    private long iterationEndPoint;
+    private final FileTxnSnapLog snapLog;
+
+    /**
+     * Constructor for TxnLogBackup
+     * @param snapLog the FileTxnSnapLog object to use
+     */
+    public TxnLogBackup(FileTxnSnapLog snapLog) {
+      super(LoggerFactory.getLogger(TxnLogBackup.class), BackupManager.this.backupStorage,
+          backupIntervalInMilliseconds);
+      this.snapLog = snapLog;
+    }
+
+    protected void initialize() throws IOException {
+      backupStorage.cleanupInvalidFiles(null);
+
+      BackupFileInfo latest =
+          BackupUtil.getLatest(backupStorage, BackupFileType.TXNLOG, IntervalEndpoint.START);
+
+      long rZxid = latest == null
+          ? BackupUtil.INVALID_LOG_ZXID
+          : latest.getIntervalEndpoint(IntervalEndpoint.END);
+
+      logger.info("Latest Zxid from storage: {}  from status: {}",
+          ZxidUtils.zxidToString(rZxid), ZxidUtils.zxidToString(backupPoint.getLogZxid()));
+
+      if (rZxid != backupPoint.getLogZxid()) {
+        synchronized (backupStatus) {
+          backupPoint.setLogZxid(rZxid);
+          backupStatus.update(backupPoint);
+        }
+      }
+    }
+
+    protected void startIteration() {
+      // Store the current last logged zxid.  This becomes the stopping point
+      // for the current iteration so we don't keep chasing our own tail as
+      // new transactions get written.
+      iterationEndPoint = snapLog.getLastLoggedZxid();
+      backupStats.setTxnLogBackupIterationStart();
+    }
+
+    protected void endIteration(boolean errorFree) {
+      backupStats.setTxnLogBackupIterationDone(errorFree);
+      iterationEndPoint = 0L;
+    }
+
+    /**
+     * Gets the next txnlog file to backup.  This is a temporary file created by copying
+     * all transaction from the previous backup point until the end zxid for this iteration, or
+     * a file indicating that some log records were lost.
+     * @return the file that needs to be backed-up.  The minZxid is the first
+     *      zxid contained in the file.  The maxZxid is the last zxid that is contained in the
+     *      file.
+     * @throws IOException
+     */
+    protected BackupFile getNextFileToBackup() throws IOException {
+      long startingZxid = backupStatus.read().getLogZxid() + 1;
+
+      // Don't keep chasing the tail so stop if past the last zxid at the time
+      // this iteration started.
+      if (startingZxid > iterationEndPoint) {
+        return null;
+      }
+
+      TxnLog.TxnIterator iter = null;
+      FileTxnLog newFile = null;
+      long lastZxid = -1;
+      int txnCopied = 0;
+      BackupFile ret;
+
+      logger.info("Creating backup file from zxid {}.", ZxidUtils.zxidToString(startingZxid));
+
+      try {
+        iter = snapLog.readTxnLog(startingZxid, true);
+
+        // Use a temp directory to avoid conflicts with live txnlog files
+        newFile = new FileTxnLog(tmpDir);
+
+        // TODO: Ideally, we should have have a way to prevent lost TxLogs
+        // Check for lost txnlogs; <=1 indicates that no backups have been done before so
+        // nothing can be considered lost.
+        // If a lost sequence is found then return a file whose name encodes the lost
+        // sequence and back that up so the backup store has a record of the lost sequence
+        if (startingZxid > 1 &&
+            iter.getHeader() != null &&
+            iter.getHeader().getZxid() > startingZxid) {
+
+          logger.error("TxnLog backups lost.  Required starting zxid={}  First available zxid={}",
+              ZxidUtils.zxidToString(startingZxid),
+              ZxidUtils.zxidToString(iter.getHeader().getZxid()));
+
+          String fileName = String.format("%s.%s",
+              BackupUtil.LOST_LOG_PREFIX,
+              Long.toHexString(startingZxid));
+          File lostZxidFile = new File(tmpDir, fileName);
+          lostZxidFile.createNewFile();
+
+          return new BackupFile(lostZxidFile, true, startingZxid, iter.getHeader().getZxid() - 1);
+        }
+
+        while (iter.getHeader() != null) {
+          TxnHeader hdr = iter.getHeader();
+
+          if (hdr.getZxid() > iterationEndPoint) {
+            break;
+          }
+
+          newFile.append(hdr, iter.getTxn());
+
+          // update position and count only AFTER the record has been successfully
+          // copied
+          lastZxid = hdr.getZxid();
+          txnCopied++;
+
+          iter.next();
+        }
+
+        ret = makeBackupFileFromCopiedLog(newFile, lastZxid);
+
+        if (ret != null) {
+          logger.info("Copied {} records starting at {} and ending at zxid {}.",
+              txnCopied,
+              ZxidUtils.zxidToString(ret.getMinZxid()),
+              ZxidUtils.zxidToString(ret.getMaxZxid()));
+        }
+
+      } catch (IOException e) {
+        logger.warn("Hit exception after {} records.  Exception: {} ", txnCopied, e.fillInStackTrace());
+
+        // If any records were copied return those and ignore the error.  Otherwise
+        // rethrow the error to be handled by the caller as a failed backup iteration.
+        if (txnCopied <= 0) {
+          throw e;
+        }
+
+        ret = makeBackupFileFromCopiedLog(newFile, lastZxid);
+      } finally {
+        if (iter != null) {
+          iter.close();
+        }
+
+        if (newFile != null) {
+          newFile.close();
+        }
+      }
+
+      return ret;
+    }
+
+    private BackupFile makeBackupFileFromCopiedLog(FileTxnLog backupTxnLog, long lastZxid) {
+
+      if (backupTxnLog == null) {
+        return null;
+      }
+
+      // The logFile gets initialized with the first transaction's zxid
+      File logFile = backupTxnLog.getCurrentFile();
+
+      if (logFile == null) {
+        return null;
+      }
+
+      long firstZxid = Util.getZxidFromName(logFile.getName(), Util.TXLOG_PREFIX);
+
+      if (lastZxid == -1) {
+        lastZxid = firstZxid;
+      }
+
+      return new BackupFile(logFile, true, new ZxidRange(firstZxid, lastZxid));
+    }
+
+    protected void backupComplete(BackupFile file) throws IOException {
+      synchronized (backupStatus) {
+        backupPoint.setLogZxid(file.getMaxZxid());
+        backupStatus.update(backupPoint);
+      }
+
+      logger.info("Updated backedup tnxlog zxid to {}",
+          ZxidUtils.zxidToString(backupPoint.getLogZxid()));
+    }
+  }
+
+  /**
+   * Implements snapshot specific logic for BackupProcess
+   */
+  protected class SnapBackup extends BackupProcess {
+    private final FileTxnSnapLog snapLog;
+    private final List<BackupFile> filesToBackup = new ArrayList<>();
+
+    /**
+     * Constructor for SnapBackup
+     * @param snapLog the FileTxnSnapLog object to use
+     */
+    public SnapBackup(FileTxnSnapLog snapLog) {
+      super(LoggerFactory.getLogger(SnapBackup.class), BackupManager.this.backupStorage,
+          backupIntervalInMilliseconds);
+      this.snapLog = snapLog;
+    }
+
+    protected void initialize() throws IOException {
+      backupStorage.cleanupInvalidFiles(null);
+
+      BackupFileInfo latest =
+          BackupUtil.getLatest(backupStorage, BackupFileType.SNAPSHOT, IntervalEndpoint.START);
+
+      long rZxid = latest == null
+          ? BackupUtil.INVALID_SNAP_ZXID
+          : latest.getIntervalEndpoint(IntervalEndpoint.START);
+
+      logger.info("Latest Zxid from storage: {}  from status: {}",
+          ZxidUtils.zxidToString(rZxid), ZxidUtils.zxidToString(backupPoint.getSnapZxid()));
+
+      if (rZxid != backupPoint.getSnapZxid()) {
+        synchronized (backupStatus) {
+          backupPoint.setSnapZxid(rZxid);
+          backupStatus.update(backupPoint);
+        }
+      }
+    }
+
+    /**
+     * Prepares snapshot files to back up and populate filesToBackup.
+     * Note that this implementation persists all snapshots instead of only persisting the
+     * latest snapshot.
+     * @throws IOException
+     */
+    protected void startIteration() throws IOException {
+      backupStats.setSnapshotBackupIterationStart();
+      filesToBackup.clear();
+
+      // Get all available snapshots excluding the ones whose lastProcessedZxid falls into the
+      // zxid range [0, backedupSnapZxid]
+      List<File> candidateSnapshots = snapLog.findValidSnapshots(0, backupPoint.getSnapZxid());
+      // Sort candidateSnapshots from oldest to newest
+      Collections.reverse(candidateSnapshots);
+
+      if (candidateSnapshots.size() == 0) {
+        // Either no snapshots or no newer snapshots to back up, so return
+        return;
+      }
+
+      for (int i = 0; i < candidateSnapshots.size(); i++) {
+        File f = candidateSnapshots.get(i);
+        ZxidRange zxidRange = Util.getZxidRangeFromName(f.getName(), Util.SNAP_PREFIX);
+
+        if (i == candidateSnapshots.size() - 1) {
+          // This is the most recent snapshot
+
+          // Handle backwards compatibility for snapshots that use old style naming where
+          // only the starting zxid is included.
+          // TODO: Can be removed after all snapshots being produced have ending zxid
+          if (!zxidRange.isHighPresent()) {
+            // Use the last logged zxid for the zxidRange for the latest snapshot as a best effort
+            // approach
+            // TODO: Because this is the best effort approach, the zxidRange will not be accurate
+            // TODO: Consider rewriting these latest snapshots to backup storage if necessary
+            // TODO: when we know the high zxid when we get a newer snapshot
+            long latestZxid = snapLog.getLastLoggedZxid();
+            long consistentAt = latestZxid == -1 ? zxidRange.getLow() : latestZxid;
+            zxidRange = new ZxidRange(zxidRange.getLow(), consistentAt);
+          }
+        } else {
+          // All newer snapshots that are not the most recent snapshot
+
+          // Handle backwards compatibility for snapshots that use old style naming where
+          // only the starting zxid is included.
+          // TODO: Can be removed after all snapshots being produced have ending zxid
+          if (!zxidRange.isHighPresent()) {
+            // ZxidRange will be [low, high] where high will be the zxid right before the next
+            // snapshot's lastProcessedZxid
+            long nextSnapshotStartZxid =
+                Util.getZxidFromName(candidateSnapshots.get(i + 1).getName(), Util.SNAP_PREFIX);
+            zxidRange = new ZxidRange(zxidRange.getLow(), nextSnapshotStartZxid - 1);
+          }
+        }
+
+        filesToBackup.add(new BackupFile(f, false, zxidRange));
+      }
+    }
+
+    protected void endIteration(boolean errorFree) {
+      backupStats.setSnapshotBackupIterationDone(errorFree);
+      filesToBackup.clear();
+    }
+
+    protected BackupFile getNextFileToBackup() {
+      if (filesToBackup.isEmpty()) {
+        return null;
+      }
+
+      return filesToBackup.remove(0);
+    }
+
+    protected void backupComplete(BackupFile file) throws IOException {
+      synchronized (backupStatus) {
+        backupPoint.setSnapZxid(file.getMinZxid());
+        backupStatus.update(backupPoint);
+      }
+      backupStats.incrementNumberOfSnapshotFilesBackedUpThisIteration();
+
+      logger.info("Updated backedup snap zxid to {}",
+          ZxidUtils.zxidToString(backupPoint.getSnapZxid()));

Review Comment:
   *THREAD_SAFETY_VIOLATION:*  Read/Write race. Non-private method `BackupManager$SnapBackup.backupComplete(...)` indirectly reads without synchronization from `this.this$0.backupPoint.snapZxid`. Potentially races with write in method `BackupManager$SnapBackup.backupComplete(...)`.
    Reporting because this access may occur on a background thread.
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621774&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621774&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621774&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621774&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621774&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/BackupManager.java:
##########
@@ -0,0 +1,661 @@
+/*
+ * 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.zookeeper.server.backup;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import javax.management.JMException;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.zookeeper.jmx.MBeanRegistry;
+import org.apache.zookeeper.server.backup.BackupUtil.BackupFileType;
+import org.apache.zookeeper.server.backup.BackupUtil.IntervalEndpoint;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.monitoring.BackupBean;
+import org.apache.zookeeper.server.backup.monitoring.BackupStats;
+import org.apache.zookeeper.server.backup.monitoring.TimetableBackupBean;
+import org.apache.zookeeper.server.backup.monitoring.TimetableBackupStats;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.timetable.TimetableBackup;
+import org.apache.zookeeper.server.persistence.FileTxnLog;
+import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
+import org.apache.zookeeper.server.persistence.TxnLog;
+import org.apache.zookeeper.server.persistence.Util;
+import org.apache.zookeeper.server.persistence.ZxidRange;
+import org.apache.zookeeper.server.util.ZxidUtils;
+import org.apache.zookeeper.txn.TxnHeader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class manages the backing up of txnlog and snap files to remote
+ * storage for longer term and durable retention than is possible on
+ * an ensemble server
+ */
+public class BackupManager {
+  private static final Logger LOG = LoggerFactory.getLogger(BackupManager.class);
+
+  private final Logger logger;
+  private final File snapDir;
+  private final File dataLogDir;
+  private final File tmpDir;
+  private final BackupConfig backupConfig;
+  private final long backupIntervalInMilliseconds;
+  private BackupProcess logBackup = null;
+  private BackupProcess snapBackup = null;
+
+  // backupStatus, backupLogZxid and backedupSnapZxid need to be access while synchronized
+  // on backupStatus.
+  private final BackupStatus backupStatus;
+  private BackupPoint backupPoint;
+
+  private final BackupStorageProvider backupStorage;
+  private final long serverId;
+  private final String namespace;
+
+  @VisibleForTesting
+  protected BackupBean backupBean = null;
+  protected TimetableBackupBean timetableBackupBean = null;
+
+  private BackupStats backupStats = null;
+
+  // Optional timetable backup
+  private BackupProcess timetableBackup = null;
+
+  /**
+   * Tracks a file that needs to be backed up, including temporary copies of the file
+   */
+  public static class BackupFile {
+    private final File file;
+    private final boolean isTemporary;
+    private final ZxidRange zxidRange;
+
+    /**
+     * Create an instance of a BackupFile for the given initial file and zxid range
+     * @param backupFile the initial/original file
+     * @param isTemporaryFile whether the file is a temporary file
+     * @param fileMinZxid the min zxid associated with this file
+     * @param fileMaxZxid the max zxid associated with this file
+     */
+    public BackupFile(File backupFile, boolean isTemporaryFile, long fileMinZxid, long fileMaxZxid) {
+      this(backupFile, isTemporaryFile, new ZxidRange(fileMinZxid, fileMaxZxid));
+    }
+
+    /**
+     * Create an instance of a BackupFile for the given initial file and zxid range
+     * @param backupFile the initial/original file
+     * @param isTemporaryFile whether the file is a temporary file
+     * @param zxidRange the zxid range associated with this file
+     */
+    public BackupFile(File backupFile, boolean isTemporaryFile, ZxidRange zxidRange) {
+      Preconditions.checkNotNull(zxidRange);
+
+      if (!zxidRange.isHighPresent()) {
+        throw new IllegalArgumentException("ZxidRange must have a high value");
+      }
+
+      this.file = backupFile;
+      this.isTemporary = isTemporaryFile;
+      this.zxidRange = zxidRange;
+    }
+
+    /**
+     * Perform cleanup including deleting temporary files.
+     */
+    public void cleanup() {
+      if (isTemporary && exists()) {
+        file.delete();
+      }
+    }
+
+    /**
+     * Whether the file representing the zxids exists
+     * @return whether the file represented exists
+     */
+    public boolean exists() {
+      return file != null && file.exists();
+    }
+
+    /**
+     * Get the current file (topmost on the stack)
+     * @return the current file
+     */
+    public File getFile() { return file; }
+
+    /**
+     * Get the zxid range associated with this file
+     * @return the zxid range
+     */
+    public ZxidRange getZxidRange() {
+      return zxidRange;
+    }
+
+    /**
+     * Get the min zxid associated with this file
+     * @return the min associated zxid
+     */
+    public long getMinZxid() { return zxidRange.getLow(); }
+
+    /**
+     * Get the max zxid associated with this file
+     * @return the max associated zxid
+     */
+    public long getMaxZxid() { return zxidRange.getHigh(); }
+
+    @Override
+    public String toString() {
+      return String.format("%s : %s : %d - %d",
+          file == null ? "[empty]" : file.getPath(),
+          isTemporary ? "temp" : "perm",
+          zxidRange.getLow(),
+          zxidRange.getHigh());
+    }
+  }
+
+  /**
+   * Implements txnlog specific logic for BackupProcess
+   */
+  protected class TxnLogBackup extends BackupProcess {
+    private long iterationEndPoint;
+    private final FileTxnSnapLog snapLog;
+
+    /**
+     * Constructor for TxnLogBackup
+     * @param snapLog the FileTxnSnapLog object to use
+     */
+    public TxnLogBackup(FileTxnSnapLog snapLog) {
+      super(LoggerFactory.getLogger(TxnLogBackup.class), BackupManager.this.backupStorage,
+          backupIntervalInMilliseconds);
+      this.snapLog = snapLog;
+    }
+
+    protected void initialize() throws IOException {
+      backupStorage.cleanupInvalidFiles(null);
+
+      BackupFileInfo latest =
+          BackupUtil.getLatest(backupStorage, BackupFileType.TXNLOG, IntervalEndpoint.START);
+
+      long rZxid = latest == null
+          ? BackupUtil.INVALID_LOG_ZXID
+          : latest.getIntervalEndpoint(IntervalEndpoint.END);
+
+      logger.info("Latest Zxid from storage: {}  from status: {}",
+          ZxidUtils.zxidToString(rZxid), ZxidUtils.zxidToString(backupPoint.getLogZxid()));

Review Comment:
   *THREAD_SAFETY_VIOLATION:*  Read/Write race. Non-private method `BackupManager$TxnLogBackup.initialize()` indirectly reads without synchronization from `this.this$0.backupPoint.logZxid`. Potentially races with write in method `BackupManager$TxnLogBackup.backupComplete(...)`.
    Reporting because this access may occur on a background thread.
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621788&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621788&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621788&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621788&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621788&lift_comment_rating=5) ]



-- 
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: notifications-unsubscribe@zookeeper.apache.org

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


[GitHub] [zookeeper] eolivelli closed pull request #1883: Add backup and restore with timetable

Posted by GitBox <gi...@apache.org>.
eolivelli closed pull request #1883: Add backup and restore with timetable
URL: https://github.com/apache/zookeeper/pull/1883


-- 
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: notifications-unsubscribe@zookeeper.apache.org

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


[GitHub] [zookeeper] eolivelli commented on pull request #1883: Add backup and restore with timetable

Posted by GitBox <gi...@apache.org>.
eolivelli commented on PR #1883:
URL: https://github.com/apache/zookeeper/pull/1883#issuecomment-1153708409

   @narendly can you please advertise this great work on dev@zookeeper.apache.org ?


-- 
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: notifications-unsubscribe@zookeeper.apache.org

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


[GitHub] [zookeeper] sonatype-lift[bot] commented on a diff in pull request #1883: Add backup and restore with timetable

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on code in PR #1883:
URL: https://github.com/apache/zookeeper/pull/1883#discussion_r878545906


##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/RestoreFromBackupTool.java:
##########
@@ -0,0 +1,716 @@
+/*
+ * 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.zookeeper.server.backup;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.nio.file.Files;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Range;
+import org.apache.commons.cli.CommandLine;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.cli.RestoreCommand;
+import org.apache.zookeeper.common.ConfigException;
+import org.apache.zookeeper.server.backup.BackupUtil.BackupFileType;
+import org.apache.zookeeper.server.backup.BackupUtil.IntervalEndpoint;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.storage.BackupStorageUtil;
+import org.apache.zookeeper.server.backup.timetable.TimetableBackup;
+import org.apache.zookeeper.server.backup.timetable.TimetableUtil;
+import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
+import org.apache.zookeeper.server.persistence.Util;
+import org.apache.zookeeper.server.util.ZxidUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ *  This is a tool to restore, from backup storage, a snapshot and set of transaction log files
+ *  that combine to represent a transactionally consistent image of a ZooKeeper ensemble at
+ *  some zxid.
+ */
+public class RestoreFromBackupTool {
+  private static final Logger LOG = LoggerFactory.getLogger(RestoreFromBackupTool.class);
+
+  private static final int MAX_RETRIES = 10;
+  private static final String HEX_PREFIX = "0x";
+  private static final int CONNECTION_TIMEOUT = 300000;
+
+  BackupStorageProvider storage;
+  FileTxnSnapLog snapLog;
+  long zxidToRestore;
+  boolean dryRun;
+  File restoreTempDir;
+  boolean overwrite = false;
+
+  // Spot restoration
+  boolean isSpotRestoration = false;
+  String znodePathToRestore;
+  String zkServerConnectionStr;
+  boolean restoreRecursively = false;
+  ZooKeeper zk;
+  SpotRestorationTool spotRestorationTool;
+
+  List<BackupFileInfo> logs;
+  List<BackupFileInfo> snaps;
+  List<BackupFileInfo> filesToCopy;
+
+  int mostRecentLogNeededIndex;
+  int snapNeededIndex;
+  int oldestLogNeededIndex;
+
+  public enum BackupStorageOption {
+    GPFS("org.apache.zookeeper.server.backup.storage.impl.FileSystemBackupStorage"),
+    NFS("org.apache.zookeeper.server.backup.storage.impl.FileSystemBackupStorage");
+
+    private String storageProviderClassName;
+
+    BackupStorageOption(String className) {
+      this.storageProviderClassName = className;
+    }
+
+    public String getStorageProviderClassName() {
+      return storageProviderClassName;
+    }
+  }
+
+  /**
+   * Default constructor; requires using parseArgs to setup state.
+   */
+  public RestoreFromBackupTool() {
+    this(null, null, -1L, false, null);
+  }
+
+  /**
+   * Constructor
+   * @param storageProvider the backup provider from which to restore the backups
+   * @param snapLog the snap and log provider to use
+   * @param zxidToRestore the zxid upto which to restore, or Long.MAX to restore to the latest
+   *                      available transactionally consistent zxid.
+   * @param dryRun whether this is a dryrun in which case no files are actually copied
+   * @param restoreTempDir a temporary, local (not remote) directory to stage the backup files from
+   *                       remote backup storage for the processing stage of restoration. Note that
+   *                       this directory and the files in it will be removed after restoration.
+   */
+  RestoreFromBackupTool(BackupStorageProvider storageProvider, FileTxnSnapLog snapLog,
+      long zxidToRestore, boolean dryRun, File restoreTempDir) {
+
+    filesToCopy = new ArrayList<>();
+    snapNeededIndex = -1;
+    mostRecentLogNeededIndex = -1;
+    oldestLogNeededIndex = -1;
+
+    this.storage = storageProvider;
+    this.zxidToRestore = zxidToRestore;
+    this.snapLog = snapLog;
+    this.dryRun = dryRun;
+    this.restoreTempDir = restoreTempDir;
+  }
+
+  /**
+   * Parse and validate arguments to the tool
+   * @param cl the command line object with user inputs
+   * @return true if the arguments parse correctly; false in all other cases.
+   * @throws IOException if the backup provider cannot be instantiated correctly.
+   */
+  public void parseArgs(CommandLine cl) {
+    String backupStoragePath = cl.getOptionValue(RestoreCommand.OptionShortForm.BACKUP_STORE);
+    createBackupStorageProvider(backupStoragePath);
+
+    // Read the restore point
+    if (cl.hasOption(RestoreCommand.OptionShortForm.RESTORE_ZXID)) {
+      parseRestoreZxid(cl);
+    } else if (cl.hasOption(RestoreCommand.OptionShortForm.RESTORE_TIMESTAMP)) {
+      parseRestoreTimestamp(cl, backupStoragePath);
+    }
+
+    parseAndValidateSpotRestorationArgs(cl);
+
+    parseAndValidateOfflineRestoreDestination(cl);
+
+    parseRestoreTempDir(cl);
+
+    // Check if overwriting the destination directories is allowed
+    if (cl.hasOption(RestoreCommand.OptionShortForm.OVERWRITE)) {
+      overwrite = true;
+    }
+
+    // Check if this is a dry-run
+    if (cl.hasOption(RestoreCommand.OptionShortForm.DRY_RUN)) {
+      dryRun = true;
+    }
+
+    System.out.println("parseArgs successful.");
+  }
+
+  private void createBackupStorageProvider(String backupStoragePath) {
+    String[] backupStorageParams = backupStoragePath.split(":");
+    if (backupStorageParams.length != 4) {
+      System.err.println(
+          "Failed to parse backup storage connection information from the backup storage path provided, please check the input.");
+      System.err.println(
+          "For example: the format for a gpfs backup storage path should be \"gpfs:<config_path>:<backup_path>:<namespace>\".");
+      System.exit(1);
+    }
+
+    String userProvidedStorageName = backupStorageParams[0].toUpperCase();
+    try {
+      BackupStorageOption storageOption = BackupStorageOption.valueOf(userProvidedStorageName);
+      String backupStorageProviderClassName = storageOption.getStorageProviderClassName();
+
+      BackupConfig.RestorationConfigBuilder configBuilder =
+          new BackupConfig.RestorationConfigBuilder()
+              .setStorageProviderClassName(backupStorageProviderClassName)
+              .setBackupStoragePath(backupStorageParams[2]).setNamespace(backupStorageParams[3]);
+      if (!backupStorageParams[1].isEmpty()) {
+        configBuilder = configBuilder.setStorageConfig(new File(backupStorageParams[1]));
+      }
+      storage = BackupUtil.createStorageProviderImpl(configBuilder.build().get());
+    } catch (IllegalArgumentException e) {
+      System.err.println("Could not find a valid backup storage option based on the input: "
+          + userProvidedStorageName + ". Error message: " + e.getMessage());
+      e.printStackTrace();
+      System.exit(1);
+    } catch (ConfigException e) {
+      System.err.println(
+          "Could not generate a backup config based on the input, error message: " + e
+              .getMessage());
+      e.getStackTrace();
+      System.exit(1);
+    } catch (InstantiationException | InvocationTargetException | NoSuchMethodException | IllegalAccessException | ClassNotFoundException e) {
+      System.err.println(
+          "Could not generate a backup storage provider based on the input, error message: " + e
+              .getMessage());
+      e.printStackTrace();
+      System.exit(1);
+    }
+  }
+
+  private void parseRestoreZxid(CommandLine cl) {
+    String zxidToRestoreStr = cl.getOptionValue(RestoreCommand.OptionShortForm.RESTORE_ZXID);
+    if (zxidToRestoreStr.equalsIgnoreCase(BackupUtil.LATEST)) {
+      zxidToRestore = Long.MAX_VALUE;
+    } else {
+      int base = 10;
+      String numStr = zxidToRestoreStr;
+
+      if (zxidToRestoreStr.startsWith(HEX_PREFIX)) {
+        numStr = zxidToRestoreStr.substring(2);
+        base = 16;
+      }
+      try {
+        zxidToRestore = Long.parseLong(numStr, base);
+      } catch (NumberFormatException nfe) {
+        System.err
+            .println("Invalid number specified for restore zxid point, the input is: " + numStr);
+        System.exit(2);
+      }
+    }
+  }
+
+  private void parseRestoreTimestamp(CommandLine cl, String backupStoragePath) {
+    String timestampStr = cl.getOptionValue(RestoreCommand.OptionShortForm.RESTORE_TIMESTAMP);
+    String timetableStoragePath = backupStoragePath;
+    if (cl.hasOption(RestoreCommand.OptionShortForm.TIMETABLE_STORAGE_PATH)) {
+      timetableStoragePath =
+          cl.getOptionValue(RestoreCommand.OptionShortForm.TIMETABLE_STORAGE_PATH);
+    }
+    File[] timetableFiles = new File(timetableStoragePath)

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/io/File.<init>(Ljava/lang/String;)V) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621135&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621135&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621135&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621135&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621135&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/RestoreFromBackupTool.java:
##########
@@ -0,0 +1,716 @@
+/*
+ * 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.zookeeper.server.backup;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.nio.file.Files;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Range;
+import org.apache.commons.cli.CommandLine;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.cli.RestoreCommand;
+import org.apache.zookeeper.common.ConfigException;
+import org.apache.zookeeper.server.backup.BackupUtil.BackupFileType;
+import org.apache.zookeeper.server.backup.BackupUtil.IntervalEndpoint;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.storage.BackupStorageUtil;
+import org.apache.zookeeper.server.backup.timetable.TimetableBackup;
+import org.apache.zookeeper.server.backup.timetable.TimetableUtil;
+import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
+import org.apache.zookeeper.server.persistence.Util;
+import org.apache.zookeeper.server.util.ZxidUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ *  This is a tool to restore, from backup storage, a snapshot and set of transaction log files
+ *  that combine to represent a transactionally consistent image of a ZooKeeper ensemble at
+ *  some zxid.
+ */
+public class RestoreFromBackupTool {
+  private static final Logger LOG = LoggerFactory.getLogger(RestoreFromBackupTool.class);
+
+  private static final int MAX_RETRIES = 10;
+  private static final String HEX_PREFIX = "0x";
+  private static final int CONNECTION_TIMEOUT = 300000;
+
+  BackupStorageProvider storage;
+  FileTxnSnapLog snapLog;
+  long zxidToRestore;
+  boolean dryRun;
+  File restoreTempDir;
+  boolean overwrite = false;
+
+  // Spot restoration
+  boolean isSpotRestoration = false;
+  String znodePathToRestore;
+  String zkServerConnectionStr;
+  boolean restoreRecursively = false;
+  ZooKeeper zk;
+  SpotRestorationTool spotRestorationTool;
+
+  List<BackupFileInfo> logs;
+  List<BackupFileInfo> snaps;
+  List<BackupFileInfo> filesToCopy;
+
+  int mostRecentLogNeededIndex;
+  int snapNeededIndex;
+  int oldestLogNeededIndex;
+
+  public enum BackupStorageOption {
+    GPFS("org.apache.zookeeper.server.backup.storage.impl.FileSystemBackupStorage"),
+    NFS("org.apache.zookeeper.server.backup.storage.impl.FileSystemBackupStorage");
+
+    private String storageProviderClassName;
+
+    BackupStorageOption(String className) {
+      this.storageProviderClassName = className;
+    }
+
+    public String getStorageProviderClassName() {
+      return storageProviderClassName;
+    }
+  }
+
+  /**
+   * Default constructor; requires using parseArgs to setup state.
+   */
+  public RestoreFromBackupTool() {
+    this(null, null, -1L, false, null);
+  }
+
+  /**
+   * Constructor
+   * @param storageProvider the backup provider from which to restore the backups
+   * @param snapLog the snap and log provider to use
+   * @param zxidToRestore the zxid upto which to restore, or Long.MAX to restore to the latest
+   *                      available transactionally consistent zxid.
+   * @param dryRun whether this is a dryrun in which case no files are actually copied
+   * @param restoreTempDir a temporary, local (not remote) directory to stage the backup files from
+   *                       remote backup storage for the processing stage of restoration. Note that
+   *                       this directory and the files in it will be removed after restoration.
+   */
+  RestoreFromBackupTool(BackupStorageProvider storageProvider, FileTxnSnapLog snapLog,
+      long zxidToRestore, boolean dryRun, File restoreTempDir) {
+
+    filesToCopy = new ArrayList<>();
+    snapNeededIndex = -1;
+    mostRecentLogNeededIndex = -1;
+    oldestLogNeededIndex = -1;
+
+    this.storage = storageProvider;
+    this.zxidToRestore = zxidToRestore;
+    this.snapLog = snapLog;
+    this.dryRun = dryRun;
+    this.restoreTempDir = restoreTempDir;
+  }
+
+  /**
+   * Parse and validate arguments to the tool
+   * @param cl the command line object with user inputs
+   * @return true if the arguments parse correctly; false in all other cases.
+   * @throws IOException if the backup provider cannot be instantiated correctly.
+   */
+  public void parseArgs(CommandLine cl) {
+    String backupStoragePath = cl.getOptionValue(RestoreCommand.OptionShortForm.BACKUP_STORE);
+    createBackupStorageProvider(backupStoragePath);
+
+    // Read the restore point
+    if (cl.hasOption(RestoreCommand.OptionShortForm.RESTORE_ZXID)) {
+      parseRestoreZxid(cl);
+    } else if (cl.hasOption(RestoreCommand.OptionShortForm.RESTORE_TIMESTAMP)) {
+      parseRestoreTimestamp(cl, backupStoragePath);
+    }
+
+    parseAndValidateSpotRestorationArgs(cl);
+
+    parseAndValidateOfflineRestoreDestination(cl);
+
+    parseRestoreTempDir(cl);
+
+    // Check if overwriting the destination directories is allowed
+    if (cl.hasOption(RestoreCommand.OptionShortForm.OVERWRITE)) {
+      overwrite = true;
+    }
+
+    // Check if this is a dry-run
+    if (cl.hasOption(RestoreCommand.OptionShortForm.DRY_RUN)) {
+      dryRun = true;
+    }
+
+    System.out.println("parseArgs successful.");
+  }
+
+  private void createBackupStorageProvider(String backupStoragePath) {
+    String[] backupStorageParams = backupStoragePath.split(":");
+    if (backupStorageParams.length != 4) {
+      System.err.println(
+          "Failed to parse backup storage connection information from the backup storage path provided, please check the input.");
+      System.err.println(
+          "For example: the format for a gpfs backup storage path should be \"gpfs:<config_path>:<backup_path>:<namespace>\".");
+      System.exit(1);
+    }
+
+    String userProvidedStorageName = backupStorageParams[0].toUpperCase();
+    try {
+      BackupStorageOption storageOption = BackupStorageOption.valueOf(userProvidedStorageName);
+      String backupStorageProviderClassName = storageOption.getStorageProviderClassName();
+
+      BackupConfig.RestorationConfigBuilder configBuilder =
+          new BackupConfig.RestorationConfigBuilder()
+              .setStorageProviderClassName(backupStorageProviderClassName)
+              .setBackupStoragePath(backupStorageParams[2]).setNamespace(backupStorageParams[3]);
+      if (!backupStorageParams[1].isEmpty()) {
+        configBuilder = configBuilder.setStorageConfig(new File(backupStorageParams[1]));
+      }
+      storage = BackupUtil.createStorageProviderImpl(configBuilder.build().get());
+    } catch (IllegalArgumentException e) {
+      System.err.println("Could not find a valid backup storage option based on the input: "
+          + userProvidedStorageName + ". Error message: " + e.getMessage());
+      e.printStackTrace();
+      System.exit(1);
+    } catch (ConfigException e) {
+      System.err.println(
+          "Could not generate a backup config based on the input, error message: " + e
+              .getMessage());
+      e.getStackTrace();
+      System.exit(1);
+    } catch (InstantiationException | InvocationTargetException | NoSuchMethodException | IllegalAccessException | ClassNotFoundException e) {
+      System.err.println(
+          "Could not generate a backup storage provider based on the input, error message: " + e
+              .getMessage());
+      e.printStackTrace();
+      System.exit(1);
+    }
+  }
+
+  private void parseRestoreZxid(CommandLine cl) {
+    String zxidToRestoreStr = cl.getOptionValue(RestoreCommand.OptionShortForm.RESTORE_ZXID);
+    if (zxidToRestoreStr.equalsIgnoreCase(BackupUtil.LATEST)) {
+      zxidToRestore = Long.MAX_VALUE;
+    } else {
+      int base = 10;
+      String numStr = zxidToRestoreStr;
+
+      if (zxidToRestoreStr.startsWith(HEX_PREFIX)) {
+        numStr = zxidToRestoreStr.substring(2);
+        base = 16;
+      }
+      try {
+        zxidToRestore = Long.parseLong(numStr, base);
+      } catch (NumberFormatException nfe) {
+        System.err
+            .println("Invalid number specified for restore zxid point, the input is: " + numStr);
+        System.exit(2);
+      }
+    }
+  }
+
+  private void parseRestoreTimestamp(CommandLine cl, String backupStoragePath) {
+    String timestampStr = cl.getOptionValue(RestoreCommand.OptionShortForm.RESTORE_TIMESTAMP);
+    String timetableStoragePath = backupStoragePath;
+    if (cl.hasOption(RestoreCommand.OptionShortForm.TIMETABLE_STORAGE_PATH)) {
+      timetableStoragePath =
+          cl.getOptionValue(RestoreCommand.OptionShortForm.TIMETABLE_STORAGE_PATH);
+    }
+    File[] timetableFiles = new File(timetableStoragePath)
+        .listFiles(file -> file.getName().startsWith(TimetableBackup.TIMETABLE_PREFIX));
+    if (timetableFiles == null || timetableFiles.length == 0) {
+      System.err.println("Could not find timetable files at the path: " + timetableStoragePath);
+      System.exit(2);
+    }
+    Map.Entry<Long, String> restorePoint;
+    String message;
+    try {
+      restorePoint = TimetableUtil.findLastZxidFromTimestamp(timetableFiles, timestampStr);
+      zxidToRestore = Long.parseLong(restorePoint.getValue(), 16);
+      String timeToRestore = new SimpleDateFormat("MM/dd/yyyy HH:mm:ss")
+          .format(new java.util.Date(restorePoint.getKey()));
+      if (timestampStr.equalsIgnoreCase(BackupUtil.LATEST)) {
+        message = "Restoring to " + timeToRestore + ", original request was to restore to latest.";
+      } else {
+        String requestedTimeToRestore = new SimpleDateFormat("MM/dd/yyyy HH:mm:ss")
+            .format(new java.util.Date(Long.decode(timestampStr)));
+        message = "Restoring to " + timeToRestore + ", original request was to restore to "
+            + requestedTimeToRestore + ".";
+      }
+      System.out.println(message);
+      LOG.info(message);
+    } catch (IllegalArgumentException | BackupException e) {
+      System.err.println(
+          "Could not find a valid zxid from timetable using the timestamp provided: " + timestampStr
+              + ". The error message is: " + e.getMessage());
+      e.printStackTrace();
+      System.exit(2);
+    }
+  }
+
+  private void parseAndValidateOfflineRestoreDestination(CommandLine cl) {
+    if (isSpotRestoration) {
+      return;
+    }
+    // Read restore destination: dataDir and logDir
+    try {
+      String snapDirPath = cl.getOptionValue(RestoreCommand.OptionShortForm.SNAP_DESTINATION);
+      String logDirPath = cl.getOptionValue(RestoreCommand.OptionShortForm.LOG_DESTINATION);
+
+      if (snapDirPath == null || logDirPath == null) {
+        throw new BackupException(
+            "Snap destination path and log destination path are not defined for offline restoration. SnapDirPath: "
+                + snapDirPath + ", logDirPath: " + logDirPath);
+      }
+
+      File snapDir = new File(snapDirPath);
+      File logDir = new File(logDirPath);
+      snapLog = new FileTxnSnapLog(logDir, snapDir);
+      checkSnapDataDirFileExistence();
+    } catch (IOException ioe) {
+      System.err.println("Could not setup transaction log utility." + ioe);
+      System.exit(3);
+    }
+  }
+
+  private void parseRestoreTempDir(CommandLine cl) {
+    if (cl.hasOption(RestoreCommand.OptionShortForm.LOCAL_RESTORE_TEMP_DIR_PATH)) {
+      String localRestoreTempDirPath =
+          cl.getOptionValue(RestoreCommand.OptionShortForm.LOCAL_RESTORE_TEMP_DIR_PATH);
+      restoreTempDir = new File(localRestoreTempDirPath);
+    }
+
+    if (restoreTempDir == null) {
+      if (isSpotRestoration) {
+        throw new BackupException(
+            "Local restore temp dir path is not defined for spot restoration.");
+      } else {
+        // This is an offline restoration
+        // If the user hasn't provided the restore temp dir parameter,
+        //then the tool will just create a temporary folder inside snapLog and delete it afterwards.
+        this.restoreTempDir = new File(snapLog.getDataDir(), "RestoreTempDir_" + zxidToRestore);
+      }
+    }
+  }
+
+  private void parseAndValidateSpotRestorationArgs(CommandLine cl) {
+    if (cl.hasOption(RestoreCommand.OptionShortForm.ZNODE_PATH_TO_RESTORE)) {
+      znodePathToRestore = cl.getOptionValue(RestoreCommand.OptionShortForm.ZNODE_PATH_TO_RESTORE);
+    }
+    if (cl.hasOption(RestoreCommand.OptionShortForm.ZK_SERVER_CONNECTION_STRING)) {
+      zkServerConnectionStr =
+          cl.getOptionValue(RestoreCommand.OptionShortForm.ZK_SERVER_CONNECTION_STRING);
+    }
+    if (cl.hasOption(RestoreCommand.OptionShortForm.RECURSIVE_SPOT_RESTORE)) {
+      restoreRecursively = true;
+    }
+    if (znodePathToRestore != null && zkServerConnectionStr != null) {
+      isSpotRestoration = true;
+    } else if (znodePathToRestore == null && zkServerConnectionStr == null) {
+      isSpotRestoration = false;
+    } else {
+      throw new BackupException(
+          "Znode path and zk server connection string must be provided in order to do spot restoration. Provided znode path: "
+              + znodePathToRestore + ", provided zk server connection string: "
+              + zkServerConnectionStr);
+    }
+  }
+
+  /**
+   * Attempts to perform a restore with up to MAX_RETRIES retries.
+   * @return true if the restore completed successfully, false in all other cases.
+   */
+  public boolean runWithRetries(CommandLine cl) {
+    parseArgs(cl);
+
+    int tries = 0;
+
+    if (dryRun) {
+      System.out.println("This is a DRYRUN, no files will actually be copied.");
+    }
+
+    while (tries < MAX_RETRIES) {
+      try {
+        run();
+        return true;
+      } catch (IllegalArgumentException re) {
+        System.err.println(
+            "Restore attempt failed, could not find all the required backup files to restore. "
+                + "Error message: " + re.getMessage());
+        re.printStackTrace();
+        return false;
+      } catch (BackupException be) {
+        System.err.println(
+            "Restoration attempt failed due to a backup exception, it's usually caused by required "
+                + "directories not existing or failure of creating directories, etc. Please check the message. "
+                + "Error message: " + be.getMessage());
+        be.printStackTrace();
+        return false;
+      } catch (Exception e) {
+        tries++;
+        System.err.println("Restore attempt failed; attempting again. " + tries + "/" + MAX_RETRIES
+            + ". Error message: " + e.getMessage());
+        e.printStackTrace();
+      }
+    }
+
+    System.err.println("Failed to restore after " + (tries + 1) + " attempts.");
+    return false;
+  }
+
+  /**
+   * Attempts to perform a restore.
+   */
+  public void run() throws IOException, InterruptedException {
+    try {
+      if (!findFilesToRestore()) {
+        throw new IllegalArgumentException("Failed to find valid snapshot and logs to restore.");
+      }
+
+      if (restoreTempDir == null || storage == null) {
+        throw new BackupException("The RestoreTempDir and BackupStorageProvider cannot be null.");
+      }
+
+      if (!restoreTempDir.exists() && !restoreTempDir.mkdirs()) {
+        throw new BackupException(
+            "Failed to create a temporary directory at path: " + restoreTempDir.getPath()
+                + " to store copied backup files.");
+      }
+
+      if (!dryRun) {
+        // This step will create a "version-2" directory inside restoreTempDir,
+        // all the selected backup files will be copied to version-2 directory
+        FileTxnSnapLog restoreTempSnapLog =
+            new FileTxnSnapLog(this.restoreTempDir, this.restoreTempDir);
+
+        copyBackupFilesToLocalTempDir(restoreTempSnapLog);
+        processCopiedBackupFiles(restoreTempSnapLog, zxidToRestore);
+        if (isSpotRestoration) {
+          performSpotRestoration(restoreTempDir);
+        } else {
+          // It is an offline restoration
+          copyProcessedRestoredFilesToDestination(restoreTempSnapLog);
+        }
+      }
+    } finally {
+      if (restoreTempDir != null && restoreTempDir.exists()) {
+        // Using recursive delete here because a "version-2" directory is created under restoreTempDir with FileTxnSnapLog
+        BackupStorageUtil.deleteDirectoryRecursively(restoreTempDir);
+      }
+    }
+  }
+
+  /**
+   * Finds the set of files (snapshot and txlog) that are required to restore to a transactionally
+   * consistent point for the requested zxid.
+   * Note that when restoring to the latest zxid, the transactionally consistent point may NOT
+   * be the latest backed up zxid if logs have been lost in between; or if there is no
+   * transactionally consistent point in which nothing will be restored but the restored will
+   * be considered successful.
+   * @return true if a transactionally consistent set of files could be found for the requested
+   *         restore point; false in all other cases.
+   * @throws IOException
+   */
+  private boolean findFilesToRestore() throws IOException {
+    snaps = BackupUtil.getBackupFiles(storage, BackupFileType.SNAPSHOT, IntervalEndpoint.START,
+        BackupUtil.SortOrder.DESCENDING);
+    logs = BackupUtil.getBackupFiles(storage,
+        new BackupFileType[]{BackupFileType.TXNLOG, BackupFileType.LOSTLOG}, IntervalEndpoint.START,
+        BackupUtil.SortOrder.DESCENDING);
+    filesToCopy = new ArrayList<>();
+
+    snapNeededIndex = -1;
+
+    while (findNextPossibleSnapshot()) {
+      if (findLogRange()) {
+        filesToCopy.add(snaps.get(snapNeededIndex));
+        filesToCopy.addAll(logs.subList(mostRecentLogNeededIndex, oldestLogNeededIndex + 1));
+        return true;
+      }
+
+      if (zxidToRestore != Long.MAX_VALUE) {
+        break;
+      }
+    }
+
+    // Restoring to an empty data tree (i.e. no backup files) is valid for restoring to
+    // latest
+    if (zxidToRestore == Long.MAX_VALUE) {
+      return true;
+    }
+
+    return false;
+  }
+
+  /**
+   * Find the next snapshot whose range is below the requested restore point.
+   * Note: in practice this only gets called once when zxidToRestore != Long.MAX
+   * @return true if a snapshot is found; false in all other cases.
+   */
+  private boolean findNextPossibleSnapshot() {
+    for (snapNeededIndex++; snapNeededIndex < snaps.size(); snapNeededIndex++) {
+      if (snaps.get(snapNeededIndex).getRange().upperEndpoint() <= zxidToRestore) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+
+  /**
+   * Find the range of log files needed to make the current proposed snapshot transactionally
+   * consistent to the requested zxid.
+   * When zxidToRestore == Long.MAX the transaction log range terminates either on the most
+   * recent backedup txnlog OR the last txnlog prior to a lost log range (assuming that log
+   * still makes the snapshot transactionally consistent).
+   * @return true if a valid log range is found; false in all other cases.
+   */
+  private boolean findLogRange() {
+    Preconditions.checkState(snapNeededIndex >= 0 && snapNeededIndex < snaps.size());
+
+    if (logs.size() == 0) {
+      return false;
+    }
+
+    BackupFileInfo snap = snaps.get(snapNeededIndex);
+    Range<Long> snapRange = snap.getRange();
+    oldestLogNeededIndex = logs.size() - 1;
+    mostRecentLogNeededIndex = 0;
+
+    // Find the first txlog that might make the snapshot valid, OR is a lost log
+    for (int i = 0; i < logs.size() - 1; i++) {
+      BackupFileInfo log = logs.get(i);
+      Range<Long> logRange = log.getRange();
+
+      if (logRange.lowerEndpoint() <= snapRange.lowerEndpoint()) {
+        oldestLogNeededIndex = i;
+        break;
+      }
+    }
+
+    // Starting if the oldest log that might allow the snapshot to be valid, find the txnlog
+    // that includes the restore point, OR is a lost log
+    for (int i = oldestLogNeededIndex; i > 0; i--) {
+      BackupFileInfo log = logs.get(i);
+      Range<Long> logRange = log.getRange();
+
+      if (log.getFileType() == BackupFileType.LOSTLOG
+          || logRange.upperEndpoint() >= zxidToRestore) {
+
+        mostRecentLogNeededIndex = i;
+        break;
+      }
+    }
+
+    return validateLogRange();
+  }
+
+  private boolean validateLogRange() {
+    Preconditions.checkState(oldestLogNeededIndex >= 0);
+    Preconditions.checkState(oldestLogNeededIndex < logs.size());
+    Preconditions.checkState(mostRecentLogNeededIndex >= 0);
+    Preconditions.checkState(mostRecentLogNeededIndex < logs.size());
+    Preconditions.checkState(oldestLogNeededIndex >= mostRecentLogNeededIndex);
+    Preconditions.checkState(snapNeededIndex >= 0);
+    Preconditions.checkState(snapNeededIndex < snaps.size());
+
+    BackupFileInfo snap = snaps.get(snapNeededIndex);
+    BackupFileInfo oldestLog = logs.get(oldestLogNeededIndex);
+    BackupFileInfo newestLog = logs.get(mostRecentLogNeededIndex);
+
+    if (oldestLog.getFileType() == BackupFileType.LOSTLOG) {
+      LOG.error("Could not find logs to make the snapshot '" + snap.getBackedUpFile()
+          + "' valid. Lost logs at " + logs.get(oldestLogNeededIndex).getRange());
+      return false;
+    }
+
+    if (newestLog.getFileType() == BackupFileType.LOSTLOG) {
+      if (zxidToRestore == Long.MAX_VALUE && oldestLogNeededIndex != mostRecentLogNeededIndex) {
+        // When restoring to the latest, we can use the last valid log prior to lost log
+        // range.
+        mostRecentLogNeededIndex++;
+      } else {
+        LOG.error("Could not find logs to make the snapshot '" + snap.getBackedUpFile()
+            + "' valid. Lost logs at " + logs.get(mostRecentLogNeededIndex).getRange() + ".");
+        return false;
+      }
+    }
+
+    Range<Long> fullRange = oldestLog.getRange().span(newestLog.getRange());
+
+    if (fullRange.lowerEndpoint() > snap.getRange().lowerEndpoint()) {
+      LOG.error("Could not find logs to make snap '" + snap.getBackedUpFile()
+          + "' valid. Logs start at zxid " + ZxidUtils.zxidToString(fullRange.lowerEndpoint())
+          + ".");
+      return false;
+    }
+
+    if (fullRange.upperEndpoint() < snap.getRange().upperEndpoint()) {
+      LOG.error("Could not find logs to make snap '" + snap.getBackedUpFile()
+          + "' valid. Logs end at zxid " + ZxidUtils.zxidToString(fullRange.upperEndpoint()) + ".");
+      return false;
+    }
+
+    if (zxidToRestore != Long.MAX_VALUE && fullRange.upperEndpoint() < zxidToRestore) {
+      LOG.error("Could not find logs to restore to zxid " + zxidToRestore + ". Logs end at zxid "
+          + ZxidUtils.zxidToString(fullRange.upperEndpoint()) + ".");
+      return false;
+    }
+
+    return true;
+  }
+
+  /**
+   * Check if the specified snap dir and data dir already have files inside.
+   * If so, ask user to confirm if they want to overwrite these two directories with restored files,
+   * which means to wipe out all existing files and the directories be populated with restored files.
+   */
+  private void checkSnapDataDirFileExistence() {
+    File dataDir = snapLog.getDataDir();
+    File snapDir = snapLog.getSnapDir();
+    if (!dataDir.exists() && !dataDir.mkdirs()) {
+      throw new BackupException("Failed to create a data directory at path: " + dataDir.getPath()
+          + " to store restored txn logs.");
+    }
+    if (!snapDir.exists() && !snapDir.mkdirs()) {
+      throw new BackupException("Failed to create a snap directory at path: " + snapDir.getPath()
+          + " to store restored snapshot files.");
+    }
+    String[] dataDirFiles = dataDir.list();
+    String[] snapDirFiles = snapDir.list();
+    if (Objects.requireNonNull(dataDirFiles).length > 0
+        || Objects.requireNonNull(snapDirFiles).length > 0) {
+      if (overwrite) {
+        LOG.warn(
+            "Overwriting the destination directories for restoration, deleting all existing files. "
+                + "The files under dataDir: " + dataDir.getPath() + " are: " + Arrays
+                .toString(dataDirFiles) + "; and files under snapDir: " + snapDir.getPath()
+                + " are: " + Arrays.toString(snapDirFiles) + ".");
+        Arrays.stream(Objects.requireNonNull(dataDir.listFiles())).forEach(File::delete);
+        Arrays.stream(Objects.requireNonNull(snapDir.listFiles())).forEach(File::delete);
+      } else {
+        throw new BackupException(
+            "The destination directories are not empty, user chose not to overwrite the entire directory, "
+                + "exiting restoration. Please check the destination directory dataDir path: "
+                + dataDir.getPath() + ", and snapDir path" + snapDir.getPath());
+      }
+    }
+  }
+
+  /**
+   * Copy selected backup files from backup storage to a local restore temporary directory for further processing later
+   * @param restoreTempSnapLog A FileTxnSnapLog instance created on the specified local temporary directory path
+   * @throws IOException
+   */
+  private void copyBackupFilesToLocalTempDir(FileTxnSnapLog restoreTempSnapLog) throws IOException {
+    // Copy backup files to local temp directory
+    for (BackupFileInfo backedupFile : filesToCopy) {
+      String standardFilename = backedupFile.getStandardFile().getName();
+      // Does not matter if we use dataDir or logDir since we make the paths of these two directories
+      // same in restoreTempSnapLog object for restoration
+      File localTempDest = new File(restoreTempSnapLog.getDataDir(), standardFilename);
+
+      if (!localTempDest.exists()) {
+        LOG.info("Copying " + backedupFile.getBackedUpFile() + " from backup storage to temp dir "
+            + localTempDest.getPath() + ".");
+        storage.copyToLocalStorage(backedupFile.getBackedUpFile(), localTempDest);
+      } else {
+        throw new BackupException("Cannot copy " + backedupFile.getBackedUpFile()
+            + " to the local temp directory because it already exists as " + localTempDest.getPath()
+            + ".");
+      }
+    }
+  }
+
+  /**
+   * Process the copied backup files stored in local restore temporary directory to get them ready to be copied to final destination
+   * The processing currently includes truncating txn logs to zxidToRestore.
+   * @param restoreTempSnapLog A FileTxnSnapLog instance created on the specified local temporary directory path
+   * @param zxidToRestore The zxid to restore to
+   * @throws IOException
+   */
+  private void processCopiedBackupFiles(FileTxnSnapLog restoreTempSnapLog, long zxidToRestore) {
+    if (zxidToRestore != Long.MAX_VALUE) {
+      restoreTempSnapLog.truncateLog(zxidToRestore);
+      LOG.info(
+          "Successfully truncate the logs inside restoreTempDir " + restoreTempDir + " to zxid "
+              + zxidToRestore);
+    }
+  }
+
+  /**
+   * Copy the processed files from local restore temp directory to final destination:
+   * snapshot to snapDir, txn logs to logDir.
+   * @param restoreTempSnapLog A FileTxnSnapLog instance created on the specified local temporary directory path
+   * @throws IOException
+   */
+  private void copyProcessedRestoredFilesToDestination(FileTxnSnapLog restoreTempSnapLog)
+      throws IOException {
+    // Does not matter if we use dataDir or logDir since we make the paths of these two directories
+    // same in restoreTempSnapLog object for restoration
+    File[] processedFiles = restoreTempSnapLog.getDataDir().listFiles();
+    if (processedFiles == null) {
+      throw new BackupException(
+          "Failed to get a list of processed files from the local temp directory.");
+    }
+    for (File processedFile : processedFiles) {
+      String fileName = processedFile.getName();
+      if (!Util.isSnapshotFileName(fileName) && !Util.isLogFileName(fileName)) {
+        // Skip files that aren't snapshot nor transaction logs. Should only happen in tests.
+        LOG.info("Skipping copying " + processedFile.getPath()
+            + " from temp dir to final destination directory.");
+        continue;
+      }
+      File finalDestinationBase =
+          Util.isSnapshotFileName(fileName) ? snapLog.getSnapDir() : snapLog.getDataDir();
+      LOG.info(
+          "Copying " + processedFile.getPath() + " from temp dir to final destination directory "
+              + finalDestinationBase.getPath() + ".");
+      Files.copy(processedFile.toPath(), new File(finalDestinationBase, fileName).toPath());

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/io/File.<init>(Ljava/io/File;Ljava/lang/String;)V) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621178&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621178&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621178&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621178&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621178&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/storage/impl/FileSystemBackupStorage.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.zookeeper.server.backup.storage.impl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.zookeeper.server.backup.BackupConfig;
+import org.apache.zookeeper.server.backup.BackupFileInfo;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.storage.BackupStorageUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation for backup storage provider for file systems that store files in a tree (hierarchical) structure
+ * To use this class for different file systems, use appropriate address for backupStoragePath in BackupConfig
+ * For example:
+ * 1. hard disk drive & solid-state drive: /mountPoint/relativePathToMountPoint
+ * 2. NFS: /nfsClientMountPoint/relativePathToMountPoint
+ * 3. local disk: an absolute path to a directory
+ */
+public class FileSystemBackupStorage implements BackupStorageProvider {
+  private static final Logger LOG = LoggerFactory.getLogger(FileSystemBackupStorage.class);
+  private final String fileRootPath;
+  private final ReadWriteLock rwLock;
+  private final Lock sharedLock;
+  private final Lock exclusiveLock;
+
+  /**
+   * Constructor using BackupConfig to get backup storage info
+   * @param backupConfig The information and settings about backup storage, to be set as a part of ZooKeeper server config
+   */
+  public FileSystemBackupStorage(BackupConfig backupConfig) {
+    if (!new File(backupConfig.getBackupStoragePath()).exists()) {
+      throw new BackupException(
+          "The backup storage is not ready, please check the path: " + backupConfig
+              .getBackupStoragePath());
+    }
+    fileRootPath = String
+        .join(File.separator, backupConfig.getBackupStoragePath(), backupConfig.getNamespace());
+    rwLock = new ReentrantReadWriteLock();
+    sharedLock = rwLock.readLock();
+    exclusiveLock = rwLock.writeLock();
+  }
+
+  @Override
+  public BackupFileInfo getBackupFileInfo(File file) throws IOException {
+    String backupFilePath = BackupStorageUtil.constructBackupFilePath(file.getName(), fileRootPath);
+    File backupFile = new File(backupFilePath);
+
+    if (!backupFile.exists()) {
+      return new BackupFileInfo(backupFile, BackupFileInfo.NOT_SET, BackupFileInfo.NOT_SET);
+    }
+
+    BasicFileAttributes fileAttributes =
+        Files.readAttributes(Paths.get(backupFilePath), BasicFileAttributes.class);
+    return new BackupFileInfo(backupFile, fileAttributes.lastModifiedTime().toMillis(),
+        fileAttributes.size());
+  }
+
+  @Override
+  public List<BackupFileInfo> getBackupFileInfos(File path, String prefix) throws IOException {
+    String filePath = path == null ? "" : path.getPath();
+    String backupDirPath = Paths.get(fileRootPath, filePath).toString();
+    File backupDir = new File(backupDirPath);

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/io/File.<init>(Ljava/lang/String;)V) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621148&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621148&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621148&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621148&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621148&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/storage/impl/FileSystemBackupStorage.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.zookeeper.server.backup.storage.impl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.zookeeper.server.backup.BackupConfig;
+import org.apache.zookeeper.server.backup.BackupFileInfo;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.storage.BackupStorageUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation for backup storage provider for file systems that store files in a tree (hierarchical) structure
+ * To use this class for different file systems, use appropriate address for backupStoragePath in BackupConfig
+ * For example:
+ * 1. hard disk drive & solid-state drive: /mountPoint/relativePathToMountPoint
+ * 2. NFS: /nfsClientMountPoint/relativePathToMountPoint
+ * 3. local disk: an absolute path to a directory
+ */
+public class FileSystemBackupStorage implements BackupStorageProvider {
+  private static final Logger LOG = LoggerFactory.getLogger(FileSystemBackupStorage.class);
+  private final String fileRootPath;
+  private final ReadWriteLock rwLock;
+  private final Lock sharedLock;
+  private final Lock exclusiveLock;
+
+  /**
+   * Constructor using BackupConfig to get backup storage info
+   * @param backupConfig The information and settings about backup storage, to be set as a part of ZooKeeper server config
+   */
+  public FileSystemBackupStorage(BackupConfig backupConfig) {
+    if (!new File(backupConfig.getBackupStoragePath()).exists()) {
+      throw new BackupException(
+          "The backup storage is not ready, please check the path: " + backupConfig
+              .getBackupStoragePath());
+    }
+    fileRootPath = String
+        .join(File.separator, backupConfig.getBackupStoragePath(), backupConfig.getNamespace());
+    rwLock = new ReentrantReadWriteLock();
+    sharedLock = rwLock.readLock();
+    exclusiveLock = rwLock.writeLock();
+  }
+
+  @Override
+  public BackupFileInfo getBackupFileInfo(File file) throws IOException {
+    String backupFilePath = BackupStorageUtil.constructBackupFilePath(file.getName(), fileRootPath);
+    File backupFile = new File(backupFilePath);
+
+    if (!backupFile.exists()) {
+      return new BackupFileInfo(backupFile, BackupFileInfo.NOT_SET, BackupFileInfo.NOT_SET);
+    }
+
+    BasicFileAttributes fileAttributes =
+        Files.readAttributes(Paths.get(backupFilePath), BasicFileAttributes.class);
+    return new BackupFileInfo(backupFile, fileAttributes.lastModifiedTime().toMillis(),
+        fileAttributes.size());
+  }
+
+  @Override
+  public List<BackupFileInfo> getBackupFileInfos(File path, String prefix) throws IOException {
+    String filePath = path == null ? "" : path.getPath();
+    String backupDirPath = Paths.get(fileRootPath, filePath).toString();
+    File backupDir = new File(backupDirPath);
+
+    if (!backupDir.exists()) {
+      return new ArrayList<>();
+    }
+
+    File[] files = BackupStorageUtil.getFilesWithPrefix(backupDir, prefix);
+
+    // Read the file info and add to the list. If an exception is thrown, the entire operation will fail
+    List<BackupFileInfo> backupFileInfos = new ArrayList<>();
+    for (File file : files) {
+      backupFileInfos.add(getBackupFileInfo(file));
+    }
+    return backupFileInfos;
+  }
+
+  @Override
+  public List<File> getDirectories(File path) {
+    String filePath = path == null ? "" : path.getPath();
+    String backupDirPath = BackupStorageUtil.constructBackupFilePath(filePath, fileRootPath);
+    File backupDir = new File(backupDirPath);
+
+    if (!backupDir.exists()) {
+      throw new BackupException(
+          "Backup directory " + filePath + " does not exist, could not get directory list.");
+    }
+
+    // Filter out all the files which are directories
+    FilenameFilter fileFilter = (dir, name) -> new File(dir, name).isDirectory();
+    File[] dirs = backupDir.listFiles(fileFilter);
+
+    if (dirs == null) {
+      return new ArrayList<>();
+    }
+    return Arrays.asList(dirs);
+  }
+
+  @Override
+  public InputStream open(File path) throws IOException {
+    if (!path.exists() || path.isDirectory()) {
+      throw new BackupException("The file with the file path " + path
+          + " does not exist or is a directory, could not open the file.");
+    }
+    return new FileInputStream(path);
+  }
+
+  @Override
+  public void copyToBackupStorage(File srcFile, File destName) throws IOException {
+    InputStream inputStream = null;
+    OutputStream outputStream = null;
+    String backupTempFilePath;
+    File backupTempFile;
+
+    sharedLock.lock();
+    try {
+      inputStream = open(srcFile);
+
+      backupTempFilePath = BackupStorageUtil
+          .constructBackupFilePath(BackupStorageUtil.constructTempFileName(destName.getName()),
+              fileRootPath);
+      backupTempFile = new File(backupTempFilePath);
+
+      BackupStorageUtil.createFile(backupTempFile, true);
+      outputStream = new FileOutputStream(backupTempFile);
+
+      BackupStorageUtil.streamData(inputStream, outputStream);
+
+      Files.move(Paths.get(backupTempFilePath),
+          Paths.get(BackupStorageUtil.constructBackupFilePath(destName.getName(), fileRootPath)),
+          StandardCopyOption.REPLACE_EXISTING);
+    } finally {
+      if (inputStream != null) {
+        inputStream.close();
+      }
+      if (outputStream != null) {
+        outputStream.close();
+      }
+      sharedLock.unlock();
+    }
+  }
+
+  @Override
+  public void copyToLocalStorage(File srcName, File destFile) throws IOException {
+    InputStream inputStream = null;
+    OutputStream outputStream = null;
+
+    // Create input stream from the source file in backup storage
+    String backupFilePath =
+        BackupStorageUtil.constructBackupFilePath(srcName.getName(), fileRootPath);
+    File backupFile = new File(backupFilePath);
+
+    try {
+      inputStream = open(backupFile);
+
+      BackupStorageUtil.createFile(destFile, true);
+      outputStream = new FileOutputStream(destFile);
+
+      BackupStorageUtil.streamData(inputStream, outputStream);
+    } finally {
+      if (inputStream != null) {
+        inputStream.close();
+      }
+      if (outputStream != null) {
+        outputStream.close();
+      }
+    }
+  }
+
+  @Override
+  public void delete(File fileToDelete) throws IOException {
+    String fileName = fileToDelete == null ? "" : fileToDelete.getName();
+    String backupFilePath = BackupStorageUtil.constructBackupFilePath(fileName, fileRootPath);
+    Files.deleteIfExists(Paths.get(backupFilePath));
+  }
+
+  @Override
+  public void cleanupInvalidFiles(File path) throws IOException {
+    if (path == null) {
+      path = new File(fileRootPath);

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/io/File.<init>(Ljava/lang/String;)V) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621167&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621167&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621167&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621167&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621167&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/storage/impl/FileSystemBackupStorage.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.zookeeper.server.backup.storage.impl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.zookeeper.server.backup.BackupConfig;
+import org.apache.zookeeper.server.backup.BackupFileInfo;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.storage.BackupStorageUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation for backup storage provider for file systems that store files in a tree (hierarchical) structure
+ * To use this class for different file systems, use appropriate address for backupStoragePath in BackupConfig
+ * For example:
+ * 1. hard disk drive & solid-state drive: /mountPoint/relativePathToMountPoint
+ * 2. NFS: /nfsClientMountPoint/relativePathToMountPoint
+ * 3. local disk: an absolute path to a directory
+ */
+public class FileSystemBackupStorage implements BackupStorageProvider {
+  private static final Logger LOG = LoggerFactory.getLogger(FileSystemBackupStorage.class);
+  private final String fileRootPath;
+  private final ReadWriteLock rwLock;
+  private final Lock sharedLock;
+  private final Lock exclusiveLock;
+
+  /**
+   * Constructor using BackupConfig to get backup storage info
+   * @param backupConfig The information and settings about backup storage, to be set as a part of ZooKeeper server config
+   */
+  public FileSystemBackupStorage(BackupConfig backupConfig) {
+    if (!new File(backupConfig.getBackupStoragePath()).exists()) {
+      throw new BackupException(
+          "The backup storage is not ready, please check the path: " + backupConfig
+              .getBackupStoragePath());
+    }
+    fileRootPath = String
+        .join(File.separator, backupConfig.getBackupStoragePath(), backupConfig.getNamespace());
+    rwLock = new ReentrantReadWriteLock();
+    sharedLock = rwLock.readLock();
+    exclusiveLock = rwLock.writeLock();
+  }
+
+  @Override
+  public BackupFileInfo getBackupFileInfo(File file) throws IOException {
+    String backupFilePath = BackupStorageUtil.constructBackupFilePath(file.getName(), fileRootPath);
+    File backupFile = new File(backupFilePath);
+
+    if (!backupFile.exists()) {
+      return new BackupFileInfo(backupFile, BackupFileInfo.NOT_SET, BackupFileInfo.NOT_SET);
+    }
+
+    BasicFileAttributes fileAttributes =
+        Files.readAttributes(Paths.get(backupFilePath), BasicFileAttributes.class);
+    return new BackupFileInfo(backupFile, fileAttributes.lastModifiedTime().toMillis(),
+        fileAttributes.size());
+  }
+
+  @Override
+  public List<BackupFileInfo> getBackupFileInfos(File path, String prefix) throws IOException {
+    String filePath = path == null ? "" : path.getPath();
+    String backupDirPath = Paths.get(fileRootPath, filePath).toString();
+    File backupDir = new File(backupDirPath);
+
+    if (!backupDir.exists()) {
+      return new ArrayList<>();
+    }
+
+    File[] files = BackupStorageUtil.getFilesWithPrefix(backupDir, prefix);
+
+    // Read the file info and add to the list. If an exception is thrown, the entire operation will fail
+    List<BackupFileInfo> backupFileInfos = new ArrayList<>();
+    for (File file : files) {
+      backupFileInfos.add(getBackupFileInfo(file));
+    }
+    return backupFileInfos;
+  }
+
+  @Override
+  public List<File> getDirectories(File path) {
+    String filePath = path == null ? "" : path.getPath();
+    String backupDirPath = BackupStorageUtil.constructBackupFilePath(filePath, fileRootPath);
+    File backupDir = new File(backupDirPath);
+
+    if (!backupDir.exists()) {
+      throw new BackupException(
+          "Backup directory " + filePath + " does not exist, could not get directory list.");
+    }
+
+    // Filter out all the files which are directories
+    FilenameFilter fileFilter = (dir, name) -> new File(dir, name).isDirectory();
+    File[] dirs = backupDir.listFiles(fileFilter);
+
+    if (dirs == null) {
+      return new ArrayList<>();
+    }
+    return Arrays.asList(dirs);
+  }
+
+  @Override
+  public InputStream open(File path) throws IOException {
+    if (!path.exists() || path.isDirectory()) {
+      throw new BackupException("The file with the file path " + path
+          + " does not exist or is a directory, could not open the file.");
+    }
+    return new FileInputStream(path);
+  }
+
+  @Override
+  public void copyToBackupStorage(File srcFile, File destName) throws IOException {
+    InputStream inputStream = null;
+    OutputStream outputStream = null;
+    String backupTempFilePath;
+    File backupTempFile;
+
+    sharedLock.lock();
+    try {
+      inputStream = open(srcFile);
+
+      backupTempFilePath = BackupStorageUtil
+          .constructBackupFilePath(BackupStorageUtil.constructTempFileName(destName.getName()),
+              fileRootPath);
+      backupTempFile = new File(backupTempFilePath);
+
+      BackupStorageUtil.createFile(backupTempFile, true);
+      outputStream = new FileOutputStream(backupTempFile);
+
+      BackupStorageUtil.streamData(inputStream, outputStream);
+
+      Files.move(Paths.get(backupTempFilePath),

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/nio/file/Paths.get(Ljava/lang/String;[Ljava/lang/String;)Ljava/nio/file/Path;) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621188&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621188&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621188&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621188&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621188&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/timetable/TimetableUtil.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.zookeeper.server.backup.timetable;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.ObjectInputStream;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.zookeeper.server.backup.BackupUtil;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+
+/**
+ * Util methods used to operate on timetable backup.
+ */
+public final class TimetableUtil {
+  private static final String TIMETABLE_PREFIX = "timetable.";
+
+  private TimetableUtil() {
+    // Util class
+  }
+
+  /**
+   * Returns the last zxid corresponding to the timestamp preceding the timestamp given as argument.
+   * Note: the given timestamp must be in the comprehensive range created from the timetable
+   * backup files unless it is "latest". This is to prevent any human mistakes of accidentally
+   * entering an arbitrarily high timestamp value and the tool restoring to the latest backup point.
+   * @param timetableBackupFiles
+   * @param timestamp timestamp string (long), or "latest"
+   * @return A map entry consists of timestamp in Long and Hex String representation of the zxid found
+   */
+  public static Map.Entry<Long, String> findLastZxidFromTimestamp(File[] timetableBackupFiles,
+      String timestamp) {
+    // Verify argument: backup files
+    if (timetableBackupFiles == null || timetableBackupFiles.length == 0) {
+      throw new IllegalArgumentException(
+          "TimetableUtil::findLastZxidFromTimestamp(): timetableBackupFiles argument is either null"
+              + " or empty!");
+    }
+
+    // Verify argument: timestamp
+    boolean isLatest = timestamp.equalsIgnoreCase(BackupUtil.LATEST);
+    long timestampLong;
+    if (isLatest) {
+      timestampLong = Long.MAX_VALUE;
+    } else {
+      try {
+        timestampLong = Long.decode(timestamp);
+      } catch (NumberFormatException e) {
+        throw new IllegalArgumentException(
+            "TimetableUtil::findLastZxidFromTimestamp(): cannot convert the given timestamp to a"
+                + " valid long! timestamp: " + timestamp, e);
+      }
+    }
+
+    // Traverse the files and find the lower bound, upper bound, and the file that contains the
+    // timestamp
+    long lowerBound = Long.MAX_VALUE, upperBound = Long.MIN_VALUE, lowestDelta = Long.MAX_VALUE;
+    File fileToRead = null;
+    for (File file : timetableBackupFiles) {
+      String[] range = file.getName().replaceAll(TIMETABLE_PREFIX, "").split("-");
+      long low = Long.parseLong(range[0]), high = Long.parseLong(range[1]);
+      lowerBound = Math.min(low, lowerBound);
+      upperBound = Math.max(high, upperBound);
+      if (isLatest) {
+        if (upperBound == high) {
+          fileToRead = file;
+        }
+      } else {
+        // Calculate the delta to find the closest available file either containing the timestamp
+        // in its range or before the timestamp
+        long delta = timestampLong - low;
+        if (delta >= 0 && delta < lowestDelta) {
+          lowestDelta = delta;
+          fileToRead = file;
+        }
+      }
+    }
+
+    // Check if the given timestamp is in range
+    if (!isLatest && (timestampLong < lowerBound || timestampLong > upperBound)) {
+      throw new IllegalArgumentException(
+          "TimetableUtil::findLastZxidFromTimestamp(): timestamp given " + timestampLong
+              + " is not in the timestamp range [" + lowerBound + " , " + upperBound
+              + "] given in the backup files!");
+    }
+
+    // Check if a file is found (this shouldn't happen if timestamp is in range)
+    if (fileToRead == null) {
+      throw new IllegalArgumentException(
+          "TimetableUtil::findLastZxidFromTimestamp(): unable to find the backup file to use!");
+    }
+
+    // Convert timetable backup files to an ordered Map<Long, String>, timestamp:zxid pairs
+    TreeMap<Long, String> timestampZxidPairs = new TreeMap<>();
+    try {
+      FileInputStream fis = new FileInputStream(fileToRead);
+      ObjectInputStream ois = new ObjectInputStream(fis);
+      @SuppressWarnings("unchecked")
+      Map<Long, String> map = (TreeMap<Long, String>) ois.readObject();

Review Comment:
   *[OBJECT_DESERIALIZATION](https://find-sec-bugs.github.io/bugs.htm#OBJECT_DESERIALIZATION):*  Object deserialization is used in org.apache.zookeeper.server.backup.timetable.TimetableUtil.findLastZxidFromTimestamp(File[], String)
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621166&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621166&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621166&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621166&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621166&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/storage/impl/FileSystemBackupStorage.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.zookeeper.server.backup.storage.impl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.zookeeper.server.backup.BackupConfig;
+import org.apache.zookeeper.server.backup.BackupFileInfo;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.storage.BackupStorageUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation for backup storage provider for file systems that store files in a tree (hierarchical) structure
+ * To use this class for different file systems, use appropriate address for backupStoragePath in BackupConfig
+ * For example:
+ * 1. hard disk drive & solid-state drive: /mountPoint/relativePathToMountPoint
+ * 2. NFS: /nfsClientMountPoint/relativePathToMountPoint
+ * 3. local disk: an absolute path to a directory
+ */
+public class FileSystemBackupStorage implements BackupStorageProvider {
+  private static final Logger LOG = LoggerFactory.getLogger(FileSystemBackupStorage.class);
+  private final String fileRootPath;
+  private final ReadWriteLock rwLock;
+  private final Lock sharedLock;
+  private final Lock exclusiveLock;
+
+  /**
+   * Constructor using BackupConfig to get backup storage info
+   * @param backupConfig The information and settings about backup storage, to be set as a part of ZooKeeper server config
+   */
+  public FileSystemBackupStorage(BackupConfig backupConfig) {
+    if (!new File(backupConfig.getBackupStoragePath()).exists()) {

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/io/File.<init>(Ljava/lang/String;)V) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621199&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621199&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621199&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621199&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621199&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/storage/impl/FileSystemBackupStorage.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.zookeeper.server.backup.storage.impl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.zookeeper.server.backup.BackupConfig;
+import org.apache.zookeeper.server.backup.BackupFileInfo;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.storage.BackupStorageUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation for backup storage provider for file systems that store files in a tree (hierarchical) structure
+ * To use this class for different file systems, use appropriate address for backupStoragePath in BackupConfig
+ * For example:
+ * 1. hard disk drive & solid-state drive: /mountPoint/relativePathToMountPoint
+ * 2. NFS: /nfsClientMountPoint/relativePathToMountPoint
+ * 3. local disk: an absolute path to a directory
+ */
+public class FileSystemBackupStorage implements BackupStorageProvider {
+  private static final Logger LOG = LoggerFactory.getLogger(FileSystemBackupStorage.class);
+  private final String fileRootPath;
+  private final ReadWriteLock rwLock;
+  private final Lock sharedLock;
+  private final Lock exclusiveLock;
+
+  /**
+   * Constructor using BackupConfig to get backup storage info
+   * @param backupConfig The information and settings about backup storage, to be set as a part of ZooKeeper server config
+   */
+  public FileSystemBackupStorage(BackupConfig backupConfig) {
+    if (!new File(backupConfig.getBackupStoragePath()).exists()) {
+      throw new BackupException(
+          "The backup storage is not ready, please check the path: " + backupConfig
+              .getBackupStoragePath());
+    }
+    fileRootPath = String
+        .join(File.separator, backupConfig.getBackupStoragePath(), backupConfig.getNamespace());
+    rwLock = new ReentrantReadWriteLock();
+    sharedLock = rwLock.readLock();
+    exclusiveLock = rwLock.writeLock();
+  }
+
+  @Override
+  public BackupFileInfo getBackupFileInfo(File file) throws IOException {
+    String backupFilePath = BackupStorageUtil.constructBackupFilePath(file.getName(), fileRootPath);
+    File backupFile = new File(backupFilePath);
+
+    if (!backupFile.exists()) {
+      return new BackupFileInfo(backupFile, BackupFileInfo.NOT_SET, BackupFileInfo.NOT_SET);
+    }
+
+    BasicFileAttributes fileAttributes =
+        Files.readAttributes(Paths.get(backupFilePath), BasicFileAttributes.class);
+    return new BackupFileInfo(backupFile, fileAttributes.lastModifiedTime().toMillis(),
+        fileAttributes.size());
+  }
+
+  @Override
+  public List<BackupFileInfo> getBackupFileInfos(File path, String prefix) throws IOException {
+    String filePath = path == null ? "" : path.getPath();
+    String backupDirPath = Paths.get(fileRootPath, filePath).toString();
+    File backupDir = new File(backupDirPath);
+
+    if (!backupDir.exists()) {
+      return new ArrayList<>();
+    }
+
+    File[] files = BackupStorageUtil.getFilesWithPrefix(backupDir, prefix);
+
+    // Read the file info and add to the list. If an exception is thrown, the entire operation will fail
+    List<BackupFileInfo> backupFileInfos = new ArrayList<>();
+    for (File file : files) {
+      backupFileInfos.add(getBackupFileInfo(file));
+    }
+    return backupFileInfos;
+  }
+
+  @Override
+  public List<File> getDirectories(File path) {
+    String filePath = path == null ? "" : path.getPath();
+    String backupDirPath = BackupStorageUtil.constructBackupFilePath(filePath, fileRootPath);
+    File backupDir = new File(backupDirPath);
+
+    if (!backupDir.exists()) {
+      throw new BackupException(
+          "Backup directory " + filePath + " does not exist, could not get directory list.");
+    }
+
+    // Filter out all the files which are directories
+    FilenameFilter fileFilter = (dir, name) -> new File(dir, name).isDirectory();
+    File[] dirs = backupDir.listFiles(fileFilter);
+
+    if (dirs == null) {
+      return new ArrayList<>();
+    }
+    return Arrays.asList(dirs);
+  }
+
+  @Override
+  public InputStream open(File path) throws IOException {
+    if (!path.exists() || path.isDirectory()) {
+      throw new BackupException("The file with the file path " + path
+          + " does not exist or is a directory, could not open the file.");
+    }
+    return new FileInputStream(path);
+  }
+
+  @Override
+  public void copyToBackupStorage(File srcFile, File destName) throws IOException {
+    InputStream inputStream = null;
+    OutputStream outputStream = null;
+    String backupTempFilePath;
+    File backupTempFile;
+
+    sharedLock.lock();
+    try {
+      inputStream = open(srcFile);
+
+      backupTempFilePath = BackupStorageUtil
+          .constructBackupFilePath(BackupStorageUtil.constructTempFileName(destName.getName()),
+              fileRootPath);
+      backupTempFile = new File(backupTempFilePath);
+
+      BackupStorageUtil.createFile(backupTempFile, true);
+      outputStream = new FileOutputStream(backupTempFile);
+
+      BackupStorageUtil.streamData(inputStream, outputStream);
+
+      Files.move(Paths.get(backupTempFilePath),
+          Paths.get(BackupStorageUtil.constructBackupFilePath(destName.getName(), fileRootPath)),
+          StandardCopyOption.REPLACE_EXISTING);
+    } finally {
+      if (inputStream != null) {
+        inputStream.close();
+      }
+      if (outputStream != null) {
+        outputStream.close();
+      }
+      sharedLock.unlock();
+    }
+  }
+
+  @Override
+  public void copyToLocalStorage(File srcName, File destFile) throws IOException {
+    InputStream inputStream = null;
+    OutputStream outputStream = null;
+
+    // Create input stream from the source file in backup storage
+    String backupFilePath =
+        BackupStorageUtil.constructBackupFilePath(srcName.getName(), fileRootPath);
+    File backupFile = new File(backupFilePath);
+
+    try {
+      inputStream = open(backupFile);
+
+      BackupStorageUtil.createFile(destFile, true);
+      outputStream = new FileOutputStream(destFile);
+
+      BackupStorageUtil.streamData(inputStream, outputStream);
+    } finally {
+      if (inputStream != null) {
+        inputStream.close();
+      }
+      if (outputStream != null) {
+        outputStream.close();
+      }
+    }
+  }
+
+  @Override
+  public void delete(File fileToDelete) throws IOException {
+    String fileName = fileToDelete == null ? "" : fileToDelete.getName();
+    String backupFilePath = BackupStorageUtil.constructBackupFilePath(fileName, fileRootPath);
+    Files.deleteIfExists(Paths.get(backupFilePath));

Review Comment:
   *[PATH_TRAVERSAL_IN](https://find-sec-bugs.github.io/bugs.htm#PATH_TRAVERSAL_IN):*  This API (java/nio/file/Paths.get(Ljava/lang/String;[Ljava/lang/String;)Ljava/nio/file/Path;) reads a file whose location might be specified by user input
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621212&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621212&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621212&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621212&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621212&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/BackupManager.java:
##########
@@ -0,0 +1,661 @@
+/*
+ * 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.zookeeper.server.backup;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import javax.management.JMException;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.zookeeper.jmx.MBeanRegistry;
+import org.apache.zookeeper.server.backup.BackupUtil.BackupFileType;
+import org.apache.zookeeper.server.backup.BackupUtil.IntervalEndpoint;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.monitoring.BackupBean;
+import org.apache.zookeeper.server.backup.monitoring.BackupStats;
+import org.apache.zookeeper.server.backup.monitoring.TimetableBackupBean;
+import org.apache.zookeeper.server.backup.monitoring.TimetableBackupStats;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.timetable.TimetableBackup;
+import org.apache.zookeeper.server.persistence.FileTxnLog;
+import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
+import org.apache.zookeeper.server.persistence.TxnLog;
+import org.apache.zookeeper.server.persistence.Util;
+import org.apache.zookeeper.server.persistence.ZxidRange;
+import org.apache.zookeeper.server.util.ZxidUtils;
+import org.apache.zookeeper.txn.TxnHeader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class manages the backing up of txnlog and snap files to remote
+ * storage for longer term and durable retention than is possible on
+ * an ensemble server
+ */
+public class BackupManager {
+  private static final Logger LOG = LoggerFactory.getLogger(BackupManager.class);
+
+  private final Logger logger;
+  private final File snapDir;
+  private final File dataLogDir;
+  private final File tmpDir;
+  private final BackupConfig backupConfig;
+  private final long backupIntervalInMilliseconds;
+  private BackupProcess logBackup = null;
+  private BackupProcess snapBackup = null;
+
+  // backupStatus, backupLogZxid and backedupSnapZxid need to be access while synchronized
+  // on backupStatus.
+  private final BackupStatus backupStatus;
+  private BackupPoint backupPoint;
+
+  private final BackupStorageProvider backupStorage;
+  private final long serverId;
+  private final String namespace;
+
+  @VisibleForTesting
+  protected BackupBean backupBean = null;
+  protected TimetableBackupBean timetableBackupBean = null;
+
+  private BackupStats backupStats = null;
+
+  // Optional timetable backup
+  private BackupProcess timetableBackup = null;
+
+  /**
+   * Tracks a file that needs to be backed up, including temporary copies of the file
+   */
+  public static class BackupFile {
+    private final File file;
+    private final boolean isTemporary;
+    private final ZxidRange zxidRange;
+
+    /**
+     * Create an instance of a BackupFile for the given initial file and zxid range
+     * @param backupFile the initial/original file
+     * @param isTemporaryFile whether the file is a temporary file
+     * @param fileMinZxid the min zxid associated with this file
+     * @param fileMaxZxid the max zxid associated with this file
+     */
+    public BackupFile(File backupFile, boolean isTemporaryFile, long fileMinZxid, long fileMaxZxid) {
+      this(backupFile, isTemporaryFile, new ZxidRange(fileMinZxid, fileMaxZxid));
+    }
+
+    /**
+     * Create an instance of a BackupFile for the given initial file and zxid range
+     * @param backupFile the initial/original file
+     * @param isTemporaryFile whether the file is a temporary file
+     * @param zxidRange the zxid range associated with this file
+     */
+    public BackupFile(File backupFile, boolean isTemporaryFile, ZxidRange zxidRange) {
+      Preconditions.checkNotNull(zxidRange);
+
+      if (!zxidRange.isHighPresent()) {
+        throw new IllegalArgumentException("ZxidRange must have a high value");
+      }
+
+      this.file = backupFile;
+      this.isTemporary = isTemporaryFile;
+      this.zxidRange = zxidRange;
+    }
+
+    /**
+     * Perform cleanup including deleting temporary files.
+     */
+    public void cleanup() {
+      if (isTemporary && exists()) {
+        file.delete();
+      }
+    }
+
+    /**
+     * Whether the file representing the zxids exists
+     * @return whether the file represented exists
+     */
+    public boolean exists() {
+      return file != null && file.exists();
+    }
+
+    /**
+     * Get the current file (topmost on the stack)
+     * @return the current file
+     */
+    public File getFile() { return file; }
+
+    /**
+     * Get the zxid range associated with this file
+     * @return the zxid range
+     */
+    public ZxidRange getZxidRange() {
+      return zxidRange;
+    }
+
+    /**
+     * Get the min zxid associated with this file
+     * @return the min associated zxid
+     */
+    public long getMinZxid() { return zxidRange.getLow(); }
+
+    /**
+     * Get the max zxid associated with this file
+     * @return the max associated zxid
+     */
+    public long getMaxZxid() { return zxidRange.getHigh(); }
+
+    @Override
+    public String toString() {
+      return String.format("%s : %s : %d - %d",
+          file == null ? "[empty]" : file.getPath(),
+          isTemporary ? "temp" : "perm",
+          zxidRange.getLow(),
+          zxidRange.getHigh());
+    }
+  }
+
+  /**
+   * Implements txnlog specific logic for BackupProcess
+   */
+  protected class TxnLogBackup extends BackupProcess {
+    private long iterationEndPoint;
+    private final FileTxnSnapLog snapLog;
+
+    /**
+     * Constructor for TxnLogBackup
+     * @param snapLog the FileTxnSnapLog object to use
+     */
+    public TxnLogBackup(FileTxnSnapLog snapLog) {
+      super(LoggerFactory.getLogger(TxnLogBackup.class), BackupManager.this.backupStorage,
+          backupIntervalInMilliseconds);
+      this.snapLog = snapLog;
+    }
+
+    protected void initialize() throws IOException {
+      backupStorage.cleanupInvalidFiles(null);
+
+      BackupFileInfo latest =
+          BackupUtil.getLatest(backupStorage, BackupFileType.TXNLOG, IntervalEndpoint.START);
+
+      long rZxid = latest == null
+          ? BackupUtil.INVALID_LOG_ZXID
+          : latest.getIntervalEndpoint(IntervalEndpoint.END);
+
+      logger.info("Latest Zxid from storage: {}  from status: {}",
+          ZxidUtils.zxidToString(rZxid), ZxidUtils.zxidToString(backupPoint.getLogZxid()));
+
+      if (rZxid != backupPoint.getLogZxid()) {
+        synchronized (backupStatus) {
+          backupPoint.setLogZxid(rZxid);
+          backupStatus.update(backupPoint);
+        }
+      }
+    }
+
+    protected void startIteration() {
+      // Store the current last logged zxid.  This becomes the stopping point
+      // for the current iteration so we don't keep chasing our own tail as
+      // new transactions get written.
+      iterationEndPoint = snapLog.getLastLoggedZxid();
+      backupStats.setTxnLogBackupIterationStart();
+    }
+
+    protected void endIteration(boolean errorFree) {
+      backupStats.setTxnLogBackupIterationDone(errorFree);
+      iterationEndPoint = 0L;
+    }
+
+    /**
+     * Gets the next txnlog file to backup.  This is a temporary file created by copying
+     * all transaction from the previous backup point until the end zxid for this iteration, or
+     * a file indicating that some log records were lost.
+     * @return the file that needs to be backed-up.  The minZxid is the first
+     *      zxid contained in the file.  The maxZxid is the last zxid that is contained in the
+     *      file.
+     * @throws IOException
+     */
+    protected BackupFile getNextFileToBackup() throws IOException {
+      long startingZxid = backupStatus.read().getLogZxid() + 1;
+
+      // Don't keep chasing the tail so stop if past the last zxid at the time
+      // this iteration started.
+      if (startingZxid > iterationEndPoint) {
+        return null;
+      }
+
+      TxnLog.TxnIterator iter = null;
+      FileTxnLog newFile = null;
+      long lastZxid = -1;
+      int txnCopied = 0;
+      BackupFile ret;
+
+      logger.info("Creating backup file from zxid {}.", ZxidUtils.zxidToString(startingZxid));
+
+      try {
+        iter = snapLog.readTxnLog(startingZxid, true);
+
+        // Use a temp directory to avoid conflicts with live txnlog files
+        newFile = new FileTxnLog(tmpDir);
+
+        // TODO: Ideally, we should have have a way to prevent lost TxLogs
+        // Check for lost txnlogs; <=1 indicates that no backups have been done before so
+        // nothing can be considered lost.
+        // If a lost sequence is found then return a file whose name encodes the lost
+        // sequence and back that up so the backup store has a record of the lost sequence
+        if (startingZxid > 1 &&
+            iter.getHeader() != null &&
+            iter.getHeader().getZxid() > startingZxid) {
+
+          logger.error("TxnLog backups lost.  Required starting zxid={}  First available zxid={}",
+              ZxidUtils.zxidToString(startingZxid),
+              ZxidUtils.zxidToString(iter.getHeader().getZxid()));
+
+          String fileName = String.format("%s.%s",
+              BackupUtil.LOST_LOG_PREFIX,
+              Long.toHexString(startingZxid));
+          File lostZxidFile = new File(tmpDir, fileName);
+          lostZxidFile.createNewFile();
+
+          return new BackupFile(lostZxidFile, true, startingZxid, iter.getHeader().getZxid() - 1);
+        }
+
+        while (iter.getHeader() != null) {
+          TxnHeader hdr = iter.getHeader();
+
+          if (hdr.getZxid() > iterationEndPoint) {
+            break;
+          }
+
+          newFile.append(hdr, iter.getTxn());
+
+          // update position and count only AFTER the record has been successfully
+          // copied
+          lastZxid = hdr.getZxid();
+          txnCopied++;
+
+          iter.next();
+        }
+
+        ret = makeBackupFileFromCopiedLog(newFile, lastZxid);
+
+        if (ret != null) {
+          logger.info("Copied {} records starting at {} and ending at zxid {}.",
+              txnCopied,
+              ZxidUtils.zxidToString(ret.getMinZxid()),
+              ZxidUtils.zxidToString(ret.getMaxZxid()));
+        }
+
+      } catch (IOException e) {
+        logger.warn("Hit exception after {} records.  Exception: {} ", txnCopied, e.fillInStackTrace());
+
+        // If any records were copied return those and ignore the error.  Otherwise
+        // rethrow the error to be handled by the caller as a failed backup iteration.
+        if (txnCopied <= 0) {
+          throw e;
+        }
+
+        ret = makeBackupFileFromCopiedLog(newFile, lastZxid);
+      } finally {
+        if (iter != null) {
+          iter.close();
+        }
+
+        if (newFile != null) {
+          newFile.close();
+        }
+      }
+
+      return ret;
+    }
+
+    private BackupFile makeBackupFileFromCopiedLog(FileTxnLog backupTxnLog, long lastZxid) {
+
+      if (backupTxnLog == null) {
+        return null;
+      }
+
+      // The logFile gets initialized with the first transaction's zxid
+      File logFile = backupTxnLog.getCurrentFile();
+
+      if (logFile == null) {
+        return null;
+      }
+
+      long firstZxid = Util.getZxidFromName(logFile.getName(), Util.TXLOG_PREFIX);
+
+      if (lastZxid == -1) {
+        lastZxid = firstZxid;
+      }
+
+      return new BackupFile(logFile, true, new ZxidRange(firstZxid, lastZxid));
+    }
+
+    protected void backupComplete(BackupFile file) throws IOException {
+      synchronized (backupStatus) {
+        backupPoint.setLogZxid(file.getMaxZxid());
+        backupStatus.update(backupPoint);
+      }
+
+      logger.info("Updated backedup tnxlog zxid to {}",
+          ZxidUtils.zxidToString(backupPoint.getLogZxid()));
+    }
+  }
+
+  /**
+   * Implements snapshot specific logic for BackupProcess
+   */
+  protected class SnapBackup extends BackupProcess {
+    private final FileTxnSnapLog snapLog;
+    private final List<BackupFile> filesToBackup = new ArrayList<>();
+
+    /**
+     * Constructor for SnapBackup
+     * @param snapLog the FileTxnSnapLog object to use
+     */
+    public SnapBackup(FileTxnSnapLog snapLog) {
+      super(LoggerFactory.getLogger(SnapBackup.class), BackupManager.this.backupStorage,
+          backupIntervalInMilliseconds);
+      this.snapLog = snapLog;
+    }
+
+    protected void initialize() throws IOException {
+      backupStorage.cleanupInvalidFiles(null);
+
+      BackupFileInfo latest =
+          BackupUtil.getLatest(backupStorage, BackupFileType.SNAPSHOT, IntervalEndpoint.START);
+
+      long rZxid = latest == null
+          ? BackupUtil.INVALID_SNAP_ZXID
+          : latest.getIntervalEndpoint(IntervalEndpoint.START);
+
+      logger.info("Latest Zxid from storage: {}  from status: {}",
+          ZxidUtils.zxidToString(rZxid), ZxidUtils.zxidToString(backupPoint.getSnapZxid()));
+
+      if (rZxid != backupPoint.getSnapZxid()) {
+        synchronized (backupStatus) {
+          backupPoint.setSnapZxid(rZxid);
+          backupStatus.update(backupPoint);
+        }
+      }
+    }
+
+    /**
+     * Prepares snapshot files to back up and populate filesToBackup.
+     * Note that this implementation persists all snapshots instead of only persisting the
+     * latest snapshot.
+     * @throws IOException
+     */
+    protected void startIteration() throws IOException {
+      backupStats.setSnapshotBackupIterationStart();
+      filesToBackup.clear();
+
+      // Get all available snapshots excluding the ones whose lastProcessedZxid falls into the
+      // zxid range [0, backedupSnapZxid]
+      List<File> candidateSnapshots = snapLog.findValidSnapshots(0, backupPoint.getSnapZxid());
+      // Sort candidateSnapshots from oldest to newest
+      Collections.reverse(candidateSnapshots);
+
+      if (candidateSnapshots.size() == 0) {
+        // Either no snapshots or no newer snapshots to back up, so return
+        return;
+      }
+
+      for (int i = 0; i < candidateSnapshots.size(); i++) {
+        File f = candidateSnapshots.get(i);
+        ZxidRange zxidRange = Util.getZxidRangeFromName(f.getName(), Util.SNAP_PREFIX);
+
+        if (i == candidateSnapshots.size() - 1) {
+          // This is the most recent snapshot
+
+          // Handle backwards compatibility for snapshots that use old style naming where
+          // only the starting zxid is included.
+          // TODO: Can be removed after all snapshots being produced have ending zxid
+          if (!zxidRange.isHighPresent()) {
+            // Use the last logged zxid for the zxidRange for the latest snapshot as a best effort
+            // approach
+            // TODO: Because this is the best effort approach, the zxidRange will not be accurate
+            // TODO: Consider rewriting these latest snapshots to backup storage if necessary
+            // TODO: when we know the high zxid when we get a newer snapshot
+            long latestZxid = snapLog.getLastLoggedZxid();
+            long consistentAt = latestZxid == -1 ? zxidRange.getLow() : latestZxid;
+            zxidRange = new ZxidRange(zxidRange.getLow(), consistentAt);
+          }
+        } else {
+          // All newer snapshots that are not the most recent snapshot
+
+          // Handle backwards compatibility for snapshots that use old style naming where
+          // only the starting zxid is included.
+          // TODO: Can be removed after all snapshots being produced have ending zxid
+          if (!zxidRange.isHighPresent()) {
+            // ZxidRange will be [low, high] where high will be the zxid right before the next
+            // snapshot's lastProcessedZxid
+            long nextSnapshotStartZxid =
+                Util.getZxidFromName(candidateSnapshots.get(i + 1).getName(), Util.SNAP_PREFIX);
+            zxidRange = new ZxidRange(zxidRange.getLow(), nextSnapshotStartZxid - 1);
+          }
+        }
+
+        filesToBackup.add(new BackupFile(f, false, zxidRange));
+      }
+    }
+
+    protected void endIteration(boolean errorFree) {
+      backupStats.setSnapshotBackupIterationDone(errorFree);
+      filesToBackup.clear();
+    }
+
+    protected BackupFile getNextFileToBackup() {
+      if (filesToBackup.isEmpty()) {
+        return null;
+      }
+
+      return filesToBackup.remove(0);
+    }
+
+    protected void backupComplete(BackupFile file) throws IOException {
+      synchronized (backupStatus) {
+        backupPoint.setSnapZxid(file.getMinZxid());
+        backupStatus.update(backupPoint);
+      }
+      backupStats.incrementNumberOfSnapshotFilesBackedUpThisIteration();
+
+      logger.info("Updated backedup snap zxid to {}",
+          ZxidUtils.zxidToString(backupPoint.getSnapZxid()));
+    }
+  }
+
+  /**
+   * Constructor for the BackupManager.
+   * @param snapDir the snapshot directory
+   * @param dataLogDir the txnlog directory
+   * @param serverId the id of the zk server
+   * @param backupConfig the backup config object
+   * @throws IOException
+   */
+  public BackupManager(File snapDir, File dataLogDir, long serverId, BackupConfig backupConfig)
+      throws IOException {
+    logger = LoggerFactory.getLogger(BackupManager.class);
+    logger.info("snapDir={}", snapDir.getPath());
+    logger.info("dataLogDir={}", dataLogDir.getPath());
+    logger.info("backupStatusDir={}", backupConfig.getStatusDir().getPath());
+    logger.info("tmpDir={}", backupConfig.getTmpDir().getPath());
+    logger.info("backupIntervalInMinutes={}", backupConfig.getBackupIntervalInMinutes());
+    logger.info("serverId={}", serverId);
+    logger.info("namespace={}", backupConfig.getNamespace());
+
+    this.snapDir = snapDir;
+    this.dataLogDir = dataLogDir;
+    this.backupConfig = backupConfig;
+    // Note: tmpDir is namespaced
+    this.tmpDir = new File(String.join(File.separator, backupConfig.getTmpDir().getAbsolutePath(),
+        backupConfig.getNamespace()));
+    // Note: statusDir is namespaced
+    this.backupStatus = new BackupStatus(new File(String
+        .join(File.separator, backupConfig.getStatusDir().getAbsolutePath(),
+            backupConfig.getNamespace())));
+    this.backupIntervalInMilliseconds =
+        TimeUnit.MINUTES.toMillis(backupConfig.getBackupIntervalInMinutes());
+    this.serverId = serverId;
+    this.namespace = backupConfig.getNamespace();
+    try {
+      backupStorage = BackupUtil.createStorageProviderImpl(backupConfig);
+    } catch (ReflectiveOperationException e) {
+      throw new BackupException(e.getMessage(), e);
+    }
+    initialize();
+  }
+
+  /**
+   * Start the backup processes.
+   * @throws IOException
+   */
+  public synchronized void start() throws IOException {
+    logger.info("BackupManager starting.");
+
+    (new Thread(logBackup)).start();
+    (new Thread(snapBackup)).start();
+    if (timetableBackup != null) {
+      (new Thread(timetableBackup)).start();
+    }
+  }
+
+  /**
+   * Stop the backup processes.
+   */
+  public void stop() {
+    logger.info("BackupManager shutting down.");
+
+    synchronized (this) {
+      logBackup.shutdown();
+      snapBackup.shutdown();
+      logBackup = null;
+      snapBackup = null;
+
+      // Unregister MBeans so they can get GC-ed
+      if (backupBean != null) {
+        MBeanRegistry.getInstance().unregister(backupBean);
+        backupBean = null;
+      }
+      if (timetableBackupBean != null) {
+        timetableBackup.shutdown();
+        MBeanRegistry.getInstance().unregister(timetableBackupBean);
+        timetableBackupBean = null;
+      }
+    }
+  }
+
+  public BackupProcess getLogBackup() { return logBackup; }
+  public BackupProcess getSnapBackup() { return snapBackup; }
+  public BackupProcess getTimetableBackup() { return timetableBackup; }
+
+  public long getBackedupLogZxid() {
+    synchronized (backupStatus) {
+      return backupPoint.getLogZxid();
+    }
+  }
+
+  public long getBackedupSnapZxid() {
+    synchronized (backupStatus) {
+      return backupPoint.getSnapZxid();
+    }
+  }
+
+  public synchronized void initialize() throws IOException {
+    try {
+      backupStats = new BackupStats();
+      backupBean = new BackupBean(backupStats, serverId);
+      MBeanRegistry.getInstance().register(backupBean, null);
+      LOG.info("Registered BackupBean {} with JMX.", backupBean.getName());
+    } catch (JMException e) {
+      LOG.error("Failed to register BackupBean with JMX for namespace {} on server {}.", namespace,
+          serverId, e);
+      backupBean = null;
+    }
+
+    synchronized (backupStatus) {
+      backupStatus.createIfNeeded();
+      // backupPoint is only to be initialized once via backupStatus.read() in initialize()
+      backupPoint = backupStatus.read();
+    }
+
+    if (!tmpDir.exists()) {
+      if (!tmpDir.mkdirs()) {
+        String errorMsg = "BackupManager::initialize(): failed to create tmpDir!";
+        logger.error(errorMsg);
+        throw new IOException(errorMsg);
+      }
+    }
+
+    logBackup = new TxnLogBackup(new FileTxnSnapLog(dataLogDir, snapDir));
+    snapBackup = new SnapBackup(new FileTxnSnapLog(dataLogDir, snapDir));

Review Comment:
   *RESOURCE_LEAK:*  resource of type `org.apache.zookeeper.server.persistence.FileTxnLog` acquired by call to `FileTxnSnapLog(...)` at line 609 is not released after line 609.
   **Note**: potential exception at line 609
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621274&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621274&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621274&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621274&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621274&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/BackupManager.java:
##########
@@ -0,0 +1,661 @@
+/*
+ * 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.zookeeper.server.backup;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import javax.management.JMException;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.zookeeper.jmx.MBeanRegistry;
+import org.apache.zookeeper.server.backup.BackupUtil.BackupFileType;
+import org.apache.zookeeper.server.backup.BackupUtil.IntervalEndpoint;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.monitoring.BackupBean;
+import org.apache.zookeeper.server.backup.monitoring.BackupStats;
+import org.apache.zookeeper.server.backup.monitoring.TimetableBackupBean;
+import org.apache.zookeeper.server.backup.monitoring.TimetableBackupStats;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.timetable.TimetableBackup;
+import org.apache.zookeeper.server.persistence.FileTxnLog;
+import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
+import org.apache.zookeeper.server.persistence.TxnLog;
+import org.apache.zookeeper.server.persistence.Util;
+import org.apache.zookeeper.server.persistence.ZxidRange;
+import org.apache.zookeeper.server.util.ZxidUtils;
+import org.apache.zookeeper.txn.TxnHeader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class manages the backing up of txnlog and snap files to remote
+ * storage for longer term and durable retention than is possible on
+ * an ensemble server
+ */
+public class BackupManager {
+  private static final Logger LOG = LoggerFactory.getLogger(BackupManager.class);
+
+  private final Logger logger;
+  private final File snapDir;
+  private final File dataLogDir;
+  private final File tmpDir;
+  private final BackupConfig backupConfig;
+  private final long backupIntervalInMilliseconds;
+  private BackupProcess logBackup = null;
+  private BackupProcess snapBackup = null;
+
+  // backupStatus, backupLogZxid and backedupSnapZxid need to be access while synchronized
+  // on backupStatus.
+  private final BackupStatus backupStatus;
+  private BackupPoint backupPoint;
+
+  private final BackupStorageProvider backupStorage;
+  private final long serverId;
+  private final String namespace;
+
+  @VisibleForTesting
+  protected BackupBean backupBean = null;
+  protected TimetableBackupBean timetableBackupBean = null;
+
+  private BackupStats backupStats = null;
+
+  // Optional timetable backup
+  private BackupProcess timetableBackup = null;
+
+  /**
+   * Tracks a file that needs to be backed up, including temporary copies of the file
+   */
+  public static class BackupFile {
+    private final File file;
+    private final boolean isTemporary;
+    private final ZxidRange zxidRange;
+
+    /**
+     * Create an instance of a BackupFile for the given initial file and zxid range
+     * @param backupFile the initial/original file
+     * @param isTemporaryFile whether the file is a temporary file
+     * @param fileMinZxid the min zxid associated with this file
+     * @param fileMaxZxid the max zxid associated with this file
+     */
+    public BackupFile(File backupFile, boolean isTemporaryFile, long fileMinZxid, long fileMaxZxid) {
+      this(backupFile, isTemporaryFile, new ZxidRange(fileMinZxid, fileMaxZxid));
+    }
+
+    /**
+     * Create an instance of a BackupFile for the given initial file and zxid range
+     * @param backupFile the initial/original file
+     * @param isTemporaryFile whether the file is a temporary file
+     * @param zxidRange the zxid range associated with this file
+     */
+    public BackupFile(File backupFile, boolean isTemporaryFile, ZxidRange zxidRange) {
+      Preconditions.checkNotNull(zxidRange);
+
+      if (!zxidRange.isHighPresent()) {
+        throw new IllegalArgumentException("ZxidRange must have a high value");
+      }
+
+      this.file = backupFile;
+      this.isTemporary = isTemporaryFile;
+      this.zxidRange = zxidRange;
+    }
+
+    /**
+     * Perform cleanup including deleting temporary files.
+     */
+    public void cleanup() {
+      if (isTemporary && exists()) {
+        file.delete();
+      }
+    }
+
+    /**
+     * Whether the file representing the zxids exists
+     * @return whether the file represented exists
+     */
+    public boolean exists() {
+      return file != null && file.exists();
+    }
+
+    /**
+     * Get the current file (topmost on the stack)
+     * @return the current file
+     */
+    public File getFile() { return file; }
+
+    /**
+     * Get the zxid range associated with this file
+     * @return the zxid range
+     */
+    public ZxidRange getZxidRange() {
+      return zxidRange;
+    }
+
+    /**
+     * Get the min zxid associated with this file
+     * @return the min associated zxid
+     */
+    public long getMinZxid() { return zxidRange.getLow(); }
+
+    /**
+     * Get the max zxid associated with this file
+     * @return the max associated zxid
+     */
+    public long getMaxZxid() { return zxidRange.getHigh(); }
+
+    @Override
+    public String toString() {
+      return String.format("%s : %s : %d - %d",
+          file == null ? "[empty]" : file.getPath(),
+          isTemporary ? "temp" : "perm",
+          zxidRange.getLow(),
+          zxidRange.getHigh());
+    }
+  }
+
+  /**
+   * Implements txnlog specific logic for BackupProcess
+   */
+  protected class TxnLogBackup extends BackupProcess {
+    private long iterationEndPoint;
+    private final FileTxnSnapLog snapLog;
+
+    /**
+     * Constructor for TxnLogBackup
+     * @param snapLog the FileTxnSnapLog object to use
+     */
+    public TxnLogBackup(FileTxnSnapLog snapLog) {
+      super(LoggerFactory.getLogger(TxnLogBackup.class), BackupManager.this.backupStorage,
+          backupIntervalInMilliseconds);
+      this.snapLog = snapLog;
+    }
+
+    protected void initialize() throws IOException {
+      backupStorage.cleanupInvalidFiles(null);
+
+      BackupFileInfo latest =
+          BackupUtil.getLatest(backupStorage, BackupFileType.TXNLOG, IntervalEndpoint.START);
+
+      long rZxid = latest == null
+          ? BackupUtil.INVALID_LOG_ZXID
+          : latest.getIntervalEndpoint(IntervalEndpoint.END);
+
+      logger.info("Latest Zxid from storage: {}  from status: {}",
+          ZxidUtils.zxidToString(rZxid), ZxidUtils.zxidToString(backupPoint.getLogZxid()));
+
+      if (rZxid != backupPoint.getLogZxid()) {
+        synchronized (backupStatus) {
+          backupPoint.setLogZxid(rZxid);
+          backupStatus.update(backupPoint);
+        }
+      }
+    }
+
+    protected void startIteration() {
+      // Store the current last logged zxid.  This becomes the stopping point
+      // for the current iteration so we don't keep chasing our own tail as
+      // new transactions get written.
+      iterationEndPoint = snapLog.getLastLoggedZxid();
+      backupStats.setTxnLogBackupIterationStart();
+    }
+
+    protected void endIteration(boolean errorFree) {
+      backupStats.setTxnLogBackupIterationDone(errorFree);
+      iterationEndPoint = 0L;
+    }
+
+    /**
+     * Gets the next txnlog file to backup.  This is a temporary file created by copying
+     * all transaction from the previous backup point until the end zxid for this iteration, or
+     * a file indicating that some log records were lost.
+     * @return the file that needs to be backed-up.  The minZxid is the first
+     *      zxid contained in the file.  The maxZxid is the last zxid that is contained in the
+     *      file.
+     * @throws IOException
+     */
+    protected BackupFile getNextFileToBackup() throws IOException {
+      long startingZxid = backupStatus.read().getLogZxid() + 1;
+
+      // Don't keep chasing the tail so stop if past the last zxid at the time
+      // this iteration started.
+      if (startingZxid > iterationEndPoint) {
+        return null;
+      }
+
+      TxnLog.TxnIterator iter = null;
+      FileTxnLog newFile = null;
+      long lastZxid = -1;
+      int txnCopied = 0;
+      BackupFile ret;
+
+      logger.info("Creating backup file from zxid {}.", ZxidUtils.zxidToString(startingZxid));
+
+      try {
+        iter = snapLog.readTxnLog(startingZxid, true);
+
+        // Use a temp directory to avoid conflicts with live txnlog files
+        newFile = new FileTxnLog(tmpDir);
+
+        // TODO: Ideally, we should have have a way to prevent lost TxLogs
+        // Check for lost txnlogs; <=1 indicates that no backups have been done before so
+        // nothing can be considered lost.
+        // If a lost sequence is found then return a file whose name encodes the lost
+        // sequence and back that up so the backup store has a record of the lost sequence
+        if (startingZxid > 1 &&
+            iter.getHeader() != null &&
+            iter.getHeader().getZxid() > startingZxid) {
+
+          logger.error("TxnLog backups lost.  Required starting zxid={}  First available zxid={}",
+              ZxidUtils.zxidToString(startingZxid),
+              ZxidUtils.zxidToString(iter.getHeader().getZxid()));
+
+          String fileName = String.format("%s.%s",
+              BackupUtil.LOST_LOG_PREFIX,
+              Long.toHexString(startingZxid));
+          File lostZxidFile = new File(tmpDir, fileName);
+          lostZxidFile.createNewFile();
+
+          return new BackupFile(lostZxidFile, true, startingZxid, iter.getHeader().getZxid() - 1);
+        }
+
+        while (iter.getHeader() != null) {
+          TxnHeader hdr = iter.getHeader();
+
+          if (hdr.getZxid() > iterationEndPoint) {
+            break;
+          }
+
+          newFile.append(hdr, iter.getTxn());
+
+          // update position and count only AFTER the record has been successfully
+          // copied
+          lastZxid = hdr.getZxid();
+          txnCopied++;
+
+          iter.next();
+        }
+
+        ret = makeBackupFileFromCopiedLog(newFile, lastZxid);
+
+        if (ret != null) {
+          logger.info("Copied {} records starting at {} and ending at zxid {}.",
+              txnCopied,
+              ZxidUtils.zxidToString(ret.getMinZxid()),
+              ZxidUtils.zxidToString(ret.getMaxZxid()));
+        }
+
+      } catch (IOException e) {
+        logger.warn("Hit exception after {} records.  Exception: {} ", txnCopied, e.fillInStackTrace());
+
+        // If any records were copied return those and ignore the error.  Otherwise
+        // rethrow the error to be handled by the caller as a failed backup iteration.
+        if (txnCopied <= 0) {
+          throw e;
+        }
+
+        ret = makeBackupFileFromCopiedLog(newFile, lastZxid);
+      } finally {
+        if (iter != null) {
+          iter.close();
+        }
+
+        if (newFile != null) {
+          newFile.close();
+        }
+      }
+
+      return ret;
+    }
+
+    private BackupFile makeBackupFileFromCopiedLog(FileTxnLog backupTxnLog, long lastZxid) {
+
+      if (backupTxnLog == null) {
+        return null;
+      }
+
+      // The logFile gets initialized with the first transaction's zxid
+      File logFile = backupTxnLog.getCurrentFile();
+
+      if (logFile == null) {
+        return null;
+      }
+
+      long firstZxid = Util.getZxidFromName(logFile.getName(), Util.TXLOG_PREFIX);
+
+      if (lastZxid == -1) {
+        lastZxid = firstZxid;
+      }
+
+      return new BackupFile(logFile, true, new ZxidRange(firstZxid, lastZxid));
+    }
+
+    protected void backupComplete(BackupFile file) throws IOException {
+      synchronized (backupStatus) {
+        backupPoint.setLogZxid(file.getMaxZxid());
+        backupStatus.update(backupPoint);
+      }
+
+      logger.info("Updated backedup tnxlog zxid to {}",
+          ZxidUtils.zxidToString(backupPoint.getLogZxid()));
+    }
+  }
+
+  /**
+   * Implements snapshot specific logic for BackupProcess
+   */
+  protected class SnapBackup extends BackupProcess {
+    private final FileTxnSnapLog snapLog;
+    private final List<BackupFile> filesToBackup = new ArrayList<>();
+
+    /**
+     * Constructor for SnapBackup
+     * @param snapLog the FileTxnSnapLog object to use
+     */
+    public SnapBackup(FileTxnSnapLog snapLog) {
+      super(LoggerFactory.getLogger(SnapBackup.class), BackupManager.this.backupStorage,
+          backupIntervalInMilliseconds);
+      this.snapLog = snapLog;
+    }
+
+    protected void initialize() throws IOException {
+      backupStorage.cleanupInvalidFiles(null);
+
+      BackupFileInfo latest =
+          BackupUtil.getLatest(backupStorage, BackupFileType.SNAPSHOT, IntervalEndpoint.START);
+
+      long rZxid = latest == null
+          ? BackupUtil.INVALID_SNAP_ZXID
+          : latest.getIntervalEndpoint(IntervalEndpoint.START);
+
+      logger.info("Latest Zxid from storage: {}  from status: {}",
+          ZxidUtils.zxidToString(rZxid), ZxidUtils.zxidToString(backupPoint.getSnapZxid()));
+
+      if (rZxid != backupPoint.getSnapZxid()) {
+        synchronized (backupStatus) {
+          backupPoint.setSnapZxid(rZxid);
+          backupStatus.update(backupPoint);
+        }
+      }
+    }
+
+    /**
+     * Prepares snapshot files to back up and populate filesToBackup.
+     * Note that this implementation persists all snapshots instead of only persisting the
+     * latest snapshot.
+     * @throws IOException
+     */
+    protected void startIteration() throws IOException {
+      backupStats.setSnapshotBackupIterationStart();
+      filesToBackup.clear();
+
+      // Get all available snapshots excluding the ones whose lastProcessedZxid falls into the
+      // zxid range [0, backedupSnapZxid]
+      List<File> candidateSnapshots = snapLog.findValidSnapshots(0, backupPoint.getSnapZxid());
+      // Sort candidateSnapshots from oldest to newest
+      Collections.reverse(candidateSnapshots);
+
+      if (candidateSnapshots.size() == 0) {
+        // Either no snapshots or no newer snapshots to back up, so return
+        return;
+      }
+
+      for (int i = 0; i < candidateSnapshots.size(); i++) {
+        File f = candidateSnapshots.get(i);
+        ZxidRange zxidRange = Util.getZxidRangeFromName(f.getName(), Util.SNAP_PREFIX);
+
+        if (i == candidateSnapshots.size() - 1) {
+          // This is the most recent snapshot
+
+          // Handle backwards compatibility for snapshots that use old style naming where
+          // only the starting zxid is included.
+          // TODO: Can be removed after all snapshots being produced have ending zxid
+          if (!zxidRange.isHighPresent()) {
+            // Use the last logged zxid for the zxidRange for the latest snapshot as a best effort
+            // approach
+            // TODO: Because this is the best effort approach, the zxidRange will not be accurate
+            // TODO: Consider rewriting these latest snapshots to backup storage if necessary
+            // TODO: when we know the high zxid when we get a newer snapshot
+            long latestZxid = snapLog.getLastLoggedZxid();
+            long consistentAt = latestZxid == -1 ? zxidRange.getLow() : latestZxid;
+            zxidRange = new ZxidRange(zxidRange.getLow(), consistentAt);
+          }
+        } else {
+          // All newer snapshots that are not the most recent snapshot
+
+          // Handle backwards compatibility for snapshots that use old style naming where
+          // only the starting zxid is included.
+          // TODO: Can be removed after all snapshots being produced have ending zxid
+          if (!zxidRange.isHighPresent()) {
+            // ZxidRange will be [low, high] where high will be the zxid right before the next
+            // snapshot's lastProcessedZxid
+            long nextSnapshotStartZxid =
+                Util.getZxidFromName(candidateSnapshots.get(i + 1).getName(), Util.SNAP_PREFIX);
+            zxidRange = new ZxidRange(zxidRange.getLow(), nextSnapshotStartZxid - 1);
+          }
+        }
+
+        filesToBackup.add(new BackupFile(f, false, zxidRange));
+      }
+    }
+
+    protected void endIteration(boolean errorFree) {
+      backupStats.setSnapshotBackupIterationDone(errorFree);
+      filesToBackup.clear();
+    }
+
+    protected BackupFile getNextFileToBackup() {
+      if (filesToBackup.isEmpty()) {
+        return null;
+      }
+
+      return filesToBackup.remove(0);
+    }
+
+    protected void backupComplete(BackupFile file) throws IOException {
+      synchronized (backupStatus) {
+        backupPoint.setSnapZxid(file.getMinZxid());
+        backupStatus.update(backupPoint);
+      }
+      backupStats.incrementNumberOfSnapshotFilesBackedUpThisIteration();
+
+      logger.info("Updated backedup snap zxid to {}",
+          ZxidUtils.zxidToString(backupPoint.getSnapZxid()));
+    }
+  }
+
+  /**
+   * Constructor for the BackupManager.
+   * @param snapDir the snapshot directory
+   * @param dataLogDir the txnlog directory
+   * @param serverId the id of the zk server
+   * @param backupConfig the backup config object
+   * @throws IOException
+   */
+  public BackupManager(File snapDir, File dataLogDir, long serverId, BackupConfig backupConfig)
+      throws IOException {
+    logger = LoggerFactory.getLogger(BackupManager.class);
+    logger.info("snapDir={}", snapDir.getPath());
+    logger.info("dataLogDir={}", dataLogDir.getPath());
+    logger.info("backupStatusDir={}", backupConfig.getStatusDir().getPath());
+    logger.info("tmpDir={}", backupConfig.getTmpDir().getPath());
+    logger.info("backupIntervalInMinutes={}", backupConfig.getBackupIntervalInMinutes());
+    logger.info("serverId={}", serverId);
+    logger.info("namespace={}", backupConfig.getNamespace());
+
+    this.snapDir = snapDir;
+    this.dataLogDir = dataLogDir;
+    this.backupConfig = backupConfig;
+    // Note: tmpDir is namespaced
+    this.tmpDir = new File(String.join(File.separator, backupConfig.getTmpDir().getAbsolutePath(),
+        backupConfig.getNamespace()));
+    // Note: statusDir is namespaced
+    this.backupStatus = new BackupStatus(new File(String
+        .join(File.separator, backupConfig.getStatusDir().getAbsolutePath(),
+            backupConfig.getNamespace())));
+    this.backupIntervalInMilliseconds =
+        TimeUnit.MINUTES.toMillis(backupConfig.getBackupIntervalInMinutes());
+    this.serverId = serverId;
+    this.namespace = backupConfig.getNamespace();
+    try {
+      backupStorage = BackupUtil.createStorageProviderImpl(backupConfig);
+    } catch (ReflectiveOperationException e) {
+      throw new BackupException(e.getMessage(), e);
+    }
+    initialize();
+  }
+
+  /**
+   * Start the backup processes.
+   * @throws IOException
+   */
+  public synchronized void start() throws IOException {
+    logger.info("BackupManager starting.");
+
+    (new Thread(logBackup)).start();
+    (new Thread(snapBackup)).start();
+    if (timetableBackup != null) {
+      (new Thread(timetableBackup)).start();
+    }
+  }
+
+  /**
+   * Stop the backup processes.
+   */
+  public void stop() {
+    logger.info("BackupManager shutting down.");
+
+    synchronized (this) {
+      logBackup.shutdown();
+      snapBackup.shutdown();
+      logBackup = null;
+      snapBackup = null;
+
+      // Unregister MBeans so they can get GC-ed
+      if (backupBean != null) {
+        MBeanRegistry.getInstance().unregister(backupBean);
+        backupBean = null;
+      }
+      if (timetableBackupBean != null) {
+        timetableBackup.shutdown();
+        MBeanRegistry.getInstance().unregister(timetableBackupBean);
+        timetableBackupBean = null;
+      }
+    }
+  }
+
+  public BackupProcess getLogBackup() { return logBackup; }

Review Comment:
   *THREAD_SAFETY_VIOLATION:*  Read/Write race. Non-private method `BackupManager.getLogBackup()` reads without synchronization from `this.logBackup`. Potentially races with write in method `BackupManager.stop()`.
    Reporting because another access to the same memory occurs on a background thread, although this access may not.
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621378&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621378&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621378&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621378&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621378&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/BackupManager.java:
##########
@@ -0,0 +1,661 @@
+/*
+ * 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.zookeeper.server.backup;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import javax.management.JMException;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.zookeeper.jmx.MBeanRegistry;
+import org.apache.zookeeper.server.backup.BackupUtil.BackupFileType;
+import org.apache.zookeeper.server.backup.BackupUtil.IntervalEndpoint;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.monitoring.BackupBean;
+import org.apache.zookeeper.server.backup.monitoring.BackupStats;
+import org.apache.zookeeper.server.backup.monitoring.TimetableBackupBean;
+import org.apache.zookeeper.server.backup.monitoring.TimetableBackupStats;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.timetable.TimetableBackup;
+import org.apache.zookeeper.server.persistence.FileTxnLog;
+import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
+import org.apache.zookeeper.server.persistence.TxnLog;
+import org.apache.zookeeper.server.persistence.Util;
+import org.apache.zookeeper.server.persistence.ZxidRange;
+import org.apache.zookeeper.server.util.ZxidUtils;
+import org.apache.zookeeper.txn.TxnHeader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class manages the backing up of txnlog and snap files to remote
+ * storage for longer term and durable retention than is possible on
+ * an ensemble server
+ */
+public class BackupManager {
+  private static final Logger LOG = LoggerFactory.getLogger(BackupManager.class);
+
+  private final Logger logger;
+  private final File snapDir;
+  private final File dataLogDir;
+  private final File tmpDir;
+  private final BackupConfig backupConfig;
+  private final long backupIntervalInMilliseconds;
+  private BackupProcess logBackup = null;
+  private BackupProcess snapBackup = null;
+
+  // backupStatus, backupLogZxid and backedupSnapZxid need to be access while synchronized
+  // on backupStatus.
+  private final BackupStatus backupStatus;
+  private BackupPoint backupPoint;
+
+  private final BackupStorageProvider backupStorage;
+  private final long serverId;
+  private final String namespace;
+
+  @VisibleForTesting
+  protected BackupBean backupBean = null;
+  protected TimetableBackupBean timetableBackupBean = null;
+
+  private BackupStats backupStats = null;
+
+  // Optional timetable backup
+  private BackupProcess timetableBackup = null;
+
+  /**
+   * Tracks a file that needs to be backed up, including temporary copies of the file
+   */
+  public static class BackupFile {
+    private final File file;
+    private final boolean isTemporary;
+    private final ZxidRange zxidRange;
+
+    /**
+     * Create an instance of a BackupFile for the given initial file and zxid range
+     * @param backupFile the initial/original file
+     * @param isTemporaryFile whether the file is a temporary file
+     * @param fileMinZxid the min zxid associated with this file
+     * @param fileMaxZxid the max zxid associated with this file
+     */
+    public BackupFile(File backupFile, boolean isTemporaryFile, long fileMinZxid, long fileMaxZxid) {
+      this(backupFile, isTemporaryFile, new ZxidRange(fileMinZxid, fileMaxZxid));
+    }
+
+    /**
+     * Create an instance of a BackupFile for the given initial file and zxid range
+     * @param backupFile the initial/original file
+     * @param isTemporaryFile whether the file is a temporary file
+     * @param zxidRange the zxid range associated with this file
+     */
+    public BackupFile(File backupFile, boolean isTemporaryFile, ZxidRange zxidRange) {
+      Preconditions.checkNotNull(zxidRange);
+
+      if (!zxidRange.isHighPresent()) {
+        throw new IllegalArgumentException("ZxidRange must have a high value");
+      }
+
+      this.file = backupFile;
+      this.isTemporary = isTemporaryFile;
+      this.zxidRange = zxidRange;
+    }
+
+    /**
+     * Perform cleanup including deleting temporary files.
+     */
+    public void cleanup() {
+      if (isTemporary && exists()) {
+        file.delete();
+      }
+    }
+
+    /**
+     * Whether the file representing the zxids exists
+     * @return whether the file represented exists
+     */
+    public boolean exists() {
+      return file != null && file.exists();
+    }
+
+    /**
+     * Get the current file (topmost on the stack)
+     * @return the current file
+     */
+    public File getFile() { return file; }
+
+    /**
+     * Get the zxid range associated with this file
+     * @return the zxid range
+     */
+    public ZxidRange getZxidRange() {
+      return zxidRange;
+    }
+
+    /**
+     * Get the min zxid associated with this file
+     * @return the min associated zxid
+     */
+    public long getMinZxid() { return zxidRange.getLow(); }
+
+    /**
+     * Get the max zxid associated with this file
+     * @return the max associated zxid
+     */
+    public long getMaxZxid() { return zxidRange.getHigh(); }
+
+    @Override
+    public String toString() {
+      return String.format("%s : %s : %d - %d",
+          file == null ? "[empty]" : file.getPath(),
+          isTemporary ? "temp" : "perm",
+          zxidRange.getLow(),
+          zxidRange.getHigh());
+    }
+  }
+
+  /**
+   * Implements txnlog specific logic for BackupProcess
+   */
+  protected class TxnLogBackup extends BackupProcess {
+    private long iterationEndPoint;
+    private final FileTxnSnapLog snapLog;
+
+    /**
+     * Constructor for TxnLogBackup
+     * @param snapLog the FileTxnSnapLog object to use
+     */
+    public TxnLogBackup(FileTxnSnapLog snapLog) {
+      super(LoggerFactory.getLogger(TxnLogBackup.class), BackupManager.this.backupStorage,
+          backupIntervalInMilliseconds);
+      this.snapLog = snapLog;
+    }
+
+    protected void initialize() throws IOException {
+      backupStorage.cleanupInvalidFiles(null);
+
+      BackupFileInfo latest =
+          BackupUtil.getLatest(backupStorage, BackupFileType.TXNLOG, IntervalEndpoint.START);
+
+      long rZxid = latest == null
+          ? BackupUtil.INVALID_LOG_ZXID
+          : latest.getIntervalEndpoint(IntervalEndpoint.END);
+
+      logger.info("Latest Zxid from storage: {}  from status: {}",
+          ZxidUtils.zxidToString(rZxid), ZxidUtils.zxidToString(backupPoint.getLogZxid()));
+
+      if (rZxid != backupPoint.getLogZxid()) {
+        synchronized (backupStatus) {
+          backupPoint.setLogZxid(rZxid);
+          backupStatus.update(backupPoint);
+        }
+      }
+    }
+
+    protected void startIteration() {
+      // Store the current last logged zxid.  This becomes the stopping point
+      // for the current iteration so we don't keep chasing our own tail as
+      // new transactions get written.
+      iterationEndPoint = snapLog.getLastLoggedZxid();
+      backupStats.setTxnLogBackupIterationStart();
+    }
+
+    protected void endIteration(boolean errorFree) {
+      backupStats.setTxnLogBackupIterationDone(errorFree);
+      iterationEndPoint = 0L;
+    }
+
+    /**
+     * Gets the next txnlog file to backup.  This is a temporary file created by copying
+     * all transaction from the previous backup point until the end zxid for this iteration, or
+     * a file indicating that some log records were lost.
+     * @return the file that needs to be backed-up.  The minZxid is the first
+     *      zxid contained in the file.  The maxZxid is the last zxid that is contained in the
+     *      file.
+     * @throws IOException
+     */
+    protected BackupFile getNextFileToBackup() throws IOException {
+      long startingZxid = backupStatus.read().getLogZxid() + 1;
+
+      // Don't keep chasing the tail so stop if past the last zxid at the time
+      // this iteration started.
+      if (startingZxid > iterationEndPoint) {
+        return null;
+      }
+
+      TxnLog.TxnIterator iter = null;
+      FileTxnLog newFile = null;
+      long lastZxid = -1;
+      int txnCopied = 0;
+      BackupFile ret;
+
+      logger.info("Creating backup file from zxid {}.", ZxidUtils.zxidToString(startingZxid));
+
+      try {
+        iter = snapLog.readTxnLog(startingZxid, true);
+
+        // Use a temp directory to avoid conflicts with live txnlog files
+        newFile = new FileTxnLog(tmpDir);
+
+        // TODO: Ideally, we should have have a way to prevent lost TxLogs
+        // Check for lost txnlogs; <=1 indicates that no backups have been done before so
+        // nothing can be considered lost.
+        // If a lost sequence is found then return a file whose name encodes the lost
+        // sequence and back that up so the backup store has a record of the lost sequence
+        if (startingZxid > 1 &&
+            iter.getHeader() != null &&
+            iter.getHeader().getZxid() > startingZxid) {
+
+          logger.error("TxnLog backups lost.  Required starting zxid={}  First available zxid={}",
+              ZxidUtils.zxidToString(startingZxid),
+              ZxidUtils.zxidToString(iter.getHeader().getZxid()));
+
+          String fileName = String.format("%s.%s",
+              BackupUtil.LOST_LOG_PREFIX,
+              Long.toHexString(startingZxid));
+          File lostZxidFile = new File(tmpDir, fileName);
+          lostZxidFile.createNewFile();
+
+          return new BackupFile(lostZxidFile, true, startingZxid, iter.getHeader().getZxid() - 1);
+        }
+
+        while (iter.getHeader() != null) {
+          TxnHeader hdr = iter.getHeader();
+
+          if (hdr.getZxid() > iterationEndPoint) {
+            break;
+          }
+
+          newFile.append(hdr, iter.getTxn());
+
+          // update position and count only AFTER the record has been successfully
+          // copied
+          lastZxid = hdr.getZxid();
+          txnCopied++;
+
+          iter.next();
+        }
+
+        ret = makeBackupFileFromCopiedLog(newFile, lastZxid);
+
+        if (ret != null) {
+          logger.info("Copied {} records starting at {} and ending at zxid {}.",
+              txnCopied,
+              ZxidUtils.zxidToString(ret.getMinZxid()),
+              ZxidUtils.zxidToString(ret.getMaxZxid()));
+        }
+
+      } catch (IOException e) {
+        logger.warn("Hit exception after {} records.  Exception: {} ", txnCopied, e.fillInStackTrace());
+
+        // If any records were copied return those and ignore the error.  Otherwise
+        // rethrow the error to be handled by the caller as a failed backup iteration.
+        if (txnCopied <= 0) {
+          throw e;
+        }
+
+        ret = makeBackupFileFromCopiedLog(newFile, lastZxid);
+      } finally {
+        if (iter != null) {
+          iter.close();
+        }
+
+        if (newFile != null) {
+          newFile.close();
+        }
+      }
+
+      return ret;
+    }
+
+    private BackupFile makeBackupFileFromCopiedLog(FileTxnLog backupTxnLog, long lastZxid) {
+
+      if (backupTxnLog == null) {
+        return null;
+      }
+
+      // The logFile gets initialized with the first transaction's zxid
+      File logFile = backupTxnLog.getCurrentFile();
+
+      if (logFile == null) {
+        return null;
+      }
+
+      long firstZxid = Util.getZxidFromName(logFile.getName(), Util.TXLOG_PREFIX);
+
+      if (lastZxid == -1) {
+        lastZxid = firstZxid;
+      }
+
+      return new BackupFile(logFile, true, new ZxidRange(firstZxid, lastZxid));
+    }
+
+    protected void backupComplete(BackupFile file) throws IOException {
+      synchronized (backupStatus) {
+        backupPoint.setLogZxid(file.getMaxZxid());
+        backupStatus.update(backupPoint);
+      }
+
+      logger.info("Updated backedup tnxlog zxid to {}",
+          ZxidUtils.zxidToString(backupPoint.getLogZxid()));
+    }
+  }
+
+  /**
+   * Implements snapshot specific logic for BackupProcess
+   */
+  protected class SnapBackup extends BackupProcess {
+    private final FileTxnSnapLog snapLog;
+    private final List<BackupFile> filesToBackup = new ArrayList<>();
+
+    /**
+     * Constructor for SnapBackup
+     * @param snapLog the FileTxnSnapLog object to use
+     */
+    public SnapBackup(FileTxnSnapLog snapLog) {
+      super(LoggerFactory.getLogger(SnapBackup.class), BackupManager.this.backupStorage,
+          backupIntervalInMilliseconds);
+      this.snapLog = snapLog;
+    }
+
+    protected void initialize() throws IOException {
+      backupStorage.cleanupInvalidFiles(null);
+
+      BackupFileInfo latest =
+          BackupUtil.getLatest(backupStorage, BackupFileType.SNAPSHOT, IntervalEndpoint.START);
+
+      long rZxid = latest == null
+          ? BackupUtil.INVALID_SNAP_ZXID
+          : latest.getIntervalEndpoint(IntervalEndpoint.START);
+
+      logger.info("Latest Zxid from storage: {}  from status: {}",
+          ZxidUtils.zxidToString(rZxid), ZxidUtils.zxidToString(backupPoint.getSnapZxid()));
+
+      if (rZxid != backupPoint.getSnapZxid()) {
+        synchronized (backupStatus) {
+          backupPoint.setSnapZxid(rZxid);
+          backupStatus.update(backupPoint);
+        }
+      }
+    }
+
+    /**
+     * Prepares snapshot files to back up and populate filesToBackup.
+     * Note that this implementation persists all snapshots instead of only persisting the
+     * latest snapshot.
+     * @throws IOException
+     */
+    protected void startIteration() throws IOException {
+      backupStats.setSnapshotBackupIterationStart();

Review Comment:
   *THREAD_SAFETY_VIOLATION:*  Unprotected write. Non-private method `BackupManager$SnapBackup.startIteration()` indirectly writes to field `this.this$0.backupStats.numberOfSnapshotFilesBackedUpThisIteration` outside of synchronization.
    Reporting because another access to the same memory occurs on a background thread, although this access may not.
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621396&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621396&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621396&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621396&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621396&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/BackupManager.java:
##########
@@ -0,0 +1,661 @@
+/*
+ * 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.zookeeper.server.backup;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import javax.management.JMException;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.zookeeper.jmx.MBeanRegistry;
+import org.apache.zookeeper.server.backup.BackupUtil.BackupFileType;
+import org.apache.zookeeper.server.backup.BackupUtil.IntervalEndpoint;
+import org.apache.zookeeper.server.backup.exception.BackupException;
+import org.apache.zookeeper.server.backup.monitoring.BackupBean;
+import org.apache.zookeeper.server.backup.monitoring.BackupStats;
+import org.apache.zookeeper.server.backup.monitoring.TimetableBackupBean;
+import org.apache.zookeeper.server.backup.monitoring.TimetableBackupStats;
+import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
+import org.apache.zookeeper.server.backup.timetable.TimetableBackup;
+import org.apache.zookeeper.server.persistence.FileTxnLog;
+import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
+import org.apache.zookeeper.server.persistence.TxnLog;
+import org.apache.zookeeper.server.persistence.Util;
+import org.apache.zookeeper.server.persistence.ZxidRange;
+import org.apache.zookeeper.server.util.ZxidUtils;
+import org.apache.zookeeper.txn.TxnHeader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class manages the backing up of txnlog and snap files to remote
+ * storage for longer term and durable retention than is possible on
+ * an ensemble server
+ */
+public class BackupManager {
+  private static final Logger LOG = LoggerFactory.getLogger(BackupManager.class);
+
+  private final Logger logger;
+  private final File snapDir;
+  private final File dataLogDir;
+  private final File tmpDir;
+  private final BackupConfig backupConfig;
+  private final long backupIntervalInMilliseconds;
+  private BackupProcess logBackup = null;
+  private BackupProcess snapBackup = null;
+
+  // backupStatus, backupLogZxid and backedupSnapZxid need to be access while synchronized
+  // on backupStatus.
+  private final BackupStatus backupStatus;
+  private BackupPoint backupPoint;
+
+  private final BackupStorageProvider backupStorage;
+  private final long serverId;
+  private final String namespace;
+
+  @VisibleForTesting
+  protected BackupBean backupBean = null;
+  protected TimetableBackupBean timetableBackupBean = null;
+
+  private BackupStats backupStats = null;
+
+  // Optional timetable backup
+  private BackupProcess timetableBackup = null;
+
+  /**
+   * Tracks a file that needs to be backed up, including temporary copies of the file
+   */
+  public static class BackupFile {
+    private final File file;
+    private final boolean isTemporary;
+    private final ZxidRange zxidRange;
+
+    /**
+     * Create an instance of a BackupFile for the given initial file and zxid range
+     * @param backupFile the initial/original file
+     * @param isTemporaryFile whether the file is a temporary file
+     * @param fileMinZxid the min zxid associated with this file
+     * @param fileMaxZxid the max zxid associated with this file
+     */
+    public BackupFile(File backupFile, boolean isTemporaryFile, long fileMinZxid, long fileMaxZxid) {
+      this(backupFile, isTemporaryFile, new ZxidRange(fileMinZxid, fileMaxZxid));
+    }
+
+    /**
+     * Create an instance of a BackupFile for the given initial file and zxid range
+     * @param backupFile the initial/original file
+     * @param isTemporaryFile whether the file is a temporary file
+     * @param zxidRange the zxid range associated with this file
+     */
+    public BackupFile(File backupFile, boolean isTemporaryFile, ZxidRange zxidRange) {
+      Preconditions.checkNotNull(zxidRange);
+
+      if (!zxidRange.isHighPresent()) {
+        throw new IllegalArgumentException("ZxidRange must have a high value");
+      }
+
+      this.file = backupFile;
+      this.isTemporary = isTemporaryFile;
+      this.zxidRange = zxidRange;
+    }
+
+    /**
+     * Perform cleanup including deleting temporary files.
+     */
+    public void cleanup() {
+      if (isTemporary && exists()) {
+        file.delete();
+      }
+    }
+
+    /**
+     * Whether the file representing the zxids exists
+     * @return whether the file represented exists
+     */
+    public boolean exists() {
+      return file != null && file.exists();
+    }
+
+    /**
+     * Get the current file (topmost on the stack)
+     * @return the current file
+     */
+    public File getFile() { return file; }
+
+    /**
+     * Get the zxid range associated with this file
+     * @return the zxid range
+     */
+    public ZxidRange getZxidRange() {
+      return zxidRange;
+    }
+
+    /**
+     * Get the min zxid associated with this file
+     * @return the min associated zxid
+     */
+    public long getMinZxid() { return zxidRange.getLow(); }
+
+    /**
+     * Get the max zxid associated with this file
+     * @return the max associated zxid
+     */
+    public long getMaxZxid() { return zxidRange.getHigh(); }
+
+    @Override
+    public String toString() {
+      return String.format("%s : %s : %d - %d",
+          file == null ? "[empty]" : file.getPath(),
+          isTemporary ? "temp" : "perm",
+          zxidRange.getLow(),
+          zxidRange.getHigh());
+    }
+  }
+
+  /**
+   * Implements txnlog specific logic for BackupProcess
+   */
+  protected class TxnLogBackup extends BackupProcess {
+    private long iterationEndPoint;
+    private final FileTxnSnapLog snapLog;
+
+    /**
+     * Constructor for TxnLogBackup
+     * @param snapLog the FileTxnSnapLog object to use
+     */
+    public TxnLogBackup(FileTxnSnapLog snapLog) {
+      super(LoggerFactory.getLogger(TxnLogBackup.class), BackupManager.this.backupStorage,
+          backupIntervalInMilliseconds);
+      this.snapLog = snapLog;
+    }
+
+    protected void initialize() throws IOException {
+      backupStorage.cleanupInvalidFiles(null);
+
+      BackupFileInfo latest =
+          BackupUtil.getLatest(backupStorage, BackupFileType.TXNLOG, IntervalEndpoint.START);
+
+      long rZxid = latest == null
+          ? BackupUtil.INVALID_LOG_ZXID
+          : latest.getIntervalEndpoint(IntervalEndpoint.END);
+
+      logger.info("Latest Zxid from storage: {}  from status: {}",
+          ZxidUtils.zxidToString(rZxid), ZxidUtils.zxidToString(backupPoint.getLogZxid()));
+
+      if (rZxid != backupPoint.getLogZxid()) {
+        synchronized (backupStatus) {
+          backupPoint.setLogZxid(rZxid);
+          backupStatus.update(backupPoint);
+        }
+      }
+    }
+
+    protected void startIteration() {
+      // Store the current last logged zxid.  This becomes the stopping point
+      // for the current iteration so we don't keep chasing our own tail as
+      // new transactions get written.
+      iterationEndPoint = snapLog.getLastLoggedZxid();
+      backupStats.setTxnLogBackupIterationStart();
+    }
+
+    protected void endIteration(boolean errorFree) {
+      backupStats.setTxnLogBackupIterationDone(errorFree);
+      iterationEndPoint = 0L;
+    }
+
+    /**
+     * Gets the next txnlog file to backup.  This is a temporary file created by copying
+     * all transaction from the previous backup point until the end zxid for this iteration, or
+     * a file indicating that some log records were lost.
+     * @return the file that needs to be backed-up.  The minZxid is the first
+     *      zxid contained in the file.  The maxZxid is the last zxid that is contained in the
+     *      file.
+     * @throws IOException
+     */
+    protected BackupFile getNextFileToBackup() throws IOException {
+      long startingZxid = backupStatus.read().getLogZxid() + 1;
+
+      // Don't keep chasing the tail so stop if past the last zxid at the time
+      // this iteration started.
+      if (startingZxid > iterationEndPoint) {
+        return null;
+      }
+
+      TxnLog.TxnIterator iter = null;
+      FileTxnLog newFile = null;
+      long lastZxid = -1;
+      int txnCopied = 0;
+      BackupFile ret;
+
+      logger.info("Creating backup file from zxid {}.", ZxidUtils.zxidToString(startingZxid));
+
+      try {
+        iter = snapLog.readTxnLog(startingZxid, true);
+
+        // Use a temp directory to avoid conflicts with live txnlog files
+        newFile = new FileTxnLog(tmpDir);
+
+        // TODO: Ideally, we should have have a way to prevent lost TxLogs
+        // Check for lost txnlogs; <=1 indicates that no backups have been done before so
+        // nothing can be considered lost.
+        // If a lost sequence is found then return a file whose name encodes the lost
+        // sequence and back that up so the backup store has a record of the lost sequence
+        if (startingZxid > 1 &&
+            iter.getHeader() != null &&
+            iter.getHeader().getZxid() > startingZxid) {
+
+          logger.error("TxnLog backups lost.  Required starting zxid={}  First available zxid={}",
+              ZxidUtils.zxidToString(startingZxid),
+              ZxidUtils.zxidToString(iter.getHeader().getZxid()));
+
+          String fileName = String.format("%s.%s",
+              BackupUtil.LOST_LOG_PREFIX,
+              Long.toHexString(startingZxid));
+          File lostZxidFile = new File(tmpDir, fileName);
+          lostZxidFile.createNewFile();
+
+          return new BackupFile(lostZxidFile, true, startingZxid, iter.getHeader().getZxid() - 1);
+        }
+
+        while (iter.getHeader() != null) {
+          TxnHeader hdr = iter.getHeader();
+
+          if (hdr.getZxid() > iterationEndPoint) {
+            break;
+          }
+
+          newFile.append(hdr, iter.getTxn());
+
+          // update position and count only AFTER the record has been successfully
+          // copied
+          lastZxid = hdr.getZxid();
+          txnCopied++;
+
+          iter.next();
+        }
+
+        ret = makeBackupFileFromCopiedLog(newFile, lastZxid);
+
+        if (ret != null) {
+          logger.info("Copied {} records starting at {} and ending at zxid {}.",
+              txnCopied,
+              ZxidUtils.zxidToString(ret.getMinZxid()),
+              ZxidUtils.zxidToString(ret.getMaxZxid()));
+        }
+
+      } catch (IOException e) {
+        logger.warn("Hit exception after {} records.  Exception: {} ", txnCopied, e.fillInStackTrace());
+
+        // If any records were copied return those and ignore the error.  Otherwise
+        // rethrow the error to be handled by the caller as a failed backup iteration.
+        if (txnCopied <= 0) {
+          throw e;
+        }
+
+        ret = makeBackupFileFromCopiedLog(newFile, lastZxid);
+      } finally {
+        if (iter != null) {
+          iter.close();
+        }
+
+        if (newFile != null) {
+          newFile.close();
+        }
+      }
+
+      return ret;
+    }
+
+    private BackupFile makeBackupFileFromCopiedLog(FileTxnLog backupTxnLog, long lastZxid) {
+
+      if (backupTxnLog == null) {
+        return null;
+      }
+
+      // The logFile gets initialized with the first transaction's zxid
+      File logFile = backupTxnLog.getCurrentFile();
+
+      if (logFile == null) {
+        return null;
+      }
+
+      long firstZxid = Util.getZxidFromName(logFile.getName(), Util.TXLOG_PREFIX);
+
+      if (lastZxid == -1) {
+        lastZxid = firstZxid;
+      }
+
+      return new BackupFile(logFile, true, new ZxidRange(firstZxid, lastZxid));
+    }
+
+    protected void backupComplete(BackupFile file) throws IOException {
+      synchronized (backupStatus) {
+        backupPoint.setLogZxid(file.getMaxZxid());
+        backupStatus.update(backupPoint);
+      }
+
+      logger.info("Updated backedup tnxlog zxid to {}",
+          ZxidUtils.zxidToString(backupPoint.getLogZxid()));

Review Comment:
   *THREAD_SAFETY_VIOLATION:*  Read/Write race. Non-private method `BackupManager$TxnLogBackup.backupComplete(...)` indirectly reads without synchronization from `this.this$0.backupPoint.logZxid`. Potentially races with write in method `BackupManager$TxnLogBackup.backupComplete(...)`.
    Reporting because this access may occur on a background thread.
   
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=226621410&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=226621410&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621410&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=226621410&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=226621410&lift_comment_rating=5) ]



-- 
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: notifications-unsubscribe@zookeeper.apache.org

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


[GitHub] [zookeeper] narendly commented on pull request #1883: Add backup and restore with timetable

Posted by GitBox <gi...@apache.org>.
narendly commented on PR #1883:
URL: https://github.com/apache/zookeeper/pull/1883#issuecomment-1153997591

   @eolivelli Will do - thanks!


-- 
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: notifications-unsubscribe@zookeeper.apache.org

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


[GitHub] [zookeeper] eolivelli commented on pull request #1883: Add backup and restore with timetable

Posted by GitBox <gi...@apache.org>.
eolivelli commented on PR #1883:
URL: https://github.com/apache/zookeeper/pull/1883#issuecomment-1305732575

   What's the status of this patch ?


-- 
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: notifications-unsubscribe@zookeeper.apache.org

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