swamirishi commented on code in PR #8477:
URL: https://github.com/apache/ozone/pull/8477#discussion_r2141143956


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServletInodeBasedXfer.java:
##########
@@ -0,0 +1,359 @@
+/*
+ * 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.hadoop.ozone.om;
+
+import static org.apache.hadoop.hdds.utils.Archiver.includeFile;
+import static org.apache.hadoop.hdds.utils.Archiver.linkAndIncludeFile;
+import static org.apache.hadoop.hdds.utils.Archiver.tar;
+import static 
org.apache.hadoop.ozone.OzoneConsts.OZONE_DB_CHECKPOINT_REQUEST_TO_EXCLUDE_SST;
+import static 
org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_RATIS_SNAPSHOT_MAX_TOTAL_SST_SIZE_DEFAULT;
+import static 
org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_RATIS_SNAPSHOT_MAX_TOTAL_SST_SIZE_KEY;
+import static 
org.apache.hadoop.ozone.om.codec.OMDBCheckpointUtils.includeSnapshotData;
+import static 
org.apache.hadoop.ozone.om.codec.OMDBCheckpointUtils.logEstimatedTarballSize;
+import static org.apache.hadoop.ozone.om.snapshot.OmSnapshotUtils.DATA_PREFIX;
+import static org.apache.hadoop.ozone.om.snapshot.OmSnapshotUtils.DATA_SUFFIX;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardOpenOption;
+import java.time.Duration;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Stream;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import org.apache.commons.compress.archivers.ArchiveOutputStream;
+import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.recon.ReconConfig;
+import org.apache.hadoop.hdds.utils.DBCheckpointServlet;
+import org.apache.hadoop.hdds.utils.db.DBCheckpoint;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.lock.BootstrapStateHandler;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.snapshot.OmSnapshotUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.ozone.rocksdiff.RocksDBCheckpointDiffer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Specialized OMDBCheckpointServlet implementation that transfers Ozone 
Manager
+ * database checkpoints using inode-based deduplication.
+ * <p>
+ * This servlet constructs checkpoint archives by examining file inodes,
+ * ensuring that files with the same inode (i.e., hardlinks or duplicates)
+ * are only transferred once. It maintains mappings from inode IDs to file
+ * paths, manages hardlink information, and enforces snapshot and SST file
+ * size constraints as needed.
+ * <p>
+ * This approach optimizes checkpoint streaming by reducing redundant data
+ * transfer, especially in environments where RocksDB and snapshotting result
+ * in multiple hardlinks to the same physical data.
+ */
+public class OMDBCheckpointServletInodeBasedXfer extends DBCheckpointServlet {
+
+  protected static final Logger LOG =
+      LoggerFactory.getLogger(OMDBCheckpointServletInodeBasedXfer.class);
+  private static final long serialVersionUID = 1L;
+
+  @Override
+  public void init() throws ServletException {
+    OzoneManager om = (OzoneManager) getServletContext()
+        .getAttribute(OzoneConsts.OM_CONTEXT_ATTRIBUTE);
+
+    if (om == null) {
+      LOG.error("Unable to initialize OMDBCheckpointServlet. OM is null");
+      return;
+    }
+
+    OzoneConfiguration conf = getConf();
+    // Only Ozone Admins and Recon are allowed
+    Collection<String> allowedUsers =
+        new LinkedHashSet<>(om.getOmAdminUsernames());
+    Collection<String> allowedGroups = om.getOmAdminGroups();
+    ReconConfig reconConfig = conf.getObject(ReconConfig.class);
+    String reconPrincipal = reconConfig.getKerberosPrincipal();
+    if (!reconPrincipal.isEmpty()) {
+      UserGroupInformation ugi =
+          UserGroupInformation.createRemoteUser(reconPrincipal);
+      allowedUsers.add(ugi.getShortUserName());
+    }
+
+    initialize(om.getMetadataManager().getStore(),
+        om.getMetrics().getDBCheckpointMetrics(),
+        om.getAclsEnabled(),
+        allowedUsers,
+        allowedGroups,
+        om.isSpnegoEnabled());
+  }
+
+  @Override
+  protected void processMetadataSnapshotRequest(HttpServletRequest request, 
HttpServletResponse response,
+      boolean isFormData, DBCheckpoint checkpoint, boolean flush) {
+    List<String> excludedSstList = new ArrayList<>();
+    String[] sstParam = isFormData ?
+        parseFormDataParameters(request) : request.getParameterValues(
+        OZONE_DB_CHECKPOINT_REQUEST_TO_EXCLUDE_SST);
+    Set<String> receivedSstFiles = fetchSstFilesReceived(sstParam);
+    Path tmpdir = null;
+    try (BootstrapStateHandler.Lock lock = getBootstrapStateLock().lock()) {
+      tmpdir = Files.createTempDirectory(getBootstrapTempData().toPath(),
+          "bootstrap-data-");
+      String tarName = "om.data-" + System.currentTimeMillis() + ".tar";
+      response.setContentType("application/x-tar");
+      response.setHeader("Content-Disposition", "attachment; filename=\"" + 
tarName + "\"");
+      Instant start = Instant.now();
+      writeDbDataToStream(request, response.getOutputStream(), 
receivedSstFiles, tmpdir);
+      Instant end = Instant.now();
+      long duration = Duration.between(start, end).toMillis();
+      LOG.info("Time taken to write the checkpoint to response output " +
+          "stream: {} milliseconds", duration);
+      logSstFileList(excludedSstList,
+          "Excluded {} SST files from the latest checkpoint{}: {}", 5);
+    } catch (Exception e) {
+      LOG.error(
+          "Unable to process metadata snapshot request. ", e);
+      response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+    } finally {
+      try {
+        if (tmpdir != null) {
+          FileUtils.deleteDirectory(tmpdir.toFile());
+        }
+      } catch (IOException e) {
+        LOG.error("unable to delete: " + tmpdir);
+      }
+    }
+    super.processMetadataSnapshotRequest(request, response, isFormData, 
checkpoint, flush);

Review Comment:
   We should be calling writeDbDataToStream after taking a checkpoint and copy 
those files consistently using the checkpoint.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServletInodeBasedXfer.java:
##########
@@ -0,0 +1,359 @@
+/*
+ * 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.hadoop.ozone.om;
+
+import static org.apache.hadoop.hdds.utils.Archiver.includeFile;
+import static org.apache.hadoop.hdds.utils.Archiver.linkAndIncludeFile;
+import static org.apache.hadoop.hdds.utils.Archiver.tar;
+import static 
org.apache.hadoop.ozone.OzoneConsts.OZONE_DB_CHECKPOINT_REQUEST_TO_EXCLUDE_SST;
+import static 
org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_RATIS_SNAPSHOT_MAX_TOTAL_SST_SIZE_DEFAULT;
+import static 
org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_RATIS_SNAPSHOT_MAX_TOTAL_SST_SIZE_KEY;
+import static 
org.apache.hadoop.ozone.om.codec.OMDBCheckpointUtils.includeSnapshotData;
+import static 
org.apache.hadoop.ozone.om.codec.OMDBCheckpointUtils.logEstimatedTarballSize;
+import static org.apache.hadoop.ozone.om.snapshot.OmSnapshotUtils.DATA_PREFIX;
+import static org.apache.hadoop.ozone.om.snapshot.OmSnapshotUtils.DATA_SUFFIX;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardOpenOption;
+import java.time.Duration;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Stream;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import org.apache.commons.compress.archivers.ArchiveOutputStream;
+import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.recon.ReconConfig;
+import org.apache.hadoop.hdds.utils.DBCheckpointServlet;
+import org.apache.hadoop.hdds.utils.db.DBCheckpoint;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.lock.BootstrapStateHandler;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.snapshot.OmSnapshotUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.ozone.rocksdiff.RocksDBCheckpointDiffer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Specialized OMDBCheckpointServlet implementation that transfers Ozone 
Manager
+ * database checkpoints using inode-based deduplication.
+ * <p>
+ * This servlet constructs checkpoint archives by examining file inodes,
+ * ensuring that files with the same inode (i.e., hardlinks or duplicates)
+ * are only transferred once. It maintains mappings from inode IDs to file
+ * paths, manages hardlink information, and enforces snapshot and SST file
+ * size constraints as needed.
+ * <p>
+ * This approach optimizes checkpoint streaming by reducing redundant data
+ * transfer, especially in environments where RocksDB and snapshotting result
+ * in multiple hardlinks to the same physical data.
+ */
+public class OMDBCheckpointServletInodeBasedXfer extends DBCheckpointServlet {
+
+  protected static final Logger LOG =
+      LoggerFactory.getLogger(OMDBCheckpointServletInodeBasedXfer.class);
+  private static final long serialVersionUID = 1L;
+
+  @Override
+  public void init() throws ServletException {
+    OzoneManager om = (OzoneManager) getServletContext()
+        .getAttribute(OzoneConsts.OM_CONTEXT_ATTRIBUTE);
+
+    if (om == null) {
+      LOG.error("Unable to initialize OMDBCheckpointServlet. OM is null");
+      return;
+    }
+
+    OzoneConfiguration conf = getConf();
+    // Only Ozone Admins and Recon are allowed
+    Collection<String> allowedUsers =
+        new LinkedHashSet<>(om.getOmAdminUsernames());
+    Collection<String> allowedGroups = om.getOmAdminGroups();
+    ReconConfig reconConfig = conf.getObject(ReconConfig.class);
+    String reconPrincipal = reconConfig.getKerberosPrincipal();
+    if (!reconPrincipal.isEmpty()) {
+      UserGroupInformation ugi =
+          UserGroupInformation.createRemoteUser(reconPrincipal);
+      allowedUsers.add(ugi.getShortUserName());
+    }
+
+    initialize(om.getMetadataManager().getStore(),
+        om.getMetrics().getDBCheckpointMetrics(),
+        om.getAclsEnabled(),
+        allowedUsers,
+        allowedGroups,
+        om.isSpnegoEnabled());
+  }
+
+  @Override
+  protected void processMetadataSnapshotRequest(HttpServletRequest request, 
HttpServletResponse response,
+      boolean isFormData, DBCheckpoint checkpoint, boolean flush) {
+    List<String> excludedSstList = new ArrayList<>();
+    String[] sstParam = isFormData ?
+        parseFormDataParameters(request) : request.getParameterValues(
+        OZONE_DB_CHECKPOINT_REQUEST_TO_EXCLUDE_SST);
+    Set<String> receivedSstFiles = fetchSstFilesReceived(sstParam);
+    Path tmpdir = null;
+    try (BootstrapStateHandler.Lock lock = getBootstrapStateLock().lock()) {
+      tmpdir = Files.createTempDirectory(getBootstrapTempData().toPath(),
+          "bootstrap-data-");
+      String tarName = "om.data-" + System.currentTimeMillis() + ".tar";
+      response.setContentType("application/x-tar");
+      response.setHeader("Content-Disposition", "attachment; filename=\"" + 
tarName + "\"");
+      Instant start = Instant.now();
+      writeDbDataToStream(request, response.getOutputStream(), 
receivedSstFiles, tmpdir);
+      Instant end = Instant.now();
+      long duration = Duration.between(start, end).toMillis();
+      LOG.info("Time taken to write the checkpoint to response output " +
+          "stream: {} milliseconds", duration);
+      logSstFileList(excludedSstList,
+          "Excluded {} SST files from the latest checkpoint{}: {}", 5);
+    } catch (Exception e) {
+      LOG.error(
+          "Unable to process metadata snapshot request. ", e);
+      response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+    } finally {
+      try {
+        if (tmpdir != null) {
+          FileUtils.deleteDirectory(tmpdir.toFile());
+        }
+      } catch (IOException e) {
+        LOG.error("unable to delete: " + tmpdir);
+      }
+    }
+    super.processMetadataSnapshotRequest(request, response, isFormData, 
checkpoint, flush);
+  }
+
+  Path getSstBackupDir() {
+    RocksDBCheckpointDiffer differ = getDbStore().getRocksDBCheckpointDiffer();
+    return new File(differ.getSSTBackupDir()).toPath();
+  }
+
+  Path getCompactionLogDir() {
+    RocksDBCheckpointDiffer differ = getDbStore().getRocksDBCheckpointDiffer();
+    return new File(differ.getCompactionLogDir()).toPath();
+  }
+
+  public void writeDbDataToStream(HttpServletRequest request, OutputStream 
destination,
+      Set<String> sstFilesToExclude, Path tmpdir) throws IOException {
+    DBCheckpoint checkpoint = null;
+    OzoneManager om = (OzoneManager) 
getServletContext().getAttribute(OzoneConsts.OM_CONTEXT_ATTRIBUTE);
+    OMMetadataManager omMetadataManager = om.getMetadataManager();
+    boolean includeSnapshotData = includeSnapshotData(request);
+    AtomicLong maxTotalSstSize = new 
AtomicLong(getConf().getLong(OZONE_OM_RATIS_SNAPSHOT_MAX_TOTAL_SST_SIZE_KEY,
+        OZONE_OM_RATIS_SNAPSHOT_MAX_TOTAL_SST_SIZE_DEFAULT));
+
+    Set<Path> snapshotPaths = Collections.emptySet();
+
+    if (!includeSnapshotData) {
+      maxTotalSstSize.set(Long.MAX_VALUE);
+    } else {
+      snapshotPaths = getSnapshotDirs(omMetadataManager);
+    }
+
+    if (sstFilesToExclude.isEmpty()) {
+      logEstimatedTarballSize(getDbStore().getDbLocation().toPath(), 
snapshotPaths);
+    }
+
+    boolean shouldContinue = true;
+
+    Map<String, String> hardLinkFileMap = new HashMap<>();
+    try (ArchiveOutputStream<TarArchiveEntry> archiveOutputStream = 
tar(destination)) {
+      if (includeSnapshotData) {
+        // Process each snapshot db path and write it to archive
+        for (Path snapshotDbPath : snapshotPaths) {
+          if (!shouldContinue) {
+            break;
+          }
+          shouldContinue = writeDBToArchive(sstFilesToExclude, snapshotDbPath,
+              maxTotalSstSize, archiveOutputStream, tmpdir, hardLinkFileMap);
+        }
+
+
+        if (shouldContinue) {
+          shouldContinue = writeDBToArchive(sstFilesToExclude, 
getSstBackupDir(),
+              maxTotalSstSize, archiveOutputStream,  tmpdir, hardLinkFileMap);
+        }
+
+        if (shouldContinue) {
+          shouldContinue = writeDBToArchive(sstFilesToExclude, 
getCompactionLogDir(),
+              maxTotalSstSize, archiveOutputStream,  tmpdir, hardLinkFileMap);
+        }
+      }
+
+      if (shouldContinue) {
+        // we finished transferring files from snapshot DB's by now and
+        // this is the last step where we transfer the active om.db contents
+        checkpoint = getCheckpoint(tmpdir, true);
+        // unlimited files as we want the Active DB contents to be transferred 
in a single batch
+        maxTotalSstSize.set(Long.MAX_VALUE);
+        Path checkpointDir = checkpoint.getCheckpointLocation();
+        writeDBToArchive(sstFilesToExclude, checkpointDir,

Review Comment:
   hardLinkFileMap will end up having the checkpointDir path instead of 
activeDB /data/om.db/1.sst it will write the sst file as 
/data/..checkpointDir/1.sst which would be wrong



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServletInodeBasedXfer.java:
##########
@@ -0,0 +1,359 @@
+/*
+ * 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.hadoop.ozone.om;
+
+import static org.apache.hadoop.hdds.utils.Archiver.includeFile;
+import static org.apache.hadoop.hdds.utils.Archiver.linkAndIncludeFile;
+import static org.apache.hadoop.hdds.utils.Archiver.tar;
+import static 
org.apache.hadoop.ozone.OzoneConsts.OZONE_DB_CHECKPOINT_REQUEST_TO_EXCLUDE_SST;
+import static 
org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_RATIS_SNAPSHOT_MAX_TOTAL_SST_SIZE_DEFAULT;
+import static 
org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_RATIS_SNAPSHOT_MAX_TOTAL_SST_SIZE_KEY;
+import static 
org.apache.hadoop.ozone.om.codec.OMDBCheckpointUtils.includeSnapshotData;
+import static 
org.apache.hadoop.ozone.om.codec.OMDBCheckpointUtils.logEstimatedTarballSize;
+import static org.apache.hadoop.ozone.om.snapshot.OmSnapshotUtils.DATA_PREFIX;
+import static org.apache.hadoop.ozone.om.snapshot.OmSnapshotUtils.DATA_SUFFIX;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardOpenOption;
+import java.time.Duration;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Stream;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import org.apache.commons.compress.archivers.ArchiveOutputStream;
+import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.recon.ReconConfig;
+import org.apache.hadoop.hdds.utils.DBCheckpointServlet;
+import org.apache.hadoop.hdds.utils.db.DBCheckpoint;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.lock.BootstrapStateHandler;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.snapshot.OmSnapshotUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.ozone.rocksdiff.RocksDBCheckpointDiffer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Specialized OMDBCheckpointServlet implementation that transfers Ozone 
Manager
+ * database checkpoints using inode-based deduplication.
+ * <p>
+ * This servlet constructs checkpoint archives by examining file inodes,
+ * ensuring that files with the same inode (i.e., hardlinks or duplicates)
+ * are only transferred once. It maintains mappings from inode IDs to file
+ * paths, manages hardlink information, and enforces snapshot and SST file
+ * size constraints as needed.
+ * <p>
+ * This approach optimizes checkpoint streaming by reducing redundant data
+ * transfer, especially in environments where RocksDB and snapshotting result
+ * in multiple hardlinks to the same physical data.
+ */
+public class OMDBCheckpointServletInodeBasedXfer extends DBCheckpointServlet {
+
+  protected static final Logger LOG =
+      LoggerFactory.getLogger(OMDBCheckpointServletInodeBasedXfer.class);
+  private static final long serialVersionUID = 1L;
+
+  @Override
+  public void init() throws ServletException {
+    OzoneManager om = (OzoneManager) getServletContext()
+        .getAttribute(OzoneConsts.OM_CONTEXT_ATTRIBUTE);
+
+    if (om == null) {
+      LOG.error("Unable to initialize OMDBCheckpointServlet. OM is null");
+      return;
+    }
+
+    OzoneConfiguration conf = getConf();
+    // Only Ozone Admins and Recon are allowed
+    Collection<String> allowedUsers =
+        new LinkedHashSet<>(om.getOmAdminUsernames());
+    Collection<String> allowedGroups = om.getOmAdminGroups();
+    ReconConfig reconConfig = conf.getObject(ReconConfig.class);
+    String reconPrincipal = reconConfig.getKerberosPrincipal();
+    if (!reconPrincipal.isEmpty()) {
+      UserGroupInformation ugi =
+          UserGroupInformation.createRemoteUser(reconPrincipal);
+      allowedUsers.add(ugi.getShortUserName());
+    }
+
+    initialize(om.getMetadataManager().getStore(),
+        om.getMetrics().getDBCheckpointMetrics(),
+        om.getAclsEnabled(),
+        allowedUsers,
+        allowedGroups,
+        om.isSpnegoEnabled());
+  }
+
+  @Override
+  protected void processMetadataSnapshotRequest(HttpServletRequest request, 
HttpServletResponse response,
+      boolean isFormData, DBCheckpoint checkpoint, boolean flush) {
+    List<String> excludedSstList = new ArrayList<>();
+    String[] sstParam = isFormData ?
+        parseFormDataParameters(request) : request.getParameterValues(
+        OZONE_DB_CHECKPOINT_REQUEST_TO_EXCLUDE_SST);
+    Set<String> receivedSstFiles = fetchSstFilesReceived(sstParam);
+    Path tmpdir = null;
+    try (BootstrapStateHandler.Lock lock = getBootstrapStateLock().lock()) {
+      tmpdir = Files.createTempDirectory(getBootstrapTempData().toPath(),
+          "bootstrap-data-");
+      String tarName = "om.data-" + System.currentTimeMillis() + ".tar";
+      response.setContentType("application/x-tar");
+      response.setHeader("Content-Disposition", "attachment; filename=\"" + 
tarName + "\"");
+      Instant start = Instant.now();
+      writeDbDataToStream(request, response.getOutputStream(), 
receivedSstFiles, tmpdir);
+      Instant end = Instant.now();
+      long duration = Duration.between(start, end).toMillis();
+      LOG.info("Time taken to write the checkpoint to response output " +
+          "stream: {} milliseconds", duration);
+      logSstFileList(excludedSstList,
+          "Excluded {} SST files from the latest checkpoint{}: {}", 5);
+    } catch (Exception e) {
+      LOG.error(
+          "Unable to process metadata snapshot request. ", e);
+      response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+    } finally {
+      try {
+        if (tmpdir != null) {
+          FileUtils.deleteDirectory(tmpdir.toFile());
+        }
+      } catch (IOException e) {
+        LOG.error("unable to delete: " + tmpdir);
+      }
+    }
+    super.processMetadataSnapshotRequest(request, response, isFormData, 
checkpoint, flush);
+  }
+
+  Path getSstBackupDir() {
+    RocksDBCheckpointDiffer differ = getDbStore().getRocksDBCheckpointDiffer();
+    return new File(differ.getSSTBackupDir()).toPath();
+  }
+
+  Path getCompactionLogDir() {
+    RocksDBCheckpointDiffer differ = getDbStore().getRocksDBCheckpointDiffer();
+    return new File(differ.getCompactionLogDir()).toPath();
+  }
+
+  public void writeDbDataToStream(HttpServletRequest request, OutputStream 
destination,
+      Set<String> sstFilesToExclude, Path tmpdir) throws IOException {
+    DBCheckpoint checkpoint = null;
+    OzoneManager om = (OzoneManager) 
getServletContext().getAttribute(OzoneConsts.OM_CONTEXT_ATTRIBUTE);
+    OMMetadataManager omMetadataManager = om.getMetadataManager();
+    boolean includeSnapshotData = includeSnapshotData(request);
+    AtomicLong maxTotalSstSize = new 
AtomicLong(getConf().getLong(OZONE_OM_RATIS_SNAPSHOT_MAX_TOTAL_SST_SIZE_KEY,
+        OZONE_OM_RATIS_SNAPSHOT_MAX_TOTAL_SST_SIZE_DEFAULT));
+
+    Set<Path> snapshotPaths = Collections.emptySet();
+
+    if (!includeSnapshotData) {
+      maxTotalSstSize.set(Long.MAX_VALUE);
+    } else {
+      snapshotPaths = getSnapshotDirs(omMetadataManager);
+    }
+
+    if (sstFilesToExclude.isEmpty()) {
+      logEstimatedTarballSize(getDbStore().getDbLocation().toPath(), 
snapshotPaths);
+    }
+
+    boolean shouldContinue = true;
+
+    Map<String, String> hardLinkFileMap = new HashMap<>();
+    try (ArchiveOutputStream<TarArchiveEntry> archiveOutputStream = 
tar(destination)) {
+      if (includeSnapshotData) {
+        // Process each snapshot db path and write it to archive
+        for (Path snapshotDbPath : snapshotPaths) {
+          if (!shouldContinue) {
+            break;
+          }
+          shouldContinue = writeDBToArchive(sstFilesToExclude, snapshotDbPath,
+              maxTotalSstSize, archiveOutputStream, tmpdir, hardLinkFileMap);
+        }
+
+
+        if (shouldContinue) {
+          shouldContinue = writeDBToArchive(sstFilesToExclude, 
getSstBackupDir(),
+              maxTotalSstSize, archiveOutputStream,  tmpdir, hardLinkFileMap);
+        }
+
+        if (shouldContinue) {
+          shouldContinue = writeDBToArchive(sstFilesToExclude, 
getCompactionLogDir(),
+              maxTotalSstSize, archiveOutputStream,  tmpdir, hardLinkFileMap);
+        }
+      }
+
+      if (shouldContinue) {
+        // we finished transferring files from snapshot DB's by now and
+        // this is the last step where we transfer the active om.db contents
+        checkpoint = getCheckpoint(tmpdir, true);
+        // unlimited files as we want the Active DB contents to be transferred 
in a single batch
+        maxTotalSstSize.set(Long.MAX_VALUE);
+        Path checkpointDir = checkpoint.getCheckpointLocation();
+        writeDBToArchive(sstFilesToExclude, checkpointDir,

Review Comment:
   I believe none of this has test cases please add test cases. We don't want 
bugs in this flow.



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

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

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@ozone.apache.org
For additional commands, e-mail: issues-h...@ozone.apache.org

Reply via email to