23
23
import com .google .common .collect .ImmutableSet ;
24
24
import java .io .ByteArrayOutputStream ;
25
25
import java .io .File ;
26
+ import java .io .IOException ;
26
27
import java .nio .file .Path ;
27
28
import java .util .ArrayList ;
28
29
import java .util .Arrays ;
102
103
@ RunWith (PowerMockRunner .class )
103
104
@ PrepareForTest ({ReflectionUtil .class , ContainerStorageManagerRestoreUtil .class })
104
105
public class TestContainerStorageManager {
105
-
106
106
private static final String STORE_NAME = "store" ;
107
107
private static final String SYSTEM_NAME = "kafka" ;
108
108
private static final String STREAM_NAME = "store-stream" ;
109
- private static final File DEFAULT_STORE_BASE_DIR = new File (System .getProperty ("java.io.tmpdir" ) + File .separator + "store" );
109
+ private static final File DEFAULT_STORE_BASE_DIR = new File (System .getProperty ("java.io.tmpdir" ) + File .separator + "store" );
110
110
private static final File
111
111
DEFAULT_LOGGED_STORE_BASE_DIR = new File (System .getProperty ("java.io.tmpdir" ) + File .separator + "loggedStore" );
112
112
@@ -116,6 +116,7 @@ public class TestContainerStorageManager {
116
116
private SamzaContainerMetrics samzaContainerMetrics ;
117
117
private Map <TaskName , TaskModel > tasks ;
118
118
private StandbyTestContext testContext ;
119
+ private CheckpointManager checkpointManager ;
119
120
120
121
private volatile int systemConsumerCreationCount ;
121
122
private volatile int systemConsumerStartCount ;
@@ -143,7 +144,7 @@ private void addMockedTask(String taskname, int changelogPartition) {
143
144
* Method to create a containerStorageManager with mocked dependencies
144
145
*/
145
146
@ Before
146
- public void setUp () throws InterruptedException {
147
+ public void setUp () throws InterruptedException , IOException {
147
148
taskRestoreMetricGauges = new HashMap <>();
148
149
this .tasks = new HashMap <>();
149
150
this .taskInstanceMetrics = new HashMap <>();
@@ -248,7 +249,7 @@ public Void answer(InvocationOnMock invocation) {
248
249
.thenReturn (
249
250
new scala .collection .immutable .Map .Map1 (new SystemStream (SYSTEM_NAME , STREAM_NAME ), systemStreamMetadata ));
250
251
251
- CheckpointManager checkpointManager = mock (CheckpointManager .class );
252
+ this . checkpointManager = mock (CheckpointManager .class );
252
253
when (checkpointManager .readLastCheckpoint (any (TaskName .class ))).thenReturn (new CheckpointV1 (new HashMap <>()));
253
254
254
255
SSPMetadataCache mockSSPMetadataCache = mock (SSPMetadataCache .class );
@@ -320,6 +321,38 @@ public void testParallelismAndMetrics() throws InterruptedException {
320
321
Assert .assertEquals ("systemConsumerStartCount count should be 1" , 1 , this .systemConsumerStartCount );
321
322
}
322
323
324
+ /**
325
+ * This test will attempt to verify if logged stores are deleted if the input checkpoints are empty.
326
+ * */
327
+ @ Test
328
+ @ SuppressWarnings ("ResultOfMethodCallIgnored" )
329
+ public void testDeleteLoggedStoreOnNoCheckpoints () {
330
+ // reset the mock to reset the stubs in setup method
331
+ reset (this .checkpointManager );
332
+ // redo stubbing to return null checkpoints
333
+ when (this .checkpointManager .readLastCheckpoint (any ())).thenReturn (null );
334
+ // create store under logged stores to demonstrate deletion
335
+ final File storeFile = new File (DEFAULT_LOGGED_STORE_BASE_DIR .getPath () + File .separator + STORE_NAME );
336
+ // add contents to store
337
+ final File storeFilePartition = new File (DEFAULT_LOGGED_STORE_BASE_DIR .getPath () + File .separator + STORE_NAME + File .separator + "Partition_0" );
338
+ storeFile .deleteOnExit ();
339
+ try {
340
+ storeFile .mkdirs ();
341
+ storeFilePartition .createNewFile ();
342
+ Assert .assertTrue ("Assert that stores are present prior to the test." , storeFile .exists ());
343
+ Assert .assertTrue ("Assert that store files are present prior to the test." , storeFilePartition .exists ());
344
+ this .containerStorageManager .start ();
345
+ this .containerStorageManager .shutdown ();
346
+ Assert .assertFalse ("Assert that stores are deleted after the test." , storeFile .exists ());
347
+ Assert .assertFalse ("Assert that store files are deleted after the test." , storeFile .exists ());
348
+ } catch (Exception e ) {
349
+ System .out .printf ("File %s could not be created." , storeFile );
350
+ Assert .fail ();
351
+ } finally {
352
+ storeFile .delete ();
353
+ }
354
+ }
355
+
323
356
@ Test
324
357
public void testNoConfiguredDurableStores () throws InterruptedException {
325
358
taskRestoreMetricGauges = new HashMap <>();
0 commit comments