Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -318,21 +318,21 @@ public static File getMetaDir(DBDefinition definition,
}

/**
* Scan the DB dir and return the existing SST files,
* including omSnapshot sst files.
* SSTs could be used for avoiding repeated download.
* Scan the DB dir and return the existing files,
* including omSnapshot files.
*
* @param db the file representing the DB to be scanned
* @return the list of SST file name. If db not exist, will return empty list
* @return the list of file names. If db not exist, will return empty list
*/
public static List<String> getExistingSstFiles(File db) throws IOException {
public static List<String> getExistingFiles(File db) throws IOException {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

To be honest getExistingFiles() shouldn't be in the HAUtils. Instead it belongs to RocksDBUtils.

List<String> sstList = new ArrayList<>();
if (!db.exists()) {
return sstList;
}
// Walk the db dir and get all sst files including omSnapshot files.
try (Stream<Path> files = Files.walk(db.toPath())) {
sstList = files.map(p -> p.getFileName().toString()).
sstList = files.filter(p -> p.toFile().isFile())
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

A potential problem is flagged by Gemini: the files object could contain relative path rather than the file name only. It's okay though for rocksdb as the directory is flat with no subdirectories.

jojochuang#598 (comment)

.map(p -> p.getFileName().toString()).
collect(Collectors.toList());
if (LOG.isDebugEnabled()) {
LOG.debug("Scanned SST files {} in {}.", sstList, db.getAbsolutePath());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -154,7 +154,7 @@ void checkLeaderConsistency(String currentLeader) throws IOException {
return;
}

List<String> files = HAUtils.getExistingSstFiles(candidateDir);
List<String> files = HAUtils.getExistingFiles(candidateDir);
if (!files.isEmpty()) {
LOG.warn("Candidate DB directory {} is not empty when last leader is " +
"null.", candidateDir);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -116,9 +116,10 @@ public void downloadSnapshot(String leaderNodeID, File targetFile)
concat(String.valueOf(a.length())))
.collect(Collectors.toList()));
try (OutputStream outputStream = Files.newOutputStream(targetFile.toPath())) {
writeDBCheckpointToStream(dbCheckpoint, outputStream,
new HashSet<>(HAUtils.getExistingSstFiles(
rdbSnapshotProvider.getCandidateDir())));
Set<String> existingSstFiles = HAUtils.getExistingFiles(rdbSnapshotProvider.getCandidateDir())
.stream().filter(fName -> fName.endsWith("sst"))
.collect(Collectors.toSet());
writeDBCheckpointToStream(dbCheckpoint, outputStream, existingSstFiles);
}
}
};
Expand All @@ -141,20 +142,20 @@ public void testDownloadDBSnapshotFromLeader() throws Exception {
assertTrue(candidateDir.exists());

DBCheckpoint checkpoint;
int before = HAUtils.getExistingSstFiles(
int before = HAUtils.getExistingFiles(
rdbSnapshotProvider.getCandidateDir()).size();
assertEquals(0, before);

// Get first snapshot
checkpoint = rdbSnapshotProvider.downloadDBSnapshotFromLeader(LEADER_ID);
File checkpointDir = checkpoint.getCheckpointLocation().toFile();
assertEquals(candidateDir, checkpointDir);
int first = HAUtils.getExistingSstFiles(
int first = HAUtils.getExistingFiles(
rdbSnapshotProvider.getCandidateDir()).size();

// Get second snapshot
checkpoint = rdbSnapshotProvider.downloadDBSnapshotFromLeader(LEADER_ID);
int second = HAUtils.getExistingSstFiles(
int second = HAUtils.getExistingFiles(
rdbSnapshotProvider.getCandidateDir()).size();
assertThat(second).withFailMessage("The second snapshot should have more SST files")
.isGreaterThan(first);
Expand All @@ -164,7 +165,7 @@ public void testDownloadDBSnapshotFromLeader() throws Exception {

// Get third snapshot
checkpoint = rdbSnapshotProvider.downloadDBSnapshotFromLeader(LEADER_ID);
int third = HAUtils.getExistingSstFiles(
int third = HAUtils.getExistingFiles(
rdbSnapshotProvider.getCandidateDir()).size();
assertThat(third).withFailMessage("The third snapshot should have more SST files")
.isGreaterThan(second);
Expand All @@ -173,7 +174,7 @@ public void testDownloadDBSnapshotFromLeader() throws Exception {

// Test cleanup candidateDB
rdbSnapshotProvider.init();
assertEquals(0, HAUtils.getExistingSstFiles(
assertEquals(0, HAUtils.getExistingFiles(
rdbSnapshotProvider.getCandidateDir()).size());
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -562,7 +562,7 @@ private IncrementData getNextIncrementalTarball(
Path increment = Paths.get(tempDir.toString(), "increment" + numKeys);
assertTrue(increment.toFile().mkdirs());
unTarLatestTarBall(followerOM, increment);
List<String> sstFiles = HAUtils.getExistingSstFiles(increment.toFile());
List<String> sstFiles = HAUtils.getExistingFiles(increment.toFile());
Path followerCandidatePath = followerOM.getOmSnapshotProvider().
getCandidateDir().toPath();

Expand Down Expand Up @@ -648,7 +648,7 @@ public void testInstallIncrementalSnapshotWithFailure() throws Exception {
// Corrupt the mixed checkpoint in the candidate DB dir
File followerCandidateDir = followerOM.getOmSnapshotProvider().
getCandidateDir();
List<String> sstList = HAUtils.getExistingSstFiles(followerCandidateDir);
List<String> sstList = HAUtils.getExistingFiles(followerCandidateDir);
assertThat(sstList.size()).isGreaterThan(0);
for (int i = 0; i < sstList.size(); i += 2) {
File victimSst = new File(followerCandidateDir, sstList.get(i));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -153,7 +153,7 @@ public void downloadSnapshot(String leaderNodeID, File targetFile)
connection.setRequestProperty("Content-Type", contentTypeValue);
connection.setDoOutput(true);
writeFormData(connection,
HAUtils.getExistingSstFiles(getCandidateDir()));
HAUtils.getExistingFiles(getCandidateDir()));

connection.connect();
int errorCode = connection.getResponseCode();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
package org.apache.hadoop.ozone.om;

import static org.apache.commons.io.file.PathUtils.copyDirectory;
import static org.apache.hadoop.hdds.utils.HAUtils.getExistingSstFiles;
import static org.apache.hadoop.hdds.utils.HAUtils.getExistingFiles;
import static org.apache.hadoop.ozone.OzoneConsts.OM_CHECKPOINT_DIR;
import static org.apache.hadoop.ozone.OzoneConsts.OM_DB_NAME;
import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
Expand All @@ -30,7 +30,6 @@
import static org.apache.hadoop.ozone.om.codec.OMDBDefinition.BUCKET_TABLE;
import static org.apache.hadoop.ozone.om.codec.OMDBDefinition.VOLUME_TABLE;
import static org.apache.hadoop.ozone.om.snapshot.OmSnapshotUtils.getINode;
import static org.apache.hadoop.ozone.om.snapshot.OmSnapshotUtils.truncateFileName;
import static org.assertj.core.api.Assertions.assertThat;
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
import static org.junit.jupiter.api.Assertions.assertEquals;
Expand Down Expand Up @@ -58,7 +57,6 @@
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import java.util.UUID;
import java.util.concurrent.TimeoutException;
import org.apache.hadoop.hdds.HddsConfigKeys;
Expand Down Expand Up @@ -417,44 +415,16 @@ public void testGetSnapshotInfo() throws IOException {
@Test
public void testExcludeUtilities() throws IOException {
File noLinkFile = new File(followerSnapDir2, "noLink.sst");

File nonSstFile = new File(followerSnapDir2, "nonSstFile");
// Confirm that the list of existing sst files is as expected.
List<String> existingSstList = getExistingSstFiles(candidateDir);
List<String> existingSstList = getExistingFiles(candidateDir);
Set<String> existingSstFiles = new HashSet<>(existingSstList);
int truncateLength = candidateDir.toString().length() + 1;
Set<String> expectedSstFiles = new HashSet<>(Arrays.asList(
s1File.toString().substring(truncateLength),
noLinkFile.toString().substring(truncateLength),
f1File.toString().substring(truncateLength)));
assertEquals(expectedSstFiles, existingSstFiles);

// Confirm that the excluded list is normalized as expected.
// (Normalizing means matches the layout on the leader.)
File leaderSstBackupDir = new File(leaderDir.toString(), "sstBackup");
assertTrue(leaderSstBackupDir.mkdirs());
File leaderTmpDir = new File(leaderDir.toString(), "tmp");
assertTrue(leaderTmpDir.mkdirs());
OMDBCheckpointServlet.DirectoryData sstBackupDir =
new OMDBCheckpointServlet.DirectoryData(leaderTmpDir.toPath(),
leaderSstBackupDir.toString());
Path srcSstBackup = Paths.get(sstBackupDir.getTmpDir().toString(),
"backup.sst");
Path destSstBackup = Paths.get(sstBackupDir.getOriginalDir().toString(),
"backup.sst");
truncateLength = leaderDir.toString().length() + 1;
existingSstList.add(truncateFileName(truncateLength, destSstBackup));
Map<String, Map<Path, Path>> normalizedMap =
OMDBCheckpointServlet.normalizeExcludeList(existingSstList,
leaderCheckpointDir.toPath(), sstBackupDir);
Map<String, Map<Path, Path>> expectedMap = new TreeMap<>();
Path s1 = Paths.get(leaderSnapDir1.toString(), "s1.sst");
Path noLink = Paths.get(leaderSnapDir2.toString(), "noLink.sst");
Path f1 = Paths.get(leaderCheckpointDir.toString(), "f1.sst");
expectedMap.put("s1.sst", ImmutableMap.of(s1, s1));
expectedMap.put("noLink.sst", ImmutableMap.of(noLink, noLink));
expectedMap.put("f1.sst", ImmutableMap.of(f1, f1));
expectedMap.put("backup.sst", ImmutableMap.of(srcSstBackup, destSstBackup));
assertEquals(expectedMap, new TreeMap<>(normalizedMap));
Set<String> expectedSstFileNames = new HashSet<>(Arrays.asList(
s1File.getName(),
noLinkFile.getName(),
f1File.getName(),
nonSstFile.getName()));
assertEquals(expectedSstFileNames, existingSstFiles);
}

/*
Expand Down