ソースを参照

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

avijayanhwx 5 年 前
コミット
8024818d33

+ 48 - 0
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/RatisServerConfiguration.java

@@ -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;
+  }
+
+}

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

@@ -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 @@ public final class XceiverServerRatis extends XceiverServer {
   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 @@ public final class XceiverServerRatis extends XceiverServer {
   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 @@ public final class XceiverServerRatis extends XceiverServer {
         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 @@ public final class XceiverServerRatis extends XceiverServer {
   }
 
   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(

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

@@ -23,15 +23,20 @@ import org.apache.hadoop.hdds.client.ReplicationType;
 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 org.junit.Test;
 
 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,7 +91,8 @@ public class TestContainerStateMachine {
     conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, TimeUnit.SECONDS);
     conf.setQuietMode(false);
     OzoneManager.setTestSecureOmFlag(true);
-  //  conf.set(HADOOP_SECURITY_AUTHENTICATION, KERBEROS.toString());
+    conf.setLong(OzoneConfigKeys.DFS_RATIS_SNAPSHOT_THRESHOLD_KEY, 1);
+    //  conf.set(HADOOP_SECURITY_AUTHENTICATION, KERBEROS.toString());
     cluster =
         MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1)
             .setHbInterval(200)
@@ -148,4 +155,57 @@ public class TestContainerStateMachine {
             .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);
+  }
+
 }