Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
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 @@ -20,7 +20,6 @@
import static org.apache.hadoop.hdds.utils.HddsServerUtil.writeDBCheckpointToStream;
import static org.apache.hadoop.ozone.OzoneConsts.OZONE_DB_CHECKPOINT_REQUEST_FLUSH;
import static org.apache.hadoop.ozone.OzoneConsts.OZONE_DB_CHECKPOINT_REQUEST_TO_EXCLUDE_SST;
import static org.apache.hadoop.ozone.OzoneConsts.ROCKSDB_SST_SUFFIX;

import com.google.common.annotations.VisibleForTesting;
import java.io.File;
Expand Down Expand Up @@ -282,7 +281,7 @@ 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()));
Arrays.stream(sstParam).distinct().collect(Collectors.toList()));
logSstFileList(receivedSstFiles, "Received list of {} SST files to be excluded{}: {}", 5);
}
return receivedSstFiles;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_INFO_WAIT_DURATION_DEFAULT;
import static org.apache.hadoop.hdds.server.ServerUtils.getOzoneMetaDirPath;
import static org.apache.hadoop.ozone.OzoneConsts.DB_TRANSIENT_MARKER;
import static org.apache.hadoop.ozone.OzoneConsts.ROCKSDB_SST_SUFFIX;
import static org.apache.hadoop.ozone.OzoneConsts.TRANSACTION_INFO_KEY;

import com.google.common.annotations.VisibleForTesting;
Expand Down Expand Up @@ -319,28 +318,24 @@ 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;
}

int truncateLength = db.toString().length() + 1;
// Walk the db dir and get all sst files including omSnapshot files.
try (Stream<Path> files = Files.walk(db.toPath())) {
sstList =
files.filter(path -> path.toString().endsWith(ROCKSDB_SST_SUFFIX)).
map(p -> p.toString().substring(truncateLength)).
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());
LOG.debug("Scanned files {} in {}.", sstList, db.getAbsolutePath());
}
}
return sstList;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -108,14 +108,15 @@ public DBCheckpoint downloadDBSnapshotFromLeader(String leaderNodeID)
LOG.info("Prepare to download the snapshot from leader OM {} and " +
"reloading state from the snapshot.", leaderNodeID);
checkLeaderConsistency(leaderNodeID);
int numParts = 0;

while (true) {
String snapshotFileName = getSnapshotFileName(leaderNodeID);
File targetFile = new File(snapshotDir, snapshotFileName);
downloadSnapshot(leaderNodeID, targetFile);
LOG.info(
"Successfully download the latest snapshot {} from leader OM: {}",
targetFile, leaderNodeID);
LOG.info("Successfully download the latest snapshot {} from leader OM: {}, part : {}",
targetFile, leaderNodeID, numParts);
numParts++;

numDownloaded.incrementAndGet();
injectPause();
Expand Down Expand Up @@ -153,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,11 @@ 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") && !fName.equals(".sst"))
.collect(Collectors.toSet());
writeDBCheckpointToStream(dbCheckpoint, outputStream, existingSstFiles);
}
}
};
Expand All @@ -141,20 +143,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 +166,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 +175,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 @@ -227,7 +227,6 @@ void testContentsOfTarballWithSnapshot() throws Exception {
.forEachRemaining(snapInfo -> snapshotPaths.add(getSnapshotDBPath(snapInfo.getCheckpointDir())));
Set<String> inodesFromOmDataDir = new HashSet<>();
Set<String> inodesFromTarball = new HashSet<>();
Set<Path> allPathsInTarball = new HashSet<>();
try (Stream<Path> filesInTarball = Files.list(newDbDir.toPath())) {
List<Path> files = filesInTarball.collect(Collectors.toList());
for (Path p : files) {
Expand All @@ -237,7 +236,6 @@ void testContentsOfTarballWithSnapshot() throws Exception {
}
String inode = getInode(file.getName());
inodesFromTarball.add(inode);
allPathsInTarball.add(p);
}
}
Map<String, List<String>> hardLinkMapFromOmData = new HashMap<>();
Expand Down Expand Up @@ -269,10 +267,7 @@ void testContentsOfTarballWithSnapshot() throws Exception {
assertTrue(inodesFromTarball.containsAll(inodesFromOmDataDir));

// create hardlinks now
OmSnapshotUtils.createHardLinks(newDbDir.toPath());
for (Path old : allPathsInTarball) {
assertTrue(old.toFile().delete());
}
OmSnapshotUtils.createHardLinks(newDbDir.toPath(), true);
assertFalse(hardlinkFilePath.toFile().exists());
}

Expand Down Expand Up @@ -301,7 +296,7 @@ public void testSnapshotDBConsistency() throws Exception {
FileUtil.unTar(tempFile, newDbDir);
Set<Path> allPathsInTarball = getAllPathsInTarball(newDbDir);
// create hardlinks now
OmSnapshotUtils.createHardLinks(newDbDir.toPath());
OmSnapshotUtils.createHardLinks(newDbDir.toPath(), false);
for (Path old : allPathsInTarball) {
assertTrue(old.toFile().delete());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -245,8 +245,8 @@ public void testInstallSnapshot(@TempDir Path tempDir) throws Exception {
// Wait & for follower to update transactions to leader snapshot index.
// Timeout error if follower does not load update within 10s
GenericTestUtils.waitFor(() -> {
return followerOM.getOmRatisServer().getLastAppliedTermIndex().getIndex()
>= leaderOMSnapshotIndex - 1;
long index = followerOM.getOmRatisServer().getLastAppliedTermIndex().getIndex();
return index >= leaderOMSnapshotIndex - 1;
}, 100, 30_000);

long followerOMLastAppliedIndex =
Expand Down 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 Expand Up @@ -1003,6 +1003,7 @@ public void testInstallCorruptedCheckpointFailure() throws Exception {
DBCheckpoint leaderDbCheckpoint = leaderOM.getMetadataManager().getStore()
.getCheckpoint(false);
Path leaderCheckpointLocation = leaderDbCheckpoint.getCheckpointLocation();
OmSnapshotUtils.createHardLinks(leaderCheckpointLocation, true);
TransactionInfo leaderCheckpointTrxnInfo = OzoneManagerRatisUtils
.getTrxnInfoFromCheckpoint(conf, leaderCheckpointLocation);

Expand Down Expand Up @@ -1157,7 +1158,7 @@ public void pause() throws IOException {
// max size config. That way next time through, we get multiple
// tarballs.
if (count == 1) {
long sstSize = getSizeOfSstFiles(tarball);
long sstSize = getSizeOfFiles(tarball);
om.getConfiguration().setLong(
OZONE_OM_RATIS_SNAPSHOT_MAX_TOTAL_SST_SIZE_KEY, sstSize / 2);
// Now empty the tarball to restart the download
Expand All @@ -1166,21 +1167,20 @@ public void pause() throws IOException {
} else {
// Each time we get a new tarball add a set of
// its sst file to the list, (i.e. one per tarball.)
sstSetList.add(getSstFilenames(tarball));
sstSetList.add(getFilenames(tarball));
}
}

// Get Size of sstfiles in tarball.
private long getSizeOfSstFiles(File tarball) throws IOException {
private long getSizeOfFiles(File tarball) throws IOException {
FileUtil.unTar(tarball, tempDir.toFile());
List<Path> sstPaths = Files.walk(tempDir).filter(
path -> path.toString().endsWith(".sst")).
List<Path> sstPaths = Files.walk(tempDir).
collect(Collectors.toList());
long sstSize = 0;
long totalFileSize = 0;
for (Path sstPath : sstPaths) {
sstSize += Files.size(sstPath);
totalFileSize += Files.size(sstPath);
}
return sstSize;
return totalFileSize;
}

private void createEmptyTarball(File dummyTarFile)
Expand All @@ -1191,21 +1191,18 @@ private void createEmptyTarball(File dummyTarFile)
archiveOutputStream.close();
}

// Return a list of sst files in tarball.
private Set<String> getSstFilenames(File tarball)
// Return a list of files in tarball.
private Set<String> getFilenames(File tarball)
throws IOException {
Set<String> sstFilenames = new HashSet<>();
Set<String> fileNames = new HashSet<>();
try (TarArchiveInputStream tarInput =
new TarArchiveInputStream(Files.newInputStream(tarball.toPath()))) {
TarArchiveEntry entry;
while ((entry = tarInput.getNextTarEntry()) != null) {
String name = entry.getName();
if (name.toLowerCase().endsWith(".sst")) {
sstFilenames.add(entry.getName());
}
fileNames.add(entry.getName());
}
}
return sstFilenames;
return fileNames;
}

// Find the tarball in the dir.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -124,6 +124,8 @@ public void testDownloadCheckpoint() throws Exception {
private long getDownloadedSnapshotIndex(DBCheckpoint dbCheckpoint)
throws Exception {

OmSnapshotUtils.createHardLinks(dbCheckpoint.getCheckpointLocation(), true);

TransactionInfo trxnInfoFromCheckpoint =
OzoneManagerRatisUtils.getTrxnInfoFromCheckpoint(conf,
dbCheckpoint.getCheckpointLocation());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,13 +42,11 @@
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.UUID;
Expand Down Expand Up @@ -128,7 +126,6 @@ public void init() throws ServletException {
@Override
public 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);
Expand All @@ -149,7 +146,7 @@ public void processMetadataSnapshotRequest(HttpServletRequest request, HttpServl
long duration = Duration.between(start, end).toMillis();
LOG.info("Time taken to write the checkpoint to response output " +
"stream: {} milliseconds", duration);
logSstFileList(excludedSstList,
logSstFileList(receivedSstFiles,
"Excluded {} SST files from the latest checkpoint{}: {}", 5);
} catch (Exception e) {
LOG.error(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3980,7 +3980,7 @@ public synchronized TermIndex installSnapshotFromLeader(String leaderId) {
TermIndex termIndex = null;
try {
// Install hard links.
OmSnapshotUtils.createHardLinks(omDBCheckpoint.getCheckpointLocation());
OmSnapshotUtils.createHardLinks(omDBCheckpoint.getCheckpointLocation(), false);
termIndex = installCheckpoint(leaderId, omDBCheckpoint);
} catch (Exception ex) {
LOG.error("Failed to install snapshot from Leader OM.", ex);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ public OzoneManagerHttpServer(MutableConfigurationSource conf,
addServlet("serviceList", OZONE_OM_SERVICE_LIST_HTTP_ENDPOINT,
ServiceListJSONServlet.class);
addServlet("dbCheckpoint", OZONE_DB_CHECKPOINT_HTTP_ENDPOINT,
OMDBCheckpointServlet.class);
OMDBCheckpointServletInodeBasedXfer.class);
getWebAppContext().setAttribute(OzoneConsts.OM_CONTEXT_ATTRIBUTE, om);
}

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
Loading