Skip to content

[GOBBLIN-2168] Add TimeBasedSnapshotCleanupPolicyTest and Fix a bug with props.containsKey() instead of props.contains() in Trash class #4070

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 9 commits into
base: master
Choose a base branch
from
Original file line number Diff line number Diff line change
Expand Up @@ -43,4 +43,4 @@ public boolean shouldDeleteSnapshot(FileStatus snapshot, Trash trash) {
DateTime snapshotTime = Trash.TRASH_SNAPSHOT_NAME_FORMATTER.parseDateTime(snapshot.getPath().getName());
return snapshotTime.plusMinutes(this.retentionMinutes).isBeforeNow();
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -178,7 +178,7 @@ protected Trash(FileSystem fs, Properties props, String user) throws IOException
}

public static Trash getTrash(FileSystem fs, Properties props, String user) throws IOException {
if (props.contains(TRASH_CLASS_KEY)) {
if (props.containsKey(TRASH_CLASS_KEY)) {
return GobblinConstructorUtils.invokeConstructor(Trash.class, props.getProperty(TRASH_CLASS_KEY), fs, props, user);
} else {
return new Trash(fs, props, user);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,71 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 org.apache.gobblin.data.management.trash;

import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
import org.mockito.Mockito;
import org.testng.Assert;
import org.testng.annotations.Test;
import org.testng.annotations.BeforeMethod;

import java.io.IOException;
import java.util.Properties;

public class TimeBasedSnapshotCleanupPolicyTest {

private TimeBasedSnapshotCleanupPolicy cleanupPolicy;

@BeforeMethod
public void setUp() {
// Initialize the cleanup policy with a retention period of 600 minutes (10 hours)
Properties properties = new Properties();
properties.setProperty(TimeBasedSnapshotCleanupPolicy.SNAPSHOT_RETENTION_POLICY_MINUTES_KEY, "600"); // 600 minutes = 10 hours
cleanupPolicy = new TimeBasedSnapshotCleanupPolicy(properties);

}

@Test
public void testShouldDeleteSnapshot() throws IOException {

// Create a Trash
TrashTestBase trashTestBase = new TrashTestBase(new Properties());
Trash trash = trashTestBase.trash;

// Get the current time
DateTime now = DateTime.now(DateTimeZone.UTC);

// Create dummy paths with timestamps between 11 and 9 hours ago
Path path1 = new Path(trash.getTrashLocation(), now.minusHours(11).toString(Trash.TRASH_SNAPSHOT_NAME_FORMATTER));
Path path2 = new Path(trash.getTrashLocation(), now.minusHours(9).toString(Trash.TRASH_SNAPSHOT_NAME_FORMATTER));

// Mock FileStatus to return the desired paths
FileStatus fs1 = Mockito.mock(FileStatus.class);
Mockito.when(fs1.getPath()).thenReturn(path1);

FileStatus fs2 = Mockito.mock(FileStatus.class);
Mockito.when(fs2.getPath()).thenReturn(path2);

// Test snapshot (should be deleted)
Assert.assertTrue(cleanupPolicy.shouldDeleteSnapshot(fs1, trash),"Snapshot should be deleted");
// Test snapshot (should NOT be deleted)
Assert.assertFalse(cleanupPolicy.shouldDeleteSnapshot(fs2, trash),"Snapshot should NOT be deleted");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -318,4 +318,27 @@ public Boolean answer(InvocationOnMock invocation)
}
}

/**
* Test Trash.getTrash() method to ensure that the correct Trash implementation is returned based on the properties evaluation
*/
@Test
public void testGetTrashEvanluateExpected() throws IOException {
Properties properties = new Properties();
properties.setProperty(Trash.TRASH_CLASS_KEY, MockTrash.class.getCanonicalName());
properties.setProperty(TrashFactory.SIMULATE, "true");
properties.setProperty(TrashFactory.SIMULATE_USING_ACTUAL_TRASH, "true");
properties.setProperty(Trash.TRASH_LOCATION_KEY, "/trash/dir");
FileSystem mockFs = mock(FileSystem.class);

Trash trash = Trash.getTrash(mockFs, properties, "testUser");

// Assert if condition is evaluated and the correct trash is returned
Assert.assertTrue(trash instanceof MockTrash);

properties.remove(Trash.TRASH_CLASS_KEY);
Trash trash2 = Trash.getTrash(mockFs, properties, "testUser");
// Assert if the trash would default to Trash if the condition is not met
Assert.assertTrue(trash2 instanceof Trash);
}

}