HDDS-1786 : Datanodes takeSnapshot should delete previously created snapshots (#1163)

This commit is contained in:
avijayanhwx 2019-09-13 10:50:31 -07:00 committed by bshashikant
parent 06ad6540b3
commit 8024818d33
3 changed files with 120 additions and 5 deletions

View File

@ -0,0 +1,48 @@
/*
* 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 org.apache.hadoop.hdds.conf.Config;
import org.apache.hadoop.hdds.conf.ConfigGroup;
import org.apache.hadoop.hdds.conf.ConfigTag;
import org.apache.hadoop.hdds.conf.ConfigType;
/**
* Holds configuration items for Ratis/Raft server.
*/
@ConfigGroup(prefix = "hdds.ratis.server")
public class RatisServerConfiguration {
private int numSnapshotsRetained;
@Config(key = "num.snapshots.retained",
type = ConfigType.INT,
defaultValue = "5",
tags = {ConfigTag.STORAGE},
description = "Config parameter to specify number of old snapshots " +
"retained at the Ratis leader.")
public void setNumSnapshotsRetained(int numSnapshotsRetained) {
this.numSnapshotsRetained = numSnapshotsRetained;
}
public int getNumSnapshotsRetained() {
return numSnapshotsRetained;
}
}

View File

@ -19,8 +19,8 @@
package org.apache.hadoop.ozone.container.common.transport.server.ratis;
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.StorageUnit;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@ -101,7 +101,7 @@ private static long nextCallId() {
private final long cacheEntryExpiryInteval;
private boolean isStarted = false;
private DatanodeDetails datanodeDetails;
private final Configuration conf;
private final OzoneConfiguration conf;
// TODO: Remove the gids set when Ratis supports an api to query active
// pipelines
private final Set<RaftGroupId> raftGids = new HashSet<>();
@ -110,7 +110,7 @@ private static long nextCallId() {
private XceiverServerRatis(DatanodeDetails dd, int port,
ContainerDispatcher dispatcher, ContainerController containerController,
StateContext context, GrpcTlsConfig tlsConfig, CertificateClient caClient,
Configuration conf)
OzoneConfiguration conf)
throws IOException {
super(conf, caClient);
this.conf = conf;
@ -255,6 +255,13 @@ private RaftProperties newRaftProperties() {
OzoneConfigKeys.DFS_CONTAINER_RATIS_LOG_PURGE_GAP_DEFAULT);
RaftServerConfigKeys.Log.setPurgeGap(properties, purgeGap);
//Set the number of Snapshots Retained.
RatisServerConfiguration ratisServerConfiguration =
conf.getObject(RatisServerConfiguration.class);
int numSnapshotsRetained =
ratisServerConfiguration.getNumSnapshotsRetained();
RaftServerConfigKeys.Snapshot.setSnapshotRetentionPolicy(properties,
numSnapshotsRetained);
return properties;
}
@ -377,7 +384,7 @@ private RpcType setRpcType(RaftProperties properties) {
}
public static XceiverServerRatis newXceiverServerRatis(
DatanodeDetails datanodeDetails, Configuration ozoneConf,
DatanodeDetails datanodeDetails, OzoneConfiguration ozoneConf,
ContainerDispatcher dispatcher, ContainerController containerController,
CertificateClient caClient, StateContext context) throws IOException {
int localPort = ozoneConf.getInt(

View File

@ -23,15 +23,20 @@
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.client.CertificateClientTestImpl;
import org.apache.hadoop.ozone.client.ObjectStore;
import org.apache.hadoop.ozone.client.OzoneClient;
import org.apache.hadoop.ozone.client.OzoneClientFactory;
import org.apache.hadoop.ozone.client.io.KeyOutputStream;
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
import org.apache.hadoop.ozone.container.ContainerTestHelper;
import org.apache.hadoop.ozone.container.common.transport.server.ratis.ContainerStateMachine;
import org.apache.hadoop.ozone.container.common.transport.server.ratis.RatisServerConfiguration;
import org.apache.hadoop.ozone.om.OzoneManager;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.ratis.statemachine.impl.SimpleStateMachineStorage;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
@ -39,6 +44,7 @@
import java.io.File;
import java.io.IOException;
import java.nio.file.Path;
import java.util.HashMap;
import java.util.List;
import java.util.concurrent.TimeUnit;
@ -85,6 +91,7 @@ public static void init() throws Exception {
conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, TimeUnit.SECONDS);
conf.setQuietMode(false);
OzoneManager.setTestSecureOmFlag(true);
conf.setLong(OzoneConfigKeys.DFS_RATIS_SNAPSHOT_THRESHOLD_KEY, 1);
// conf.set(HADOOP_SECURITY_AUTHENTICATION, KERBEROS.toString());
cluster =
MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1)
@ -148,4 +155,57 @@ public void testContainerStateMachineFailures() throws Exception {
.getContainerState()
== ContainerProtos.ContainerDataProto.State.UNHEALTHY);
}
@Test
public void testRatisSnapshotRetention() throws Exception {
ContainerStateMachine stateMachine =
(ContainerStateMachine) ContainerTestHelper.getStateMachine(cluster);
SimpleStateMachineStorage storage =
(SimpleStateMachineStorage) stateMachine.getStateMachineStorage();
Assert.assertNull(storage.findLatestSnapshot());
// Write 10 keys. Num snapshots should be equal to config value.
for (int i = 1; i <= 10; i++) {
OzoneOutputStream key =
objectStore.getVolume(volumeName).getBucket(bucketName)
.createKey(("ratis" + i), 1024, ReplicationType.RATIS,
ReplicationFactor.ONE, new HashMap<>());
// First write and flush creates a container in the datanode
key.write(("ratis" + i).getBytes());
key.flush();
key.write(("ratis" + i).getBytes());
}
RatisServerConfiguration ratisServerConfiguration =
conf.getObject(RatisServerConfiguration.class);
stateMachine =
(ContainerStateMachine) ContainerTestHelper.getStateMachine(cluster);
storage = (SimpleStateMachineStorage) stateMachine.getStateMachineStorage();
Path parentPath = storage.findLatestSnapshot().getFile().getPath();
int numSnapshots = parentPath.getParent().toFile().listFiles().length;
Assert.assertTrue(Math.abs(ratisServerConfiguration
.getNumSnapshotsRetained() - numSnapshots) <= 1);
// Write 10 more keys. Num Snapshots should remain the same.
for (int i = 11; i <= 20; i++) {
OzoneOutputStream key =
objectStore.getVolume(volumeName).getBucket(bucketName)
.createKey(("ratis" + i), 1024, ReplicationType.RATIS,
ReplicationFactor.ONE, new HashMap<>());
// First write and flush creates a container in the datanode
key.write(("ratis" + i).getBytes());
key.flush();
key.write(("ratis" + i).getBytes());
}
stateMachine =
(ContainerStateMachine) ContainerTestHelper.getStateMachine(cluster);
storage = (SimpleStateMachineStorage) stateMachine.getStateMachineStorage();
parentPath = storage.findLatestSnapshot().getFile().getPath();
numSnapshots = parentPath.getParent().toFile().listFiles().length;
Assert.assertTrue(Math.abs(ratisServerConfiguration
.getNumSnapshotsRetained() - numSnapshots) <= 1);
}
}