Skip to content

Commit

Permalink
Add support to skip pinned timestamp in remote segment garbage collector
Browse files Browse the repository at this point in the history
Signed-off-by: Sachin Kale <[email protected]>
  • Loading branch information
Sachin Kale committed Aug 2, 2024
1 parent 67a2e4c commit b8cbc22
Show file tree
Hide file tree
Showing 2 changed files with 414 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -51,15 +51,19 @@
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.TreeSet;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.stream.Collectors;

import static org.opensearch.index.store.RemoteSegmentStoreDirectory.MetadataFilenameUtils.SEPARATOR;

/**
* A RemoteDirectory extension for remote segment store. We need to make sure we don't overwrite a segment file once uploaded.
* In order to prevent segment overwrite which can occur due to two primary nodes for the same shard at the same time,
Expand Down Expand Up @@ -91,6 +95,8 @@ public final class RemoteSegmentStoreDirectory extends FilterDirectory implement

private final RemoteStoreLockManager mdLockManager;

private final Map<Long, String> metadataFilePinnedTimestampMap;

private final ThreadPool threadPool;

/**
Expand Down Expand Up @@ -132,6 +138,7 @@ public RemoteSegmentStoreDirectory(
this.remoteMetadataDirectory = remoteMetadataDirectory;
this.mdLockManager = mdLockManager;
this.threadPool = threadPool;
this.metadataFilePinnedTimestampMap = new HashMap<>();
this.logger = Loggers.getLogger(getClass(), shardId);
init();
}
Expand Down Expand Up @@ -176,6 +183,33 @@ public RemoteSegmentMetadata initializeToSpecificCommit(long primaryTerm, long c
return remoteSegmentMetadata;
}

/**
* Initializes the remote segment metadata to a specific timestamp.
*
* @param timestamp The timestamp to initialize the remote segment metadata to.
* @return The RemoteSegmentMetadata object corresponding to the specified timestamp, or null if no metadata file is found for that timestamp.
* @throws IOException If an I/O error occurs while reading the metadata file.
*/
public RemoteSegmentMetadata initializeToSpecificTimestamp(long timestamp) throws IOException {
List<String> metadataFiles = remoteMetadataDirectory.listFilesByPrefixInLexicographicOrder(
MetadataFilenameUtils.METADATA_PREFIX,
Integer.MAX_VALUE
);
Set<String> lockedMetadataFiles = getPinnedTimestampLockedFiles(metadataFiles, Set.of(timestamp));
if (lockedMetadataFiles.isEmpty()) {
return null;
}
assert lockedMetadataFiles.size() == 1 : "Expected exactly one metadata file but got " + lockedMetadataFiles;
String metadataFile = lockedMetadataFiles.iterator().next();
RemoteSegmentMetadata remoteSegmentMetadata = readMetadataFile(metadataFile);
if (remoteSegmentMetadata != null) {
this.segmentsUploadedToRemoteStore = new ConcurrentHashMap<>(remoteSegmentMetadata.getMetadata());
} else {
this.segmentsUploadedToRemoteStore = new ConcurrentHashMap<>();
}
return remoteSegmentMetadata;
}

/**
* Read the latest metadata file to get the list of segments uploaded to the remote segment store.
* We upload a metadata file per refresh, but it is not unique per refresh. Refresh metadata file is unique for a given commit.
Expand Down Expand Up @@ -349,6 +383,10 @@ static long getGeneration(String[] filenameTokens) {
return RemoteStoreUtils.invertLong(filenameTokens[2]);
}

static long getTimestamp(String[] filenameTokens) {
return RemoteStoreUtils.invertLong(filenameTokens[6]);
}

public static Tuple<String, String> getNodeIdByPrimaryTermAndGen(String filename) {
String[] tokens = filename.split(SEPARATOR);
if (tokens.length < 8) {
Expand Down Expand Up @@ -773,6 +811,9 @@ public void deleteStaleSegments(int lastNMetadataFilesToKeep) throws IOException
);
return;
}

// ToDo: Check last fetch status of pinned timestamps. If stale, return.

List<String> sortedMetadataFileList = remoteMetadataDirectory.listFilesByPrefixInLexicographicOrder(
MetadataFilenameUtils.METADATA_PREFIX,
Integer.MAX_VALUE
Expand All @@ -789,13 +830,21 @@ public void deleteStaleSegments(int lastNMetadataFilesToKeep) throws IOException
List<String> metadataFilesEligibleToDelete = new ArrayList<>(
sortedMetadataFileList.subList(lastNMetadataFilesToKeep, sortedMetadataFileList.size())
);
Set<String> allLockFiles;

// ToDo: fetch pinned timestamps along with last fetch status of pinned timestamps. If stale, return.
Set<Long> pinnedTimestamps = new HashSet<>();
Set<String> implicitLockedFiles = getPinnedTimestampLockedFiles(metadataFilesEligibleToDelete, pinnedTimestamps);
final Set<String> allLockFiles = new HashSet<>(implicitLockedFiles);

try {
allLockFiles = ((RemoteStoreMetadataLockManager) mdLockManager).fetchLockedMetadataFiles(MetadataFilenameUtils.METADATA_PREFIX);
allLockFiles.addAll(
((RemoteStoreMetadataLockManager) mdLockManager).fetchLockedMetadataFiles(MetadataFilenameUtils.METADATA_PREFIX)
);
} catch (Exception e) {
logger.error("Exception while fetching segment metadata lock files, skipping deleteStaleSegments", e);
return;
}

List<String> metadataFilesToBeDeleted = metadataFilesEligibleToDelete.stream()
.filter(metadataFile -> allLockFiles.contains(metadataFile) == false)
.collect(Collectors.toList());
Expand Down Expand Up @@ -859,6 +908,84 @@ public void deleteStaleSegments(int lastNMetadataFilesToKeep) throws IOException
logger.debug("deletedSegmentFiles={}", deletedSegmentFiles);
}

/**
* Determines and returns a set of metadata files that match provided pinned timestamps.
*
* This method identifies metadata files that are considered implicitly locked due to their timestamps
* matching or being the closest preceding timestamp to the pinned timestamps. It uses a caching mechanism
* to improve performance for previously processed timestamps.
*
* @param metadataFiles A list of metadata file names. Expected to be sorted in descending order of timestamp.
* @param pinnedTimestampSet A set of timestamps representing pinned points in time.
* @return A set of metadata file names that are implicitly locked based on the pinned timestamps.
*
* @implSpec The method performs the following steps:
* 1. Validates input parameters.
* 2. Updates the cache (metadataFilePinnedTimestampMap) to remove outdated entries.
* 3. Processes cached entries and identifies new timestamps to process.
* 4. For new timestamps, iterates through metadata files to find matching or closest preceding files.
* 5. Updates the cache with newly processed timestamps and their corresponding metadata files.
*
* @implNote The method currently sorts the metadata files, but this may be unnecessary if files are
* already sorted when fetched from the remote store.
*/
Set<String> getPinnedTimestampLockedFiles(List<String> metadataFiles, Set<Long> pinnedTimestampSet) {
Set<String> implicitLockedFiles = new HashSet<>();

if (metadataFiles == null || metadataFiles.isEmpty() || pinnedTimestampSet == null) {
return implicitLockedFiles;
}

// Remove entries for timestamps that are no longer pinned
metadataFilePinnedTimestampMap.keySet().retainAll(pinnedTimestampSet);

// Add cached entries and collect new timestamps
Set<Long> newPinnedTimestamps = new TreeSet<>(Collections.reverseOrder());
for (Long pinnedTimestamp : pinnedTimestampSet) {
String cachedFile = metadataFilePinnedTimestampMap.get(pinnedTimestamp);
if (cachedFile != null) {
implicitLockedFiles.add(cachedFile);
} else {
newPinnedTimestamps.add(pinnedTimestamp);
}
}

if (newPinnedTimestamps.isEmpty()) {
return implicitLockedFiles;
}

// Sort metadata files in descending order of timestamp
// ToDo: Do we really need this? Files fetched from remote store are already lexicographically sorted.
metadataFiles.sort(String::compareTo);

Iterator<Long> timestampIterator = newPinnedTimestamps.iterator();
Long currentPinnedTimestamp = timestampIterator.next();
long prevMdTimestamp = Long.MAX_VALUE;

for (String metadataFileName : metadataFiles) {
long currentMdTimestamp = MetadataFilenameUtils.getTimestamp(metadataFileName.split(SEPARATOR));

while (currentMdTimestamp <= currentPinnedTimestamp && prevMdTimestamp > currentPinnedTimestamp) {
implicitLockedFiles.add(metadataFileName);
// Do not cache entry for latest metadata file as the next metadata can also match the same pinned timestamp
if (prevMdTimestamp != Long.MAX_VALUE) {
metadataFilePinnedTimestampMap.put(currentPinnedTimestamp, metadataFileName);
}
if (timestampIterator.hasNext() == false) {
return implicitLockedFiles;
}
currentPinnedTimestamp = timestampIterator.next();
}
prevMdTimestamp = currentMdTimestamp;
}

return implicitLockedFiles;
}

public Map<Long, String> getMetadataFilePinnedTimestampMap() {
return Collections.unmodifiableMap(metadataFilePinnedTimestampMap);
}

public void deleteStaleSegmentsAsync(int lastNMetadataFilesToKeep) {
deleteStaleSegmentsAsync(lastNMetadataFilesToKeep, ActionListener.wrap(r -> {}, e -> {}));
}
Expand Down
Loading

0 comments on commit b8cbc22

Please sign in to comment.