Skip to content

Commit 8024818

Browse files
avijayanhwxbshashikant
authored andcommitted
HDDS-1786 : Datanodes takeSnapshot should delete previously created snapshots (#1163)
1 parent 06ad654 commit 8024818

File tree

3 files changed

+120
-5
lines changed

3 files changed

+120
-5
lines changed
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,48 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package org.apache.hadoop.ozone.container.common.transport.server.ratis;
20+
21+
import org.apache.hadoop.hdds.conf.Config;
22+
import org.apache.hadoop.hdds.conf.ConfigGroup;
23+
import org.apache.hadoop.hdds.conf.ConfigTag;
24+
import org.apache.hadoop.hdds.conf.ConfigType;
25+
26+
/**
27+
* Holds configuration items for Ratis/Raft server.
28+
*/
29+
@ConfigGroup(prefix = "hdds.ratis.server")
30+
public class RatisServerConfiguration {
31+
32+
private int numSnapshotsRetained;
33+
34+
@Config(key = "num.snapshots.retained",
35+
type = ConfigType.INT,
36+
defaultValue = "5",
37+
tags = {ConfigTag.STORAGE},
38+
description = "Config parameter to specify number of old snapshots " +
39+
"retained at the Ratis leader.")
40+
public void setNumSnapshotsRetained(int numSnapshotsRetained) {
41+
this.numSnapshotsRetained = numSnapshotsRetained;
42+
}
43+
44+
public int getNumSnapshotsRetained() {
45+
return numSnapshotsRetained;
46+
}
47+
48+
}

hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java

Lines changed: 11 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -19,8 +19,8 @@
1919
package org.apache.hadoop.ozone.container.common.transport.server.ratis;
2020

2121
import com.google.common.annotations.VisibleForTesting;
22-
import org.apache.hadoop.conf.Configuration;
2322
import org.apache.hadoop.conf.StorageUnit;
23+
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
2424
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
2525
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto;
2626
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -101,7 +101,7 @@ private static long nextCallId() {
101101
private final long cacheEntryExpiryInteval;
102102
private boolean isStarted = false;
103103
private DatanodeDetails datanodeDetails;
104-
private final Configuration conf;
104+
private final OzoneConfiguration conf;
105105
// TODO: Remove the gids set when Ratis supports an api to query active
106106
// pipelines
107107
private final Set<RaftGroupId> raftGids = new HashSet<>();
@@ -110,7 +110,7 @@ private static long nextCallId() {
110110
private XceiverServerRatis(DatanodeDetails dd, int port,
111111
ContainerDispatcher dispatcher, ContainerController containerController,
112112
StateContext context, GrpcTlsConfig tlsConfig, CertificateClient caClient,
113-
Configuration conf)
113+
OzoneConfiguration conf)
114114
throws IOException {
115115
super(conf, caClient);
116116
this.conf = conf;
@@ -255,6 +255,13 @@ private RaftProperties newRaftProperties() {
255255
OzoneConfigKeys.DFS_CONTAINER_RATIS_LOG_PURGE_GAP_DEFAULT);
256256
RaftServerConfigKeys.Log.setPurgeGap(properties, purgeGap);
257257

258+
//Set the number of Snapshots Retained.
259+
RatisServerConfiguration ratisServerConfiguration =
260+
conf.getObject(RatisServerConfiguration.class);
261+
int numSnapshotsRetained =
262+
ratisServerConfiguration.getNumSnapshotsRetained();
263+
RaftServerConfigKeys.Snapshot.setSnapshotRetentionPolicy(properties,
264+
numSnapshotsRetained);
258265
return properties;
259266
}
260267

@@ -377,7 +384,7 @@ private RpcType setRpcType(RaftProperties properties) {
377384
}
378385

379386
public static XceiverServerRatis newXceiverServerRatis(
380-
DatanodeDetails datanodeDetails, Configuration ozoneConf,
387+
DatanodeDetails datanodeDetails, OzoneConfiguration ozoneConf,
381388
ContainerDispatcher dispatcher, ContainerController containerController,
382389
CertificateClient caClient, StateContext context) throws IOException {
383390
int localPort = ozoneConf.getInt(

hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachine.java

Lines changed: 61 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -23,22 +23,28 @@
2323
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
2424
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
2525
import org.apache.hadoop.ozone.MiniOzoneCluster;
26+
import org.apache.hadoop.ozone.OzoneConfigKeys;
2627
import org.apache.hadoop.ozone.client.CertificateClientTestImpl;
2728
import org.apache.hadoop.ozone.client.ObjectStore;
2829
import org.apache.hadoop.ozone.client.OzoneClient;
2930
import org.apache.hadoop.ozone.client.OzoneClientFactory;
3031
import org.apache.hadoop.ozone.client.io.KeyOutputStream;
3132
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
33+
import org.apache.hadoop.ozone.container.ContainerTestHelper;
34+
import org.apache.hadoop.ozone.container.common.transport.server.ratis.ContainerStateMachine;
35+
import org.apache.hadoop.ozone.container.common.transport.server.ratis.RatisServerConfiguration;
3236
import org.apache.hadoop.ozone.om.OzoneManager;
3337
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
3438
import org.apache.hadoop.test.GenericTestUtils;
39+
import org.apache.ratis.statemachine.impl.SimpleStateMachineStorage;
3540
import org.junit.AfterClass;
3641
import org.junit.Assert;
3742
import org.junit.BeforeClass;
3843
import org.junit.Test;
3944

4045
import java.io.File;
4146
import java.io.IOException;
47+
import java.nio.file.Path;
4248
import java.util.HashMap;
4349
import java.util.List;
4450
import java.util.concurrent.TimeUnit;
@@ -85,7 +91,8 @@ public static void init() throws Exception {
8591
conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, TimeUnit.SECONDS);
8692
conf.setQuietMode(false);
8793
OzoneManager.setTestSecureOmFlag(true);
88-
// conf.set(HADOOP_SECURITY_AUTHENTICATION, KERBEROS.toString());
94+
conf.setLong(OzoneConfigKeys.DFS_RATIS_SNAPSHOT_THRESHOLD_KEY, 1);
95+
// conf.set(HADOOP_SECURITY_AUTHENTICATION, KERBEROS.toString());
8996
cluster =
9097
MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1)
9198
.setHbInterval(200)
@@ -148,4 +155,57 @@ public void testContainerStateMachineFailures() throws Exception {
148155
.getContainerState()
149156
== ContainerProtos.ContainerDataProto.State.UNHEALTHY);
150157
}
158+
159+
@Test
160+
public void testRatisSnapshotRetention() throws Exception {
161+
162+
ContainerStateMachine stateMachine =
163+
(ContainerStateMachine) ContainerTestHelper.getStateMachine(cluster);
164+
SimpleStateMachineStorage storage =
165+
(SimpleStateMachineStorage) stateMachine.getStateMachineStorage();
166+
Assert.assertNull(storage.findLatestSnapshot());
167+
168+
// Write 10 keys. Num snapshots should be equal to config value.
169+
for (int i = 1; i <= 10; i++) {
170+
OzoneOutputStream key =
171+
objectStore.getVolume(volumeName).getBucket(bucketName)
172+
.createKey(("ratis" + i), 1024, ReplicationType.RATIS,
173+
ReplicationFactor.ONE, new HashMap<>());
174+
// First write and flush creates a container in the datanode
175+
key.write(("ratis" + i).getBytes());
176+
key.flush();
177+
key.write(("ratis" + i).getBytes());
178+
}
179+
180+
RatisServerConfiguration ratisServerConfiguration =
181+
conf.getObject(RatisServerConfiguration.class);
182+
183+
stateMachine =
184+
(ContainerStateMachine) ContainerTestHelper.getStateMachine(cluster);
185+
storage = (SimpleStateMachineStorage) stateMachine.getStateMachineStorage();
186+
Path parentPath = storage.findLatestSnapshot().getFile().getPath();
187+
int numSnapshots = parentPath.getParent().toFile().listFiles().length;
188+
Assert.assertTrue(Math.abs(ratisServerConfiguration
189+
.getNumSnapshotsRetained() - numSnapshots) <= 1);
190+
191+
// Write 10 more keys. Num Snapshots should remain the same.
192+
for (int i = 11; i <= 20; i++) {
193+
OzoneOutputStream key =
194+
objectStore.getVolume(volumeName).getBucket(bucketName)
195+
.createKey(("ratis" + i), 1024, ReplicationType.RATIS,
196+
ReplicationFactor.ONE, new HashMap<>());
197+
// First write and flush creates a container in the datanode
198+
key.write(("ratis" + i).getBytes());
199+
key.flush();
200+
key.write(("ratis" + i).getBytes());
201+
}
202+
stateMachine =
203+
(ContainerStateMachine) ContainerTestHelper.getStateMachine(cluster);
204+
storage = (SimpleStateMachineStorage) stateMachine.getStateMachineStorage();
205+
parentPath = storage.findLatestSnapshot().getFile().getPath();
206+
numSnapshots = parentPath.getParent().toFile().listFiles().length;
207+
Assert.assertTrue(Math.abs(ratisServerConfiguration
208+
.getNumSnapshotsRetained() - numSnapshots) <= 1);
209+
}
210+
151211
}

0 commit comments

Comments
 (0)