From 955727593c900edf1a4edddb03859b48c1d56c29 Mon Sep 17 00:00:00 2001 From: Matt Lehman Date: Thu, 2 May 2024 09:29:57 -0700 Subject: [PATCH] Cease stopping Cassandra in the restore process and reveal the ability to restore to a staging area and then import or move the data depending on whether Cassandra is running. --- .../priam/backup/AbstractBackupPath.java | 9 +- .../priam/backup/BackupRestoreUtil.java | 10 +- .../netflix/priam/config/IConfiguration.java | 4 + .../priam/config/PriamConfiguration.java | 5 + .../priam/connection/CassandraOperations.java | 61 ++++++++ .../connection/ICassandraOperations.java | 6 + .../priam/health/CassandraMonitor.java | 4 +- .../priam/restore/AbstractRestore.java | 29 ++-- .../com/netflix/priam/restore/Restore.java | 7 +- .../cluser/management/TestCompaction.groovy | 2 +- .../netflix/priam/backup/BRTestModule.java | 3 + .../priam/backupv2/TestBackupV2Service.java | 2 + .../priam/config/FakeConfiguration.java | 13 ++ .../connection/TestCassandraOperations.java | 136 +++++++++++++++++- .../priam/health/TestCassandraMonitor.java | 2 +- 15 files changed, 261 insertions(+), 32 deletions(-) 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 4ddbd22c5..579bced47 100644 --- a/priam/src/main/java/com/netflix/priam/backup/AbstractBackupPath.java +++ b/priam/src/main/java/com/netflix/priam/backup/AbstractBackupPath.java @@ -146,12 +146,13 @@ protected String match(Date start, Date end) { /** Local restore file */ public File newRestoreFile() { File return_; - String dataDir = config.getDataFileLocation(); + String dataDir = config.getRestoreDataLocation(); switch (type) { case SECONDARY_INDEX_V2: - String restoreFileName = - PATH_JOINER.join(dataDir, keyspace, columnFamily, indexDir, fileName); - return_ = new File(restoreFileName); + return_ = + new File( + PATH_JOINER.join( + dataDir, keyspace, columnFamily, indexDir, fileName)); break; case META_V2: return_ = new File(PATH_JOINER.join(config.getDataFileLocation(), fileName)); 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 635d91b26..1e17cb80f 100644 --- a/priam/src/main/java/com/netflix/priam/backup/BackupRestoreUtil.java +++ b/priam/src/main/java/com/netflix/priam/backup/BackupRestoreUtil.java @@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableSet; import com.netflix.priam.backupv2.IMetaProxy; import com.netflix.priam.utils.DateUtil; +import java.io.IOException; import java.nio.file.DirectoryStream; import java.nio.file.Files; import java.nio.file.Path; @@ -141,19 +142,18 @@ public final boolean isFiltered(String keyspace, String columnFamilyDir) { * @param dataDirectory the location of the data folder. * @param monitoringFolder folder where cassandra backup's are configured. * @return Set of the path(s) containing the backup folder for each columnfamily. - * @throws Exception incase of IOException. + * @throws IOException */ public static ImmutableSet getBackupDirectories( - String dataDirectory, String monitoringFolder) throws Exception { + String dataDirectory, String monitoringFolder) throws IOException { ImmutableSet.Builder backupPaths = ImmutableSet.builder(); Path dataPath = Paths.get(dataDirectory); if (Files.exists(dataPath) && Files.isDirectory(dataPath)) try (DirectoryStream directoryStream = - Files.newDirectoryStream(dataPath, path -> Files.isDirectory(path))) { + Files.newDirectoryStream(dataPath, Files::isDirectory)) { for (Path keyspaceDirPath : directoryStream) { try (DirectoryStream keyspaceStream = - Files.newDirectoryStream( - keyspaceDirPath, path -> Files.isDirectory(path))) { + Files.newDirectoryStream(keyspaceDirPath, Files::isDirectory)) { for (Path columnfamilyDirPath : keyspaceStream) { Path backupDirPath = Paths.get(columnfamilyDirPath.toString(), monitoringFolder); 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 7c611f0fe..c9f17eb06 100644 --- a/priam/src/main/java/com/netflix/priam/config/IConfiguration.java +++ b/priam/src/main/java/com/netflix/priam/config/IConfiguration.java @@ -398,6 +398,10 @@ default String getRestoreSnapshot() { return StringUtils.EMPTY; } + default String getRestoreDataLocation() { + return getCassandraBaseDirectory() + "/restore"; + } + /** @return Get the region to connect to SDB for instance identity */ default String getSDBInstanceIdentityRegion() { return "us-east-1"; 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 d565eb1f2..cfd3eae39 100644 --- a/priam/src/main/java/com/netflix/priam/config/PriamConfiguration.java +++ b/priam/src/main/java/com/netflix/priam/config/PriamConfiguration.java @@ -278,6 +278,11 @@ public String getRestoreSnapshot() { return config.get(PRIAM_PRE + ".restore.snapshot", ""); } + @Override + public String getRestoreDataLocation() { + return config.get(PRIAM_PRE + ".restore.data.location"); + } + @Override public boolean isRestoreEncrypted() { return config.get(PRIAM_PRE + ".encrypted.restore.enabled", false); diff --git a/priam/src/main/java/com/netflix/priam/connection/CassandraOperations.java b/priam/src/main/java/com/netflix/priam/connection/CassandraOperations.java index 299610e27..dbdb02653 100644 --- a/priam/src/main/java/com/netflix/priam/connection/CassandraOperations.java +++ b/priam/src/main/java/com/netflix/priam/connection/CassandraOperations.java @@ -16,8 +16,17 @@ */ package com.netflix.priam.connection; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableSet; +import com.netflix.priam.backup.BackupRestoreUtil; import com.netflix.priam.config.IConfiguration; +import com.netflix.priam.health.CassandraMonitor; import com.netflix.priam.utils.RetryableCallable; +import java.io.IOException; +import java.nio.file.DirectoryStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; import java.util.*; import javax.inject.Inject; import org.apache.cassandra.db.ColumnFamilyStoreMBean; @@ -202,4 +211,56 @@ public List> gossipInfo() throws Exception { } return returnPublicIpSourceIpMap; } + + @Override + public List importAll(String srcDir) throws IOException { + List failedImports = new ArrayList<>(); + if (CassandraMonitor.hasCassadraStarted()) { + for (Path tableDir : BackupRestoreUtil.getBackupDirectories(srcDir, "")) { + String keyspace = tableDir.getParent().getFileName().toString(); + String table = tableDir.getFileName().toString().split("-")[0]; + failedImports.addAll(importData(keyspace, table, tableDir.toString())); + } + } else { + recursiveMove(Paths.get(srcDir), Paths.get(configuration.getDataFileLocation())); + } + return failedImports; + } + + private List importData(String keyspace, String table, String source) + throws IOException { + try (JMXNodeTool nodeTool = JMXNodeTool.instance(configuration)) { + return nodeTool.importNewSSTables( + keyspace, + table, + ImmutableSet.of(source), + false /* resetLevel */, + false /* clearRepaired */, + true /* verifySSTables */, + true /* verifyTokens */, + true /* invalidateCaches */, + false /* extendedVerify */, + false /* copyData */); + } + } + + private void recursiveMove(Path source, Path destination) throws IOException { + Preconditions.checkState(Files.exists(source)); + if (!Files.exists(destination)) { + if (!destination.toFile().mkdirs()) { + throw new IOException("Failed creating " + destination); + } + } + try (DirectoryStream directoryStream = Files.newDirectoryStream(source)) { + for (Path path : directoryStream) { + if (Files.isRegularFile(path)) { + Files.move(path, destination.resolve(path.getFileName())); + } else if (Files.isDirectory(path)) { + recursiveMove(path, destination.resolve(path.getFileName())); + } else { + throw new IOException("Failed determining type of inode is " + path); + } + } + } + } } diff --git a/priam/src/main/java/com/netflix/priam/connection/ICassandraOperations.java b/priam/src/main/java/com/netflix/priam/connection/ICassandraOperations.java index ae0c97201..481198e1d 100644 --- a/priam/src/main/java/com/netflix/priam/connection/ICassandraOperations.java +++ b/priam/src/main/java/com/netflix/priam/connection/ICassandraOperations.java @@ -61,4 +61,10 @@ public interface ICassandraOperations { void forceKeyspaceFlush(String keyspaceName) throws Exception; List> gossipInfo() throws Exception; + + /** + * import sstables from the directory at srcDir into the configured data directory. importAll + * Will just move them if Cassandra hasn't started. + */ + List importAll(String srcDir) throws Exception; } diff --git a/priam/src/main/java/com/netflix/priam/health/CassandraMonitor.java b/priam/src/main/java/com/netflix/priam/health/CassandraMonitor.java index 4f580e8af..1955e4aa5 100644 --- a/priam/src/main/java/com/netflix/priam/health/CassandraMonitor.java +++ b/priam/src/main/java/com/netflix/priam/health/CassandraMonitor.java @@ -169,8 +169,8 @@ public static Boolean hasCassadraStarted() { } // Added for testing only - public static void setIsCassadraStarted() { + public static void setIsCassandraStarted(boolean newStartedState) { // Setting cassandra flag to true - isCassandraStarted.set(true); + isCassandraStarted.set(newStartedState); } } 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 5d1ecc0c5..6b6d067f9 100644 --- a/priam/src/main/java/com/netflix/priam/restore/AbstractRestore.java +++ b/priam/src/main/java/com/netflix/priam/restore/AbstractRestore.java @@ -22,16 +22,17 @@ import com.netflix.priam.backup.Status; import com.netflix.priam.backupv2.IMetaProxy; import com.netflix.priam.config.IConfiguration; +import com.netflix.priam.connection.ICassandraOperations; import com.netflix.priam.defaultimpl.ICassandraProcess; +import com.netflix.priam.health.CassandraMonitor; import com.netflix.priam.health.InstanceState; import com.netflix.priam.identity.InstanceIdentity; import com.netflix.priam.scheduler.Task; import com.netflix.priam.utils.DateUtil; import com.netflix.priam.utils.RetryableCallable; import com.netflix.priam.utils.Sleeper; -import java.io.IOException; import java.math.BigInteger; -import java.nio.file.Path; +import java.nio.file.*; import java.time.LocalDateTime; import java.util.*; import java.util.concurrent.Future; @@ -59,6 +60,7 @@ public abstract class AbstractRestore extends Task implements IRestoreStrategy { private final ICassandraProcess cassProcess; private final InstanceState instanceState; private final IPostRestoreHook postRestoreHook; + private final ICassandraOperations cassOps; @Inject @Named("v2") @@ -73,7 +75,8 @@ public AbstractRestore( RestoreTokenSelector tokenSelector, ICassandraProcess cassProcess, InstanceState instanceState, - IPostRestoreHook postRestoreHook) { + IPostRestoreHook postRestoreHook, + ICassandraOperations cassOps) { super(config); this.fs = fs; this.sleeper = sleeper; @@ -86,6 +89,7 @@ public AbstractRestore( new BackupRestoreUtil( config.getRestoreIncludeCFList(), config.getRestoreExcludeCFList()); this.postRestoreHook = postRestoreHook; + this.cassOps = cassOps; } public static boolean isRestoreEnabled(IConfiguration conf) { @@ -108,10 +112,6 @@ private void waitForCompletion(List> futureList) throws Exception { for (Future future : futureList) future.get(); } - private void stopCassProcess() throws IOException { - cassProcess.stop(true); - } - @Override public void execute() throws Exception { if (!isRestoreEnabled(config)) return; @@ -144,7 +144,6 @@ public void restore(DateUtil.DateRange dateRange) throws Exception { new Date(dateRange.getStartTime().toEpochMilli())); instanceIdentity.getInstance().setToken(restoreToken.toString()); } - stopCassProcess(); Optional latestValidMetaFile = BackupRestoreUtil.getLatestValidMetaPath(metaProxy, dateRange); if (!latestValidMetaFile.isPresent()) { @@ -165,9 +164,17 @@ public void restore(DateUtil.DateRange dateRange) throws Exception { } List> futureList = new ArrayList<>(download(allFiles.iterator())); waitForCompletion(futureList); - postRestoreHook.execute(); - instanceState.endRestore(Status.FINISHED, LocalDateTime.now()); - if (!config.doesCassandraStartManually()) cassProcess.start(true); + List failedImports = cassOps.importAll(config.getRestoreDataLocation()); + if (!failedImports.isEmpty()) { + instanceState.endRestore(Status.FAILED, LocalDateTime.now()); + } else { + postRestoreHook.execute(); + if (!config.doesCassandraStartManually() + && !CassandraMonitor.hasCassadraStarted()) { + cassProcess.start(true); + } + instanceState.endRestore(Status.FINISHED, LocalDateTime.now()); + } } catch (Exception e) { instanceState.endRestore(Status.FAILED, LocalDateTime.now()); throw e; diff --git a/priam/src/main/java/com/netflix/priam/restore/Restore.java b/priam/src/main/java/com/netflix/priam/restore/Restore.java index 4c23d2e60..dceaabe5c 100644 --- a/priam/src/main/java/com/netflix/priam/restore/Restore.java +++ b/priam/src/main/java/com/netflix/priam/restore/Restore.java @@ -19,6 +19,7 @@ import com.netflix.priam.backup.AbstractBackupPath; import com.netflix.priam.backup.IBackupFileSystem; import com.netflix.priam.config.IConfiguration; +import com.netflix.priam.connection.ICassandraOperations; import com.netflix.priam.defaultimpl.ICassandraProcess; import com.netflix.priam.health.InstanceState; import com.netflix.priam.identity.InstanceIdentity; @@ -46,7 +47,8 @@ public Restore( InstanceIdentity instanceIdentity, RestoreTokenSelector tokenSelector, InstanceState instanceState, - IPostRestoreHook postRestoreHook) { + IPostRestoreHook postRestoreHook, + ICassandraOperations cassandraOperations) { super( config, fs, @@ -56,7 +58,8 @@ public Restore( tokenSelector, cassProcess, instanceState, - postRestoreHook); + postRestoreHook, + cassandraOperations); } @Override diff --git a/priam/src/test/groovy/com/netflix/priam/cluser/management/TestCompaction.groovy b/priam/src/test/groovy/com/netflix/priam/cluser/management/TestCompaction.groovy index 90fe7db80..1e073b95a 100644 --- a/priam/src/test/groovy/com/netflix/priam/cluser/management/TestCompaction.groovy +++ b/priam/src/test/groovy/com/netflix/priam/cluser/management/TestCompaction.groovy @@ -150,7 +150,7 @@ class TestCompaction extends Specification { private static int concurrentRuns(int size) { - CassandraMonitor.setIsCassadraStarted() + CassandraMonitor.setIsCassandraStarted(true) ExecutorService threads = Executors.newFixedThreadPool(size) List> torun = new ArrayList<>(size) for (int i = 0; i < size; i++) { 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 0e87702ee..939dbc63d 100644 --- a/priam/src/test/java/com/netflix/priam/backup/BRTestModule.java +++ b/priam/src/test/java/com/netflix/priam/backup/BRTestModule.java @@ -28,6 +28,8 @@ import com.netflix.priam.config.FakeConfiguration; import com.netflix.priam.config.IBackupRestoreConfig; import com.netflix.priam.config.IConfiguration; +import com.netflix.priam.connection.CassandraOperations; +import com.netflix.priam.connection.ICassandraOperations; import com.netflix.priam.cred.ICredential; import com.netflix.priam.defaultimpl.FakeCassandraProcess; import com.netflix.priam.defaultimpl.ICassandraProcess; @@ -78,5 +80,6 @@ protected void configure() { 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())); + bind(ICassandraOperations.class).to(CassandraOperations.class).in(Scopes.SINGLETON); } } 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 26f3409f9..9a483b04c 100644 --- a/priam/src/test/java/com/netflix/priam/backupv2/TestBackupV2Service.java +++ b/priam/src/test/java/com/netflix/priam/backupv2/TestBackupV2Service.java @@ -201,6 +201,8 @@ public void updateService( result = "-1"; configuration.isIncrementalBackupEnabled(); result = true; + configuration.getDataFileLocation(); + result = "target/data"; backupRestoreConfig.enableV2Backups(); result = true; backupRestoreConfig.getBackupVerificationCronExpression(); diff --git a/priam/src/test/java/com/netflix/priam/config/FakeConfiguration.java b/priam/src/test/java/com/netflix/priam/config/FakeConfiguration.java index 4d6f65c8b..36f358e90 100644 --- a/priam/src/test/java/com/netflix/priam/config/FakeConfiguration.java +++ b/priam/src/test/java/com/netflix/priam/config/FakeConfiguration.java @@ -44,6 +44,7 @@ public class FakeConfiguration implements IConfiguration { private String partitioner; private String diskFailurePolicy; private int blockForPeersTimeoutInSecs; + private String dataFileLocation; public Map fakeProperties = new HashMap<>(); @@ -345,4 +346,16 @@ public FakeConfiguration setBlockForPeersTimeoutInSecs(int timeout) { public int getBlockForPeersTimeoutInSecs() { return this.blockForPeersTimeoutInSecs; } + + public FakeConfiguration setDataFileLocation(String dataFileLocation) { + this.dataFileLocation = dataFileLocation; + return this; + } + + @Override + public String getDataFileLocation() { + return dataFileLocation == null + ? IConfiguration.super.getDataFileLocation() + : dataFileLocation; + } } diff --git a/priam/src/test/java/com/netflix/priam/connection/TestCassandraOperations.java b/priam/src/test/java/com/netflix/priam/connection/TestCassandraOperations.java index 89c9e5e59..2a2b0a6e9 100644 --- a/priam/src/test/java/com/netflix/priam/connection/TestCassandraOperations.java +++ b/priam/src/test/java/com/netflix/priam/connection/TestCassandraOperations.java @@ -17,12 +17,20 @@ package com.netflix.priam.connection; +import com.google.common.collect.ImmutableSet; +import com.google.common.io.Files; +import com.google.common.truth.Truth; import com.google.inject.Guice; import com.google.inject.Injector; import com.mchange.io.FileUtils; import com.netflix.priam.backup.BRTestModule; +import com.netflix.priam.config.FakeConfiguration; import com.netflix.priam.config.IConfiguration; +import com.netflix.priam.health.CassandraMonitor; import java.io.File; +import java.io.IOException; +import java.nio.file.Paths; +import java.util.ArrayList; import java.util.List; import java.util.Map; import mockit.Expectations; @@ -30,17 +38,34 @@ import mockit.MockUp; import mockit.Mocked; import org.apache.cassandra.tools.NodeProbe; -import org.junit.Assert; -import org.junit.Test; +import org.junit.*; +import org.junit.rules.TestName; /** Created by aagrawal on 3/1/19. */ public class TestCassandraOperations { private final String gossipInfo1 = "src/test/resources/gossipInfoSample_1.txt"; @Mocked private NodeProbe nodeProbe; @Mocked private JMXNodeTool jmxNodeTool; + private FakeConfiguration config; private static CassandraOperations cassandraOperations; + private static final String BASE_DIR = "base"; + private static final String RESTORE_DIR = "restore"; + private static final String DATA_DIR = "data"; + private static final String KS = "ks"; + private static final String TB = "tb"; + private static final String SI = "si"; + private static final String DATAFILE = "datafile"; + private static final String SI_DATAFILE = "si_datafile"; - public TestCassandraOperations() { + @Rule public TestName name = new TestName(); + + @BeforeClass + public static void prepBeforeAllTests() { + deleteBaseDirectoryForAllTests(); + } + + @Before + public void prepareTest() throws IOException { new MockUp() { @Mock NodeProbe instance(IConfiguration config) { @@ -48,13 +73,29 @@ NodeProbe instance(IConfiguration config) { } }; Injector injector = Guice.createInjector(new BRTestModule()); - if (cassandraOperations == null) - cassandraOperations = injector.getInstance(CassandraOperations.class); + cassandraOperations = injector.getInstance(CassandraOperations.class); + Paths.get(getRestoreDir(), KS, TB, SI).toFile().mkdirs(); + Files.touch(Paths.get(getRestoreDir(), KS, TB, DATAFILE).toFile()); + Files.touch(Paths.get(getRestoreDir(), KS, TB, SI, SI_DATAFILE).toFile()); + config = (FakeConfiguration) injector.getInstance(IConfiguration.class); + config.setDataFileLocation(getDataDir()); + CassandraMonitor.setIsCassandraStarted(false); + } + + @After + public void cleanup() { + org.apache.commons.io.FileUtils.deleteQuietly(Paths.get(getRestoreDir()).toFile()); + org.apache.commons.io.FileUtils.deleteQuietly(Paths.get(getDataDir()).toFile()); + config.setDataFileLocation(null); + } + + @AfterClass + public static void cleanupAfterAllTests() { + deleteBaseDirectoryForAllTests(); } @Test public void testGossipInfo() throws Exception { - String gossipInfoFromNodetool = FileUtils.getContentsAsString(new File(gossipInfo1)); new Expectations() { { @@ -80,4 +121,87 @@ public void testGossipInfo() throws Exception { Assert.assertEquals("[123,234]", gossipInfo.get("TOKENS")); }); } + + @Test + public void testRestoreViaMove_dataDirectoryIsCreated() throws IOException { + cassandraOperations.importAll(getRestoreDir()); + Truth.assertThat(Paths.get(getDataDir()).toFile().exists()); + } + + @Test + public void testRestoreViaMove_keyspaceDirectoryIsCreated() throws IOException { + cassandraOperations.importAll(getRestoreDir()); + Truth.assertThat(Paths.get(getDataDir(), KS).toFile().exists()); + } + + @Test + public void testRestoreViaMove_tableDirectoryIsCreated() throws IOException { + cassandraOperations.importAll(getRestoreDir()); + Truth.assertThat(Paths.get(getDataDir(), KS, TB).toFile().exists()); + } + + @Test + public void testRestoreViaMove_secondaryIndexDirectoryIsCreated() throws IOException { + cassandraOperations.importAll(getRestoreDir()); + Truth.assertThat(Paths.get(getDataDir(), KS, TB, SI).toFile().exists()); + } + + @Test + public void testRestoreViaMove_dataFileIsMoved() throws IOException { + cassandraOperations.importAll(getRestoreDir()); + Truth.assertThat(Paths.get(getDataDir(), KS, TB, DATAFILE).toFile().exists()); + } + + @Test + public void testRestoreViaMove_dataFileIsRemovedFromOrigin() throws IOException { + cassandraOperations.importAll(getRestoreDir()); + Truth.assertThat(Paths.get(getRestoreDir(), KS, TB, DATAFILE).toFile().exists()).isFalse(); + } + + @Test + public void testRestoreViaMove_secondaryIndexFileIsMoved() throws IOException { + cassandraOperations.importAll(getRestoreDir()); + Truth.assertThat(Paths.get(getDataDir(), KS, TB, SI, SI_DATAFILE).toFile().exists()); + } + + @Test + public void testRestoreViaMove_secondaryIndexFileIsRemovedFromOrigin() throws IOException { + cassandraOperations.importAll(getRestoreDir()); + Truth.assertThat(Paths.get(getRestoreDir(), KS, TB, SI, SI_DATAFILE).toFile().exists()) + .isFalse(); + } + + @Test + public void testRestoreViaImport() throws IOException { + CassandraMonitor.setIsCassandraStarted(true); + new Expectations() { + { + nodeProbe.importNewSSTables( + KS, + TB, + ImmutableSet.of(Paths.get(getRestoreDir(), KS, TB).toString()), + false /* resetLevel */, + false /* clearRepaired */, + true /* verifySSTables */, + true /* verifyTokens */, + true /* invalidateCaches */, + false /* extendedVerify */, + false /* copyData */); + result = new ArrayList<>(); + } + }; + cassandraOperations.importAll(getRestoreDir()); + } + + private static void deleteBaseDirectoryForAllTests() { + org.apache.commons.io.FileUtils.deleteQuietly(Paths.get(BASE_DIR).toFile()); + } + + private String getRestoreDir() { + return BASE_DIR + "/" + name.getMethodName() + RESTORE_DIR; + } + + private String getDataDir() { + return BASE_DIR + "/" + name.getMethodName() + DATA_DIR; + } } diff --git a/priam/src/test/java/com/netflix/priam/health/TestCassandraMonitor.java b/priam/src/test/java/com/netflix/priam/health/TestCassandraMonitor.java index 4bf3fa61c..dfcf1a142 100644 --- a/priam/src/test/java/com/netflix/priam/health/TestCassandraMonitor.java +++ b/priam/src/test/java/com/netflix/priam/health/TestCassandraMonitor.java @@ -63,7 +63,7 @@ public void testCassandraMonitor() throws Exception { Assert.assertFalse(CassandraMonitor.hasCassadraStarted()); - CassandraMonitor.setIsCassadraStarted(); + CassandraMonitor.setIsCassandraStarted(true); Assert.assertTrue(CassandraMonitor.hasCassadraStarted()); monitor.execute();