Skip to content
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,8 @@
import com.google.common.base.Preconditions;
import com.google.common.cache.Cache;
import com.google.common.cache.CacheBuilder;

import org.apache.commons.io.FileUtils;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.HddsUtils;
Expand Down Expand Up @@ -72,6 +74,7 @@

import java.io.File;
import java.io.IOException;
import java.nio.file.Path;
import java.util.Collection;
import java.util.List;
import java.util.Map;
Expand Down Expand Up @@ -256,6 +259,7 @@ public void persistContainerSet(OutputStream out) throws IOException {
public long takeSnapshot() throws IOException {
TermIndex ti = getLastAppliedTermIndex();
long startTime = Time.monotonicNow();
SingleFileSnapshotInfo lastSnapshot = storage.findLatestSnapshot();
if (ti != null && ti.getIndex() != RaftLog.INVALID_LOG_INDEX) {
final File snapshotFile =
storage.getSnapshotFile(ti.getTerm(), ti.getIndex());
Expand All @@ -265,6 +269,14 @@ public long takeSnapshot() throws IOException {
fos.flush();
// make sure the snapshot file is synced
fos.getFD().sync();

//delete old snapshot only if the above creation step was successful.
if (lastSnapshot != null && lastSnapshot.getFile() != null) {
Path lastSnapshotFile = lastSnapshot.getFile().getPath();
LOG.info("Deleting last snapshot at {} ",
lastSnapshotFile.toString());
FileUtils.deleteQuietly(lastSnapshotFile.toFile());
}
} catch (IOException ioe) {
LOG.info("{}: Failed to write snapshot at:{} file {}", gid, ti,
snapshotFile);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,112 @@
/*
* 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.hadoop.ozone.container.common.transport.server.ratis;

import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;

import java.io.File;
import java.io.IOException;
import java.lang.reflect.Field;
import java.nio.file.Paths;

import org.apache.ratis.server.protocol.TermIndex;
import org.apache.ratis.server.storage.FileInfo;
import org.apache.ratis.statemachine.impl.SimpleStateMachineStorage;
import org.apache.ratis.statemachine.impl.SingleFileSnapshotInfo;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;

/**
* Unit test methods of ContainerStateMachine.
*/
public class TestContainerStateMachine {

@Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder();

@Test
public void testDeleteOldSnapshot() throws Exception {

File firstSnapshotFile = temporaryFolder.newFile();
File secondSnapshotFile = temporaryFolder.newFile();
File thirdSnapshotFile = temporaryFolder.newFile();

FileInfo fileInfoMock = mock(FileInfo.class);
when(fileInfoMock.getPath()).thenReturn(firstSnapshotFile.toPath())
.thenReturn(secondSnapshotFile.toPath());

SingleFileSnapshotInfo singleFileSnapshotInfoMock = mock(
SingleFileSnapshotInfo.class);
when(singleFileSnapshotInfoMock.getFile()).thenReturn(fileInfoMock);

TermIndex termIndexMock = mock(TermIndex.class);
when(termIndexMock.getIndex()).thenReturn(1L);
when(termIndexMock.getTerm()).thenReturn(1L);

SimpleStateMachineStorage simpleStateMachineStorageMock =
mock(SimpleStateMachineStorage.class);
when(simpleStateMachineStorageMock.findLatestSnapshot())
.thenReturn(singleFileSnapshotInfoMock);
when(simpleStateMachineStorageMock.getSnapshotFile(1L, 1L))
.thenReturn(secondSnapshotFile)
.thenReturn(thirdSnapshotFile)
//Return non-existent file while taking 3rd snapshot.
.thenReturn(Paths.get("NonExistentDir", "NonExistentFile")
.toFile());

ContainerStateMachine containerStateMachine =
mock(ContainerStateMachine.class);
when(containerStateMachine.getLastAppliedTermIndex()).thenReturn(
termIndexMock);
when(containerStateMachine.takeSnapshot()).thenCallRealMethod();

// Have to use reflections here since storage is baked into
// ContainerStateMachine class.
Field f1 = containerStateMachine.getClass().getSuperclass()
.getDeclaredField("storage");
f1.setAccessible(true);
f1.set(containerStateMachine, simpleStateMachineStorageMock);

// Verify last snapshot deletion while calling takeSnapshot() API.
assertTrue(firstSnapshotFile.exists());
containerStateMachine.takeSnapshot();
assertFalse(firstSnapshotFile.exists());

// Verify current snapshot deletion while calling takeSnapshot() API once
// more.
assertTrue(secondSnapshotFile.exists());
containerStateMachine.takeSnapshot();
assertFalse(secondSnapshotFile.exists());

// Now, takeSnapshot throws IOException.
try {
containerStateMachine.takeSnapshot();
Assert.fail();
} catch (IOException ioEx) {
//Verify the old snapshot file still exists.
assertTrue(thirdSnapshotFile.exists());
}

}
}
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@
* Tests the containerStateMachine failure handling.
*/

public class TestContainerStateMachine {
public class TestContainerStateMachineInt {

private static MiniOzoneCluster cluster;
private static OzoneConfiguration conf = new OzoneConfiguration();
Expand All @@ -71,7 +71,7 @@ public class TestContainerStateMachine {
@BeforeClass
public static void init() throws Exception {
path = GenericTestUtils
.getTempPath(TestContainerStateMachine.class.getSimpleName());
.getTempPath(TestContainerStateMachineInt.class.getSimpleName());
File baseDir = new File(path);
baseDir.mkdirs();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,8 +28,6 @@
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;

import org.apache.hadoop.ozone.container.common.transport.server.ratis
.ContainerStateMachine;
import org.apache.hadoop.ozone.om.OzoneManager;
import org.apache.hadoop.ozone.om.exceptions.OMException;
import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
Expand Down Expand Up @@ -64,7 +62,7 @@
public class OzoneManagerStateMachine extends BaseStateMachine {

static final Logger LOG =
LoggerFactory.getLogger(ContainerStateMachine.class);
LoggerFactory.getLogger(OzoneManagerStateMachine.class);
private final SimpleStateMachineStorage storage =
new SimpleStateMachineStorage();
private final OzoneManagerRatisServer omRatisServer;
Expand Down