Skip to content

Commit

Permalink
Merge pull request #767 from arunagrawal84/3.11
Browse files Browse the repository at this point in the history
Backup 2.0
  • Loading branch information
arunagrawal-84 authored Jan 11, 2019
2 parents eaad292 + 43c7f1c commit 63674fa
Show file tree
Hide file tree
Showing 60 changed files with 1,615 additions and 2,129 deletions.
10 changes: 10 additions & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
@@ -1,4 +1,14 @@
# Changelog

## 2018/01/11 3.11.38
(#761) Add new file format (SST_V2) and methods to get/parse remote locations.
(#761) Upload files from SnapshotMetaService in backup version 2.0, if enabled.
(#761) Process older SNAPSHOT_V2 at the restart of Priam.
(#767) Backup Verification for Backup 2.0.
(#767) Restore for Backup 2.0
(#767) Some API changes for Snapshot Verification
(#767) Remove deprecated code like flush hour or snapshot hour.

## 2018/10/29 3.11.37
* Bug Fix: SnapshotMetaService can leave snapshots if there is any error.
* Bug Fix: SnapshotMetaService should continue building snapshot even if an unexpected file is found in snapshot.
Expand Down
18 changes: 9 additions & 9 deletions build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -30,9 +30,9 @@ allprojects {
}

dependencies {
compile 'org.apache.commons:commons-lang3:3.5'
compile 'org.apache.commons:commons-lang3:3.8.1'
compile 'commons-logging:commons-logging:1.2'
compile 'org.apache.commons:commons-collections4:4.1'
compile 'org.apache.commons:commons-collections4:4.2'
compile 'commons-io:commons-io:2.6'
compile 'commons-cli:commons-cli:1.4'
compile 'commons-httpclient:commons-httpclient:3.1'
Expand All @@ -42,7 +42,7 @@ allprojects {
compile 'com.sun.jersey.contribs:jersey-guice:1.19.4'
compile 'com.google.guava:guava:21.0'
compile 'com.google.code.findbugs:jsr305:3.0.2'
compile 'com.amazonaws:aws-java-sdk:1.11.467'
compile 'com.amazonaws:aws-java-sdk:1.11.475'
compile 'com.google.inject:guice:4.2.2'
compile 'com.google.inject.extensions:guice-servlet:4.2.2'
compile 'org.quartz-scheduler:quartz:2.3.0'
Expand All @@ -52,11 +52,11 @@ allprojects {
compile 'org.apache.cassandra:cassandra-all:3.0.17'
compile 'javax.ws.rs:jsr311-api:1.1.1'
compile 'joda-time:joda-time:2.10.1'
compile 'org.apache.commons:commons-configuration2:2.1.1'
compile 'org.apache.commons:commons-configuration2:2.4'
compile 'xerces:xercesImpl:2.12.0'
compile 'net.java.dev.jna:jna:4.4.0'
compile 'org.apache.httpcomponents:httpclient:4.5.3'
compile 'org.apache.httpcomponents:httpcore:4.4.6'
compile 'net.java.dev.jna:jna:5.2.0'
compile 'org.apache.httpcomponents:httpclient:4.5.6'
compile 'org.apache.httpcomponents:httpcore:4.4.10'
compile 'com.ning:compress-lzf:1.0.4'
compile 'com.google.code.gson:gson:2.8.5'
compile 'org.slf4j:slf4j-api:1.7.25'
Expand All @@ -66,9 +66,9 @@ allprojects {
compile ('com.google.appengine.tools:appengine-gcs-client:0.7') {
exclude module: 'guava'
}
compile 'com.google.apis:google-api-services-storage:v1-rev100-1.22.0'
compile 'com.google.apis:google-api-services-storage:v1-rev141-1.25.0'
compile 'com.google.http-client:google-http-client-jackson2:1.22.0'
compile 'com.netflix.spectator:spectator-api:0.81.2'
compile 'com.netflix.spectator:spectator-api:0.82.0'
compileOnly 'javax.servlet:javax.servlet-api:3.1.0'
testCompile 'org.jmockit:jmockit:1.31'
testCompile "org.spockframework:spock-core:1.1-groovy-2.4"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import com.netflix.priam.config.IConfiguration;
import com.netflix.priam.cryptography.IFileCryptography;
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;
Expand Down Expand Up @@ -128,7 +129,11 @@ private void parseV2Location(String remoteFile) {
}

private Path getV1Location() {
Path path = Paths.get(getV1Prefix().toString(), formatDate(time), type.toString());
Path path =
Paths.get(
getV1Prefix().toString(),
DateUtil.formatyyyyMMddHHmm(time),
type.toString());
if (BackupFileType.isDataFile(type))
path = Paths.get(path.toString(), keyspace, columnFamily);
return Paths.get(path.toString(), fileName);
Expand All @@ -141,7 +146,7 @@ private void parseV1Location(Path remoteFilePath) {
String.format(
"Too few elements (expected: [%d]) in path: %s", 7, remoteFilePath));

time = parseDate(remoteFilePath.getName(4).toString());
time = DateUtil.getDate(remoteFilePath.getName(4).toString());
type = BackupFileType.valueOf(remoteFilePath.getName(5).toString());
if (BackupFileType.isDataFile(type)) {
keyspace = remoteFilePath.getName(6).toString();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -106,7 +106,6 @@ protected List<AbstractBackupPath> upload(
true);

bps.add(bp);
addToRemotePath(bp.getRemotePath());
}
}

Expand Down Expand Up @@ -183,7 +182,4 @@ private boolean isValidBackupDir(File keyspaceDir, File backupDir) {

return true;
}

/** Adds Remote path to the list of Remote Paths */
protected abstract void addToRemotePath(String remotePath);
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,23 +22,19 @@
import com.netflix.priam.config.IConfiguration;
import com.netflix.priam.cryptography.IFileCryptography;
import com.netflix.priam.identity.InstanceIdentity;
import com.netflix.priam.utils.DateUtil;
import java.io.File;
import java.nio.file.Path;
import java.text.ParseException;
import java.time.Instant;
import java.util.Date;
import org.apache.commons.lang3.StringUtils;
import org.joda.time.DateTime;
import org.joda.time.format.DateTimeFormat;
import org.joda.time.format.DateTimeFormatter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

@ImplementedBy(RemoteBackupPath.class)
public abstract class AbstractBackupPath implements Comparable<AbstractBackupPath> {
private static final Logger logger = LoggerFactory.getLogger(AbstractBackupPath.class);
private static final String FMT = "yyyyMMddHHmm";
private static final DateTimeFormatter DATE_FORMAT = DateTimeFormat.forPattern(FMT);
public static final char PATH_SEP = File.separatorChar;

public enum BackupFileType {
Expand Down Expand Up @@ -82,18 +78,6 @@ public AbstractBackupPath(IConfiguration config, InstanceIdentity instanceIdenti
this.config = config;
}

// TODO: This is so wrong as it completely depends on the timezone where application is running.
// Hopefully everyone running Priam has their clocks set to UTC.
public static String formatDate(Date d) {
return new DateTime(d).toString(FMT);
}

// TODO: This is so wrong as it completely depends on the timezone where application is running.
// Hopefully everyone running Priam has their clocks set to UTC.
public Date parseDate(String s) {
return DATE_FORMAT.parseDateTime(s).toDate();
}

public void parseLocal(File file, BackupFileType type) throws ParseException {
this.backupFile = file;

Expand All @@ -117,7 +101,7 @@ public void parseLocal(File file, BackupFileType type) throws ParseException {
2. This is to ensure that all the files from the snapshot are uploaded under single directory in remote file system.
3. For META file we always override the time field via @link{Metadata#decorateMetaJson}
*/
if (type == BackupFileType.SNAP) time = parseDate(elements[3]);
if (type == BackupFileType.SNAP) time = DateUtil.getDate(elements[3]);

this.lastModified = Instant.ofEpochMilli(file.lastModified());
this.fileName = file.getName();
Expand All @@ -126,8 +110,8 @@ public void parseLocal(File file, BackupFileType type) throws ParseException {

/** Given a date range, find a common string prefix Eg: 20120212, 20120213 = 2012021 */
protected String match(Date start, Date end) {
String sString = formatDate(start);
String eString = formatDate(end);
String sString = DateUtil.formatyyyyMMddHHmm(start); // formatDate(start);
String eString = DateUtil.formatyyyyMMddHHmm(end); // formatDate(end);
int diff = StringUtils.indexOfDifference(sString, eString);
if (diff < 0) return sString;
return sString.substring(0, diff);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -115,8 +115,8 @@ public Future<Path> asyncDownloadFile(
public void downloadFile(final Path remotePath, final Path localPath, final int retry)
throws BackupRestoreException {
// TODO: Should we download the file if localPath already exists?
if (remotePath == null) return;

if (remotePath == null || localPath == null) return;
localPath.toFile().getParentFile().mkdirs();
logger.info("Downloading file: {} to location: {}", remotePath, localPath);
try {
new BoundedExponentialRetryCallable<Void>(500, 10000, retry) {
Expand Down
148 changes: 28 additions & 120 deletions priam/src/main/java/com/netflix/priam/backup/BackupVerification.java
Original file line number Diff line number Diff line change
Expand Up @@ -14,150 +14,58 @@
package com.netflix.priam.backup;

import com.google.inject.Inject;
import com.google.inject.Provider;
import com.google.inject.Singleton;
import com.google.inject.name.Named;
import com.netflix.priam.config.IConfiguration;
import com.netflix.priam.utils.DateUtil;
import java.io.FileReader;
import java.nio.file.FileSystems;
import com.netflix.priam.backupv2.IMetaProxy;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.*;
import java.util.stream.Collectors;
import org.apache.commons.collections4.CollectionUtils;
import org.json.simple.parser.JSONParser;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* Created by aagrawal on 2/16/17. This class validates the backup by doing listing of files in the
* backup destination and comparing with meta.json by downloading from the location. Input:
* BackupMetadata that needs to be verified. Since one backupmetadata can have multiple start time,
* provide one startTime if interested in verifying one particular backup. Leave startTime as null
* to get the latest snapshot for the provided BackupMetadata.
* BackupMetadata that needs to be verified.
*/
@Singleton
public class BackupVerification {

private static final Logger logger = LoggerFactory.getLogger(BackupVerification.class);
private final IBackupFileSystem bkpStatusFs;
private final IConfiguration config;
private final IMetaProxy metaProxy;
private final Provider<AbstractBackupPath> abstractBackupPathProvider;

@Inject
BackupVerification(@Named("backup") IBackupFileSystem bkpStatusFs, IConfiguration config) {
this.bkpStatusFs = bkpStatusFs;
this.config = config;
BackupVerification(
@Named("v1") IMetaProxy metaProxy,
Provider<AbstractBackupPath> abstractBackupPathProvider) {
this.metaProxy = metaProxy;
this.abstractBackupPathProvider = abstractBackupPathProvider;
}

public BackupVerificationResult verifyBackup(List<BackupMetadata> metadata, Date startTime) {
BackupVerificationResult result = new BackupVerificationResult();

if (metadata == null || metadata.isEmpty()) return result;

result.snapshotAvailable = true;
// All the dates should be same.
result.selectedDate = metadata.get(0).getSnapshotDate();

List<String> backups =
metadata.stream()
.map(
backupMetadata ->
DateUtil.formatyyyyMMddHHmm(backupMetadata.getStart()))
.collect(Collectors.toList());
logger.info("Snapshots found for {} : [{}]", result.selectedDate, backups);

// find the latest date (default) or verify if one provided
Date latestDate = null;
for (BackupMetadata backupMetadata : metadata) {
if (latestDate == null || latestDate.before(backupMetadata.getStart()))
latestDate = backupMetadata.getStart();

if (startTime != null
&& DateUtil.formatyyyyMMddHHmm(backupMetadata.getStart())
.equals(DateUtil.formatyyyyMMddHHmm(startTime))) {
latestDate = startTime;
break;
}
}

result.snapshotTime = DateUtil.formatyyyyMMddHHmm(latestDate);
logger.info(
"Latest/Requested snapshot date found: {}, for selected/provided date: {}",
result.snapshotTime,
result.selectedDate);

// Get Backup File Iterator
String prefix = config.getBackupPrefix();
logger.info("Looking for meta file in the location: {}", prefix);

Date strippedMsSnapshotTime = DateUtil.getDate(result.snapshotTime);
Iterator<AbstractBackupPath> backupfiles =
bkpStatusFs.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;
}

result.backupFileListAvail = true;

List<AbstractBackupPath> metas = new LinkedList<>();
List<String> s3Listing = new ArrayList<>();

while (backupfiles.hasNext()) {
AbstractBackupPath path = backupfiles.next();
if (path.getFileName().equalsIgnoreCase("meta.json")) metas.add(path);
else s3Listing.add(path.getRemotePath());
}

if (metas.size() == 0) {
logger.error(
"No meta found for snapshotdate: {}", DateUtil.formatyyyyMMddHHmm(latestDate));
return result;
}

result.metaFileFound = true;
// Download meta.json from backup location and uncompress it.
List<String> metaFileList = new ArrayList<>();
try {
Path metaFileLocation =
FileSystems.getDefault().getPath(config.getDataFileLocation(), "tmp_meta.json");
bkpStatusFs.downloadFile(Paths.get(metas.get(0).getRemotePath()), metaFileLocation, 5);
logger.info(
"Meta file successfully downloaded to localhost: {}",
metaFileLocation.toString());
public Optional<BackupMetadata> getLatestBackupMetaData(List<BackupMetadata> metadata) {
return metadata.stream()
.filter(backupMetadata -> backupMetadata != null)
.filter(backupMetadata -> backupMetadata.getStatus() == Status.FINISHED)
.sorted(Comparator.comparing(BackupMetadata::getStart).reversed())
.findFirst();
}

JSONParser jsonParser = new JSONParser();
org.json.simple.JSONArray fileList =
(org.json.simple.JSONArray)
jsonParser.parse(new FileReader(metaFileLocation.toFile()));
for (Object aFileList : fileList) metaFileList.add(aFileList.toString());
public Optional<BackupVerificationResult> verifyBackup(List<BackupMetadata> metadata) {
if (metadata == null || metadata.isEmpty()) return Optional.empty();

} catch (Exception e) {
logger.error("Error while fetching meta.json from path: {}", metas.get(0), e);
return result;
}
Optional<BackupMetadata> latestBackupMetaData = getLatestBackupMetaData(metadata);

if (metaFileList.isEmpty() && s3Listing.isEmpty()) {
logger.info(
"Uncommon Scenario: Both meta file and backup filesystem listing is empty. Considering this as success");
result.valid = true;
return result;
if (!latestBackupMetaData.isPresent()) {
logger.error("No backup found which finished during the time provided.");
return Optional.empty();
}

// Atleast meta file or s3 listing contains some file.
result.filesInS3Only = new ArrayList<>(s3Listing);
result.filesInS3Only.removeAll(metaFileList);
result.filesInMetaOnly = new ArrayList<>(metaFileList);
result.filesInMetaOnly.removeAll(s3Listing);
result.filesMatched =
(ArrayList<String>) CollectionUtils.intersection(metaFileList, s3Listing);

// There could be a scenario that backupfilesystem has more files than meta file. e.g. some
// leftover objects
if (result.filesInMetaOnly.size() == 0) result.valid = true;

return result;
Path metadataLocation = Paths.get(latestBackupMetaData.get().getSnapshotLocation());
metadataLocation = metadataLocation.subpath(1, metadataLocation.getNameCount());
AbstractBackupPath abstractBackupPath = abstractBackupPathProvider.get();
abstractBackupPath.parseRemote(metadataLocation.toString());
return Optional.of((metaProxy.isMetaFileValid(abstractBackupPath)));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -13,20 +13,25 @@
*/
package com.netflix.priam.backup;

import com.netflix.priam.utils.GsonJsonSerializer;
import java.time.Instant;
import java.util.ArrayList;
import java.util.List;

/**
* Created by aagrawal on 2/16/17. This class holds the result from BackupVerification. The default
* are all null and false.
*/
public class BackupVerificationResult {
public boolean snapshotAvailable = false;
public boolean valid = false;
public boolean metaFileFound = false;
public boolean backupFileListAvail = false;
public String selectedDate = null;
public String snapshotTime = null;
public List<String> filesInMetaOnly = null;
public List<String> filesInS3Only = null;
public List<String> filesMatched = null;
public String remotePath = null;
public Instant snapshotInstant = null;
public boolean manifestAvailable = false;
public List<String> filesInMetaOnly = new ArrayList<>();
public int filesMatched = 0;

@Override
public String toString() {
return GsonJsonSerializer.getGson().toJson(this);
}
}
Loading

0 comments on commit 63674fa

Please sign in to comment.