From 281cd177fbc1ef35b951e0ca6669cff9cc636613 Mon Sep 17 00:00:00 2001 From: Matt Lehman Date: Thu, 29 Feb 2024 14:16:25 -0800 Subject: [PATCH] Remove remaining vestiges of backup v1. --- .../netflix/priam/aws/RemoteBackupPath.java | 102 ++--- .../priam/backup/AbstractBackupPath.java | 34 +- .../priam/backup/AbstractFileSystem.java | 24 -- .../priam/backup/BackupHelperImpl.java | 4 +- .../priam/backup/BackupRestoreUtil.java | 4 +- .../priam/backup/IBackupFileSystem.java | 11 - .../priam/backup/IncrementalBackup.java | 15 +- .../netflix/priam/backup/SnapshotBackup.java | 227 ----------- .../priam/backupv2/ForgottenFilesManager.java | 212 ----------- .../netflix/priam/backupv2/MetaV1Proxy.java | 192 ---------- .../netflix/priam/config/IConfiguration.java | 12 - .../priam/config/PriamConfiguration.java | 5 - .../priam/defaultimpl/PriamGuiceModule.java | 2 - .../priam/resources/BackupServlet.java | 285 -------------- .../priam/resources/BackupServletV2.java | 4 +- .../priam/restore/AbstractRestore.java | 10 +- .../backup/TestBackupScheduler.groovy | 73 ---- .../priam/aws/TestRemoteBackupPath.java | 87 ----- .../netflix/priam/backup/BRTestModule.java | 2 - .../priam/backup/FakeBackupFileSystem.java | 59 +-- .../com/netflix/priam/backup/TestBackup.java | 37 +- .../netflix/priam/backup/TestBackupFile.java | 81 ++-- .../priam/backup/TestBackupVerification.java | 9 - .../priam/backup/TestFileIterator.java | 353 ------------------ .../priam/backup/TestS3FileSystem.java | 6 +- .../priam/backupv2/TestBackupV2Service.java | 4 - .../backupv2/TestForgottenFileManager.java | 235 ------------ .../TestBackupNotificationMgr.java | 8 +- .../netflix/priam/restore/TestRestore.java | 107 ++++-- .../netflix/priam/stream/StreamingTest.java | 60 +-- .../fake-app2_meta_v2_201108110130.json | 65 ++++ .../fake-app_meta_v2_201108110029.json | 0 .../fake-app_meta_v2_201108110130.json | 65 ++++ .../fake-app_meta_v2_201108110615.json | 34 ++ 34 files changed, 342 insertions(+), 2086 deletions(-) delete mode 100644 priam/src/main/java/com/netflix/priam/backup/SnapshotBackup.java delete mode 100644 priam/src/main/java/com/netflix/priam/backupv2/ForgottenFilesManager.java delete mode 100644 priam/src/main/java/com/netflix/priam/backupv2/MetaV1Proxy.java delete mode 100644 priam/src/main/java/com/netflix/priam/resources/BackupServlet.java delete mode 100644 priam/src/test/groovy/com.netflix.priam/backup/TestBackupScheduler.groovy delete mode 100644 priam/src/test/java/com/netflix/priam/backup/TestFileIterator.java delete mode 100644 priam/src/test/java/com/netflix/priam/backupv2/TestForgottenFileManager.java create mode 100644 priam/src/test/resources/fake-app2_meta_v2_201108110130.json create mode 100644 priam/src/test/resources/fake-app_meta_v2_201108110029.json create mode 100644 priam/src/test/resources/fake-app_meta_v2_201108110130.json create mode 100644 priam/src/test/resources/fake-app_meta_v2_201108110615.json 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 2eb1fd09f..ef8a6234c 100644 --- a/priam/src/main/java/com/netflix/priam/aws/RemoteBackupPath.java +++ b/priam/src/main/java/com/netflix/priam/aws/RemoteBackupPath.java @@ -19,19 +19,15 @@ import com.google.api.client.util.Lists; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; import com.netflix.priam.backup.AbstractBackupPath; import com.netflix.priam.compress.CompressionType; import com.netflix.priam.config.IConfiguration; import com.netflix.priam.identity.InstanceIdentity; -import com.netflix.priam.utils.DateUtil; import java.nio.file.Path; import java.nio.file.Paths; import java.time.Instant; -import java.util.Arrays; import java.util.Date; import java.util.List; -import java.util.Optional; import javax.inject.Inject; /** @@ -40,18 +36,13 @@ * this instance. */ public class RemoteBackupPath extends AbstractBackupPath { - private static final ImmutableSet V2_ONLY_FILE_TYPES = - ImmutableSet.of( - BackupFileType.META_V2, - BackupFileType.SST_V2, - BackupFileType.SECONDARY_INDEX_V2); @Inject public RemoteBackupPath(IConfiguration config, InstanceIdentity factory) { super(config, factory); } - private ImmutableList.Builder getV2Prefix() { + private ImmutableList.Builder getPrefix() { ImmutableList.Builder prefix = ImmutableList.builder(); prefix.add(baseDir, prependHash(clusterName), token); return prefix; @@ -82,8 +73,8 @@ private String removeHash(String appNameWithHash) { * Another major difference w.r.t. V1 is having no distinction between SNAP and SST files as we upload SSTables only * once to remote file system. */ - private String getV2Location() { - ImmutableList.Builder parts = getV2Prefix(); + private String getLocation() { + ImmutableList.Builder parts = getPrefix(); // JDK-8177809 truncate to seconds to ensure consistent behavior with our old method of // getting lastModified time (File::lastModified) in Java 8. long lastModified = getLastModified().toEpochMilli() / 1_000L * 1_000L; @@ -98,7 +89,25 @@ private String getV2Location() { return toPath(parts.build()).toString(); } - private void parseV2Location(Path remotePath) { + private Path toPath(ImmutableList parts) { + return Paths.get(parts.get(0), parts.subList(1, parts.size()).toArray(new String[0])); + } + + /** + * Format of backup path: 1. For old style backups: + * BASE/REGION/CLUSTER/TOKEN/[SNAPSHOTTIME]/[SST|SNAP|META]/KEYSPACE/COLUMNFAMILY/FILE + * + *

2. For new style backups (SnapshotMetaService) + * BASE/[cluster_name_hash]_cluster/TOKEN//[META_V2|SST_V2]/KEYSPACE/COLUMNFAMILY/[last_modified_time_ms]/FILE.compression + */ + @Override + public String getRemotePath() { + return getLocation(); + } + + @Override + public void parseRemote(String remoteFilepath) { + Path remotePath = Paths.get(remoteFilepath); Preconditions.checkArgument( remotePath.getNameCount() >= 8, String.format("%s has fewer than %d parts", remotePath, 8)); @@ -128,36 +137,9 @@ private void parseV2Location(Path remotePath) { Paths.get(config.getDataFileLocation(), parts.toArray(new String[] {})).toFile(); } - private String getV1Location() { - ImmutableList.Builder parts = ImmutableList.builder(); - String timeString = DateUtil.formatyyyyMMddHHmm(time); - parts.add(baseDir, region, clusterName, token, timeString, type.toString()); - if (BackupFileType.isDataFile(type)) { - parts.add(keyspace, columnFamily); - } - parts.add(fileName); - return toPath(parts.build()).toString(); - } - - private Path toPath(ImmutableList parts) { - return Paths.get(parts.get(0), parts.subList(1, parts.size()).toArray(new String[0])); - } - - private void parseV1Location(Path remotePath) { - Preconditions.checkArgument( - remotePath.getNameCount() >= 7, - String.format("%s has fewer than %d parts", remotePath, 7)); - parseV1Prefix(remotePath); - time = DateUtil.getDate(remotePath.getName(4).toString()); - type = BackupFileType.valueOf(remotePath.getName(5).toString()); - if (BackupFileType.isDataFile(type)) { - keyspace = remotePath.getName(6).toString(); - columnFamily = remotePath.getName(7).toString(); - } - fileName = remotePath.getName(remotePath.getNameCount() - 1).toString(); - } - - private void parseV1Prefix(Path remotePath) { + @Override + public void parsePartialPrefix(String remoteFilePath) { + Path remotePath = Paths.get(remoteFilePath); Preconditions.checkArgument( remotePath.getNameCount() >= 4, String.format("%s needs %d parts to parse prefix", remotePath, 4)); @@ -167,38 +149,6 @@ private void parseV1Prefix(Path remotePath) { token = remotePath.getName(3).toString(); } - /** - * Format of backup path: 1. For old style backups: - * BASE/REGION/CLUSTER/TOKEN/[SNAPSHOTTIME]/[SST|SNAP|META]/KEYSPACE/COLUMNFAMILY/FILE - * - *

2. For new style backups (SnapshotMetaService) - * BASE/[cluster_name_hash]_cluster/TOKEN//[META_V2|SST_V2]/KEYSPACE/COLUMNFAMILY/[last_modified_time_ms]/FILE.compression - */ - @Override - public String getRemotePath() { - return V2_ONLY_FILE_TYPES.contains(type) ? getV2Location() : getV1Location(); - } - - @Override - public void parseRemote(String remotePath) { - // Hack to determine type in advance of parsing. Will disappear once v1 is retired - Optional inferredType = - Arrays.stream(BackupFileType.values()) - .filter(bft -> remotePath.contains(PATH_SEP + bft.toString() + PATH_SEP)) - .findAny() - .filter(V2_ONLY_FILE_TYPES::contains); - if (inferredType.isPresent()) { - parseV2Location(Paths.get(remotePath)); - } else { - parseV1Location(Paths.get(remotePath)); - } - } - - @Override - public void parsePartialPrefix(String remoteFilePath) { - parseV1Prefix(Paths.get(remoteFilePath)); - } - @Override public String remotePrefix(Date start, Date end, String location) { return PATH_JOINER.join( @@ -217,7 +167,7 @@ public Path remoteV2Prefix(Path location, BackupFileType fileType) { clusterName = removeHash(location.getName(2).toString()); } token = instanceIdentity.getInstance().getToken(); - ImmutableList.Builder parts = getV2Prefix(); + ImmutableList.Builder parts = getPrefix(); parts.add(fileType.toString()); return toPath(parts.build()); } 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 e5c9a69fa..7e61ccada 100644 --- a/priam/src/main/java/com/netflix/priam/backup/AbstractBackupPath.java +++ b/priam/src/main/java/com/netflix/priam/backup/AbstractBackupPath.java @@ -48,28 +48,18 @@ public abstract class AbstractBackupPath implements Comparable DATA_FILE_TYPES = - ImmutableSet.of(SECONDARY_INDEX_V2, SNAP, SST, SST_V2); - - private static ImmutableSet V2_FILE_TYPES = - ImmutableSet.of(SECONDARY_INDEX_V2, SST_V2, META_V2); + ImmutableSet.of(SECONDARY_INDEX_V2, SST_V2); public static boolean isDataFile(BackupFileType type) { return DATA_FILE_TYPES.contains(type); } - public static boolean isV2(BackupFileType type) { - return V2_FILE_TYPES.contains(type); - } - public static BackupFileType fromString(String s) throws BackupRestoreException { try { return BackupFileType.valueOf(s); @@ -88,7 +78,6 @@ public static BackupFileType fromString(String s) throws BackupRestoreException protected String token; protected String region; protected String indexDir; - protected Date time; private long size; // uncompressed file size private long compressedFileSize = 0; protected final InstanceIdentity instanceIdentity; @@ -136,8 +125,6 @@ public void parseLocal(File file, BackupFileType type) { if (BackupFileType.isDataFile(type)) { this.keyspace = parts[0]; this.columnFamily = parts[1]; - } - if (BackupFileType.isDataFile(type)) { Optional folder = BackupFolder.fromName(parts[2]); this.isIncremental = folder.filter(BackupFolder.BACKUPS::equals).isPresent(); if (type == BackupFileType.SECONDARY_INDEX_V2) { @@ -146,16 +133,6 @@ public void parseLocal(File file, BackupFileType type) { this.indexDir = parts[index]; } } - - /* - 1. For old style snapshots, make this value to time at which backup was executed. - 2. This is to ensure that all the files from the snapshot are uploaded under single directory in remote file system. - 3. For META files we always override the time field via @link{Metadata#decorateMetaJson} - */ - this.time = - type == BackupFileType.SNAP - ? DateUtil.getDate(parts[3]) - : new Date(lastModified.toEpochMilli()); } /** Given a date range, find a common string prefix Eg: 20120212, 20120213 = 2012021 */ @@ -180,7 +157,6 @@ public File newRestoreFile() { PATH_JOINER.join(dataDir, keyspace, columnFamily, indexDir, fileName); return_ = new File(restoreFileName); break; - case META: case META_V2: return_ = new File(PATH_JOINER.join(config.getDataFileLocation(), fileName)); break; @@ -254,14 +230,6 @@ public String getRegion() { return region; } - public Date getTime() { - return time; - } - - public void setTime(Date time) { - this.time = time; - } - /* @return original, uncompressed file size */ diff --git a/priam/src/main/java/com/netflix/priam/backup/AbstractFileSystem.java b/priam/src/main/java/com/netflix/priam/backup/AbstractFileSystem.java index ba7f94865..a0a3eccb6 100644 --- a/priam/src/main/java/com/netflix/priam/backup/AbstractFileSystem.java +++ b/priam/src/main/java/com/netflix/priam/backup/AbstractFileSystem.java @@ -44,7 +44,6 @@ import java.util.concurrent.*; 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.lang3.StringUtils; @@ -308,29 +307,6 @@ public Iterator listPrefixes(Date date) { }); } - @Override - public Iterator list(String path, Date start, Date till) { - String prefix = pathProvider.get().remotePrefix(start, till, path); - Iterator fileIterator = listFileSystem(prefix, null, null); - - @SuppressWarnings("unchecked") - TransformIterator transformIterator = - new TransformIterator( - fileIterator, - remotePath -> { - AbstractBackupPath abstractBackupPath = pathProvider.get(); - abstractBackupPath.parseRemote(remotePath.toString()); - return abstractBackupPath; - }); - - return new FilterIterator<>( - transformIterator, - abstractBackupPath -> - (abstractBackupPath.getTime().after(start) - && abstractBackupPath.getTime().before(till)) - || abstractBackupPath.getTime().equals(start)); - } - @Override public int getUploadTasksQueued() { return tasksQueued.size(); diff --git a/priam/src/main/java/com/netflix/priam/backup/BackupHelperImpl.java b/priam/src/main/java/com/netflix/priam/backup/BackupHelperImpl.java index 50ce7d533..8b469f296 100644 --- a/priam/src/main/java/com/netflix/priam/backup/BackupHelperImpl.java +++ b/priam/src/main/java/com/netflix/priam/backup/BackupHelperImpl.java @@ -86,9 +86,7 @@ public ImmutableSet getBackupPaths( private CompressionType getCorrectCompressionAlgorithm( AbstractBackupPath path, Set compressedFiles) { - if (!AbstractBackupPath.BackupFileType.isV2(path.getType()) - || path.getLastModified().toEpochMilli() - < config.getCompressionTransitionEpochMillis()) { + if (path.getLastModified().toEpochMilli() < config.getCompressionTransitionEpochMillis()) { return CompressionType.SNAPPY; } String file = path.getFileName(); diff --git a/priam/src/main/java/com/netflix/priam/backup/BackupRestoreUtil.java b/priam/src/main/java/com/netflix/priam/backup/BackupRestoreUtil.java index c97ba72e3..5c8d01202 100644 --- a/priam/src/main/java/com/netflix/priam/backup/BackupRestoreUtil.java +++ b/priam/src/main/java/com/netflix/priam/backup/BackupRestoreUtil.java @@ -19,7 +19,6 @@ import com.google.common.collect.ImmutableMap; import com.netflix.priam.backupv2.IMetaProxy; -import com.netflix.priam.backupv2.MetaV2Proxy; import com.netflix.priam.utils.DateUtil; import java.nio.file.Path; import java.time.Instant; @@ -84,8 +83,7 @@ public static List getIncrementalPaths( DateUtil.DateRange dateRange, IMetaProxy metaProxy) { Instant snapshotTime; - if (metaProxy instanceof MetaV2Proxy) snapshotTime = latestValidMetaFile.getLastModified(); - else snapshotTime = latestValidMetaFile.getTime().toInstant(); + snapshotTime = latestValidMetaFile.getLastModified(); DateUtil.DateRange incrementalDateRange = new DateUtil.DateRange(snapshotTime, dateRange.getEndTime()); List incrementalPaths = new ArrayList<>(); diff --git a/priam/src/main/java/com/netflix/priam/backup/IBackupFileSystem.java b/priam/src/main/java/com/netflix/priam/backup/IBackupFileSystem.java index e33ff3bfc..84a75b4f4 100644 --- a/priam/src/main/java/com/netflix/priam/backup/IBackupFileSystem.java +++ b/priam/src/main/java/com/netflix/priam/backup/IBackupFileSystem.java @@ -106,17 +106,6 @@ default String getShard() { */ Path getPrefix(); - /** - * List all files in the backup location for the specified time range. - * - * @param path This is used as the `prefix` for listing files in the filesystem. All the files - * that start with this prefix will be returned. - * @param start Start date of the file upload. - * @param till End date of the file upload. - * @return Iterator of the AbstractBackupPath matching the criteria. - */ - Iterator list(String path, Date start, Date till); - /** Get a list of prefixes for the cluster available in backup for the specified date */ Iterator listPrefixes(Date date); diff --git a/priam/src/main/java/com/netflix/priam/backup/IncrementalBackup.java b/priam/src/main/java/com/netflix/priam/backup/IncrementalBackup.java index 8e2a9646e..c757db36e 100644 --- a/priam/src/main/java/com/netflix/priam/backup/IncrementalBackup.java +++ b/priam/src/main/java/com/netflix/priam/backup/IncrementalBackup.java @@ -90,10 +90,8 @@ public static boolean isEnabled( // Once backup 1.0 is gone, we should not check for enableV2Backups. enabled = (configuration.isIncrementalBackupEnabled() - && (SnapshotBackup.isBackupEnabled(configuration) - || (backupRestoreConfig.enableV2Backups() - && SnapshotMetaTask.isBackupEnabled( - backupRestoreConfig)))); + && (backupRestoreConfig.enableV2Backups() + && SnapshotMetaTask.isBackupEnabled(backupRestoreConfig))); logger.info("Incremental backups are enabled: {}", enabled); if (!enabled) { @@ -115,21 +113,18 @@ public String getName() { @Override protected void processColumnFamily(File backupDir) throws Exception { - BackupFileType fileType = - backupRestoreConfig.enableV2Backups() ? BackupFileType.SST_V2 : BackupFileType.SST; - // upload SSTables and components ImmutableList> futures = backupHelper.uploadAndDeleteAllFiles( - backupDir, fileType, config.enableAsyncIncremental()); + backupDir, BackupFileType.SST_V2, config.enableAsyncIncremental()); Futures.whenAllComplete(futures).call(() -> null, MoreExecutors.directExecutor()); // Next, upload secondary indexes - fileType = BackupFileType.SECONDARY_INDEX_V2; + boolean async = config.enableAsyncIncremental(); for (File directory : getSecondaryIndexDirectories(backupDir)) { futures = backupHelper.uploadAndDeleteAllFiles( - directory, fileType, config.enableAsyncIncremental()); + directory, BackupFileType.SECONDARY_INDEX_V2, async); if (futures.stream().allMatch(ListenableFuture::isDone)) { deleteIfEmpty(directory); } else { diff --git a/priam/src/main/java/com/netflix/priam/backup/SnapshotBackup.java b/priam/src/main/java/com/netflix/priam/backup/SnapshotBackup.java deleted file mode 100644 index ff82758ab..000000000 --- a/priam/src/main/java/com/netflix/priam/backup/SnapshotBackup.java +++ /dev/null @@ -1,227 +0,0 @@ -/* - * Copyright 2013 Netflix, Inc. - * - * Licensed 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 com.netflix.priam.backup; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; -import com.google.common.util.concurrent.ListenableFuture; -import com.netflix.priam.backup.AbstractBackupPath.BackupFileType; -import com.netflix.priam.backupv2.ForgottenFilesManager; -import com.netflix.priam.config.IConfiguration; -import com.netflix.priam.connection.CassandraOperations; -import com.netflix.priam.health.CassandraMonitor; -import com.netflix.priam.identity.InstanceIdentity; -import com.netflix.priam.scheduler.CronTimer; -import com.netflix.priam.scheduler.TaskTimer; -import com.netflix.priam.utils.DateUtil; -import com.netflix.priam.utils.ThreadSleeper; -import java.io.File; -import java.nio.file.DirectoryStream; -import java.nio.file.Files; -import java.nio.file.Path; -import java.time.Instant; -import java.util.*; -import java.util.concurrent.Future; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; -import javax.inject.Inject; -import javax.inject.Singleton; -import org.apache.commons.io.FileUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** Task for running daily snapshots */ -@Singleton -public class SnapshotBackup extends AbstractBackup { - private static final Logger logger = LoggerFactory.getLogger(SnapshotBackup.class); - public static final String JOBNAME = "SnapshotBackup"; - private final MetaData metaData; - private final ThreadSleeper sleeper = new ThreadSleeper(); - private static final long WAIT_TIME_MS = 60 * 1000 * 10; - private final InstanceIdentity instanceIdentity; - private final IBackupStatusMgr snapshotStatusMgr; - private final BackupRestoreUtil backupRestoreUtil; - private final ForgottenFilesManager forgottenFilesManager; - private String snapshotName = null; - private Instant snapshotInstant = DateUtil.getInstant(); - private List abstractBackupPaths = null; - private final CassandraOperations cassandraOperations; - private final BackupHelper backupHelper; - private static final Lock lock = new ReentrantLock(); - - @Inject - public SnapshotBackup( - IConfiguration config, - BackupHelper backupHelper, - MetaData metaData, - IBackupStatusMgr snapshotStatusMgr, - InstanceIdentity instanceIdentity, - CassandraOperations cassandraOperations, - ForgottenFilesManager forgottenFilesManager) { - super(config); - this.backupHelper = backupHelper; - this.metaData = metaData; - this.snapshotStatusMgr = snapshotStatusMgr; - this.instanceIdentity = instanceIdentity; - this.cassandraOperations = cassandraOperations; - backupRestoreUtil = - new BackupRestoreUtil( - config.getSnapshotIncludeCFList(), config.getSnapshotExcludeCFList()); - this.forgottenFilesManager = forgottenFilesManager; - } - - @Override - public void execute() throws Exception { - // If Cassandra is started then only start Snapshot Backup - while (!CassandraMonitor.hasCassadraStarted()) { - logger.debug( - "Cassandra has not yet started, hence Snapshot Backup will start after [" - + WAIT_TIME_MS / 1000 - + "] secs ..."); - sleeper.sleep(WAIT_TIME_MS); - } - - // Do not allow more than one snapshot to run at the same time. This is possible as this - // happens on CRON. - if (!lock.tryLock()) { - logger.warn("Snapshot Operation is already running! Try again later."); - throw new Exception("Snapshot Operation already running"); - } - - try { - // Clean up all the backup directories, if any. - cleanOldBackups(config); - executeSnapshot(); - } finally { - lock.unlock(); - } - } - - private void executeSnapshot() throws Exception { - Date startTime = Calendar.getInstance(TimeZone.getTimeZone("GMT")).getTime(); - snapshotName = DateUtil.formatyyyyMMddHHmm(startTime); - snapshotInstant = DateUtil.getInstant(); - String token = instanceIdentity.getInstance().getToken(); - - // Save start snapshot status - BackupMetadata backupMetadata = - new BackupMetadata(BackupVersion.SNAPSHOT_BACKUP, token, startTime); - snapshotStatusMgr.start(backupMetadata); - - try { - logger.info("Starting snapshot {}", snapshotName); - cassandraOperations.takeSnapshot(snapshotName); - backupMetadata.setCassandraSnapshotSuccess(true); - - // Collect all snapshot dir's under keyspace dir's - abstractBackupPaths = Lists.newArrayList(); - // Try to upload all the files as part of snapshot. If there is any error, there will be - // an exception and snapshot will be considered as failure. - initiateBackup(SNAPSHOT_FOLDER, backupRestoreUtil); - - // All the files are uploaded successfully as part of snapshot. - // pre condition notify of meta.json upload - File tmpMetaFile = metaData.createTmpMetaFile(); - // Note: no need to remove this temp as it is done within createTmpMetaFile() - AbstractBackupPath metaJsonAbp = metaData.decorateMetaJson(tmpMetaFile, snapshotName); - - // Upload meta file - AbstractBackupPath metaJson = metaData.set(abstractBackupPaths, snapshotName); - - logger.info("Snapshot upload complete for {}", snapshotName); - backupMetadata.setSnapshotLocation( - config.getBackupPrefix() + File.separator + metaJson.getRemotePath()); - snapshotStatusMgr.finish(backupMetadata); - } catch (Exception e) { - logger.error( - "Exception occurred while taking snapshot: {}. Exception: {}", - snapshotName, - e.getLocalizedMessage()); - snapshotStatusMgr.failed(backupMetadata); - throw e; - } finally { - try { - cassandraOperations.clearSnapshot(snapshotName); - } catch (Exception e) { - logger.error(e.getMessage(), e); - } - } - } - - private File getValidSnapshot(File snpDir, String snapshotName) { - for (File snapshotDir : snpDir.listFiles()) - if (snapshotDir.getName().matches(snapshotName)) return snapshotDir; - return null; - } - - @Override - public String getName() { - return JOBNAME; - } - - public static boolean isBackupEnabled(IConfiguration config) throws Exception { - return (getTimer(config) != null); - } - - public static TaskTimer getTimer(IConfiguration config) throws Exception { - TaskTimer timer = CronTimer.getCronTimer(JOBNAME, config.getBackupCronExpression()); - if (timer == null) { - // Clean up all the backup directories, if any. - cleanOldBackups(config); - } - return timer; - } - - private static void cleanOldBackups(IConfiguration configuration) throws Exception { - Set backupPaths = AbstractBackup.getBackupDirectories(configuration, SNAPSHOT_FOLDER); - for (Path backupDirPath : backupPaths) - try (DirectoryStream directoryStream = - Files.newDirectoryStream(backupDirPath, path -> Files.isDirectory(path))) { - for (Path backupDir : directoryStream) { - if (isValidBackupDir(backupDir)) { - FileUtils.deleteDirectory(backupDir.toFile()); - } - } - } - } - - @Override - protected void processColumnFamily(File backupDir) throws Exception { - File snapshotDir = getValidSnapshot(backupDir, snapshotName); - - if (snapshotDir == null) { - logger.warn("{} folder does not contain {} snapshots", backupDir, snapshotName); - return; - } - - forgottenFilesManager.findAndMoveForgottenFiles(snapshotInstant, snapshotDir); - // Add files to this dir - - ImmutableList> futures = - backupHelper.uploadAndDeleteAllFiles( - snapshotDir, BackupFileType.SNAP, config.enableAsyncSnapshot()); - for (Future future : futures) { - abstractBackupPaths.add(future.get()); - } - } - - private static boolean isValidBackupDir(Path backupDir) { - String backupDirName = backupDir.toFile().getName(); - // Check if it of format yyyyMMddHHmm - return (DateUtil.getDate(backupDirName) != null); - } -} diff --git a/priam/src/main/java/com/netflix/priam/backupv2/ForgottenFilesManager.java b/priam/src/main/java/com/netflix/priam/backupv2/ForgottenFilesManager.java deleted file mode 100644 index 1aa75b5c0..000000000 --- a/priam/src/main/java/com/netflix/priam/backupv2/ForgottenFilesManager.java +++ /dev/null @@ -1,212 +0,0 @@ -/* - * Copyright 2019 Netflix, Inc. - * - * Licensed 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 com.netflix.priam.backupv2; - -import com.netflix.priam.config.IConfiguration; -import com.netflix.priam.merics.BackupMetrics; -import com.netflix.priam.utils.DateUtil; -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.LinkOption; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.time.Instant; -import java.time.temporal.ChronoUnit; -import java.util.Collection; -import java.util.regex.Pattern; -import java.util.stream.Collectors; -import javax.inject.Inject; -import org.apache.commons.io.FileUtils; -import org.apache.commons.io.filefilter.FileFilterUtils; -import org.apache.commons.io.filefilter.IOFileFilter; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** Created by aagrawal on 1/1/19. */ -public class ForgottenFilesManager { - private static final Logger logger = LoggerFactory.getLogger(ForgottenFilesManager.class); - - private BackupMetrics backupMetrics; - private IConfiguration config; - private static final String TMP_EXT = ".tmp"; - - private static final Pattern tmpFilePattern = - Pattern.compile("^((.*)\\-(.*)\\-)?tmp(link)?\\-((?:l|k).)\\-(\\d)*\\-(.*)$"); - - protected static final String LOST_FOUND = "lost+found"; - - @Inject - public ForgottenFilesManager(IConfiguration configuration, BackupMetrics backupMetrics) { - this.config = configuration; - this.backupMetrics = backupMetrics; - } - - public void findAndMoveForgottenFiles(Instant snapshotInstant, File snapshotDir) { - try { - Collection snapshotFiles = - FileUtils.listFiles(snapshotDir, FileFilterUtils.fileFileFilter(), null); - File columnfamilyDir = snapshotDir.getParentFile().getParentFile(); - Collection columnfamilyFiles = - getColumnfamilyFiles(snapshotInstant, columnfamilyDir); - - // Remove the SSTable(s) which are part of snapshot from the CF file list. - // This cannot be a simple removeAll as snapshot files have "different" file folder - // prefix. - for (File file : snapshotFiles) { - // Get its parent directory file based on this file. - File originalFile = new File(columnfamilyDir, file.getName()); - columnfamilyFiles.remove(originalFile); - } - - // If there are no "extra" SSTables in CF data folder, we are done. - if (columnfamilyFiles.size() == 0) return; - - logger.warn( - "# of potential forgotten files: {} found for CF: {}", - columnfamilyFiles.size(), - columnfamilyDir.getName()); - - // Move the files to lost_found directory if configured. - moveForgottenFiles(columnfamilyDir, columnfamilyFiles); - - } catch (Exception e) { - // Eat the exception, if there, for any reason. This should not stop the snapshot for - // any reason. - logger.error( - "Exception occurred while trying to find forgottenFile. Ignoring the error and continuing with remaining backup", - e); - e.printStackTrace(); - } - } - - protected Collection getColumnfamilyFiles(Instant snapshotInstant, File columnfamilyDir) { - // Find all the files in columnfamily folder which is : - // 1. Not a temp file. - // 2. Is a file. (we don't care about directories) - // 3. Is older than snapshot time, as new files keep getting created after taking a - // snapshot. - IOFileFilter tmpFileFilter1 = FileFilterUtils.suffixFileFilter(TMP_EXT); - IOFileFilter tmpFileFilter2 = - FileFilterUtils.asFileFilter( - pathname -> tmpFilePattern.matcher(pathname.getName()).matches()); - IOFileFilter tmpFileFilter = FileFilterUtils.or(tmpFileFilter1, tmpFileFilter2); - /* - Here we are allowing files which were more than - @link{IConfiguration#getGracePeriodDaysForCompaction}. We do this to allow cassandra - to have files which were generated as part of long running compaction. - Refer to https://issues.apache.org/jira/browse/CASSANDRA-6756 and - https://issues.apache.org/jira/browse/CASSANDRA-7066 - for more information. - */ - IOFileFilter ageFilter = - FileFilterUtils.ageFileFilter( - snapshotInstant - .minus(config.getGracePeriodDaysForCompaction(), ChronoUnit.DAYS) - .toEpochMilli()); - IOFileFilter fileFilter = - FileFilterUtils.and( - FileFilterUtils.notFileFilter(tmpFileFilter), - FileFilterUtils.fileFileFilter(), - ageFilter); - - return FileUtils.listFiles(columnfamilyDir, fileFilter, null); - } - - protected void moveForgottenFiles(File columnfamilyDir, Collection columnfamilyFiles) - throws IOException { - // This is a list of potential forgotten file(s). Note that C* might still be using - // files as part of read, so we really do not want to move them until we meet the - // @link{IConfiguration#getForgottenFileGracePeriodDaysForRead} window elapses. - - final Path destDir = Paths.get(columnfamilyDir.getAbsolutePath(), LOST_FOUND); - FileUtils.forceMkdir(destDir.toFile()); - final Collection columnfamilyPaths = - columnfamilyFiles - .parallelStream() - .map(file -> Paths.get(file.getAbsolutePath())) - .collect(Collectors.toList()); - - for (Path file : columnfamilyPaths) { - try { - final Path symbolic_link = - Paths.get(destDir.toFile().getAbsolutePath(), file.toFile().getName()); - // Lets see if there is a symbolic link to this file already? - if (!Files.exists(symbolic_link)) { - // If not, lets create one and work on next file. - Files.createSymbolicLink(symbolic_link, file); - continue; - } else if (Files.isSymbolicLink(symbolic_link)) { - // Symbolic link exists, is it older than our timeframe? - Instant last_modified_time = - Files.getLastModifiedTime(symbolic_link, LinkOption.NOFOLLOW_LINKS) - .toInstant(); - if (DateUtil.getInstant() - .isAfter( - last_modified_time.plus( - config.getForgottenFileGracePeriodDaysForRead(), - ChronoUnit.DAYS))) { - // Eligible for move. - logger.info( - "Eligible for move: Forgotten file: {} found for CF: {}", - file, - columnfamilyDir.getName()); - backupMetrics.incrementForgottenFiles(1); - if (config.isForgottenFileMoveEnabled()) { - try { - // Remove our symbolic link. Note that deletion of symbolic link - // does not remove the original file. - Files.delete(symbolic_link); - FileUtils.moveFileToDirectory( - file.toFile(), destDir.toFile(), true); - logger.warn( - "Successfully moved forgotten file: {} found for CF: {}", - file, - columnfamilyDir.getName()); - } catch (IOException e) { - logger.error( - "Exception occurred while trying to move forgottenFile: {}. Ignoring the error and continuing with remaining backup/forgotten files.", - file); - e.printStackTrace(); - } - } - } - } - - } catch (IOException e) { - logger.error("Forgotten file: Error while trying to process the file: {}", file); - e.printStackTrace(); - } - } - - // Clean LOST_FOUND directory of any previous symbolic link files which are not considered - // lost any more. - for (File file : FileUtils.listFiles(destDir.toFile(), null, false)) { - Path filePath = Paths.get(file.getAbsolutePath()); - if (Files.isSymbolicLink(filePath)) { - Path originalFile = Files.readSymbolicLink(filePath); - if (!columnfamilyPaths.contains(originalFile)) { - Files.delete(filePath); - logger.info( - "Deleting the symbolic link as it is not considered as lost anymore. filePath: {}", - filePath); - } - } - } - } -} diff --git a/priam/src/main/java/com/netflix/priam/backupv2/MetaV1Proxy.java b/priam/src/main/java/com/netflix/priam/backupv2/MetaV1Proxy.java deleted file mode 100644 index 589d2a839..000000000 --- a/priam/src/main/java/com/netflix/priam/backupv2/MetaV1Proxy.java +++ /dev/null @@ -1,192 +0,0 @@ -/* - * Copyright 2018 Netflix, Inc. - * - * Licensed 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 com.netflix.priam.backupv2; - -import com.google.common.collect.Lists; -import com.netflix.priam.backup.AbstractBackupPath; -import com.netflix.priam.backup.BackupRestoreException; -import com.netflix.priam.backup.BackupVerificationResult; -import com.netflix.priam.backup.IBackupFileSystem; -import com.netflix.priam.config.IConfiguration; -import com.netflix.priam.utils.DateUtil; -import java.io.FileReader; -import java.nio.file.InvalidPathException; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.time.temporal.ChronoUnit; -import java.util.*; -import javax.inject.Inject; -import javax.inject.Named; -import org.apache.commons.collections4.CollectionUtils; -import org.apache.commons.collections4.iterators.FilterIterator; -import org.apache.commons.io.FileUtils; -import org.json.simple.parser.JSONParser; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** Created by aagrawal on 12/18/18. */ -public class MetaV1Proxy implements IMetaProxy { - private static final Logger logger = LoggerFactory.getLogger(MetaV1Proxy.class); - private final IBackupFileSystem fs; - - @Inject - public MetaV1Proxy( - IConfiguration configuration, @Named("backup") IBackupFileSystem fileSystem) { - fs = fileSystem; - } - - @Override - public Path getLocalMetaFileDirectory() { - return null; - } - - @Override - public String getMetaPrefix(DateUtil.DateRange dateRange) { - return null; - } - - @Override - public List findMetaFiles(DateUtil.DateRange dateRange) { - Date startTime = new Date(dateRange.getStartTime().toEpochMilli()); - Date endTime = new Date(dateRange.getEndTime().toEpochMilli()); - String restorePrefix = fs.getPrefix().toString(); - logger.debug("Looking for snapshot meta file within restore prefix: {}", restorePrefix); - List metas = Lists.newArrayList(); - - Iterator backupfiles = fs.list(restorePrefix, startTime, endTime); - - while (backupfiles.hasNext()) { - AbstractBackupPath path = backupfiles.next(); - if (path.getType() == AbstractBackupPath.BackupFileType.META) - // Since there are now meta file for incrementals as well as snapshot, we need to - // find the correct one (i.e. the snapshot meta file (meta.json)) - if (path.getFileName().equalsIgnoreCase("meta.json")) { - metas.add(path); - } - } - - metas.sort(Collections.reverseOrder()); - - if (metas.size() == 0) { - logger.info( - "No meta v1 file found on remote file system for the time period: {}", - dateRange); - } - - return metas; - } - - @Override - public BackupVerificationResult isMetaFileValid(AbstractBackupPath metaBackupPath) { - BackupVerificationResult result = new BackupVerificationResult(); - result.remotePath = metaBackupPath.getRemotePath(); - result.snapshotInstant = metaBackupPath.getTime().toInstant(); - - try { - // Download the meta file. - Path metaFile = downloadMetaFile(metaBackupPath); - // Read the local meta file. - List metaFileList = getSSTFilesFromMeta(metaFile); - FileUtils.deleteQuietly(metaFile.toFile()); - result.manifestAvailable = true; - - // List the remote file system to validate the backup. - String prefix = fs.getPrefix().toString(); - Date strippedMsSnapshotTime = - new Date(result.snapshotInstant.truncatedTo(ChronoUnit.MINUTES).toEpochMilli()); - Iterator backupfiles = - fs.list(prefix, strippedMsSnapshotTime, strippedMsSnapshotTime); - - // Return validation fail if backup filesystem listing failed. - if (!backupfiles.hasNext()) { - logger.warn( - "ERROR: No files available while doing backup filesystem listing. Declaring the verification failed."); - return result; - } - - // Convert the remote listing to String. - List remoteListing = new ArrayList<>(); - while (backupfiles.hasNext()) { - AbstractBackupPath path = backupfiles.next(); - if (path.getType() == AbstractBackupPath.BackupFileType.SNAP) - remoteListing.add(path.getRemotePath()); - } - - if (metaFileList.isEmpty() && remoteListing.isEmpty()) { - logger.info( - "Uncommon Scenario: Both meta file and backup filesystem listing is empty. Considering this as success"); - result.valid = true; - return result; - } - - ArrayList filesMatched = - (ArrayList) CollectionUtils.intersection(metaFileList, remoteListing); - result.filesMatched = filesMatched.size(); - result.filesInMetaOnly = metaFileList; - result.filesInMetaOnly.removeAll(filesMatched); - - // There could be a scenario that backupfilesystem has more files than meta file. e.g. - // some leftover objects - result.valid = (result.filesInMetaOnly.isEmpty()); - } catch (Exception e) { - logger.error( - "Error while processing meta file: " + metaBackupPath, e.getLocalizedMessage()); - e.printStackTrace(); - } - - return result; - } - - @Override - public Path downloadMetaFile(AbstractBackupPath meta) throws BackupRestoreException { - fs.downloadFile(meta, ".download" /* suffix */, 10 /* retries */); - return Paths.get(meta.newRestoreFile().getAbsolutePath() + ".download"); - } - - @Override - public List getSSTFilesFromMeta(Path localMetaPath) throws Exception { - if (localMetaPath.toFile().isDirectory() || !localMetaPath.toFile().exists()) - throw new InvalidPathException( - localMetaPath.toString(), "Input path is either directory or do not exist"); - - List result = new ArrayList<>(); - JSONParser jsonParser = new JSONParser(); - org.json.simple.JSONArray fileList = - (org.json.simple.JSONArray) - jsonParser.parse(new FileReader(localMetaPath.toFile())); - fileList.forEach(entry -> result.add(entry.toString())); - return result; - } - - @Override - public Iterator getIncrementals(DateUtil.DateRange dateRange) { - String prefix = fs.getPrefix().toString(); - Iterator iterator = - fs.list( - prefix, - new Date(dateRange.getStartTime().toEpochMilli()), - new Date(dateRange.getEndTime().toEpochMilli())); - return new FilterIterator<>( - iterator, - abstractBackupPath -> - abstractBackupPath.getType() == AbstractBackupPath.BackupFileType.SST); - } - - @Override - public void cleanupOldMetaFiles() {} -} diff --git a/priam/src/main/java/com/netflix/priam/config/IConfiguration.java b/priam/src/main/java/com/netflix/priam/config/IConfiguration.java index 61981ac8b..405687edb 100644 --- a/priam/src/main/java/com/netflix/priam/config/IConfiguration.java +++ b/priam/src/main/java/com/netflix/priam/config/IConfiguration.java @@ -297,18 +297,6 @@ default String getCompactionExcludeCFList() { return null; } - /** - * Cron expression to be used for snapshot backups. - * - * @return Backup cron expression for snapshots - * @see quartz-scheduler - * @see http://www.cronmaker.com To build new cron timer - */ - default String getBackupCronExpression() { - return "0 0 12 1/1 * ? *"; - } - /** * Column Family(ies), comma delimited, to include during snapshot backup. Note 1: The expected * format is keyspace.cfname. If no value is provided then snapshot contains all KS,CF(s) Note diff --git a/priam/src/main/java/com/netflix/priam/config/PriamConfiguration.java b/priam/src/main/java/com/netflix/priam/config/PriamConfiguration.java index 481236049..9e699f0c9 100644 --- a/priam/src/main/java/com/netflix/priam/config/PriamConfiguration.java +++ b/priam/src/main/java/com/netflix/priam/config/PriamConfiguration.java @@ -212,11 +212,6 @@ public String getMaxDirectMemory() { (PRIAM_PRE + ".direct.memory.size.") + instanceInfo.getInstanceType(), "50G"); } - @Override - public String getBackupCronExpression() { - return config.get(PRIAM_PRE + ".backup.cron", "0 0 12 1/1 * ? *"); // Backup daily at 12 - } - @Override public GCType getGCType() throws UnsupportedTypeException { String gcType = config.get(PRIAM_PRE + ".gc.type", GCType.CMS.getGcType()); diff --git a/priam/src/main/java/com/netflix/priam/defaultimpl/PriamGuiceModule.java b/priam/src/main/java/com/netflix/priam/defaultimpl/PriamGuiceModule.java index fda267097..19ff006de 100644 --- a/priam/src/main/java/com/netflix/priam/defaultimpl/PriamGuiceModule.java +++ b/priam/src/main/java/com/netflix/priam/defaultimpl/PriamGuiceModule.java @@ -24,7 +24,6 @@ import com.netflix.priam.aws.auth.S3RoleAssumptionCredential; import com.netflix.priam.backup.IBackupFileSystem; import com.netflix.priam.backupv2.IMetaProxy; -import com.netflix.priam.backupv2.MetaV1Proxy; import com.netflix.priam.backupv2.MetaV2Proxy; import com.netflix.priam.cred.ICredential; import com.netflix.spectator.api.NoopRegistry; @@ -44,7 +43,6 @@ protected void configure() { bind(ICredential.class) .annotatedWith(Names.named("awsec2roleassumption")) .to(EC2RoleAssumptionCredential.class); - bind(IMetaProxy.class).annotatedWith(Names.named("v1")).to(MetaV1Proxy.class); bind(IMetaProxy.class).annotatedWith(Names.named("v2")).to(MetaV2Proxy.class); bind(Registry.class).toInstance(new NoopRegistry()); } diff --git a/priam/src/main/java/com/netflix/priam/resources/BackupServlet.java b/priam/src/main/java/com/netflix/priam/resources/BackupServlet.java deleted file mode 100644 index 6d8e20cfa..000000000 --- a/priam/src/main/java/com/netflix/priam/resources/BackupServlet.java +++ /dev/null @@ -1,285 +0,0 @@ -/* - * Copyright 2017 Netflix, Inc. - * - * Licensed 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 com.netflix.priam.resources; - -import com.netflix.priam.backup.*; -import com.netflix.priam.backup.AbstractBackupPath.BackupFileType; -import com.netflix.priam.config.IBackupRestoreConfig; -import com.netflix.priam.config.IConfiguration; -import com.netflix.priam.scheduler.PriamScheduler; -import com.netflix.priam.utils.DateUtil; -import com.netflix.priam.utils.DateUtil.DateRange; -import java.time.Instant; -import java.time.temporal.ChronoUnit; -import java.util.*; -import java.util.stream.Collectors; -import javax.inject.Inject; -import javax.inject.Named; -import javax.ws.rs.*; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.Response; -import org.codehaus.jettison.json.JSONArray; -import org.codehaus.jettison.json.JSONException; -import org.codehaus.jettison.json.JSONObject; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@Path("/v1/backup") -@Produces(MediaType.APPLICATION_JSON) -public class BackupServlet { - private static final Logger logger = LoggerFactory.getLogger(BackupServlet.class); - - private static final String REST_SUCCESS = "[\"ok\"]"; - private static final String REST_HEADER_RANGE = "daterange"; - private static final String REST_HEADER_FILTER = "filter"; - private final IConfiguration config; - private final IBackupRestoreConfig backupRestoreConfig; - private final IBackupFileSystem backupFs; - private final SnapshotBackup snapshotBackup; - private final BackupVerification backupVerification; - @Inject private PriamScheduler scheduler; - private final IBackupStatusMgr completedBkups; - private final BackupService backupService; - @Inject private MetaData metaData; - - @Inject - public BackupServlet( - IConfiguration config, - IBackupRestoreConfig backupRestoreConfig, - @Named("backup") IBackupFileSystem backupFs, - SnapshotBackup snapshotBackup, - IBackupStatusMgr completedBkups, - BackupVerification backupVerification, - BackupService backupService) { - this.config = config; - this.backupRestoreConfig = backupRestoreConfig; - this.backupFs = backupFs; - this.snapshotBackup = snapshotBackup; - this.completedBkups = completedBkups; - this.backupVerification = backupVerification; - this.backupService = backupService; - } - - @GET - @Path("/do_snapshot") - public Response backup() throws Exception { - snapshotBackup.execute(); - return Response.ok(REST_SUCCESS, MediaType.APPLICATION_JSON).build(); - } - - @GET - @Path("/incremental_backup") - public Response backupIncrementals() throws Exception { - scheduler.addTask( - "IncrementalBackup", - IncrementalBackup.class, - IncrementalBackup.getTimer(config, backupRestoreConfig)); - return Response.ok(REST_SUCCESS, MediaType.APPLICATION_JSON).build(); - } - - @GET - @Path("/updateService") - public Response updateService() throws Exception { - backupService.onChangeUpdateService(); - return Response.ok(REST_SUCCESS, MediaType.APPLICATION_JSON).build(); - } - - @GET - @Path("/list") - /* - * Fetch the list of files for the requested date range. - * - * @param date range - * @param filter. The type of data files fetched. E.g. META will only fetch the daily snapshot meta data file (meta.json). - * @return the list of files in json format as part of the Http response body. - */ - public Response list( - @QueryParam(REST_HEADER_RANGE) String daterange, - @QueryParam(REST_HEADER_FILTER) @DefaultValue("") String filter) - throws Exception { - - logger.info( - "Parameters: {backupPrefix: [{}], daterange: [{}], filter: [{}]}", - config.getBackupPrefix(), - daterange, - filter); - - DateUtil.DateRange dateRange = new DateUtil.DateRange(daterange); - - Iterator it = - backupFs.list( - config.getBackupPrefix(), - Date.from(dateRange.getStartTime()), - Date.from(dateRange.getEndTime())); - JSONObject object = new JSONObject(); - object = constructJsonResponse(object, it, filter); - return Response.ok(object.toString(2), MediaType.APPLICATION_JSON).build(); - } - - @GET - @Path("/status") - @Produces(MediaType.APPLICATION_JSON) - public Response status() throws Exception { - JSONObject object = new JSONObject(); - object.put("SnapshotStatus", snapshotBackup.state().toString()); - return Response.ok(object.toString(), MediaType.APPLICATION_JSON).build(); - } - - /* - * Determines the status of a snapshot for a date. If there was at least one successful snpashot for the date, snapshot - * for the date is considered completed. - * @param date date of the snapshot. Format of date is yyyymmdd - * @return {"Snapshotstatus":false} or {"Snapshotstatus":true} - */ - @GET - @Path("/status/{date}") - @Produces(MediaType.APPLICATION_JSON) - public Response statusByDate(@PathParam("date") String date) throws Exception { - Instant startTime = DateUtil.parseInstant(date); - Optional backupMetadataOptional = - this.completedBkups - .getLatestBackupMetadata( - BackupVersion.SNAPSHOT_BACKUP, - new DateRange( - startTime.truncatedTo(ChronoUnit.DAYS), - startTime - .plus(1, ChronoUnit.DAYS) - .truncatedTo(ChronoUnit.DAYS))) - .stream() - .findFirst(); - - JSONObject object = new JSONObject(); - if (!backupMetadataOptional.isPresent()) { - object.put("Snapshotstatus", false); - } else { - - object.put("Snapshotstatus", true); - object.put("Details", new JSONObject(backupMetadataOptional.get().toString())); - } - return Response.ok(object.toString(), MediaType.APPLICATION_JSON).build(); - } - - /* - * Determines the status of a snapshot for a date. If there was at least one successful snpashot for the date, snapshot - * for the date is considered completed. - * @param date date of the snapshot. Format of date is yyyymmdd - * @return {"Snapshots":["201606060450","201606060504"]} or "Snapshots":[]} - */ - @GET - @Path("/status/{date}/snapshots") - @Produces(MediaType.APPLICATION_JSON) - public Response snapshotsByDate(@PathParam("date") String date) throws Exception { - List metadata = this.completedBkups.locate(date); - JSONObject object = new JSONObject(); - List snapshots = new ArrayList<>(); - - if (metadata != null && !metadata.isEmpty()) - snapshots.addAll( - metadata.stream() - .filter( - backupMetadata -> - backupMetadata - .getBackupVersion() - .equals(BackupVersion.SNAPSHOT_BACKUP)) - .map( - backupMetadata -> - DateUtil.formatyyyyMMddHHmm(backupMetadata.getStart())) - .collect(Collectors.toList())); - - object.put("Snapshots", snapshots); - return Response.ok(object.toString(), MediaType.APPLICATION_JSON).build(); - } - - /* - * Determines the validity of the backup by i) Downloading meta.json file ii) Listing of the backup directory - * iii) Find the missing or extra files in backup location. - * This by default takes the latest snapshot of the application. One can provide exact hour and min to check specific backup. - * Input: Daterange in the format of yyyyMMddHHmm,yyyyMMddHHmm OR yyyyMMdd,yyyyMMdd OR default - */ - @GET - @Path("/validate/snapshot/{daterange}") - @Produces(MediaType.APPLICATION_JSON) - public Response validateSnapshotByDate( - @PathParam("daterange") String daterange, - @DefaultValue("false") @QueryParam("force") boolean force) - throws Exception { - DateUtil.DateRange dateRange = new DateUtil.DateRange(daterange); - Optional result = - backupVerification.verifyLatestBackup( - BackupVersion.SNAPSHOT_BACKUP, force, dateRange); - if (!result.isPresent()) { - return Response.noContent() - .entity("No valid meta found for provided time range") - .build(); - } - - return Response.ok(result.get().toString()).build(); - } - - /* - * A list of files for requested filter. Currently, the only supported filter is META, all others will be ignore. - * For filter of META, ONLY the daily snapshot meta file (meta.json) are accounted for, not the incremental meta file. - * In addition, we do ONLY list the name of the meta data file, not the list of data files within it. - * - * @param handle to the json response - * @param a list of all files (data (*.db), and meta data file (*.json)) from S3 for requested dates. - * @param backup meta data file filter. Currently, the only supported filter is META, all others will be ignore. - * @return a list of files in Json format. - */ - private JSONObject constructJsonResponse( - JSONObject object, Iterator it, String filter) throws Exception { - int fileCnt = 0; - filter = filter.contains("?") ? filter.substring(0, filter.indexOf("?")) : filter; - - try { - JSONArray jArray = new JSONArray(); - while (it.hasNext()) { - AbstractBackupPath p = it.next(); - if (!filter.isEmpty() && BackupFileType.valueOf(filter) != p.getType()) continue; - JSONObject backupJSON = new JSONObject(); - backupJSON.put("bucket", config.getBackupPrefix()); - backupJSON.put("filename", p.getRemotePath()); - backupJSON.put("app", p.getClusterName()); - backupJSON.put("region", p.getRegion()); - backupJSON.put("token", p.getToken()); - backupJSON.put("ts", DateUtil.formatyyyyMMddHHmm(p.getTime())); - backupJSON.put( - "instance_id", p.getInstanceIdentity().getInstance().getInstanceId()); - backupJSON.put("uploaded_ts", DateUtil.formatyyyyMMddHHmm(p.getUploadedTs())); - if ("meta".equalsIgnoreCase(filter)) { // only check for existence of meta file - p.setFileName( - "meta.json"); // ignore incremental meta files, we are only interested - // in daily snapshot - if (metaData.doesExist(p)) { - // if here, snapshot completed. - fileCnt++; - jArray.put(backupJSON); - backupJSON.put("num_files", "1"); - } - } else { // account for every file (data, and meta) . - fileCnt++; - jArray.put(backupJSON); - } - } - object.put("files", jArray); - object.put("num_files", fileCnt); - } catch (JSONException jse) { - logger.info("Caught JSON Exception --> {}", jse.getMessage()); - } - return object; - } -} 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 791ebc6f3..5d2b6a4f6 100644 --- a/priam/src/main/java/com/netflix/priam/resources/BackupServletV2.java +++ b/priam/src/main/java/com/netflix/priam/resources/BackupServletV2.java @@ -110,7 +110,6 @@ public Response info(@PathParam("date") String date) { Instant instant = DateUtil.parseInstant(date); List metadataList = backupStatusMgr.getLatestBackupMetadata( - BackupVersion.SNAPSHOT_META_SERVICE, new DateRange( instant, instant.plus(1, ChronoUnit.DAYS).truncatedTo(ChronoUnit.DAYS))); @@ -125,8 +124,7 @@ public Response validateV2SnapshotByDate( throws Exception { DateUtil.DateRange dateRange = new DateUtil.DateRange(daterange); Optional result = - backupVerification.verifyLatestBackup( - BackupVersion.SNAPSHOT_META_SERVICE, force, dateRange); + backupVerification.verifyLatestBackup(force, dateRange); if (!result.isPresent()) { return Response.noContent() .entity("No valid meta found for provided time range") 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 8ac714f2e..11a133adf 100644 --- a/priam/src/main/java/com/netflix/priam/restore/AbstractRestore.java +++ b/priam/src/main/java/com/netflix/priam/restore/AbstractRestore.java @@ -21,7 +21,6 @@ import com.netflix.priam.backup.IBackupFileSystem; import com.netflix.priam.backup.Status; import com.netflix.priam.backupv2.IMetaProxy; -import com.netflix.priam.config.IBackupRestoreConfig; import com.netflix.priam.config.IConfiguration; import com.netflix.priam.defaultimpl.ICassandraProcess; import com.netflix.priam.health.InstanceState; @@ -69,16 +68,10 @@ public abstract class AbstractRestore extends Task implements IRestoreStrategy { private final InstanceState instanceState; private final IPostRestoreHook postRestoreHook; - @Inject - @Named("v1") - IMetaProxy metaV1Proxy; - @Inject @Named("v2") IMetaProxy metaV2Proxy; - @Inject IBackupRestoreConfig backupRestoreConfig; - public AbstractRestore( IConfiguration config, IBackupFileSystem fs, @@ -177,8 +170,7 @@ public void restore(DateUtil.DateRange dateRange) throws Exception { } Date endTime = new Date(dateRange.getEndTime().toEpochMilli()); - IMetaProxy metaProxy = metaV1Proxy; - if (backupRestoreConfig.enableV2Restore()) metaProxy = metaV2Proxy; + IMetaProxy metaProxy = metaV2Proxy; // Set the restore status. instanceState.getRestoreStatus().resetStatus(); diff --git a/priam/src/test/groovy/com.netflix.priam/backup/TestBackupScheduler.groovy b/priam/src/test/groovy/com.netflix.priam/backup/TestBackupScheduler.groovy deleted file mode 100644 index d5baaf0b1..000000000 --- a/priam/src/test/groovy/com.netflix.priam/backup/TestBackupScheduler.groovy +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Copyright 2017 Netflix, Inc. - * - * Licensed 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 com.netflix.priam.backup - -import com.netflix.priam.config.FakeConfiguration -import spock.lang.Specification -import spock.lang.Unroll - -/** - Created by aagrawal on 11/7/17. - */ -@Unroll -class TestBackupScheduler extends Specification { - def "IsBackupEnabled CRON #configCRON is #result"() { - expect: - SnapshotBackup.isBackupEnabled(new BackupConfiguration(configCRON)) == result - - where: - configCRON || result - "-1" || false - "0 0 9 1/1 * ? *" || true - } - - def "Exception for illegal value of Snapshot CRON expression , #configCRON"() { - when: - SnapshotBackup.isBackupEnabled(new BackupConfiguration(configCRON)) - - then: - thrown(expectedException) - - where: - configCRON || expectedException - "abc" || Exception - "0 9 1/1 * ? *"|| Exception - } - - def "Validate CRON for backup CRON #configCRON is #result"() { - expect: - SnapshotBackup.getTimer(new BackupConfiguration(configCRON)).cronExpression == result - - where: - configCRON || result - "0 0 9 1/1 * ? *" || "0 0 9 1/1 * ? *" - } - - private class BackupConfiguration extends FakeConfiguration { - private String backupCronExpression - - BackupConfiguration(String backupCronExpression) { - this.backupCronExpression = backupCronExpression - } - - @Override - String getBackupCronExpression() { - return backupCronExpression - } - } - -} diff --git a/priam/src/test/java/com/netflix/priam/aws/TestRemoteBackupPath.java b/priam/src/test/java/com/netflix/priam/aws/TestRemoteBackupPath.java index bc75b66a2..166826059 100644 --- a/priam/src/test/java/com/netflix/priam/aws/TestRemoteBackupPath.java +++ b/priam/src/test/java/com/netflix/priam/aws/TestRemoteBackupPath.java @@ -43,93 +43,6 @@ public TestRemoteBackupPath() { Guice.createInjector(new BRTestModule()).getProvider(AbstractBackupPath.class); } - @Test - public void testV1BackupPathsSST() { - Path path = - Paths.get("target/data", "keyspace1", "columnfamily1", "backup", "mc-1234-Data.db"); - AbstractBackupPath abstractBackupPath = pathFactory.get(); - abstractBackupPath.parseLocal(path.toFile(), BackupFileType.SST); - - // Verify parse local - Assert.assertEquals( - 0, abstractBackupPath.getLastModified().toEpochMilli()); // File do not exist. - Assert.assertEquals("keyspace1", abstractBackupPath.getKeyspace()); - Assert.assertEquals("columnfamily1", abstractBackupPath.getColumnFamily()); - Assert.assertEquals(BackupFileType.SST, abstractBackupPath.getType()); - Assert.assertEquals(path.toFile(), abstractBackupPath.getBackupFile()); - Assert.assertEquals( - 0, - abstractBackupPath - .getTime() - .toInstant() - .toEpochMilli()); // Since file do not exist. - - // Verify toRemote and parseRemote. - String remotePath = abstractBackupPath.getRemotePath(); - logger.info(remotePath); - AbstractBackupPath abstractBackupPath2 = pathFactory.get(); - abstractBackupPath2.parseRemote(remotePath); - validateAbstractBackupPath(abstractBackupPath, abstractBackupPath2); - Assert.assertEquals(abstractBackupPath.getTime(), abstractBackupPath2.getTime()); - } - - @Test - public void testV1BackupPathsSnap() { - Path path = - Paths.get( - "target/data", - "keyspace1", - "columnfamily1", - "snapshot", - "201801011201", - "mc-1234-Data.db"); - AbstractBackupPath abstractBackupPath = pathFactory.get(); - abstractBackupPath.parseLocal(path.toFile(), BackupFileType.SNAP); - - // Verify parse local - Assert.assertEquals( - 0, abstractBackupPath.getLastModified().toEpochMilli()); // File do not exist. - Assert.assertEquals("keyspace1", abstractBackupPath.getKeyspace()); - Assert.assertEquals("columnfamily1", abstractBackupPath.getColumnFamily()); - Assert.assertEquals(BackupFileType.SNAP, abstractBackupPath.getType()); - Assert.assertEquals(path.toFile(), abstractBackupPath.getBackupFile()); - Assert.assertEquals( - "201801011201", DateUtil.formatyyyyMMddHHmm(abstractBackupPath.getTime())); - - // Verify toRemote and parseRemote. - String remotePath = abstractBackupPath.getRemotePath(); - logger.info(remotePath); - - AbstractBackupPath abstractBackupPath2 = pathFactory.get(); - abstractBackupPath2.parseRemote(remotePath); - validateAbstractBackupPath(abstractBackupPath, abstractBackupPath2); - Assert.assertEquals(abstractBackupPath.getTime(), abstractBackupPath2.getTime()); - } - - @Test - public void testV1BackupPathsMeta() { - Path path = Paths.get("target/data", "meta.json"); - AbstractBackupPath abstractBackupPath = pathFactory.get(); - abstractBackupPath.parseLocal(path.toFile(), BackupFileType.META); - - // Verify parse local - Assert.assertEquals( - 0, abstractBackupPath.getLastModified().toEpochMilli()); // File do not exist. - Assert.assertNull(abstractBackupPath.getKeyspace()); - Assert.assertNull(abstractBackupPath.getColumnFamily()); - Assert.assertEquals(BackupFileType.META, abstractBackupPath.getType()); - Assert.assertEquals(path.toFile(), abstractBackupPath.getBackupFile()); - - // Verify toRemote and parseRemote. - String remotePath = abstractBackupPath.getRemotePath(); - logger.info(remotePath); - - AbstractBackupPath abstractBackupPath2 = pathFactory.get(); - abstractBackupPath2.parseRemote(remotePath); - validateAbstractBackupPath(abstractBackupPath, abstractBackupPath2); - Assert.assertEquals(abstractBackupPath.getTime(), abstractBackupPath2.getTime()); - } - @Test public void testV2BackupPathSST() { Path path = diff --git a/priam/src/test/java/com/netflix/priam/backup/BRTestModule.java b/priam/src/test/java/com/netflix/priam/backup/BRTestModule.java index ec626b72f..0e87702ee 100644 --- a/priam/src/test/java/com/netflix/priam/backup/BRTestModule.java +++ b/priam/src/test/java/com/netflix/priam/backup/BRTestModule.java @@ -23,7 +23,6 @@ import com.netflix.priam.aws.auth.IS3Credential; import com.netflix.priam.aws.auth.S3RoleAssumptionCredential; import com.netflix.priam.backupv2.IMetaProxy; -import com.netflix.priam.backupv2.MetaV1Proxy; import com.netflix.priam.backupv2.MetaV2Proxy; import com.netflix.priam.config.FakeBackupRestoreConfig; import com.netflix.priam.config.FakeConfiguration; @@ -76,7 +75,6 @@ protected void configure() { bind(ICassandraProcess.class).to(FakeCassandraProcess.class); bind(IPostRestoreHook.class).to(FakePostRestoreHook.class); bind(Registry.class).toInstance(new DefaultRegistry()); - bind(IMetaProxy.class).annotatedWith(Names.named("v1")).to(MetaV1Proxy.class); bind(IMetaProxy.class).annotatedWith(Names.named("v2")).to(MetaV2Proxy.class); bind(DynamicRateLimiter.class).to(FakeDynamicRateLimiter.class); bind(Clock.class).toInstance(Clock.fixed(Instant.EPOCH, ZoneId.systemDefault())); diff --git a/priam/src/test/java/com/netflix/priam/backup/FakeBackupFileSystem.java b/priam/src/test/java/com/netflix/priam/backup/FakeBackupFileSystem.java index 0750179fd..91501aa02 100644 --- a/priam/src/test/java/com/netflix/priam/backup/FakeBackupFileSystem.java +++ b/priam/src/test/java/com/netflix/priam/backup/FakeBackupFileSystem.java @@ -17,20 +17,18 @@ package com.netflix.priam.backup; -import com.netflix.priam.aws.RemoteBackupPath; import com.netflix.priam.config.IConfiguration; import com.netflix.priam.merics.BackupMetrics; import com.netflix.priam.notification.BackupNotificationMgr; -import java.io.File; -import java.io.FileWriter; import java.io.IOException; +import java.nio.file.Files; import java.nio.file.Path; +import java.nio.file.Paths; import java.time.Instant; import java.util.*; import javax.inject.Inject; import javax.inject.Provider; import javax.inject.Singleton; -import org.json.simple.JSONArray; @Singleton public class FakeBackupFileSystem extends AbstractFileSystem { @@ -71,31 +69,6 @@ public void addFile(String file) { flist.add(path); } - @SuppressWarnings("unchecked") - @Override - public Iterator list(String bucket, Date start, Date till) { - String[] paths = bucket.split(String.valueOf(RemoteBackupPath.PATH_SEP)); - - if (paths.length > 1) { - baseDir = paths[1]; - region = paths[2]; - clusterName = paths[3]; - } - - List tmpList = new ArrayList<>(); - for (AbstractBackupPath path : flist) { - - if ((path.time.after(start) && path.time.before(till)) - || path.time.equals(start) - && path.baseDir.equals(baseDir) - && path.clusterName.equals(clusterName) - && path.region.equals(region)) { - tmpList.add(path); - } - } - return tmpList.iterator(); - } - @Override public Iterator listFileSystem(String prefix, String delimiter, String marker) { ArrayList items = new ArrayList<>(); @@ -146,21 +119,21 @@ public void cleanup() { @Override protected void downloadFileImpl(AbstractBackupPath path, String suffix) throws BackupRestoreException { - File localFile = new File(path.newRestoreFile().getAbsolutePath() + suffix); - if (path.getType() == AbstractBackupPath.BackupFileType.META) { - // List all files and generate the file - try (FileWriter fr = new FileWriter(localFile)) { - JSONArray jsonObj = new JSONArray(); - for (AbstractBackupPath filePath : flist) { - if (filePath.type == AbstractBackupPath.BackupFileType.SNAP - && filePath.time.equals(path.time)) { - jsonObj.add(filePath.getRemotePath()); - } + if (path.getType() == AbstractBackupPath.BackupFileType.META_V2) { + Path destination = Paths.get(path.newRestoreFile().getAbsolutePath() + suffix); + if (!destination.toFile().exists()) { + Path origin = + Paths.get( + "src/test/resources/" + + path.getClusterName() + + "_" + + Paths.get(path.getRemotePath()).getFileName()) + .toAbsolutePath(); + try { + Files.copy(origin, destination); + } catch (IOException io) { + throw new BackupRestoreException(io.getMessage(), io); } - fr.write(jsonObj.toJSONString()); - fr.flush(); - } catch (IOException io) { - throw new BackupRestoreException(io.getMessage(), io); } } downloadedFiles.add(path.getRemotePath()); diff --git a/priam/src/test/java/com/netflix/priam/backup/TestBackup.java b/priam/src/test/java/com/netflix/priam/backup/TestBackup.java index 276ace860..672fa0ab4 100644 --- a/priam/src/test/java/com/netflix/priam/backup/TestBackup.java +++ b/priam/src/test/java/com/netflix/priam/backup/TestBackup.java @@ -21,7 +21,6 @@ import com.google.inject.Guice; import com.google.inject.Injector; import com.google.inject.Key; -import com.google.inject.name.Names; import java.io.BufferedOutputStream; import java.io.File; import java.io.FileOutputStream; @@ -47,19 +46,20 @@ * @author Praveen Sadhu */ public class TestBackup { - private static Injector injector; - private static FakeBackupFileSystem filesystem; + private Injector injector; + private FakeBackupFileSystem filesystem; private static final Logger logger = LoggerFactory.getLogger(TestBackup.class); private static final Set expectedFiles = new HashSet<>(); @BeforeClass public static void setup() throws InterruptedException, IOException { new MockNodeProbe(); + } + + @Before + public void init() { injector = Guice.createInjector(new BRTestModule()); - filesystem = - (FakeBackupFileSystem) - injector.getInstance( - Key.get(IBackupFileSystem.class, Names.named("backup"))); + filesystem = (FakeBackupFileSystem) injector.getInstance(Key.get(IBackupFileSystem.class)); } @AfterClass @@ -68,29 +68,6 @@ public static void cleanup() throws IOException { FileUtils.deleteQuietly(file); } - @Test - public void testSnapshotBackup() throws Exception { - filesystem.cleanup(); - SnapshotBackup backup = injector.getInstance(SnapshotBackup.class); - - // - // backup.execute(); - // Assert.assertEquals(3, filesystem.uploadedFiles.size()); - // System.out.println("***** "+filesystem.uploadedFiles.size()); - // boolean metafile = false; - // for (String filePath : expectedFiles) - // Assert.assertTrue(filesystem.uploadedFiles.contains(filePath)); - // - // for(String filepath : filesystem.uploadedFiles){ - // if( filepath.endsWith("meta.json")){ - // metafile = true; - // break; - // } - // } - // Assert.assertTrue(metafile); - - } - @Test public void testIncrementalBackup() throws Exception { filesystem.cleanup(); diff --git a/priam/src/test/java/com/netflix/priam/backup/TestBackupFile.java b/priam/src/test/java/com/netflix/priam/backup/TestBackupFile.java index 52478f3f6..97bc60e6e 100644 --- a/priam/src/test/java/com/netflix/priam/backup/TestBackupFile.java +++ b/priam/src/test/java/com/netflix/priam/backup/TestBackupFile.java @@ -22,13 +22,8 @@ import com.netflix.priam.aws.RemoteBackupPath; import com.netflix.priam.backup.AbstractBackupPath.BackupFileType; import com.netflix.priam.identity.InstanceIdentity; -import com.netflix.priam.utils.DateUtil; -import java.io.BufferedOutputStream; import java.io.File; -import java.io.FileOutputStream; import java.io.IOException; -import java.sql.Date; -import java.text.ParseException; import org.apache.commons.io.FileUtils; import org.junit.AfterClass; import org.junit.Assert; @@ -42,40 +37,24 @@ public class TestBackupFile { @BeforeClass public static void setup() throws IOException { injector = Guice.createInjector(new BRTestModule()); - File file = - new File("target/data/Keyspace1/Standard1/", "Keyspace1-Standard1-ia-5-Data.db"); - if (!file.exists()) { - File dir1 = new File("target/data/Keyspace1/Standard1/"); - if (!dir1.exists()) dir1.mkdirs(); - byte b = 8; - long oneKB = (1024L); - System.out.println(oneKB); - BufferedOutputStream bos1 = new BufferedOutputStream(new FileOutputStream(file)); - for (long i = 0; i < oneKB; i++) { - bos1.write(b); - } - bos1.flush(); - bos1.close(); - } InstanceIdentity factory = injector.getInstance(InstanceIdentity.class); - factory.getInstance().setToken("1234567"); // Token + factory.getInstance().setToken("1234567"); region = factory.getInstanceInfo().getRegion(); } @AfterClass public static void cleanup() throws IOException { - File file = new File("Keyspace1-Standard1-ia-5-Data.db"); - FileUtils.deleteQuietly(file); + FileUtils.deleteDirectory(new File("target")); } @Test - public void testBackupFileCreation() throws ParseException { - // Test snapshot file - String snapshotfile = - "target/data/Keyspace1/Standard1/snapshots/201108082320/Keyspace1-Standard1-ia-5-Data.db"; + public void testBackupFileCreation() throws IOException { + File file = + createFile( + "target/data/Keyspace1/Standard1/snapshots/201108082320/Keyspace1-Standard1-ia-5-Data.db"); RemoteBackupPath backupfile = injector.getInstance(RemoteBackupPath.class); - backupfile.parseLocal(new File(snapshotfile), BackupFileType.SNAP); - Assert.assertEquals(BackupFileType.SNAP, backupfile.type); + backupfile.parseLocal(file, BackupFileType.SST_V2); + Assert.assertEquals(BackupFileType.SST_V2, backupfile.type); Assert.assertEquals("Keyspace1", backupfile.keyspace); Assert.assertEquals("Standard1", backupfile.columnFamily); Assert.assertEquals("1234567", backupfile.token); @@ -83,50 +62,50 @@ public void testBackupFileCreation() throws ParseException { Assert.assertEquals(region, backupfile.region); Assert.assertEquals("casstestbackup", backupfile.baseDir); Assert.assertEquals( - "casstestbackup/" - + region - + "/fake-app/1234567/201108082320/SNAP/Keyspace1/Standard1/Keyspace1-Standard1-ia-5-Data.db", + "casstestbackup/1049_fake-app/1234567/SST_V2/1699206297000/Keyspace1/Standard1/SNAPPY/PLAINTEXT/Keyspace1-Standard1-ia-5-Data.db", backupfile.getRemotePath()); } @Test - public void testIncBackupFileCreation() throws ParseException { - // Test incremental file - File bfile = new File("target/data/Keyspace1/Standard1/Keyspace1-Standard1-ia-5-Data.db"); + public void testIncBackupFileCreation() throws IOException { + File file = + createFile( + "target/data/Keyspace1/Standard1/backups/Keyspace1-Standard1-ia-5-Data.db"); RemoteBackupPath backupfile = injector.getInstance(RemoteBackupPath.class); - backupfile.parseLocal(bfile, BackupFileType.SST); - Assert.assertEquals(BackupFileType.SST, backupfile.type); + backupfile.parseLocal(file, BackupFileType.SST_V2); + Assert.assertEquals(BackupFileType.SST_V2, backupfile.type); Assert.assertEquals("Keyspace1", backupfile.keyspace); Assert.assertEquals("Standard1", backupfile.columnFamily); Assert.assertEquals("1234567", backupfile.token); Assert.assertEquals("fake-app", backupfile.clusterName); Assert.assertEquals(region, backupfile.region); Assert.assertEquals("casstestbackup", backupfile.baseDir); - String datestr = DateUtil.formatyyyyMMddHHmm(new Date(bfile.lastModified())); Assert.assertEquals( - "casstestbackup/" - + region - + "/fake-app/1234567/" - + datestr - + "/SST/Keyspace1/Standard1/Keyspace1-Standard1-ia-5-Data.db", + "casstestbackup/1049_fake-app/1234567/SST_V2/1699206297000/Keyspace1/Standard1/SNAPPY/PLAINTEXT/Keyspace1-Standard1-ia-5-Data.db", backupfile.getRemotePath()); } @Test - public void testMetaFileCreation() throws ParseException { - // Test snapshot file - String filestr = "cass/data/1234567.meta"; - File bfile = new File(filestr); + public void testMetaFileCreation() throws IOException { + File file = createFile("target/data/1234567.meta"); RemoteBackupPath backupfile = injector.getInstance(RemoteBackupPath.class); - backupfile.parseLocal(bfile, BackupFileType.META); - backupfile.setTime(DateUtil.getDate("201108082320")); - Assert.assertEquals(BackupFileType.META, backupfile.type); + backupfile.parseLocal(file, BackupFileType.META_V2); + Assert.assertEquals(BackupFileType.META_V2, backupfile.type); Assert.assertEquals("1234567", backupfile.token); Assert.assertEquals("fake-app", backupfile.clusterName); Assert.assertEquals(region, backupfile.region); Assert.assertEquals("casstestbackup", backupfile.baseDir); Assert.assertEquals( - "casstestbackup/" + region + "/fake-app/1234567/201108082320/META/1234567.meta", + "casstestbackup/1049_fake-app/1234567/META_V2/1699206297000/SNAPPY/PLAINTEXT/1234567.meta", backupfile.getRemotePath()); } + + private File createFile(String path) throws IOException { + File file = new File(path); + File dir = file.getParentFile(); + dir.mkdirs(); + file.createNewFile(); + file.setLastModified(1699206297000L); + return file; + } } diff --git a/priam/src/test/java/com/netflix/priam/backup/TestBackupVerification.java b/priam/src/test/java/com/netflix/priam/backup/TestBackupVerification.java index 72e079f88..07daf7acd 100644 --- a/priam/src/test/java/com/netflix/priam/backup/TestBackupVerification.java +++ b/priam/src/test/java/com/netflix/priam/backup/TestBackupVerification.java @@ -20,7 +20,6 @@ import com.google.inject.Guice; import com.google.inject.Injector; import com.netflix.priam.backup.AbstractBackupPath.BackupFileType; -import com.netflix.priam.backupv2.MetaV1Proxy; import com.netflix.priam.backupv2.MetaV2Proxy; import com.netflix.priam.config.IConfiguration; import com.netflix.priam.utils.DateUtil; @@ -67,13 +66,6 @@ public TestBackupVerification() { backupStatusMgr = injector.getInstance(IBackupStatusMgr.class); } - static class MockMetaV1Proxy extends MockUp { - @Mock - public BackupVerificationResult isMetaFileValid(AbstractBackupPath metaBackupPath) { - return getBackupVerificationResult(); - } - } - static class MockMetaV2Proxy extends MockUp { @Mock public BackupVerificationResult isMetaFileValid(AbstractBackupPath metaBackupPath) { @@ -84,7 +76,6 @@ public BackupVerificationResult isMetaFileValid(AbstractBackupPath metaBackupPat @Before @After public void cleanup() { - new MockMetaV1Proxy(); new MockMetaV2Proxy(); FileUtils.deleteQuietly(new File(configuration.getBackupStatusFileLoc())); } diff --git a/priam/src/test/java/com/netflix/priam/backup/TestFileIterator.java b/priam/src/test/java/com/netflix/priam/backup/TestFileIterator.java deleted file mode 100644 index e79b45f51..000000000 --- a/priam/src/test/java/com/netflix/priam/backup/TestFileIterator.java +++ /dev/null @@ -1,353 +0,0 @@ -/* - * Copyright 2017 Netflix, Inc. - * - * Licensed 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 com.netflix.priam.backup; - -import com.amazonaws.AmazonClientException; -import com.amazonaws.services.s3.AmazonS3Client; -import com.amazonaws.services.s3.model.ListObjectsRequest; -import com.amazonaws.services.s3.model.ObjectListing; -import com.amazonaws.services.s3.model.S3ObjectSummary; -import com.google.inject.Guice; -import com.google.inject.Injector; -import com.netflix.priam.aws.S3FileSystem; -import com.netflix.priam.identity.InstanceIdentity; -import com.netflix.priam.utils.DateUtil; -import java.io.IOException; -import java.util.*; -import mockit.Mock; -import mockit.MockUp; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Ignore; -import org.junit.Test; - -/** - * Unit test for backup file iterator - * - * @author Praveen Sadhu - */ -public class TestFileIterator { - private static Date startTime, endTime; - - private static S3FileSystem s3FileSystem; - private static String region; - private static String bucket = "TESTBUCKET"; - - @BeforeClass - public static void setup() throws InterruptedException, IOException { - AmazonS3Client s3client = new MockAmazonS3Client().getMockInstance(); - new MockObjectListing(); - - Injector injector = Guice.createInjector(new BRTestModule()); - InstanceIdentity factory = injector.getInstance(InstanceIdentity.class); - region = factory.getInstanceInfo().getRegion(); - s3FileSystem = injector.getInstance(S3FileSystem.class); - s3FileSystem.setS3Client(s3client); - - DateUtil.DateRange dateRange = new DateUtil.DateRange("201108110030,201108110530"); - startTime = new Date(dateRange.getStartTime().toEpochMilli()); - endTime = new Date(dateRange.getEndTime().toEpochMilli()); - } - - static class MockAmazonS3Client extends MockUp { - @Mock - public ObjectListing listObjects(ListObjectsRequest listObjectsRequest) - throws AmazonClientException { - ObjectListing listing = new ObjectListing(); - listing.setBucketName(listObjectsRequest.getBucketName()); - listing.setPrefix(listObjectsRequest.getPrefix()); - return listing; - } - - @Mock - public ObjectListing listNextBatchOfObjects(ObjectListing previousObjectListing) - throws AmazonClientException { - ObjectListing listing = new ObjectListing(); - listing.setBucketName(previousObjectListing.getBucketName()); - listing.setPrefix(previousObjectListing.getPrefix()); - return new ObjectListing(); - } - } - - // MockObjectListing class - @Ignore - public static class MockObjectListing extends MockUp { - public static boolean truncated = true; - public static boolean firstcall = true; - public static boolean simfilter = false; // Simulate filtering - - @Mock - public List getObjectSummaries() { - if (firstcall) { - firstcall = false; - if (simfilter) return getObjectSummaryEmpty(); - return getObjectSummary(); - } else { - if (simfilter) { - simfilter = false; // reset - return getObjectSummaryEmpty(); - } else truncated = false; - return getNextObjectSummary(); - } - } - - @Mock - public boolean isTruncated() { - return truncated; - } - } - - @Test - public void testIteratorEmptySet() { - DateUtil.DateRange dateRange = new DateUtil.DateRange("201107110601,201107111101"); - Date stime = new Date(dateRange.getStartTime().toEpochMilli()); - Date etime = new Date(dateRange.getEndTime().toEpochMilli()); - - Iterator fileIterator = s3FileSystem.list(bucket, stime, etime); - Set files = new HashSet<>(); - while (fileIterator.hasNext()) files.add(fileIterator.next().getRemotePath()); - Assert.assertEquals(0, files.size()); - } - - @Test - public void testIterator() { - MockObjectListing.truncated = false; - MockObjectListing.firstcall = true; - MockObjectListing.simfilter = false; - - Iterator fileIterator = s3FileSystem.list(bucket, startTime, endTime); - - Set files = new HashSet<>(); - while (fileIterator.hasNext()) files.add(fileIterator.next().getRemotePath()); - Assert.assertEquals(3, files.size()); - Assert.assertTrue( - files.contains( - "test_backup/" - + region - + "/fakecluster/123456/201108110030/SNAP/ks1/cf1/f1.db")); - Assert.assertTrue( - files.contains( - "test_backup/" - + region - + "/fakecluster/123456/201108110430/SST/ks1/cf1/f2.db")); - Assert.assertTrue( - files.contains( - "test_backup/" - + region - + "/fakecluster/123456/201108110030/META/meta.json")); - Assert.assertFalse( - files.contains( - "test_backup/" - + region - + "/fakecluster/123456/201108110600/SST/ks1/cf1/f3.db")); - } - - @Test - public void testIteratorTruncated() { - MockObjectListing.truncated = true; - MockObjectListing.firstcall = true; - MockObjectListing.simfilter = false; - - Iterator fileIterator = s3FileSystem.list(bucket, startTime, endTime); - - Set files = new HashSet<>(); - while (fileIterator.hasNext()) files.add(fileIterator.next().getRemotePath()); - Assert.assertEquals(5, files.size()); - Assert.assertTrue( - files.contains( - "test_backup/" - + region - + "/fakecluster/123456/201108110030/SNAP/ks1/cf1/f1.db")); - Assert.assertTrue( - files.contains( - "test_backup/" - + region - + "/fakecluster/123456/201108110430/SST/ks1/cf1/f2.db")); - Assert.assertTrue( - files.contains( - "test_backup/" - + region - + "/fakecluster/123456/201108110030/META/meta.json")); - Assert.assertFalse( - files.contains( - "test_backup/" - + region - + "/fakecluster/123456/201108110600/SST/ks1/cf1/f3.db")); - - Assert.assertTrue( - files.contains( - "test_backup/" - + region - + "/fakecluster/123456/201108110030/SNAP/ks2/cf1/f1.db")); - Assert.assertTrue( - files.contains( - "test_backup/" - + region - + "/fakecluster/123456/201108110430/SST/ks2/cf1/f2.db")); - Assert.assertFalse( - files.contains( - "test_backup/" - + region - + "/fakecluster/123456/201108110600/SST/ks2/cf1/f3.db")); - } - - @Test - public void testIteratorTruncatedOOR() { - MockObjectListing.truncated = true; - MockObjectListing.firstcall = true; - MockObjectListing.simfilter = true; - - Iterator fileIterator = s3FileSystem.list(bucket, startTime, endTime); - - Set files = new HashSet<>(); - while (fileIterator.hasNext()) files.add(fileIterator.next().getRemotePath()); - Assert.assertEquals(2, files.size()); - Assert.assertFalse( - files.contains( - "test_backup/" - + region - + "/fakecluster/123456/201107110030/SNAP/ks1/cf1/f1.db")); - Assert.assertFalse( - files.contains( - "test_backup/" - + region - + "/fakecluster/123456/201107110430/SST/ks1/cf1/f2.db")); - Assert.assertFalse( - files.contains( - "test_backup/" - + region - + "/fakecluster/123456/201107110030/META/meta.json")); - Assert.assertFalse( - files.contains( - "test_backup/" - + region - + "/fakecluster/123456/201107110600/SST/ks1/cf1/f3.db")); - - Assert.assertTrue( - files.contains( - "test_backup/" - + region - + "/fakecluster/123456/201108110030/SNAP/ks2/cf1/f1.db")); - Assert.assertTrue( - files.contains( - "test_backup/" - + region - + "/fakecluster/123456/201108110430/SST/ks2/cf1/f2.db")); - Assert.assertFalse( - files.contains( - "test_backup/" - + region - + "/fakecluster/123456/201108110600/SST/ks2/cf1/f3.db")); - } - - @Test - public void testRestorePathIteration() { - MockObjectListing.truncated = true; - MockObjectListing.firstcall = true; - MockObjectListing.simfilter = false; - - Iterator fileIterator = - s3FileSystem.list( - "RESTOREBUCKET/test_restore_backup/fake-restore-region/fakerestorecluster", - startTime, - endTime); - - Set files = new HashSet<>(); - while (fileIterator.hasNext()) files.add(fileIterator.next().getRemotePath()); - while (fileIterator.hasNext()) files.add(fileIterator.next().getRemotePath()); - - Assert.assertEquals(5, files.size()); - Assert.assertTrue( - files.contains( - "test_backup/" - + region - + "/fakecluster/123456/201108110030/SNAP/ks1/cf1/f1.db")); - Assert.assertTrue( - files.contains( - "test_backup/" - + region - + "/fakecluster/123456/201108110430/SST/ks1/cf1/f2.db")); - Assert.assertTrue( - files.contains( - "test_backup/" - + region - + "/fakecluster/123456/201108110030/META/meta.json")); - Assert.assertFalse( - files.contains( - "test_backup/" - + region - + "/fakecluster/123456/201108110600/SST/ks1/cf1/f3.db")); - - Assert.assertTrue( - files.contains( - "test_backup/" - + region - + "/fakecluster/123456/201108110030/SNAP/ks2/cf1/f1.db")); - Assert.assertTrue( - files.contains( - "test_backup/" - + region - + "/fakecluster/123456/201108110430/SST/ks2/cf1/f2.db")); - Assert.assertFalse( - files.contains( - "test_backup/" - + region - + "/fakecluster/123456/201108110600/SST/ks2/cf1/f3.db")); - } - - private static List getObjectSummary() { - List list = new ArrayList<>(); - S3ObjectSummary summary = new S3ObjectSummary(); - summary.setKey( - "test_backup/" + region + "/fakecluster/123456/201108110030/SNAP/ks1/cf1/f1.db"); - list.add(summary); - summary = new S3ObjectSummary(); - summary.setKey( - "test_backup/" + region + "/fakecluster/123456/201108110430/SST/ks1/cf1/f2.db"); - list.add(summary); - summary = new S3ObjectSummary(); - summary.setKey( - "test_backup/" + region + "/fakecluster/123456/201108110600/SST/ks1/cf1/f3.db"); - list.add(summary); - summary = new S3ObjectSummary(); - summary.setKey("test_backup/" + region + "/fakecluster/123456/201108110030/META/meta.json"); - list.add(summary); - return list; - } - - private static List getObjectSummaryEmpty() { - return new ArrayList<>(); - } - - private static List getNextObjectSummary() { - List list = new ArrayList<>(); - S3ObjectSummary summary = new S3ObjectSummary(); - summary.setKey( - "test_backup/" + region + "/fakecluster/123456/201108110030/SNAP/ks2/cf1/f1.db"); - list.add(summary); - summary = new S3ObjectSummary(); - summary.setKey( - "test_backup/" + region + "/fakecluster/123456/201108110430/SST/ks2/cf1/f2.db"); - list.add(summary); - summary = new S3ObjectSummary(); - summary.setKey( - "test_backup/" + region + "/fakecluster/123456/201108110600/SST/ks2/cf1/f3.db"); - list.add(summary); - return list; - } -} diff --git a/priam/src/test/java/com/netflix/priam/backup/TestS3FileSystem.java b/priam/src/test/java/com/netflix/priam/backup/TestS3FileSystem.java index 0a297433b..418c1ea9b 100644 --- a/priam/src/test/java/com/netflix/priam/backup/TestS3FileSystem.java +++ b/priam/src/test/java/com/netflix/priam/backup/TestS3FileSystem.java @@ -95,7 +95,7 @@ public void testFileUpload() throws Exception { MockS3PartUploader.setup(); AbstractFileSystem fs = injector.getInstance(NullBackupFileSystem.class); RemoteBackupPath backupfile = injector.getInstance(RemoteBackupPath.class); - backupfile.parseLocal(localFile(), BackupFileType.SNAP); + backupfile.parseLocal(localFile(), BackupFileType.META_V2); long noOfFilesUploaded = backupMetrics.getUploadRate().count(); // temporary hack to allow tests to complete in a timely fashion // This will be removed once we stop inheriting from AbstractFileSystem @@ -125,7 +125,7 @@ public void testFileUploadFailures() throws Exception { long noOfFailures = backupMetrics.getInvalidUploads().count(); S3FileSystem fs = injector.getInstance(S3FileSystem.class); RemoteBackupPath backupfile = injector.getInstance(RemoteBackupPath.class); - backupfile.parseLocal(localFile(), BackupFileType.SNAP); + backupfile.parseLocal(localFile(), BackupFileType.META_V2); try { // temporary hack to allow tests to complete in a timely fashion // This will be removed once we stop inheriting from AbstractFileSystem @@ -144,7 +144,7 @@ public void testFileUploadCompleteFailure() throws Exception { S3FileSystem fs = injector.getInstance(S3FileSystem.class); fs.setS3Client(new MockAmazonS3Client().getMockInstance()); RemoteBackupPath backupfile = injector.getInstance(RemoteBackupPath.class); - backupfile.parseLocal(localFile(), BackupFileType.SNAP); + backupfile.parseLocal(localFile(), BackupFileType.META_V2); try { // temporary hack to allow tests to complete in a timely fashion // This will be removed once we stop inheriting from AbstractFileSystem diff --git a/priam/src/test/java/com/netflix/priam/backupv2/TestBackupV2Service.java b/priam/src/test/java/com/netflix/priam/backupv2/TestBackupV2Service.java index a287a394b..e017ef295 100644 --- a/priam/src/test/java/com/netflix/priam/backupv2/TestBackupV2Service.java +++ b/priam/src/test/java/com/netflix/priam/backupv2/TestBackupV2Service.java @@ -139,8 +139,6 @@ public void testBackupEnabled( result = true; configuration.isIncrementalBackupEnabled(); result = true; - configuration.getBackupCronExpression(); - result = "-1"; } }; IService backupService = @@ -207,8 +205,6 @@ public void updateService( result = "-1"; backupRestoreConfig.getBackupTTLMonitorPeriodInSec(); result = 600; - configuration.getBackupCronExpression(); - result = "-1"; } }; IService backupService = diff --git a/priam/src/test/java/com/netflix/priam/backupv2/TestForgottenFileManager.java b/priam/src/test/java/com/netflix/priam/backupv2/TestForgottenFileManager.java deleted file mode 100644 index 869891d99..000000000 --- a/priam/src/test/java/com/netflix/priam/backupv2/TestForgottenFileManager.java +++ /dev/null @@ -1,235 +0,0 @@ -/* - * Copyright 2019 Netflix, Inc. - * - * Licensed 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 com.netflix.priam.backupv2; - -import com.google.inject.Guice; -import com.google.inject.Injector; -import com.netflix.priam.backup.BRTestModule; -import com.netflix.priam.config.FakeConfiguration; -import com.netflix.priam.merics.BackupMetrics; -import com.netflix.priam.utils.DateUtil; -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.time.Instant; -import java.time.temporal.ChronoUnit; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.commons.io.FileUtils; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -/** Created by aagrawal on 1/1/19. */ -public class TestForgottenFileManager { - private ForgottenFilesManager forgottenFilesManager; - private TestBackupUtils testBackupUtils; - private ForgottenFilesConfiguration configuration; - private List allFiles = new ArrayList<>(); - private Instant snapshotInstant; - private Path snapshotDir; - - public TestForgottenFileManager() { - Injector injector = Guice.createInjector(new BRTestModule()); - BackupMetrics backupMetrics = injector.getInstance(BackupMetrics.class); - configuration = new ForgottenFilesConfiguration(); - forgottenFilesManager = new ForgottenFilesManager(configuration, backupMetrics); - testBackupUtils = injector.getInstance(TestBackupUtils.class); - } - - @Before - public void prep() throws Exception { - cleanup(); - Instant now = DateUtil.getInstant(); - snapshotInstant = now; - Path file1 = Paths.get(testBackupUtils.createFile("file1", now.minus(10, ChronoUnit.DAYS))); - Path file2 = Paths.get(testBackupUtils.createFile("file2", now.minus(8, ChronoUnit.DAYS))); - Path file3 = Paths.get(testBackupUtils.createFile("file3", now.minus(6, ChronoUnit.DAYS))); - Path file4 = Paths.get(testBackupUtils.createFile("file4", now.minus(4, ChronoUnit.DAYS))); - Path file5 = Paths.get(testBackupUtils.createFile("file5", now.minus(1, ChronoUnit.DAYS))); - Path file6 = - Paths.get( - testBackupUtils.createFile( - "tmplink-lb-59516-big-Index.db", now.minus(3, ChronoUnit.DAYS))); - Path file7 = - Paths.get(testBackupUtils.createFile("file7.tmp", now.minus(3, ChronoUnit.DAYS))); - - allFiles.add(file1); - allFiles.add(file2); - allFiles.add(file3); - allFiles.add(file4); - allFiles.add(file5); - allFiles.add(file6); - allFiles.add(file7); - - // Create a snapshot with file2, file3, file4. - Path columnfamilyDir = file1.getParent(); - snapshotDir = - Paths.get( - columnfamilyDir.toString(), - "snapshot", - "snap_v2_" + DateUtil.formatInstant(DateUtil.yyyyMMddHHmm, now)); - snapshotDir.toFile().mkdirs(); - Files.createLink(Paths.get(snapshotDir.toString(), file2.getFileName().toString()), file2); - Files.createLink(Paths.get(snapshotDir.toString(), file3.getFileName().toString()), file3); - Files.createLink(Paths.get(snapshotDir.toString(), file4.getFileName().toString()), file4); - } - - @After - public void cleanup() throws Exception { - String dataDir = configuration.getDataFileLocation(); - org.apache.commons.io.FileUtils.cleanDirectory(new File(dataDir)); - } - - @Test - public void testMoveForgottenFiles() throws IOException, InterruptedException { - Collection files = allFiles.stream().map(Path::toFile).collect(Collectors.toList()); - Path lostFoundDir = - Paths.get(configuration.getDataFileLocation(), forgottenFilesManager.LOST_FOUND); - - // Lets create some extra symlinks in the LOST_FOUND folder. They should not exist anymore - Path randomSymlink = Paths.get(lostFoundDir.toFile().getAbsolutePath(), "random"); - Files.createDirectory(lostFoundDir); - Files.createSymbolicLink(randomSymlink, lostFoundDir); - - forgottenFilesManager.moveForgottenFiles( - new File(configuration.getDataFileLocation()), files); - - // Extra symlinks are deleted. - Assert.assertFalse(Files.exists(randomSymlink)); - - // Symlinks are created for all the files. They are not moved yet. - Collection symlinkFiles = FileUtils.listFiles(lostFoundDir.toFile(), null, false); - Assert.assertEquals(allFiles.size(), symlinkFiles.size()); - for (Path file : allFiles) { - Path symlink = Paths.get(lostFoundDir.toString(), file.getFileName().toString()); - Assert.assertTrue(symlinkFiles.contains(symlink.toFile())); - Assert.assertTrue(Files.isSymbolicLink(symlink)); - Assert.assertTrue(Files.exists(file)); - } - - // Lets change the configuration and try again!! - configuration.setGracePeriodForgottenFileInDaysForRead(0); - forgottenFilesManager.moveForgottenFiles( - new File(configuration.getDataFileLocation()), files); - Collection movedFiles = FileUtils.listFiles(lostFoundDir.toFile(), null, false); - Assert.assertEquals(allFiles.size(), movedFiles.size()); - movedFiles - .stream() - .forEach( - file -> { - Assert.assertTrue( - Files.isRegularFile(Paths.get(file.getAbsolutePath()))); - }); - allFiles.stream() - .forEach( - file -> { - Assert.assertFalse(file.toFile().exists()); - }); - - configuration.setGracePeriodForgottenFileInDaysForRead( - ForgottenFilesConfiguration.DEFAULT_GRACE_PERIOD); - } - - @Test - public void getColumnfamilyFiles() { - - Path columnfamilyDir = allFiles.get(0).getParent(); - Collection columnfamilyFiles = - forgottenFilesManager.getColumnfamilyFiles( - snapshotInstant, columnfamilyDir.toFile()); - Assert.assertEquals(3, columnfamilyFiles.size()); - Assert.assertTrue(columnfamilyFiles.contains(allFiles.get(0).toFile())); - Assert.assertTrue(columnfamilyFiles.contains(allFiles.get(1).toFile())); - Assert.assertTrue(columnfamilyFiles.contains(allFiles.get(2).toFile())); - } - - @Test - public void findAndMoveForgottenFiles() { - Path lostFoundDir = - Paths.get(allFiles.get(0).getParent().toString(), forgottenFilesManager.LOST_FOUND); - forgottenFilesManager.findAndMoveForgottenFiles(snapshotInstant, snapshotDir.toFile()); - - // Only one potential forgotten file - file1. It will be symlink here. - Collection movedFiles = FileUtils.listFiles(lostFoundDir.toFile(), null, false); - Assert.assertEquals(1, movedFiles.size()); - Assert.assertTrue( - movedFiles - .iterator() - .next() - .getName() - .equals(allFiles.get(0).getFileName().toString())); - Assert.assertTrue( - Files.isSymbolicLink(Paths.get(movedFiles.iterator().next().getAbsolutePath()))); - - // All files still remain in columnfamily dir. - Collection cfFiles = - FileUtils.listFiles(new File(allFiles.get(0).getParent().toString()), null, false); - Assert.assertEquals(allFiles.size(), cfFiles.size()); - - // Snapshot is untouched. - Collection snapshotFiles = FileUtils.listFiles(snapshotDir.toFile(), null, false); - Assert.assertEquals(3, snapshotFiles.size()); - - // Lets change the configuration and try again!! - configuration.setGracePeriodForgottenFileInDaysForRead(0); - forgottenFilesManager.findAndMoveForgottenFiles(snapshotInstant, snapshotDir.toFile()); - configuration.setGracePeriodForgottenFileInDaysForRead( - ForgottenFilesConfiguration.DEFAULT_GRACE_PERIOD); - movedFiles = FileUtils.listFiles(lostFoundDir.toFile(), null, false); - Assert.assertEquals(1, movedFiles.size()); - Assert.assertTrue( - Files.isRegularFile(Paths.get(movedFiles.iterator().next().getAbsolutePath()))); - cfFiles = - FileUtils.listFiles(new File(allFiles.get(0).getParent().toString()), null, false); - Assert.assertEquals(6, cfFiles.size()); - int temp_file_name = 1; - for (File file : cfFiles) { - file.getName().equals(allFiles.get(temp_file_name++).getFileName().toString()); - } - - // Snapshot is untouched. - snapshotFiles = FileUtils.listFiles(snapshotDir.toFile(), null, false); - Assert.assertEquals(3, snapshotFiles.size()); - } - - private class ForgottenFilesConfiguration extends FakeConfiguration { - protected static final int DEFAULT_GRACE_PERIOD = 3; - private int gracePeriodForgottenFileInDaysForRead = DEFAULT_GRACE_PERIOD; - - @Override - public boolean isForgottenFileMoveEnabled() { - return true; - } - - @Override - public int getForgottenFileGracePeriodDaysForRead() { - return gracePeriodForgottenFileInDaysForRead; - } - - public void setGracePeriodForgottenFileInDaysForRead( - int gracePeriodForgottenFileInDaysForRead) { - this.gracePeriodForgottenFileInDaysForRead = gracePeriodForgottenFileInDaysForRead; - } - } -} diff --git a/priam/src/test/java/com/netflix/priam/notification/TestBackupNotificationMgr.java b/priam/src/test/java/com/netflix/priam/notification/TestBackupNotificationMgr.java index 2dbece5de..cf4f05a75 100644 --- a/priam/src/test/java/com/netflix/priam/notification/TestBackupNotificationMgr.java +++ b/priam/src/test/java/com/netflix/priam/notification/TestBackupNotificationMgr.java @@ -108,7 +108,7 @@ public void testNoNotificationsNonEmptyFilter( "fakeBackup", "fakeData.db"); AbstractBackupPath abstractBackupPath = abstractBackupPathProvider.get(); - abstractBackupPath.parseLocal(path.toFile(), AbstractBackupPath.BackupFileType.SST); + abstractBackupPath.parseLocal(path.toFile(), AbstractBackupPath.BackupFileType.SST_V2); backupNotificationMgr.notify(abstractBackupPath, UploadStatus.STARTED); new Verifications() { { @@ -149,7 +149,7 @@ public void testNotificationsEmptyFilter( "fakeBackup", "fakeData.db"); AbstractBackupPath abstractBackupPath = abstractBackupPathProvider.get(); - abstractBackupPath.parseLocal(path.toFile(), AbstractBackupPath.BackupFileType.SST); + abstractBackupPath.parseLocal(path.toFile(), AbstractBackupPath.BackupFileType.SST_V2); backupNotificationMgr.notify(abstractBackupPath, UploadStatus.STARTED); new Verifications() { { @@ -190,7 +190,7 @@ public void testNotificationsInvalidFilter( "fakeBackup", "fakeData.db"); AbstractBackupPath abstractBackupPath = abstractBackupPathProvider.get(); - abstractBackupPath.parseLocal(path.toFile(), AbstractBackupPath.BackupFileType.SST); + abstractBackupPath.parseLocal(path.toFile(), AbstractBackupPath.BackupFileType.SST_V2); backupNotificationMgr.notify(abstractBackupPath, UploadStatus.STARTED); new Verifications() { { @@ -254,7 +254,7 @@ public void testNoNotificationsPartiallyValidFilter( new Expectations() { { backupRestoreConfig.getBackupNotifyComponentIncludeList(); - result = "SOME_FAKE_FILE_TYPE_1, SOME_FAKE_FILE_TYPE_2, SST"; + result = "SOME_FAKE_FILE_TYPE_1, SOME_FAKE_FILE_TYPE_2, SST_V2"; maxTimes = 2; } }; diff --git a/priam/src/test/java/com/netflix/priam/restore/TestRestore.java b/priam/src/test/java/com/netflix/priam/restore/TestRestore.java index aeb169911..41c434229 100644 --- a/priam/src/test/java/com/netflix/priam/restore/TestRestore.java +++ b/priam/src/test/java/com/netflix/priam/restore/TestRestore.java @@ -17,6 +17,7 @@ package com.netflix.priam.restore; +import com.google.common.truth.Truth; import com.google.inject.Guice; import com.google.inject.Injector; import com.netflix.priam.backup.BRTestModule; @@ -25,7 +26,6 @@ import com.netflix.priam.config.FakeConfiguration; import com.netflix.priam.config.IConfiguration; import com.netflix.priam.health.InstanceState; -import com.netflix.priam.identity.config.InstanceInfo; import com.netflix.priam.utils.DateUtil; import java.io.IOException; import java.util.ArrayList; @@ -37,7 +37,6 @@ public class TestRestore { private static FakeBackupFileSystem filesystem; private static ArrayList fileList = new ArrayList<>(); private static FakeConfiguration conf; - private static String region; private static Restore restore; private static InstanceState instanceState; @@ -46,29 +45,55 @@ public static void setup() throws InterruptedException, IOException { Injector injector = Guice.createInjector(new BRTestModule()); if (filesystem == null) filesystem = injector.getInstance(FakeBackupFileSystem.class); if (conf == null) conf = (FakeConfiguration) injector.getInstance(IConfiguration.class); - region = injector.getInstance(InstanceInfo.class).getRegion(); if (restore == null) restore = injector.getInstance(Restore.class); if (instanceState == null) instanceState = injector.getInstance(InstanceState.class); } - private static void populateBackupFileSystem(String baseDir) { + private static void populateBackupFileSystem(String cluster) { fileList.clear(); - fileList.add(baseDir + "/" + region + "/fakecluster/123456/201108110030/META/meta.json"); fileList.add( - baseDir + "/" + region + "/fakecluster/123456/201108110030/SNAP/ks1/cf1/f1.db"); + "test_backup/" + + cluster + + "/1808575600/META_V2/1313026200000/SNAPPY/PLAINTEXT/meta_v2_201108110130.json"); fileList.add( - baseDir + "/" + region + "/fakecluster/123456/201108110030/SNAP/ks1/cf1/f2.db"); + "test_backup/" + + cluster + + "/1808575600/SST_V2/1313022601000/ks1/cf1/SNAPPY/PLAINTEXT/me-1-big-Data.db"); fileList.add( - baseDir + "/" + region + "/fakecluster/123456/201108110030/SNAP/ks2/cf1/f2.db"); - fileList.add(baseDir + "/" + region + "/fakecluster/123456/201108110530/SST/ks2/cf1/f3.db"); - fileList.add(baseDir + "/" + region + "/fakecluster/123456/201108110600/SST/ks2/cf1/f4.db"); + "test_backup/" + + cluster + + "/1808575600/SST_V2/1313022601000/ks1/cf1/SNAPPY/PLAINTEXT/me-1-big-Index.db"); + fileList.add( + "test_backup/" + + cluster + + "/1808575600/SST_V2/1313022601000/ks2/cf1/SNAPPY/PLAINTEXT/me-2-big-Data.db"); + fileList.add( + "test_backup/" + + cluster + + "/1808575600/SST_V2/1313040601000/ks2/cf1/SNAPPY/PLAINTEXT/me-2-big-Index.db"); + fileList.add( + "test_backup/" + + cluster + + "/1808575600/SST_V2/1313042400000/ks2/cf1/SNAPPY/PLAINTEXT/me-2-big-Summary.db"); + fileList.add( + "test_backup/" + + cluster + + "/1808575600/SST_V2/1313043000000/ks1/cf1/SNAPPY/PLAINTEXT/me-3-big-Data.db"); + fileList.add( + "test_backup/" + + cluster + + "/1808575600/META_V2/1313043300000/SNAPPY/PLAINTEXT/meta_v2_201108110615.json"); + fileList.add( + "test_backup/" + + cluster + + "/1808575600/META_V2/1313022540000/SNAPPY/PLAINTEXT/meta_v2_201108110029.json"); filesystem.setupTest(fileList); - conf.setRestorePrefix("RESTOREBUCKET/" + baseDir + "/" + region + "/fakecluster"); + conf.setRestorePrefix("RESTOREBUCKET/test_backup/" + cluster + ""); } @Test public void testRestore() throws Exception { - populateBackupFileSystem("test_backup"); + populateBackupFileSystem("1049_fake-app"); String dateRange = "201108110030,201108110530"; restore.restore(new DateUtil.DateRange(dateRange)); Assert.assertTrue(filesystem.downloadedFiles.contains(fileList.get(0))); @@ -82,8 +107,8 @@ public void testRestore() throws Exception { @Test public void testRestoreWithIncremental() throws Exception { - populateBackupFileSystem("test_backup"); - String dateRange = "201108110030,201108110730"; + populateBackupFileSystem("1049_fake-app"); + String dateRange = "201108110030,201108110601"; restore.restore(new DateUtil.DateRange(dateRange)); Assert.assertTrue(filesystem.downloadedFiles.contains(fileList.get(0))); Assert.assertTrue(filesystem.downloadedFiles.contains(fileList.get(1))); @@ -95,50 +120,53 @@ public void testRestoreWithIncremental() throws Exception { } @Test - public void testRestoreLatestWithEmptyMeta() throws Exception { - populateBackupFileSystem("test_backup"); - String metafile = - "test_backup/" + region + "/fakecluster/123456/201108110130/META/meta.json"; - filesystem.addFile(metafile); - String dateRange = "201108110030,201108110530"; + public void testRestoreWithIncrementalFromDifferentCluster() throws Exception { + populateBackupFileSystem("6089_fake-app2"); + String dateRange = "201108110030,201108110601"; restore.restore(new DateUtil.DateRange(dateRange)); - Assert.assertFalse(filesystem.downloadedFiles.contains(fileList.get(0))); - Assert.assertTrue(filesystem.downloadedFiles.contains(metafile)); - Assert.assertFalse(filesystem.downloadedFiles.contains(fileList.get(1))); - Assert.assertFalse(filesystem.downloadedFiles.contains(fileList.get(2))); - Assert.assertFalse(filesystem.downloadedFiles.contains(fileList.get(3))); - Assert.assertFalse(filesystem.downloadedFiles.contains(fileList.get(4))); - Assert.assertFalse(filesystem.downloadedFiles.contains(fileList.get(5))); + Assert.assertTrue(filesystem.downloadedFiles.contains(fileList.get(0))); + Assert.assertTrue(filesystem.downloadedFiles.contains(fileList.get(1))); + Assert.assertTrue(filesystem.downloadedFiles.contains(fileList.get(2))); + Assert.assertTrue(filesystem.downloadedFiles.contains(fileList.get(3))); + Assert.assertTrue(filesystem.downloadedFiles.contains(fileList.get(4))); + Assert.assertTrue(filesystem.downloadedFiles.contains(fileList.get(5))); Assert.assertEquals(Status.FINISHED, instanceState.getRestoreStatus().getStatus()); - Assert.assertEquals(metafile, instanceState.getRestoreStatus().getSnapshotMetaFile()); + } + + @Test + public void testRestoreEmptyMeta() throws Exception { + populateBackupFileSystem("1049_fake-app"); + String metafile = + "test_backup/1049_fake-app/1808575600/META_V2/1313022540000/SNAPPY/PLAINTEXT/meta_v2_201108110029.json"; + String dateRange = "201108110029,201108110030"; + restore.restore(new DateUtil.DateRange(dateRange)); + Truth.assertThat(filesystem.downloadedFiles).containsExactly(metafile); + Assert.assertEquals(Status.FAILED, instanceState.getRestoreStatus().getStatus()); + Assert.assertNull(instanceState.getRestoreStatus().getSnapshotMetaFile()); } @Test public void testRestoreLatest() throws Exception { - populateBackupFileSystem("test_backup"); + populateBackupFileSystem("1049_fake-app"); String metafile = - "test_backup/" + region + "/fakecluster/123456/201108110130/META/meta.json"; - filesystem.addFile(metafile); - String snapFile = - "test_backup/" + region + "/fakecluster/123456/201108110130/SNAP/ks1/cf1/f9.db"; - filesystem.addFile(snapFile); - String dateRange = "201108110030,201108110530"; + "test_backup/1049_fake-app/1808575600/META_V2/1313043300000/SNAPPY/PLAINTEXT/meta_v2_201108110615.json"; + String dateRange = "201108110030,201108110620"; restore.restore(new DateUtil.DateRange(dateRange)); Assert.assertFalse(filesystem.downloadedFiles.contains(fileList.get(0))); - Assert.assertTrue(filesystem.downloadedFiles.contains(metafile)); - Assert.assertTrue(filesystem.downloadedFiles.contains(snapFile)); Assert.assertFalse(filesystem.downloadedFiles.contains(fileList.get(1))); Assert.assertFalse(filesystem.downloadedFiles.contains(fileList.get(2))); Assert.assertFalse(filesystem.downloadedFiles.contains(fileList.get(3))); Assert.assertFalse(filesystem.downloadedFiles.contains(fileList.get(4))); Assert.assertFalse(filesystem.downloadedFiles.contains(fileList.get(5))); + Assert.assertTrue(filesystem.downloadedFiles.contains(fileList.get(6))); + Assert.assertTrue(filesystem.downloadedFiles.contains(fileList.get(7))); Assert.assertEquals(Status.FINISHED, instanceState.getRestoreStatus().getStatus()); Assert.assertEquals(metafile, instanceState.getRestoreStatus().getSnapshotMetaFile()); } @Test public void testNoSnapshots() throws Exception { - populateBackupFileSystem("test_backup"); + populateBackupFileSystem("1049_fake-app"); filesystem.setupTest(fileList); String dateRange = "201109110030,201109110530"; restore.restore(new DateUtil.DateRange(dateRange)); @@ -147,10 +175,9 @@ public void testNoSnapshots() throws Exception { @Test public void testRestoreFromDiffCluster() throws Exception { - populateBackupFileSystem("test_backup_new"); + populateBackupFileSystem("6089_fake-app2"); String dateRange = "201108110030,201108110530"; restore.restore(new DateUtil.DateRange(dateRange)); - System.out.println("Downloaded files: " + filesystem.downloadedFiles); Assert.assertTrue(filesystem.downloadedFiles.contains(fileList.get(0))); Assert.assertTrue(filesystem.downloadedFiles.contains(fileList.get(1))); Assert.assertTrue(filesystem.downloadedFiles.contains(fileList.get(2))); diff --git a/priam/src/test/java/com/netflix/priam/stream/StreamingTest.java b/priam/src/test/java/com/netflix/priam/stream/StreamingTest.java index c88953c2c..2d2a6f5a3 100644 --- a/priam/src/test/java/com/netflix/priam/stream/StreamingTest.java +++ b/priam/src/test/java/com/netflix/priam/stream/StreamingTest.java @@ -43,18 +43,16 @@ public void testAbstractPath() { Injector injector = Guice.createInjector(new BRTestModule()); IConfiguration conf = injector.getInstance(IConfiguration.class); InstanceIdentity factory = injector.getInstance(InstanceIdentity.class); - String region = factory.getInstanceInfo().getRegion(); FifoQueue queue = new FifoQueue<>(10); for (int i = 10; i < 30; i++) { RemoteBackupPath path = new RemoteBackupPath(conf, factory); path.parseRemote( "test_backup/" - + region - + "/fakecluster/123456/201108" + + "1941_fakecluster/123456/SST_V2/201108" + i + "0000" - + "/SNAP/ks1/cf2/f1" + + "/ks1/cf2/SNAPPY/PLAINTEXT/f1" + i + ".db"); queue.adjustAndAdd(path); @@ -63,12 +61,10 @@ public void testAbstractPath() { for (int i = 10; i < 30; i++) { RemoteBackupPath path = new RemoteBackupPath(conf, factory); path.parseRemote( - "test_backup/" - + region - + "/fakecluster/123456/201108" + "test_backup/1941_fakecluster/123456/SST_V2/201108" + i + "0000" - + "/SNAP/ks1/cf2/f2" + + "/ks1/cf2/SNAPPY/PLAINTEXT/f2" + i + ".db"); queue.adjustAndAdd(path); @@ -77,12 +73,10 @@ public void testAbstractPath() { for (int i = 10; i < 30; i++) { RemoteBackupPath path = new RemoteBackupPath(conf, factory); path.parseRemote( - "test_backup/" - + region - + "/fakecluster/123456/201108" + "test_backup/1941_fakecluster/123456/SST_V2/201108" + i + "0000" - + "/SNAP/ks1/cf2/f3" + + "/ks1/cf2/SNAPPY/PLAINTEXT/f3" + i + ".db"); queue.adjustAndAdd(path); @@ -90,43 +84,19 @@ public void testAbstractPath() { RemoteBackupPath path = new RemoteBackupPath(conf, factory); path.parseRemote( - "test_backup/" - + region - + "/fakecluster/123456/201108290000" - + "/SNAP/ks1/cf2/f129.db"); + "test_backup" + + "/1941_fakecluster/123456/SST_V2/201108290000" + + "/ks1/cf2/SNAPPY/PLAINTEXT/f129.db"); Assert.assertTrue(queue.contains(path)); path.parseRemote( - "test_backup/" - + region - + "/fakecluster/123456/201108290000" - + "/SNAP/ks1/cf2/f229.db"); + "test_backup" + + "/1941_fakecluster/123456/SST_V2/201108290000" + + "/ks1/cf2/SNAPPY/PLAINTEXT/f229.db"); Assert.assertTrue(queue.contains(path)); path.parseRemote( - "test_backup/" - + region - + "/fakecluster/123456/201108290000" - + "/SNAP/ks1/cf2/f329.db"); + "test_backup" + + "/1941_fakecluster/123456/SST_V2/201108290000" + + "/ks1/cf2/SNAPPY/PLAINTEXT/f329.db"); Assert.assertTrue(queue.contains(path)); - - path.parseRemote( - "test_backup/" - + region - + "/fakecluster/123456/201108260000/SNAP/ks1/cf2/f326.db To: cass/data/ks1/cf2/f326.db"); - Assert.assertEquals(path, queue.first()); - } - - @Test - public void testIgnoreIndexFiles() { - String[] testInputs = - new String[] { - "User_Authentication_Audit.User_Authentication_Audit_appkey_idx-hc-93-Digest.sha1", - "User_Authentication_Audit.User_Authentication_Audit_appkey_idx-hc-93-Filter.db", - "User_Authentication_Audit.User_Authentication_Audit_appkey_idx-hc-93-Data.db", - "User_Authentication_Audit.User_Authentication_Audit_appkey_idx-hc-93-Statistics.db", - "CS_Agents.CS_Agents_supervisorEmpSk_idx-hc-1-Filter.db", - "CS_Agents.CS_Agents_supervisorEmpSk_idx-hc-1-Digest.sha1", - "CS_Agents.CS_Agents_supervisorEmpSk_idx-hc-1-Statistics.db", - "CS_Agents.CS_Agents_supervisorEmpSk_idx-hc-1-Data.db" - }; } } diff --git a/priam/src/test/resources/fake-app2_meta_v2_201108110130.json b/priam/src/test/resources/fake-app2_meta_v2_201108110130.json new file mode 100644 index 000000000..c23752b94 --- /dev/null +++ b/priam/src/test/resources/fake-app2_meta_v2_201108110130.json @@ -0,0 +1,65 @@ +{ + "info": { + "version": 1, + "appName": "fake-app2", + "region": "us-east-1", + "rack": "us-east-1c", + "backupIdentifier": [ + "1808575600" + ] + }, + "data": [ + { + "keyspaceName": "ks1", + "columnfamilyName": "cf1", + "sstables": [ + { + "prefix": "me-1-big", + "sstableComponents": [ + { + "fileName": "me-1-big-Data.db", + "lastModifiedTime": 1313022601000, + "fileCreationTime": 1313022601000, + "fileSizeOnDisk": 4921307, + "compression": "SNAPPY", + "encryption": "PLAINTEXT", + "isUploaded": false, + "backupPath": "test_backup/6089_fake-app2/1808575600/SST_V2/1313022601000/ks1/cf1/SNAPPY/PLAINTEXT/me-1-big-Data.db" + }, + { + "fileName": "me-1-big-Index.db", + "lastModifiedTime": 1313022601000, + "fileCreationTime": 1313022601000, + "fileSizeOnDisk": 1828266, + "compression": "SNAPPY", + "encryption": "PLAINTEXT", + "isUploaded": false, + "backupPath": "test_backup/6089_fake-app2/1808575600/SST_V2/1313022601000/ks1/cf1/SNAPPY/PLAINTEXT/me-1-big-Index.db" + } + ] + } + ] + }, + { + "keyspaceName": "ks2", + "columnfamilyName": "cf1", + "sstables": [ + { + "prefix": "me-2-big", + "sstableComponents": [ + { + "fileName": "me-2-big-Data.db", + "lastModifiedTime": 1313022601000, + "fileCreationTime": 1313022601000, + "fileSizeOnDisk": 4921307, + "compression": "SNAPPY", + "encryption": "PLAINTEXT", + "isUploaded": false, + "backupPath": "test_backup/6089_fake-app2/1808575600/SST_V2/1313022601000/ks2/cf1/SNAPPY/PLAINTEXT/me-2-big-Data.db" + } + ] + } + ] + } + ] +} diff --git a/priam/src/test/resources/fake-app_meta_v2_201108110029.json b/priam/src/test/resources/fake-app_meta_v2_201108110029.json new file mode 100644 index 000000000..e69de29bb diff --git a/priam/src/test/resources/fake-app_meta_v2_201108110130.json b/priam/src/test/resources/fake-app_meta_v2_201108110130.json new file mode 100644 index 000000000..cd8484c5c --- /dev/null +++ b/priam/src/test/resources/fake-app_meta_v2_201108110130.json @@ -0,0 +1,65 @@ +{ + "info": { + "version": 1, + "appName": "fake-app", + "region": "us-east-1", + "rack": "us-east-1c", + "backupIdentifier": [ + "1808575600" + ] + }, + "data": [ + { + "keyspaceName": "ks1", + "columnfamilyName": "cf1", + "sstables": [ + { + "prefix": "me-1-big", + "sstableComponents": [ + { + "fileName": "me-1-big-Data.db", + "lastModifiedTime": 1313022601000, + "fileCreationTime": 1313022601000, + "fileSizeOnDisk": 4921307, + "compression": "SNAPPY", + "encryption": "PLAINTEXT", + "isUploaded": false, + "backupPath": "test_backup/1049_fake-app/1808575600/SST_V2/1313022601000/ks1/cf1/SNAPPY/PLAINTEXT/me-1-big-Data.db" + }, + { + "fileName": "me-1-big-Index.db", + "lastModifiedTime": 1313022601000, + "fileCreationTime": 1313022601000, + "fileSizeOnDisk": 1828266, + "compression": "SNAPPY", + "encryption": "PLAINTEXT", + "isUploaded": false, + "backupPath": "test_backup/1049_fake-app/1808575600/SST_V2/1313022601000/ks1/cf1/SNAPPY/PLAINTEXT/me-1-big-Index.db" + } + ] + } + ] + }, + { + "keyspaceName": "ks2", + "columnfamilyName": "cf1", + "sstables": [ + { + "prefix": "me-2-big", + "sstableComponents": [ + { + "fileName": "me-2-big-Data.db", + "lastModifiedTime": 1313022601000, + "fileCreationTime": 1313022601000, + "fileSizeOnDisk": 4921307, + "compression": "SNAPPY", + "encryption": "PLAINTEXT", + "isUploaded": false, + "backupPath": "test_backup/1049_fake-app/1808575600/SST_V2/1313022601000/ks2/cf1/SNAPPY/PLAINTEXT/me-2-big-Data.db" + } + ] + } + ] + } + ] +} diff --git a/priam/src/test/resources/fake-app_meta_v2_201108110615.json b/priam/src/test/resources/fake-app_meta_v2_201108110615.json new file mode 100644 index 000000000..65022c727 --- /dev/null +++ b/priam/src/test/resources/fake-app_meta_v2_201108110615.json @@ -0,0 +1,34 @@ +{ + "info": { + "version": 1, + "appName": "fake-app", + "region": "us-east-1", + "rack": "us-east-1c", + "backupIdentifier": [ + "1808575600" + ] + }, + "data": [ + { + "keyspaceName": "ks1", + "columnfamilyName": "cf1", + "sstables": [ + { + "prefix": "me-3-big", + "sstableComponents": [ + { + "fileName": "me-3-big-Data.db", + "lastModifiedTime": 1313043000000, + "fileCreationTime": 1313043000000, + "fileSizeOnDisk": 4921307, + "compression": "SNAPPY", + "encryption": "PLAINTEXT", + "isUploaded": false, + "backupPath": "test_backup/1049_fake-app/1808575600/SST_V2/1313043000000/ks1/cf1/SNAPPY/PLAINTEXT/me-3-big-Data.db" + } + ] + } + ] + } + ] +}