You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2020/04/23 05:01:08 UTC

[GitHub] [hadoop-ozone] adoroszlai commented on a change in pull request #850: HDDS-3222. Add integration test for Recon FSCK.

adoroszlai commented on a change in pull request #850:
URL: https://github.com/apache/hadoop-ozone/pull/850#discussion_r413507894



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskConfig.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.recon.tasks;
+
+import java.util.concurrent.TimeUnit;
+
+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;
+
+/**
+ * The configuration class for the Recon tasks.
+ */
+@ConfigGroup(prefix = "ozone.recon.task")
+public class ReconTaskConfig {
+
+  @Config(key = "pipelinesync.interval",
+      type = ConfigType.TIME, timeUnit = TimeUnit.SECONDS,
+      defaultValue = "600",
+      tags = { ConfigTag.RECON, ConfigTag.OZONE },
+      description = "Interval of the PipelineSyncTask in seconds."

Review comment:
       How about something like "The time interval of periodic sync of pipeline state from SCM to Recon"?  I think that's more friendly.
   
   Also, as config type is time, values can be specified with units for clarity, eg. `10m` (= `600s`), so text doesn't need to state it's "in seconds".

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconScmTask.java
##########
@@ -34,26 +32,30 @@
   private Thread taskThread;
   private ReconTaskStatusDao reconTaskStatusDao;
   private volatile boolean running;
+  private volatile boolean registered;
 
-  @Inject
-  public ReconScmTask(ReconTaskStatusDao reconTaskStatusDao) {
+  protected ReconScmTask(ReconTaskStatusDao reconTaskStatusDao) {
     this.reconTaskStatusDao = reconTaskStatusDao;
   }
 
-  public void register() {
+  private void register() {
     String taskName = getTaskName();
     if (!reconTaskStatusDao.existsById(taskName)) {
       ReconTaskStatus reconTaskStatusRecord = new ReconTaskStatus(
           taskName, 0L, 0L);
       reconTaskStatusDao.insert(reconTaskStatusRecord);
       LOG.info("Registered {} task ", taskName);
+      registered = true;
     }

Review comment:
       If the task is persistent, restarting the task thread while keeping Recon running (which I don't know if is a possible scenario currently) will always check the DB.  If so, `registered` flag is unnecessary if DB check is cheap, otherwise it should be set outside of `if`.

##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/recon/TestReconTasks.java
##########
@@ -0,0 +1,125 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.recon;
+
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL;
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_PIPELINE_REPORT_INTERVAL;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.RATIS;
+import static org.apache.hadoop.ozone.container.ozoneimpl.TestOzoneContainer.runTestOzoneContainerViaDataNode;
+import static org.junit.Assert.assertEquals;
+
+import java.util.List;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.scm.XceiverClientGrpc;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
+import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.recon.scm.ReconContainerManager;
+import org.apache.hadoop.ozone.recon.scm.ReconStorageContainerManagerFacade;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.hadoop.ozone.recon.schema.tables.pojos.MissingContainers;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+/**
+ * Integration Tests for Recon's tasks.
+ */
+public class TestReconTasks {
+
+  private MiniOzoneCluster cluster = null;
+  private OzoneConfiguration conf;
+
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Before
+  public void init() throws Exception {
+    conf = new OzoneConfiguration();
+    conf.set(HDDS_CONTAINER_REPORT_INTERVAL, "5s");
+    conf.set(HDDS_PIPELINE_REPORT_INTERVAL, "5s");
+    conf.set("ozone.recon.task.missingcontainer.interval", "15s");
+    conf.set("ozone.scm.stale.node.interval", "10s");
+    conf.set("ozone.scm.dead.node.interval", "20s");
+    cluster =  MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1)
+        .includeRecon(true).build();
+    cluster.waitForClusterToBeReady();
+  }
+
+  @After
+  public void shutdown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testMissingContainerDownNode() throws Exception {
+    ReconStorageContainerManagerFacade reconScm =
+        (ReconStorageContainerManagerFacade)
+            cluster.getReconServer().getReconStorageContainerManager();
+    StorageContainerManager scm = cluster.getStorageContainerManager();
+    PipelineManager reconPipelineManager = reconScm.getPipelineManager();
+    PipelineManager scmPipelineManager = scm.getPipelineManager();
+
+    // Make sure Recon's pipeline state is initialized.
+    LambdaTestUtils.await(60000, 5000,
+        () -> (reconPipelineManager.getPipelines().size() == 1));

Review comment:
       `>= 1` may be safer (don't know if multiple pipelines are allowed for single datanode).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org