This is an automated email from the ASF dual-hosted git repository.
weichiu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git
The following commit(s) were added to refs/heads/master by this push:
new de683aad88f HDDS-13263. Refactor DB Checkpoint Utilities. (#8620)
de683aad88f is described below
commit de683aad88fc4911d69892efe237d35429c8efd9
Author: Sadanand Shenoy <[email protected]>
AuthorDate: Wed Jun 18 10:19:02 2025 +0530
HDDS-13263. Refactor DB Checkpoint Utilities. (#8620)
---
.../hadoop/hdds/utils/DBCheckpointServlet.java | 50 ++++++-----
.../apache/hadoop/hdds/utils/HddsServerUtil.java | 8 +-
.../hadoop/hdds/utils/TestRDBSnapshotProvider.java | 5 +-
.../hdds/scm/ha/SCMDBCheckpointProvider.java | 5 +-
.../hdds/scm/TestSCMDbCheckpointServlet.java | 17 ++--
.../hadoop/ozone/om/TestOMDbCheckpointServlet.java | 19 ++--
.../ozone/om/snapshot/TestOMDBCheckpointUtils.java | 100 +++++++++++++++++++++
.../hadoop/ozone/om/OMDBCheckpointServlet.java | 74 ++++-----------
.../ozone/om/snapshot/OMDBCheckpointUtils.java | 80 +++++++++++++++++
.../hadoop/ozone/om/TestOmSnapshotManager.java | 47 +++-------
10 files changed, 261 insertions(+), 144 deletions(-)
diff --git
a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/DBCheckpointServlet.java
b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/DBCheckpointServlet.java
index 0b05eec14a9..629fba1772a 100644
---
a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/DBCheckpointServlet.java
+++
b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/DBCheckpointServlet.java
@@ -33,8 +33,10 @@
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
+import java.util.HashSet;
import java.util.List;
import java.util.Objects;
+import java.util.Set;
import java.util.stream.Collectors;
import javax.servlet.ServletException;
import javax.servlet.http.HttpServlet;
@@ -130,12 +132,12 @@ private boolean hasPermission(UserGroupInformation user) {
}
}
- private static void logSstFileList(List<String>sstList, String msg, int
sampleSize) {
+ private static void logSstFileList(Collection<String> sstList, String msg,
int sampleSize) {
int count = sstList.size();
if (LOG.isDebugEnabled()) {
LOG.debug(msg, count, "", sstList);
} else if (count > sampleSize) {
- List<String> sample = sstList.subList(0, sampleSize);
+ List<String> sample =
sstList.stream().limit(sampleSize).collect(Collectors.toList());
LOG.info(msg, count, ", sample", sample);
} else {
LOG.info(msg, count, "", sstList);
@@ -187,8 +189,6 @@ private void generateSnapshotCheckpoint(HttpServletRequest
request,
}
}
- DBCheckpoint checkpoint = null;
-
boolean flush = false;
String flushParam =
request.getParameter(OZONE_DB_CHECKPOINT_REQUEST_FLUSH);
@@ -196,21 +196,17 @@ private void
generateSnapshotCheckpoint(HttpServletRequest request,
flush = Boolean.parseBoolean(flushParam);
}
- List<String> receivedSstList = new ArrayList<>();
+ processMetadataSnapshotRequest(request, response, isFormData, flush);
+ }
+
+ private void processMetadataSnapshotRequest(HttpServletRequest request,
HttpServletResponse response,
+ boolean isFormData, boolean flush) {
List<String> excludedSstList = new ArrayList<>();
String[] sstParam = isFormData ?
parseFormDataParameters(request) : request.getParameterValues(
OZONE_DB_CHECKPOINT_REQUEST_TO_EXCLUDE_SST);
- if (sstParam != null) {
- receivedSstList.addAll(
- Arrays.stream(sstParam)
- .filter(s -> s.endsWith(ROCKSDB_SST_SUFFIX))
- .distinct()
- .collect(Collectors.toList()));
- logSstFileList(receivedSstList,
- "Received list of {} SST files to be excluded{}: {}", 5);
- }
-
+ Set<String> receivedSstFiles = extractSstFilesToExclude(sstParam);
+ DBCheckpoint checkpoint = null;
Path tmpdir = null;
try (BootstrapStateHandler.Lock lock = getBootstrapStateLock().lock()) {
tmpdir = Files.createTempDirectory(bootstrapTempData.toPath(),
@@ -235,8 +231,8 @@ private void generateSnapshotCheckpoint(HttpServletRequest
request,
file + ".tar\"");
Instant start = Instant.now();
- writeDbDataToStream(checkpoint, request,
- response.getOutputStream(), receivedSstList, excludedSstList,
tmpdir);
+ writeDbDataToStream(checkpoint, request, response.getOutputStream(),
+ receivedSstFiles, tmpdir);
Instant end = Instant.now();
long duration = Duration.between(start, end).toMillis();
@@ -276,6 +272,16 @@ private void generateSnapshotCheckpoint(HttpServletRequest
request,
}
}
+ protected static Set<String> extractSstFilesToExclude(String[] sstParam) {
+ Set<String> receivedSstFiles = new HashSet<>();
+ if (sstParam != null) {
+ receivedSstFiles.addAll(
+ Arrays.stream(sstParam).filter(s ->
s.endsWith(ROCKSDB_SST_SUFFIX)).distinct().collect(Collectors.toList()));
+ logSstFileList(receivedSstFiles, "Received list of {} SST files to be
excluded{}: {}", 5);
+ }
+ return receivedSstFiles;
+ }
+
public DBCheckpoint getCheckpoint(Path ignoredTmpdir, boolean flush)
throws IOException {
return dbStore.getCheckpoint(flush);
@@ -346,20 +352,16 @@ public void doPost(HttpServletRequest request,
HttpServletResponse response) {
* (Parameter is ignored in this class but used in child classes).
* @param destination The stream to write to.
* @param toExcludeList the files to be excluded
- * @param excludedList the files excluded
*
*/
public void writeDbDataToStream(DBCheckpoint checkpoint,
HttpServletRequest ignoredRequest,
OutputStream destination,
- List<String> toExcludeList,
- List<String> excludedList, Path tmpdir)
+ Set<String> toExcludeList,
+ Path tmpdir)
throws IOException, InterruptedException {
Objects.requireNonNull(toExcludeList);
- Objects.requireNonNull(excludedList);
-
- writeDBCheckpointToStream(checkpoint, destination,
- toExcludeList, excludedList);
+ writeDBCheckpointToStream(checkpoint, destination, toExcludeList);
}
public DBStore getDbStore() {
diff --git
a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/HddsServerUtil.java
b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/HddsServerUtil.java
index 9a780be8b3d..3d41df36285 100644
---
a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/HddsServerUtil.java
+++
b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/HddsServerUtil.java
@@ -58,10 +58,10 @@
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
-import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.OptionalInt;
+import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@@ -617,14 +617,12 @@ public static MetricsSystem initializeMetrics(
* @param checkpoint checkpoint file
* @param destination destination output stream.
* @param toExcludeList the files to be excluded
- * @param excludedList the files excluded
* @throws IOException
*/
public static void writeDBCheckpointToStream(
DBCheckpoint checkpoint,
OutputStream destination,
- List<String> toExcludeList,
- List<String> excludedList)
+ Set<String> toExcludeList)
throws IOException {
try (ArchiveOutputStream<TarArchiveEntry> archiveOutputStream =
tar(destination);
Stream<Path> files =
@@ -636,8 +634,6 @@ public static void writeDBCheckpointToStream(
String fileName = fileNamePath.toString();
if (!toExcludeList.contains(fileName)) {
includeFile(path.toFile(), fileName, archiveOutputStream);
- } else {
- excludedList.add(fileName);
}
}
}
diff --git
a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/TestRDBSnapshotProvider.java
b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/TestRDBSnapshotProvider.java
index 61652dbd7ca..4d149cec7e1 100644
---
a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/TestRDBSnapshotProvider.java
+++
b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/TestRDBSnapshotProvider.java
@@ -31,7 +31,6 @@
import java.io.OutputStream;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
-import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
@@ -118,8 +117,8 @@ public void downloadSnapshot(String leaderNodeID, File
targetFile)
.collect(Collectors.toList()));
try (OutputStream outputStream =
Files.newOutputStream(targetFile.toPath())) {
writeDBCheckpointToStream(dbCheckpoint, outputStream,
- HAUtils.getExistingSstFiles(
- rdbSnapshotProvider.getCandidateDir()), new ArrayList<>());
+ new HashSet<>(HAUtils.getExistingSstFiles(
+ rdbSnapshotProvider.getCandidateDir())));
}
}
};
diff --git
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMDBCheckpointProvider.java
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMDBCheckpointProvider.java
index d5579f1eb13..ba9fd6ff3d6 100644
---
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMDBCheckpointProvider.java
+++
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMDBCheckpointProvider.java
@@ -22,7 +22,7 @@
import java.nio.file.Path;
import java.time.Duration;
import java.time.Instant;
-import java.util.ArrayList;
+import java.util.HashSet;
import org.apache.hadoop.hdds.utils.HddsServerUtil;
import org.apache.hadoop.hdds.utils.db.DBCheckpoint;
import org.apache.hadoop.hdds.utils.db.DBStore;
@@ -66,8 +66,7 @@ public void writeDBCheckPointToSream(OutputStream stream,
boolean flush)
}
Instant start = Instant.now();
- HddsServerUtil.writeDBCheckpointToStream(checkpoint, stream,
- new ArrayList<>(), new ArrayList<>());
+ HddsServerUtil.writeDBCheckpointToStream(checkpoint, stream, new
HashSet<>());
Instant end = Instant.now();
long duration = Duration.between(start, end).toMillis();
diff --git
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestSCMDbCheckpointServlet.java
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestSCMDbCheckpointServlet.java
index 8a4f0ca5600..fbd02d6f8e5 100644
---
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestSCMDbCheckpointServlet.java
+++
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestSCMDbCheckpointServlet.java
@@ -41,9 +41,10 @@
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Path;
-import java.util.ArrayList;
+import java.util.Collection;
import java.util.Collections;
-import java.util.List;
+import java.util.HashSet;
+import java.util.Set;
import java.util.stream.Stream;
import javax.servlet.ServletContext;
import javax.servlet.ServletException;
@@ -114,7 +115,7 @@ public void init() throws Exception {
Collections.emptyList(),
false);
doCallRealMethod().when(scmDbCheckpointServletMock)
- .writeDbDataToStream(any(), any(), any(), any(), any(), any());
+ .writeDbDataToStream(any(), any(), any(), any(), any());
doCallRealMethod().when(scmDbCheckpointServletMock).doPost(requestMock,
responseMock);
doCallRealMethod().when(scmDbCheckpointServletMock).doGet(requestMock,
@@ -145,7 +146,7 @@ void testEndpoint(String httpMethod, @TempDir Path tempDir)
throws ServletException, IOException, InterruptedException {
this.method = httpMethod;
- List<String> toExcludeList = new ArrayList<>();
+ Set<String> toExcludeList = new HashSet<>();
toExcludeList.add("sstFile1.sst");
toExcludeList.add("sstFile2.sst");
@@ -197,7 +198,7 @@ public void write(int b) throws IOException {
.isGreaterThan(initialCheckpointCount);
verify(scmDbCheckpointServletMock).writeDbDataToStream(any(),
- any(), any(), eq(toExcludeList), any(), any());
+ any(), any(), eq(toExcludeList), any());
}
@Test
@@ -235,7 +236,7 @@ private static Stream<Arguments> getHttpMethods() {
* @param toExcludeList SST file names to be excluded.
* @throws IOException
*/
- private void setupHttpMethod(List<String> toExcludeList) throws IOException {
+ private void setupHttpMethod(Collection<String> toExcludeList) throws
IOException {
if (method.equals("POST")) {
setupPostMethod(toExcludeList);
} else {
@@ -248,7 +249,7 @@ private void setupHttpMethod(List<String> toExcludeList)
throws IOException {
* @param toExcludeList SST file names to be excluded.
* @throws IOException
*/
- private void setupPostMethod(List<String> toExcludeList)
+ private void setupPostMethod(Collection<String> toExcludeList)
throws IOException {
when(requestMock.getMethod()).thenReturn("POST");
when(requestMock.getContentType()).thenReturn("multipart/form-data; " +
@@ -286,7 +287,7 @@ private void setupPostMethod(List<String> toExcludeList)
* Setups details for HTTP GET request.
* @param toExcludeList SST file names to be excluded.
*/
- private void setupGetMethod(List<String> toExcludeList) {
+ private void setupGetMethod(Collection<String> toExcludeList) {
when(requestMock.getMethod()).thenReturn("GET");
when(requestMock
.getParameterValues(OZONE_DB_CHECKPOINT_REQUEST_TO_EXCLUDE_SST))
diff --git
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMDbCheckpointServlet.java
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMDbCheckpointServlet.java
index e0b2a8129e0..a9440ef1515 100644
---
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMDbCheckpointServlet.java
+++
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMDbCheckpointServlet.java
@@ -222,7 +222,7 @@ public void write(int b) throws IOException {
responseMock);
doCallRealMethod().when(omDbCheckpointServletMock)
- .writeDbDataToStream(any(), any(), any(), any(), any(), any());
+ .writeDbDataToStream(any(), any(), any(), any(), any());
when(omDbCheckpointServletMock.getBootstrapStateLock())
.thenReturn(lock);
@@ -265,7 +265,7 @@ private void testEndpoint(String method) throws Exception {
doNothing().when(responseMock).setContentType("application/x-tar");
doNothing().when(responseMock).setHeader(anyString(), anyString());
- List<String> toExcludeList = new ArrayList<>();
+ Set<String> toExcludeList = new HashSet<>();
toExcludeList.add("sstFile1.sst");
toExcludeList.add("sstFile2.sst");
@@ -288,7 +288,7 @@ private void testEndpoint(String method) throws Exception {
.isGreaterThan(initialCheckpointCount);
verify(omDbCheckpointServletMock).writeDbDataToStream(any(),
- any(), any(), eq(toExcludeList), any(), any());
+ any(), any(), eq(toExcludeList), any());
}
private void testDoPostWithInvalidContentType() throws Exception {
@@ -528,7 +528,7 @@ private void testWriteDbDataWithoutOmSnapshot()
Path tmpdir = folder.resolve("bootstrapData");
try (OutputStream fileOutputStream =
Files.newOutputStream(tempFile.toPath())) {
omDbCheckpointServletMock.writeDbDataToStream(dbCheckpoint, requestMock,
- fileOutputStream, new ArrayList<>(), new ArrayList<>(), tmpdir);
+ fileOutputStream, new HashSet<>(), tmpdir);
}
// Untar the file into a temp folder to be examined.
@@ -562,8 +562,7 @@ private void testWriteDbDataWithToExcludeFileList()
writer.write("Dummy data.");
}
assertTrue(dummyFile.exists());
- List<String> toExcludeList = new ArrayList<>();
- List<String> excludedList = new ArrayList<>();
+ Set<String> toExcludeList = new HashSet<>();
toExcludeList.add(dummyFile.getName());
// Set http param to exclude snapshot data.
@@ -574,7 +573,7 @@ private void testWriteDbDataWithToExcludeFileList()
Path tmpdir = folder.resolve("bootstrapData");
try (OutputStream fileOutputStream =
Files.newOutputStream(tempFile.toPath())) {
omDbCheckpointServletMock.writeDbDataToStream(dbCheckpoint, requestMock,
- fileOutputStream, toExcludeList, excludedList, tmpdir);
+ fileOutputStream, toExcludeList, tmpdir);
}
// Untar the file into a temp folder to be examined.
@@ -611,7 +610,7 @@ private void doEndpoint(String method) {
* @param toExcludeList SST file names to be excluded.
* @throws IOException
*/
- private void setupHttpMethod(String method, List<String> toExcludeList)
throws IOException {
+ private void setupHttpMethod(String method, Collection <String>
toExcludeList) throws IOException {
if (method.equals("POST")) {
setupPostMethod(toExcludeList);
} else {
@@ -624,7 +623,7 @@ private void setupHttpMethod(String method, List<String>
toExcludeList) throws I
* @param toExcludeList SST file names to be excluded.
* @throws IOException
*/
- private void setupPostMethod(List<String> toExcludeList)
+ private void setupPostMethod(Collection<String> toExcludeList)
throws IOException {
when(requestMock.getMethod()).thenReturn("POST");
when(requestMock.getContentType()).thenReturn("multipart/form-data; " +
@@ -662,7 +661,7 @@ private void setupPostMethod(List<String> toExcludeList)
* Setups details for HTTP GET request.
* @param toExcludeList SST file names to be excluded.
*/
- private void setupGetMethod(List<String> toExcludeList) {
+ private void setupGetMethod(Collection<String> toExcludeList) {
when(requestMock.getMethod()).thenReturn("GET");
when(requestMock
.getParameterValues(OZONE_DB_CHECKPOINT_REQUEST_TO_EXCLUDE_SST))
diff --git
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestOMDBCheckpointUtils.java
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestOMDBCheckpointUtils.java
new file mode 100644
index 00000000000..249e9285d7d
--- /dev/null
+++
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestOMDBCheckpointUtils.java
@@ -0,0 +1,100 @@
+/*
+ * 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.snapshot;
+
+import static
org.apache.hadoop.ozone.OzoneConsts.OZONE_DB_CHECKPOINT_INCLUDE_SNAPSHOT_DATA;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.HashSet;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.TimeoutException;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+/**
+ * This class provides utilities to test the behavior of methods in the
+ * OMDBCheckpointUtils class related to ozone database checkpoint tarball
+ * logging and snapshot data inclusion.
+ */
+public class TestOMDBCheckpointUtils {
+
+ @TempDir
+ private Path dbDir;
+
+ /**
+ * Writes a given number of fake sst files with a given size to a given
directory.
+ */
+ private void writeSstFilesToDirectory(Path dir, int numFiles, int fileSize)
throws IOException {
+ for (int i = 0; i < numFiles; i++) {
+ byte[] data = new byte[fileSize]; // 10KB
+ Random random = new Random();
+ random.nextBytes(data);
+ Files.write(dir.resolve(i + ".sst"), data);
+ }
+ }
+
+ @Test
+ public void testlogEstimatedTarballSize() throws IOException,
InterruptedException, TimeoutException {
+ GenericTestUtils.LogCapturer logCapturer =
+ GenericTestUtils.LogCapturer.captureLogs(OMDBCheckpointUtils.class);
+ writeSstFilesToDirectory(dbDir, 10, 10 * 1024);
+ Set<Path> snapshotDirs = new HashSet<>();
+ // without snapshots
+ OMDBCheckpointUtils.logEstimatedTarballSize(dbDir, snapshotDirs);
+ // 100KB checkpoint
+ // 10 in checkpoint
+ String expectedLogLine = getExpectedLogLine("100 KB", 10, 0);
+ GenericTestUtils.waitFor(() ->
logCapturer.getOutput().contains(expectedLogLine), 1000, 5000);
+ // with snapshots
+ snapshotDirs.add(dbDir);
+ OMDBCheckpointUtils.logEstimatedTarballSize(dbDir, snapshotDirs);
+ // 100KB checkpoint + 100KB snapshot
+ // 10 in checkpoint + 10 in snapshot
+ String expectedLogLineWithSnapshots = getExpectedLogLine("200 KB", 20, 1);
+ GenericTestUtils.waitFor(() ->
logCapturer.getOutput().contains(expectedLogLineWithSnapshots),
+ 1000, 5000);
+ }
+
+ private static String getExpectedLogLine(String expectedDataSize, int
expectedSSTFiles,
+ int expectedSnapshots) {
+ String baseMessage = String.format("Estimates for Checkpoint Tarball
Stream - Data size: %s, SST files: ",
+ expectedDataSize);
+ if (expectedSnapshots <= 0) {
+ return baseMessage;
+ }
+ return String.format("%s%d, snapshots: %d", baseMessage, expectedSSTFiles,
expectedSnapshots);
+ }
+
+ @Test
+ public void testIncludeSnapshotData() {
+ HttpServletRequest httpServletRequest = mock(HttpServletRequest.class);
+
when(httpServletRequest.getParameter(OZONE_DB_CHECKPOINT_INCLUDE_SNAPSHOT_DATA)).thenReturn("true");
+ assertTrue(OMDBCheckpointUtils.includeSnapshotData(httpServletRequest));
+
when(httpServletRequest.getParameter(OZONE_DB_CHECKPOINT_INCLUDE_SNAPSHOT_DATA)).thenReturn("false");
+ assertFalse(OMDBCheckpointUtils.includeSnapshotData(httpServletRequest));
+ }
+}
diff --git
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java
index bcff75fd039..4d85e9f0747 100644
---
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java
+++
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java
@@ -17,18 +17,17 @@
package org.apache.hadoop.ozone.om;
-import static org.apache.commons.io.filefilter.TrueFileFilter.TRUE;
import static org.apache.hadoop.hdds.utils.Archiver.includeFile;
import static org.apache.hadoop.hdds.utils.Archiver.tar;
import static
org.apache.hadoop.hdds.utils.HddsServerUtil.includeRatisSnapshotCompleteFlag;
import static org.apache.hadoop.ozone.OzoneConsts.OM_CHECKPOINT_DIR;
import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_CHECKPOINT_DIR;
import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_DIR;
-import static
org.apache.hadoop.ozone.OzoneConsts.OZONE_DB_CHECKPOINT_INCLUDE_SNAPSHOT_DATA;
import static org.apache.hadoop.ozone.OzoneConsts.ROCKSDB_SST_SUFFIX;
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.OmSnapshotManager.getSnapshotPath;
+import static
org.apache.hadoop.ozone.om.snapshot.OMDBCheckpointUtils.includeSnapshotData;
import static
org.apache.hadoop.ozone.om.snapshot.OmSnapshotUtils.createHardLinkList;
import static
org.apache.hadoop.ozone.om.snapshot.OmSnapshotUtils.truncateFileName;
@@ -58,11 +57,6 @@
import org.apache.commons.compress.archivers.ArchiveOutputStream;
import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
import org.apache.commons.io.FileUtils;
-import org.apache.commons.io.IOCase;
-import org.apache.commons.io.file.Counters;
-import org.apache.commons.io.file.CountingPathVisitor;
-import org.apache.commons.io.file.PathFilter;
-import org.apache.commons.io.filefilter.SuffixFileFilter;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.recon.ReconConfig;
import org.apache.hadoop.hdds.utils.DBCheckpointServlet;
@@ -74,6 +68,7 @@
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.OMDBCheckpointUtils;
import org.apache.hadoop.ozone.om.snapshot.OmSnapshotUtils;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.Time;
@@ -101,8 +96,6 @@ public class OMDBCheckpointServlet extends
DBCheckpointServlet {
private static final long serialVersionUID = 1L;
private transient BootstrapStateHandler.Lock lock;
private long maxTotalSstSize = 0;
- private static final PathFilter SST_FILE_FILTER =
- new SuffixFileFilter(ROCKSDB_SST_SUFFIX, IOCase.INSENSITIVE);
@Override
public void init() throws ServletException {
@@ -142,12 +135,10 @@ public void init() throws ServletException {
public void writeDbDataToStream(DBCheckpoint checkpoint,
HttpServletRequest request,
OutputStream destination,
- List<String> toExcludeList,
- List<String> excludedList,
+ Set<String> toExcludeList,
Path tmpdir)
throws IOException, InterruptedException {
Objects.requireNonNull(toExcludeList);
- Objects.requireNonNull(excludedList);
// copyFiles is a map of files to be added to tarball. The keys
// are the src path of the file, (where they are copied from on
@@ -175,7 +166,7 @@ public void writeDbDataToStream(DBCheckpoint checkpoint,
boolean completed = getFilesForArchive(checkpoint, copyFiles,
hardLinkFiles, sstFilesToExclude, includeSnapshotData(request),
- excludedList, sstBackupDir, compactionLogDir);
+ sstBackupDir, compactionLogDir);
Map<Path, Path> flatCopyFiles = copyFiles.values().stream().flatMap(map
-> map.entrySet().stream())
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
writeFilesToArchive(flatCopyFiles, hardLinkFiles, archiveOutputStream,
@@ -200,7 +191,7 @@ hardLinkFiles, sstFilesToExclude,
includeSnapshotData(request),
*/
@VisibleForTesting
public static Map<String, Map<Path, Path>> normalizeExcludeList(
- List<String> toExcludeList,
+ Collection<String> toExcludeList,
Path checkpointLocation,
DirectoryData sstBackupDir) {
Map<String, Map<Path, Path>> paths = new HashMap<>();
@@ -280,7 +271,6 @@ private boolean getFilesForArchive(DBCheckpoint checkpoint,
Map<Path, Path> hardLinkFiles,
Map<String, Map<Path, Path>> sstFilesToExclude,
boolean includeSnapshotData,
- List<String> excluded,
DirectoryData sstBackupDir,
DirectoryData compactionLogDir)
throws IOException {
@@ -305,7 +295,7 @@ private boolean getFilesForArchive(DBCheckpoint checkpoint,
// Get the active fs files.
Path dir = checkpoint.getCheckpointLocation();
if (!processDir(dir, copyFiles, hardLinkFiles, sstFilesToExclude,
- new HashSet<>(), excluded, copySize, null)) {
+ new HashSet<>(), copySize, null)) {
return false;
}
@@ -317,12 +307,12 @@ private boolean getFilesForArchive(DBCheckpoint
checkpoint,
Set<Path> snapshotPaths = getSnapshotDirs(checkpoint, true);
Path snapshotDir = getSnapshotDir();
if (!processDir(snapshotDir, copyFiles, hardLinkFiles, sstFilesToExclude,
- snapshotPaths, excluded, copySize, null)) {
+ snapshotPaths, copySize, null)) {
return false;
}
// Process the tmp sst compaction dir.
if (!processDir(sstBackupDir.getTmpDir().toPath(), copyFiles,
hardLinkFiles,
- sstFilesToExclude, new HashSet<>(), excluded, copySize,
+ sstFilesToExclude, new HashSet<>(), copySize,
sstBackupDir.getOriginalDir().toPath())) {
return false;
}
@@ -330,32 +320,18 @@ private boolean getFilesForArchive(DBCheckpoint
checkpoint,
// Process the tmp compaction log dir.
return processDir(compactionLogDir.getTmpDir().toPath(), copyFiles,
hardLinkFiles, sstFilesToExclude,
- new HashSet<>(), excluded, copySize,
+ new HashSet<>(), copySize,
compactionLogDir.getOriginalDir().toPath());
}
- private void logEstimatedTarballSize(
- DBCheckpoint checkpoint, boolean includeSnapshotData) {
- try {
- Counters.PathCounters counters = Counters.longPathCounters();
- CountingPathVisitor visitor = new CountingPathVisitor(
- counters, SST_FILE_FILTER, TRUE);
- Files.walkFileTree(checkpoint.getCheckpointLocation(), visitor);
- int totalSnapshots = 0;
- if (includeSnapshotData) {
- Set<Path> snapshotPaths = getSnapshotDirs(checkpoint, false);
- totalSnapshots = snapshotPaths.size();
- for (Path snapshotDir: snapshotPaths) {
- Files.walkFileTree(snapshotDir, visitor);
- }
- }
- LOG.info("Estimates for Checkpoint Tarball Stream - Data size: {} KB, "
+ "SST files: {}{}",
- counters.getByteCounter().get() / (1024),
- counters.getFileCounter().get(),
- (includeSnapshotData ? ", snapshots: " + totalSnapshots : ""));
- } catch (Exception e) {
- LOG.error("Could not estimate size of transfer to Checkpoint Tarball
Stream.", e);
+ private void logEstimatedTarballSize(DBCheckpoint checkpoint, boolean
includeSnapshotData)
+ throws IOException {
+ Set<Path> snapshotPaths = new HashSet<>();
+ if (includeSnapshotData) {
+ // since this is an estimate we can avoid waiting for dir to exist.
+ snapshotPaths = getSnapshotDirs(checkpoint, false);
}
+
OMDBCheckpointUtils.logEstimatedTarballSize(checkpoint.getCheckpointLocation(),
snapshotPaths);
}
/**
@@ -406,7 +382,6 @@ private boolean processDir(Path dir, Map<String, Map<Path,
Path>> copyFiles,
Map<Path, Path> hardLinkFiles,
Map<String, Map<Path, Path>> sstFilesToExclude,
Set<Path> snapshotPaths,
- List<String> excluded,
AtomicLong copySize,
Path destDir)
throws IOException {
@@ -450,12 +425,12 @@ private boolean processDir(Path dir, Map<String,
Map<Path, Path>> copyFiles,
filename.toString());
}
if (!processDir(file, copyFiles, hardLinkFiles, sstFilesToExclude,
- snapshotPaths, excluded, copySize, destSubDir)) {
+ snapshotPaths, copySize, destSubDir)) {
return false;
}
} else {
long fileSize = processFile(file, copyFiles, hardLinkFiles,
- sstFilesToExclude, excluded, destDir);
+ sstFilesToExclude, destDir);
if (copySize.get() + fileSize > maxTotalSstSize) {
return false;
} else {
@@ -476,13 +451,11 @@ private boolean processDir(Path dir, Map<String,
Map<Path, Path>> copyFiles,
* @param copyFiles The db files to be added to tarball.
* @param hardLinkFiles The db files to be added as hard links.
* @param sstFilesToExclude The db files to be excluded from tarball.
- * @param excluded The list of db files that actually were excluded.
*/
@VisibleForTesting
public static long processFile(Path file, Map<String, Map<Path, Path>>
copyFiles,
Map<Path, Path> hardLinkFiles,
Map<String, Map<Path, Path>>
sstFilesToExclude,
- List<String> excluded,
Path destDir)
throws IOException {
long fileSize = 0;
@@ -500,9 +473,7 @@ public static long processFile(Path file, Map<String,
Map<Path, Path>> copyFiles
if (destDir != null) {
destFile = Paths.get(destDir.toString(), fileName);
}
- if (sstFilesToExclude.getOrDefault(fileNamePath.toString(),
Collections.emptyMap()).containsKey(file)) {
- excluded.add(destFile.toString());
- } else {
+ if (!sstFilesToExclude.getOrDefault(fileNamePath.toString(),
Collections.emptyMap()).containsKey(file)) {
if (fileName.endsWith(ROCKSDB_SST_SUFFIX)) {
// If same as existing excluded file, add a link for it.
Path linkPath = findLinkPath(sstFilesToExclude, file);
@@ -568,13 +539,6 @@ private static Path findLinkPath(Map<String, Map<Path,
Path>> files, Path file)
return null;
}
- // Returns value of http request parameter.
- private boolean includeSnapshotData(HttpServletRequest request) {
- String includeParam =
- request.getParameter(OZONE_DB_CHECKPOINT_INCLUDE_SNAPSHOT_DATA);
- return Boolean.parseBoolean(includeParam);
- }
-
private void writeFilesToArchive(
Map<Path, Path> copyFiles,
Map<Path, Path> hardLinkFiles,
diff --git
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/OMDBCheckpointUtils.java
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/OMDBCheckpointUtils.java
new file mode 100644
index 00000000000..07b994964f5
--- /dev/null
+++
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/OMDBCheckpointUtils.java
@@ -0,0 +1,80 @@
+/*
+ * 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.snapshot;
+
+import static org.apache.commons.io.filefilter.TrueFileFilter.TRUE;
+import static
org.apache.hadoop.ozone.OzoneConsts.OZONE_DB_CHECKPOINT_INCLUDE_SNAPSHOT_DATA;
+import static org.apache.hadoop.ozone.OzoneConsts.ROCKSDB_SST_SUFFIX;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Set;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.io.IOCase;
+import org.apache.commons.io.file.Counters;
+import org.apache.commons.io.file.CountingPathVisitor;
+import org.apache.commons.io.file.PathFilter;
+import org.apache.commons.io.filefilter.SuffixFileFilter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utility class for handling operations related to OM DB Checkpoints.
+ * This includes extracting metadata directory paths, handling snapshot data,
+ * and logging estimated sizes of checkpoint tarball streams.
+ */
+public final class OMDBCheckpointUtils {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(OMDBCheckpointUtils.class);
+
+ private static final PathFilter SST_FILE_FILTER =
+ new SuffixFileFilter(ROCKSDB_SST_SUFFIX, IOCase.INSENSITIVE);
+
+ private OMDBCheckpointUtils() {
+ }
+
+ public static boolean includeSnapshotData(HttpServletRequest request) {
+ String includeParam =
+ request.getParameter(OZONE_DB_CHECKPOINT_INCLUDE_SNAPSHOT_DATA);
+ return Boolean.parseBoolean(includeParam);
+ }
+
+ public static void logEstimatedTarballSize(Path dbLocation, Set<Path>
snapshotPaths) {
+ try {
+ Counters.PathCounters counters = Counters.longPathCounters();
+ CountingPathVisitor visitor = new CountingPathVisitor(
+ counters, SST_FILE_FILTER, TRUE);
+ Files.walkFileTree(dbLocation, visitor);
+ boolean includeSnapshotData = !snapshotPaths.isEmpty();
+ long totalSnapshots = snapshotPaths.size();
+ if (includeSnapshotData) {
+ for (Path snapshotDir: snapshotPaths) {
+ Files.walkFileTree(snapshotDir, visitor);
+ }
+ }
+ LOG.info("Estimates for Checkpoint Tarball Stream - Data size: {} KB,
SST files: {}{}",
+ counters.getByteCounter().get() / (1024),
+ counters.getFileCounter().get(),
+ (includeSnapshotData ? ", snapshots: " + totalSnapshots : ""));
+ } catch (Exception e) {
+ LOG.error("Could not estimate size of transfer to Checkpoint Tarball
Stream for dbLocation:{} snapshotPaths:{}",
+ dbLocation, snapshotPaths, e);
+ }
+ }
+}
diff --git
a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshotManager.java
b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshotManager.java
index f1c42449b18..ff65978728c 100644
---
a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshotManager.java
+++
b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshotManager.java
@@ -51,7 +51,6 @@
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
-import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
@@ -501,24 +500,17 @@ void testProcessFileWithNullDestDirParameter(@TempDir
File testDir) throws IOExc
Map<String, Map<Path, Path>> copyFiles = new HashMap<>();
copyFiles.computeIfAbsent(copyFileName.toString(), (k) -> new
HashMap<>()).put(copyFile,
copyFile);
- List<String> excluded = new ArrayList<>();
Map<Path, Path> hardLinkFiles = new HashMap<>();
long fileSize;
- // Confirm the exclude file gets added to the excluded list,
- // (and thus is excluded.)
fileSize = processFile(excludeFile, copyFiles, hardLinkFiles,
- toExcludeFiles, excluded, null);
- assertEquals(excluded.size(), 1);
- assertEquals((excluded.get(0)), excludeFile.toString());
+ toExcludeFiles, null);
assertEquals(copyFiles.size(), 1);
assertEquals(hardLinkFiles.size(), 0);
assertEquals(fileSize, 0);
- excluded = new ArrayList<>();
// Confirm the linkToExcludedFile gets added as a link.
fileSize = processFile(linkToExcludedFile, copyFiles, hardLinkFiles,
- toExcludeFiles, excluded, null);
- assertEquals(excluded.size(), 0);
+ toExcludeFiles, null);
assertEquals(copyFiles.size(), 1);
assertEquals(hardLinkFiles.size(), 1);
assertEquals(hardLinkFiles.get(linkToExcludedFile), excludeFile);
@@ -527,8 +519,7 @@ void testProcessFileWithNullDestDirParameter(@TempDir File
testDir) throws IOExc
// Confirm the linkToCopiedFile gets added as a link.
fileSize = processFile(linkToCopiedFile, copyFiles, hardLinkFiles,
- toExcludeFiles, excluded, null);
- assertEquals(excluded.size(), 0);
+ toExcludeFiles, null);
assertEquals(copyFiles.size(), 1);
assertEquals(hardLinkFiles.size(), 1);
assertEquals(hardLinkFiles.get(linkToCopiedFile), copyFile);
@@ -537,8 +528,7 @@ void testProcessFileWithNullDestDirParameter(@TempDir File
testDir) throws IOExc
// Confirm the addToCopiedFiles gets added to list of copied files
fileSize = processFile(addToCopiedFiles, copyFiles, hardLinkFiles,
- toExcludeFiles, excluded, null);
- assertEquals(excluded.size(), 0);
+ toExcludeFiles, null);
assertEquals(copyFiles.size(), 2);
assertEquals(copyFiles.get(addToCopiedFiles.getFileName().toString()).get(addToCopiedFiles),
addToCopiedFiles);
assertEquals(fileSize, expectedFileSize);
@@ -547,8 +537,7 @@ void testProcessFileWithNullDestDirParameter(@TempDir File
testDir) throws IOExc
// Confirm the addNonSstToCopiedFiles gets added to list of copied files
fileSize = processFile(addNonSstToCopiedFiles, copyFiles, hardLinkFiles,
- toExcludeFiles, excluded, null);
- assertEquals(excluded.size(), 0);
+ toExcludeFiles, null);
assertEquals(copyFiles.size(), 2);
assertEquals(fileSize, 0);
assertEquals(copyFiles.get(addNonSstToCopiedFiles.getFileName().toString()).get(addNonSstToCopiedFiles),
@@ -626,25 +615,18 @@ void testProcessFileWithDestDirParameter(@TempDir File
testDir) throws IOExcepti
toExcludeFiles.put(excludeFileName.toString(),
ImmutableMap.of(excludeFile, destExcludeFile));
Map<String, Map<Path, Path>> copyFiles = new HashMap<>();
copyFiles.computeIfAbsent(copyFileName.toString(), (k) -> new
HashMap<>()).put(copyFile, destCopyFile);
- List<String> excluded = new ArrayList<>();
Map<Path, Path> hardLinkFiles = new HashMap<>();
long fileSize;
- // Confirm the exclude file gets added to the excluded list,
- // (and thus is excluded.)
fileSize = processFile(excludeFile, copyFiles, hardLinkFiles,
- toExcludeFiles, excluded, destExcludeFile.getParent());
- assertEquals(excluded.size(), 1);
- assertEquals((excluded.get(0)), destExcludeFile.toString());
+ toExcludeFiles, destExcludeFile.getParent());
assertEquals(copyFiles.size(), 1);
assertEquals(hardLinkFiles.size(), 0);
assertEquals(fileSize, 0);
- excluded = new ArrayList<>();
// Confirm the linkToExcludedFile gets added as a link.
fileSize = processFile(linkToExcludedFile, copyFiles, hardLinkFiles,
- toExcludeFiles, excluded, destLinkToExcludedFile.getParent());
- assertEquals(excluded.size(), 0);
+ toExcludeFiles, destLinkToExcludedFile.getParent());
assertEquals(copyFiles.size(), 1);
assertEquals(hardLinkFiles.size(), 1);
assertEquals(hardLinkFiles.get(destLinkToExcludedFile),
@@ -654,8 +636,7 @@ void testProcessFileWithDestDirParameter(@TempDir File
testDir) throws IOExcepti
// Confirm the file with same name as excluded file gets copied.
fileSize = processFile(sameNameAsExcludeFile, copyFiles, hardLinkFiles,
- toExcludeFiles, excluded, destSameNameAsExcludeFile.getParent());
- assertEquals(excluded.size(), 0);
+ toExcludeFiles, destSameNameAsExcludeFile.getParent());
assertEquals(copyFiles.size(), 2);
assertEquals(hardLinkFiles.size(), 0);
assertEquals(copyFiles.get(sameNameAsExcludeFile.getFileName().toString()).get(sameNameAsExcludeFile),
@@ -667,8 +648,7 @@ void testProcessFileWithDestDirParameter(@TempDir File
testDir) throws IOExcepti
// Confirm the file with same name as copy file gets copied.
fileSize = processFile(sameNameAsCopyFile, copyFiles, hardLinkFiles,
- toExcludeFiles, excluded, destSameNameAsCopyFile.getParent());
- assertEquals(excluded.size(), 0);
+ toExcludeFiles, destSameNameAsCopyFile.getParent());
assertEquals(copyFiles.size(), 2);
assertEquals(hardLinkFiles.size(), 0);
assertEquals(copyFiles.get(sameNameAsCopyFile.getFileName().toString()).get(sameNameAsCopyFile),
@@ -680,8 +660,7 @@ void testProcessFileWithDestDirParameter(@TempDir File
testDir) throws IOExcepti
// Confirm the linkToCopiedFile gets added as a link.
fileSize = processFile(linkToCopiedFile, copyFiles, hardLinkFiles,
- toExcludeFiles, excluded, destLinkToCopiedFile.getParent());
- assertEquals(excluded.size(), 0);
+ toExcludeFiles, destLinkToCopiedFile.getParent());
assertEquals(copyFiles.size(), 1);
assertEquals(hardLinkFiles.size(), 1);
assertEquals(hardLinkFiles.get(destLinkToCopiedFile),
@@ -691,8 +670,7 @@ void testProcessFileWithDestDirParameter(@TempDir File
testDir) throws IOExcepti
// Confirm the addToCopiedFiles gets added to list of copied files
fileSize = processFile(addToCopiedFiles, copyFiles, hardLinkFiles,
- toExcludeFiles, excluded, destAddToCopiedFiles.getParent());
- assertEquals(excluded.size(), 0);
+ toExcludeFiles, destAddToCopiedFiles.getParent());
assertEquals(copyFiles.size(), 2);
assertEquals(copyFiles.get(addToCopiedFiles.getFileName().toString()).get(addToCopiedFiles),
destAddToCopiedFiles);
@@ -702,8 +680,7 @@ void testProcessFileWithDestDirParameter(@TempDir File
testDir) throws IOExcepti
// Confirm the addNonSstToCopiedFiles gets added to list of copied files
fileSize = processFile(addNonSstToCopiedFiles, copyFiles, hardLinkFiles,
- toExcludeFiles, excluded, destAddNonSstToCopiedFiles.getParent());
- assertEquals(excluded.size(), 0);
+ toExcludeFiles, destAddNonSstToCopiedFiles.getParent());
assertEquals(copyFiles.size(), 2);
assertEquals(fileSize, 0);
assertEquals(copyFiles.get(addNonSstToCopiedFiles.getFileName().toString()).get(addNonSstToCopiedFiles),
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]