Skip to content
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

HDDS-1786 : Datanodes takeSnapshot should delete previously created s… #1163

Merged
merged 10 commits into from
Sep 13, 2019
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();
avijayanhwx marked this conversation as resolved.
Show resolved Hide resolved
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 {
avijayanhwx marked this conversation as resolved.
Show resolved Hide resolved

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