From e93d85ed8befbd674b53bda2db4d7dffa04d1ee0 Mon Sep 17 00:00:00 2001 From: mattl-netflix <63665634+mattl-netflix@users.noreply.github.com> Date: Sat, 1 Jun 2024 15:50:31 -0700 Subject: [PATCH] Restore incremental backups of secondary index sstables. (#1093) * Remove last usage of CL BackupFileType. * Remove unused remotePrefix() method. * Create DateRange::contains to consolidate recurring logic in a consistent way. * Change IMetaProxy API to return an ImmutableList of AbstractBackupPaths when fetching incrementals. The iterators are always fully materialized. Also, remove the now-redundant method from BackupRestoreUtil that merely wrapped the MetaProxy call. * Include incremental secondary index files in restore. --- .../netflix/priam/aws/RemoteBackupPath.java | 9 --- .../priam/backup/AbstractBackupPath.java | 9 --- .../priam/backup/BackupRestoreUtil.java | 14 ----- .../netflix/priam/backup/BackupStatusMgr.java | 13 +---- .../netflix/priam/backupv2/IMetaProxy.java | 4 +- .../netflix/priam/backupv2/MetaV2Proxy.java | 58 ++++++++----------- .../priam/resources/BackupServletV2.java | 26 ++++----- .../priam/restore/AbstractRestore.java | 8 ++- .../com/netflix/priam/utils/DateUtil.java | 4 ++ .../priam/backupv2/TestMetaV2Proxy.java | 31 +++++++--- .../netflix/priam/utils/TestDateUtils.java | 31 ++++++++++ 11 files changed, 100 insertions(+), 107 deletions(-) diff --git a/priam/src/main/java/com/netflix/priam/aws/RemoteBackupPath.java b/priam/src/main/java/com/netflix/priam/aws/RemoteBackupPath.java index ef8a6234c..061a0caa6 100644 --- a/priam/src/main/java/com/netflix/priam/aws/RemoteBackupPath.java +++ b/priam/src/main/java/com/netflix/priam/aws/RemoteBackupPath.java @@ -26,7 +26,6 @@ import java.nio.file.Path; import java.nio.file.Paths; import java.time.Instant; -import java.util.Date; import java.util.List; import javax.inject.Inject; @@ -149,14 +148,6 @@ public void parsePartialPrefix(String remoteFilePath) { token = remotePath.getName(3).toString(); } - @Override - public String remotePrefix(Date start, Date end, String location) { - return PATH_JOINER.join( - clusterPrefix(location), - instanceIdentity.getInstance().getToken(), - match(start, end)); - } - @Override public Path remoteV2Prefix(Path location, BackupFileType fileType) { if (location.getNameCount() <= 1) { diff --git a/priam/src/main/java/com/netflix/priam/backup/AbstractBackupPath.java b/priam/src/main/java/com/netflix/priam/backup/AbstractBackupPath.java index 7e61ccada..4ddbd22c5 100644 --- a/priam/src/main/java/com/netflix/priam/backup/AbstractBackupPath.java +++ b/priam/src/main/java/com/netflix/priam/backup/AbstractBackupPath.java @@ -47,7 +47,6 @@ public abstract class AbstractBackupPath implements Comparable getMostRecentSnapshotPaths( return snapshotPaths; } - public static List getIncrementalPaths( - AbstractBackupPath latestValidMetaFile, - DateUtil.DateRange dateRange, - IMetaProxy metaProxy) { - Instant snapshotTime; - snapshotTime = latestValidMetaFile.getLastModified(); - DateUtil.DateRange incrementalDateRange = - new DateUtil.DateRange(snapshotTime, dateRange.getEndTime()); - List incrementalPaths = new ArrayList<>(); - metaProxy.getIncrementals(incrementalDateRange).forEachRemaining(incrementalPaths::add); - return incrementalPaths; - } - public static Map> getFilter(String inputFilter) throws IllegalArgumentException { if (StringUtils.isEmpty(inputFilter)) return null; diff --git a/priam/src/main/java/com/netflix/priam/backup/BackupStatusMgr.java b/priam/src/main/java/com/netflix/priam/backup/BackupStatusMgr.java index 891a8a727..01f95c710 100644 --- a/priam/src/main/java/com/netflix/priam/backup/BackupStatusMgr.java +++ b/priam/src/main/java/com/netflix/priam/backup/BackupStatusMgr.java @@ -209,18 +209,7 @@ public List getLatestBackupMetadata(DateUtil.DateRange dateRange .stream() .filter(Objects::nonNull) .filter(backupMetadata -> backupMetadata.getStatus() == Status.FINISHED) - .filter( - backupMetadata -> - backupMetadata - .getStart() - .toInstant() - .compareTo(dateRange.getStartTime()) - >= 0 - && backupMetadata - .getStart() - .toInstant() - .compareTo(dateRange.getEndTime()) - <= 0) + .filter(backupMetadata -> dateRange.contains(backupMetadata.getStart().toInstant())) .sorted(Comparator.comparing(BackupMetadata::getStart).reversed()) .collect(Collectors.toList()); } diff --git a/priam/src/main/java/com/netflix/priam/backupv2/IMetaProxy.java b/priam/src/main/java/com/netflix/priam/backupv2/IMetaProxy.java index 2f79852b2..d16b3aa73 100644 --- a/priam/src/main/java/com/netflix/priam/backupv2/IMetaProxy.java +++ b/priam/src/main/java/com/netflix/priam/backupv2/IMetaProxy.java @@ -17,12 +17,12 @@ package com.netflix.priam.backupv2; +import com.google.common.collect.ImmutableList; import com.netflix.priam.backup.AbstractBackupPath; import com.netflix.priam.backup.BackupRestoreException; import com.netflix.priam.backup.BackupVerificationResult; import com.netflix.priam.utils.DateUtil; import java.nio.file.Path; -import java.util.Iterator; import java.util.List; /** Proxy to do management tasks for meta files. Created by aagrawal on 12/18/18. */ @@ -78,7 +78,7 @@ public interface IMetaProxy { * @param dateRange the time period to scan in the remote file system for incremental files. * @return iterator containing the list of path on the remote file system satisfying criteria. */ - Iterator getIncrementals(DateUtil.DateRange dateRange); + ImmutableList getIncrementals(DateUtil.DateRange dateRange); /** * Validate that all the files mentioned in the meta file actually exists on remote file system. diff --git a/priam/src/main/java/com/netflix/priam/backupv2/MetaV2Proxy.java b/priam/src/main/java/com/netflix/priam/backupv2/MetaV2Proxy.java index 6e5850c08..1073681c0 100644 --- a/priam/src/main/java/com/netflix/priam/backupv2/MetaV2Proxy.java +++ b/priam/src/main/java/com/netflix/priam/backupv2/MetaV2Proxy.java @@ -17,6 +17,7 @@ package com.netflix.priam.backupv2; +import com.google.common.collect.ImmutableList; import com.netflix.priam.backup.AbstractBackupPath; import com.netflix.priam.backup.BackupRestoreException; import com.netflix.priam.backup.BackupVerificationResult; @@ -31,8 +32,6 @@ import java.util.*; import javax.inject.Inject; import javax.inject.Provider; -import org.apache.commons.collections4.iterators.FilterIterator; -import org.apache.commons.collections4.iterators.TransformIterator; import org.apache.commons.io.FileUtils; import org.apache.commons.io.filefilter.FileFilterUtils; import org.apache.commons.io.filefilter.IOFileFilter; @@ -84,40 +83,34 @@ private String getMatch( } @Override - public Iterator getIncrementals(DateUtil.DateRange dateRange) { - String incrementalPrefix = getMatch(dateRange, AbstractBackupPath.BackupFileType.SST_V2); - String marker = - getMatch( - new DateUtil.DateRange(dateRange.getStartTime(), null), - AbstractBackupPath.BackupFileType.SST_V2); + public ImmutableList getIncrementals(DateUtil.DateRange dateRange) { + return new ImmutableList.Builder() + .addAll(getIncrementals(dateRange, AbstractBackupPath.BackupFileType.SST_V2)) + .addAll( + getIncrementals( + dateRange, AbstractBackupPath.BackupFileType.SECONDARY_INDEX_V2)) + .build(); + } + + private ImmutableList getIncrementals( + DateUtil.DateRange dateRange, AbstractBackupPath.BackupFileType type) { + String incrementalPrefix = getMatch(dateRange, type); + String marker = getMatch(new DateUtil.DateRange(dateRange.getStartTime(), null), type); logger.info( "Listing filesystem with prefix: {}, marker: {}, daterange: {}", incrementalPrefix, marker, dateRange); Iterator iterator = fs.listFileSystem(incrementalPrefix, null, marker); - Iterator transformIterator = - new TransformIterator<>( - iterator, - s -> { - AbstractBackupPath path = abstractBackupPathProvider.get(); - path.parseRemote(s); - return path; - }); - - return new FilterIterator<>( - transformIterator, - abstractBackupPath -> - (abstractBackupPath.getLastModified().isAfter(dateRange.getStartTime()) - && abstractBackupPath - .getLastModified() - .isBefore(dateRange.getEndTime())) - || abstractBackupPath - .getLastModified() - .equals(dateRange.getStartTime()) - || abstractBackupPath - .getLastModified() - .equals(dateRange.getEndTime())); + ImmutableList.Builder results = ImmutableList.builder(); + while (iterator.hasNext()) { + AbstractBackupPath path = abstractBackupPathProvider.get(); + path.parseRemote(iterator.next()); + if (dateRange.contains(path.getLastModified())) { + results.add(path); + } + } + return results.build(); } @Override @@ -136,10 +129,7 @@ public List findMetaFiles(DateUtil.DateRange dateRange) { AbstractBackupPath abstractBackupPath = abstractBackupPathProvider.get(); abstractBackupPath.parseRemote(iterator.next()); logger.debug("Meta file found: {}", abstractBackupPath); - if (abstractBackupPath.getLastModified().toEpochMilli() - >= dateRange.getStartTime().toEpochMilli() - && abstractBackupPath.getLastModified().toEpochMilli() - <= dateRange.getEndTime().toEpochMilli()) { + if (dateRange.contains(abstractBackupPath.getLastModified())) { metas.add(abstractBackupPath); } } diff --git a/priam/src/main/java/com/netflix/priam/resources/BackupServletV2.java b/priam/src/main/java/com/netflix/priam/resources/BackupServletV2.java index 5813b7599..70ea8bfd8 100644 --- a/priam/src/main/java/com/netflix/priam/resources/BackupServletV2.java +++ b/priam/src/main/java/com/netflix/priam/resources/BackupServletV2.java @@ -160,25 +160,19 @@ public Response validateV2SnapshotByDate( @Path("/list/{daterange}") public Response list(@PathParam("daterange") String daterange) throws Exception { DateUtil.DateRange dateRange = new DateUtil.DateRange(daterange); - // Find latest valid meta file. - Optional latestValidMetaFile = + Optional metaFile = BackupRestoreUtil.getLatestValidMetaPath(metaProxy, dateRange); - if (!latestValidMetaFile.isPresent()) { + if (!metaFile.isPresent()) { return Response.ok("No valid meta found!").build(); } - List allFiles = + List files = BackupRestoreUtil.getMostRecentSnapshotPaths( - latestValidMetaFile.get(), metaProxy, pathProvider); - allFiles.addAll( - BackupRestoreUtil.getIncrementalPaths( - latestValidMetaFile.get(), dateRange, metaProxy)); - - return Response.ok( - GsonJsonSerializer.getGson() - .toJson( - allFiles.stream() - .map(AbstractBackupPath::getRemotePath) - .collect(Collectors.toList()))) - .build(); + metaFile.get(), metaProxy, pathProvider); + DateUtil.DateRange incrementalDateRange = + new DateRange(metaFile.get().getLastModified(), dateRange.getEndTime()); + files.addAll(metaProxy.getIncrementals(incrementalDateRange)); + List remotePaths = + files.stream().map(AbstractBackupPath::getRemotePath).collect(Collectors.toList()); + return Response.ok(GsonJsonSerializer.getGson().toJson(remotePaths)).build(); } } diff --git a/priam/src/main/java/com/netflix/priam/restore/AbstractRestore.java b/priam/src/main/java/com/netflix/priam/restore/AbstractRestore.java index 11a133adf..1a427ff2c 100644 --- a/priam/src/main/java/com/netflix/priam/restore/AbstractRestore.java +++ b/priam/src/main/java/com/netflix/priam/restore/AbstractRestore.java @@ -220,9 +220,11 @@ public void restore(DateUtil.DateRange dateRange) throws Exception { BackupRestoreUtil.getMostRecentSnapshotPaths( latestValidMetaFile.get(), metaProxy, pathProvider); if (!config.skipIncrementalRestore()) { - allFiles.addAll( - BackupRestoreUtil.getIncrementalPaths( - latestValidMetaFile.get(), dateRange, metaProxy)); + DateUtil.DateRange incrementalDateRange = + new DateUtil.DateRange( + latestValidMetaFile.get().getLastModified(), + dateRange.getEndTime()); + allFiles.addAll(metaProxy.getIncrementals(incrementalDateRange)); } // Download snapshot which is listed in the meta file. diff --git a/priam/src/main/java/com/netflix/priam/utils/DateUtil.java b/priam/src/main/java/com/netflix/priam/utils/DateUtil.java index ff28cf96f..54c7e264c 100644 --- a/priam/src/main/java/com/netflix/priam/utils/DateUtil.java +++ b/priam/src/main/java/com/netflix/priam/utils/DateUtil.java @@ -215,6 +215,10 @@ public Instant getEndTime() { return endTime; } + public boolean contains(Instant instant) { + return startTime.compareTo(instant) <= 0 && endTime.compareTo(instant) >= 0; + } + public String toString() { return GsonJsonSerializer.getGson().toJson(this); } diff --git a/priam/src/test/java/com/netflix/priam/backupv2/TestMetaV2Proxy.java b/priam/src/test/java/com/netflix/priam/backupv2/TestMetaV2Proxy.java index 417fc7444..2ac6f5454 100644 --- a/priam/src/test/java/com/netflix/priam/backupv2/TestMetaV2Proxy.java +++ b/priam/src/test/java/com/netflix/priam/backupv2/TestMetaV2Proxy.java @@ -17,6 +17,8 @@ package com.netflix.priam.backupv2; +import com.google.common.collect.ImmutableList; +import com.google.common.truth.Truth; import com.google.inject.Guice; import com.google.inject.Injector; import com.netflix.priam.backup.AbstractBackupPath; @@ -32,7 +34,6 @@ import java.time.Instant; import java.time.temporal.ChronoUnit; import java.util.ArrayList; -import java.util.Iterator; import java.util.List; import java.util.stream.Collectors; import javax.inject.Provider; @@ -131,13 +132,16 @@ public void testGetSSTFilesFromMeta() throws Exception { @Test public void testGetIncrementalFiles() throws Exception { DateUtil.DateRange dateRange = new DateUtil.DateRange("202812071820,20281229"); - Iterator incrementals = metaProxy.getIncrementals(dateRange); - int i = 0; - while (incrementals.hasNext()) { - System.out.println(incrementals.next()); - i++; - } - Assert.assertEquals(3, i); + ImmutableList paths = metaProxy.getIncrementals(dateRange); + Truth.assertThat(paths).hasSize(4); + } + + @Test + public void testGetIncrementalFilesIncludesSecondaryIndexes() throws Exception { + DateUtil.DateRange dateRange = new DateUtil.DateRange("202812071820,20281229"); + ImmutableList paths = metaProxy.getIncrementals(dateRange); + Truth.assertThat(paths.get(3).getType()) + .isEqualTo(AbstractBackupPath.BackupFileType.SECONDARY_INDEX_V2); } @Test @@ -227,6 +231,17 @@ private List getRemoteFakeFiles() { "SNAPPY", "PLAINTEXT", "file4-Data.db")); + files.add( + Paths.get( + getPrefix(), + AbstractBackupPath.BackupFileType.SECONDARY_INDEX_V2.toString(), + "1859828420000", + "keyspace1", + "columnfamily1", + "index1", + "SNAPPY", + "PLAINTEXT", + "file5-Data.db")); files.add( Paths.get( getPrefix(), diff --git a/priam/src/test/java/com/netflix/priam/utils/TestDateUtils.java b/priam/src/test/java/com/netflix/priam/utils/TestDateUtils.java index 15cc254b3..a9701173a 100644 --- a/priam/src/test/java/com/netflix/priam/utils/TestDateUtils.java +++ b/priam/src/test/java/com/netflix/priam/utils/TestDateUtils.java @@ -17,6 +17,7 @@ package com.netflix.priam.utils; +import com.google.common.truth.Truth; import java.time.Duration; import java.time.Instant; import java.time.temporal.ChronoUnit; @@ -100,4 +101,34 @@ public void testFutureDateRangeValues() { Assert.assertEquals(Instant.ofEpochSecond(1830686460), dateRange.getEndTime()); Assert.assertEquals("1830", dateRange.match()); } + + @Test + public void testContainsInstantBeforeStartTime() { + DateUtil.DateRange dateRange = new DateUtil.DateRange("202404301200,202405011200"); + Truth.assertThat(dateRange.contains(Instant.parse("2024-04-29T12:00:00Z"))).isFalse(); + } + + @Test + public void testContainsInstantEqualToStartTime() { + DateUtil.DateRange dateRange = new DateUtil.DateRange("202404301200,202405011200"); + Truth.assertThat(dateRange.contains(Instant.parse("2024-04-30T12:00:00Z"))).isTrue(); + } + + @Test + public void testContainsInstantBetweenStartAndEndTimes() { + DateUtil.DateRange dateRange = new DateUtil.DateRange("202404301200,202405011200"); + Truth.assertThat(dateRange.contains(Instant.parse("2024-05-01T00:00:00Z"))).isTrue(); + } + + @Test + public void testContainsInstantEqualToEndTime() { + DateUtil.DateRange dateRange = new DateUtil.DateRange("202404301200,202405011200"); + Truth.assertThat(dateRange.contains(Instant.parse("2024-05-01T12:00:00Z"))).isTrue(); + } + + @Test + public void testContainsInstantAfterEndTime() { + DateUtil.DateRange dateRange = new DateUtil.DateRange("202404301200,202405011200"); + Truth.assertThat(dateRange.contains(Instant.parse("2024-05-02T12:00:00Z"))).isFalse(); + } }