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 2022/08/29 15:36:47 UTC

[GitHub] [ozone] Galsza opened a new pull request, #3727: HDDS-2476. Share more code between metadata and data scanners.

Galsza opened a new pull request, #3727:
URL: https://github.com/apache/ozone/pull/3727

   ## What changes were proposed in this pull request?
   
   Clean up container scanner code so it contains less duplication. No functional change.
   
   ## What is the link to the Apache JIRA
   
   [HDDS-2476: Share more code between metadata and data scanners
   ](https://issues.apache.org/jira/browse/HDDS-2476)
   
   ## How was this patch tested?
   
   No functional changes, unit/integration tests are green.
   


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] adoroszlai commented on a diff in pull request #3727: HDDS-2476. Share more code between metadata and data scanners.

Posted by GitBox <gi...@apache.org>.
adoroszlai commented on code in PR #3727:
URL: https://github.com/apache/ozone/pull/3727#discussion_r965888717


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/AbstractContainerScanner.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.ozoneimpl;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Base class for scheduled scanners on a Datanode.
+ */
+public abstract class AbstractContainerScanner extends Thread {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(AbstractContainerScanner.class);
+
+  private final long dataScanInterval;
+
+  /**
+   * True if the thread is stopping.<p/>
+   * Protected by this object's lock.
+   */
+  private volatile boolean stopping = false;
+
+  public AbstractContainerScanner(String name, long dataScanInterval) {
+    this.dataScanInterval = dataScanInterval;
+    setName(name);
+    setDaemon(true);
+  }
+
+  @Override
+  public final void run() {
+    AbstractContainerScannerMetrics metrics = getMetrics();
+    try {
+      while (!stopping) {
+        runIteration();
+        metrics.resetNumContainersScanned();
+        metrics.resetNumUnhealthyContainers();
+      }
+      LOG.info("{} exiting.", this);
+    } catch (Exception e) {
+      LOG.error("{} exiting because of exception ", this, e);
+    } finally {
+      if (metrics != null) {
+        metrics.unregister();
+      }
+    }
+  }
+
+  @VisibleForTesting
+  public final void runIteration() {
+    if (stopping) {
+      return;
+    }

Review Comment:
   This is already checked in `run()` right before calling `runIteration()`.  Original code was also checking  _after_ the iteration, before logging and updating the metrics.  I think the intention was to prevent incomplete iteration being logged as "Completed an iteration ...".
   
   Therefore I think we should move it back.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] adoroszlai merged pull request #3727: HDDS-2476. Share more code between metadata and data scanners.

Posted by GitBox <gi...@apache.org>.
adoroszlai merged PR #3727:
URL: https://github.com/apache/ozone/pull/3727


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] adoroszlai commented on pull request #3727: HDDS-2476. Share more code between metadata and data scanners.

Posted by GitBox <gi...@apache.org>.
adoroszlai commented on PR #3727:
URL: https://github.com/apache/ozone/pull/3727#issuecomment-1241099493

   Thanks @Galsza for the refactoring, @errose28, @sodonnel for the review.


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] adoroszlai commented on a diff in pull request #3727: HDDS-2476. Share more code between metadata and data scanners.

Posted by GitBox <gi...@apache.org>.
adoroszlai commented on code in PR #3727:
URL: https://github.com/apache/ozone/pull/3727#discussion_r965023673


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/AbstractContainerScanner.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.ozoneimpl;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Base class for scheduled scanners on a Datanode.
+ */
+public abstract class AbstractContainerScanner extends Thread {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(AbstractContainerScanner.class);
+
+  private final AbstractContainerScannerMetric metrics;
+  private final long dataScanInterval;
+
+  /**
+   * True if the thread is stopping.<p/>
+   * Protected by this object's lock.
+   */
+  private volatile boolean stopping = false;
+
+  public AbstractContainerScanner(String name, long dataScanInterval,
+                                  AbstractContainerScannerMetric metrics) {
+    this.dataScanInterval = dataScanInterval;
+    this.metrics = metrics;
+    setName(name);
+    setDaemon(true);
+  }
+
+  @Override
+  public final void run() {
+    try {
+      while (!stopping) {
+        runIteration();
+        metrics.resetNumContainersScanned();
+        metrics.resetNumUnhealthyContainers();
+      }
+      LOG.info("{} exiting.", this);
+    } catch (Exception e) {
+      LOG.error("{} exiting because of exception ", this, e);
+    } finally {
+      if (metrics != null) {
+        metrics.unregister();
+      }
+    }
+  }
+
+  @VisibleForTesting
+  public final void runIteration() {
+    long startTime = System.nanoTime();
+    scanContainers();
+    long totalDuration = System.nanoTime() - startTime;
+    if (stopping) {
+      return;
+    }
+    metrics.incNumScanIterations();
+    LOG.info("Completed an iteration of " + this.getClass().toString() +
+            " in {} minutes." +

Review Comment:
   Actual output:
   
   ```
   datanode_1  | 2022-09-07 15:48:00,816 [ContainerDataScanner(/data/hdds/hdds)] INFO ozoneimpl.AbstractContainerScanner: Completed an iteration of class org.apache.hadoop.ozone.container.ozoneimpl.ContainerDataScanner in 0 minutes. Number of iterations (since the data-node restart) : 13, Number of containers scanned in this iteration : 0, Number of unhealthy containers found in this iteration : 0
   datanode_1  | 2022-09-07 15:48:00,831 [ContainerMetadataScanner] INFO ozoneimpl.AbstractContainerScanner: Completed an iteration of class org.apache.hadoop.ozone.container.ozoneimpl.ContainerMetadataScanner in 0 minutes. Number of iterations (since the data-node restart) : 25, Number of containers scanned in this iteration : 1, Number of unhealthy containers found in this iteration : 0
   ```
   
   The part `class org.apache.hadoop.ozone.container.ozoneimpl.ContainerDataScanner` looks a bit odd in the message.  I think instance name would be better than class name.  And since it is already included due to thread name pattern, we might omit it.
   
   ```suggestion
       LOG.info("Completed an iteration in {} minutes" +
   ```



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerDataScanner.java:
##########
@@ -163,23 +102,17 @@ private static void logScanCompleted(
     }
   }
 
+  @Override
   public synchronized void shutdown() {
-    this.stopping = true;
     this.canceler.cancel(
         String.format(NAME_FORMAT, volume) + " is shutting down");
-    this.interrupt();
-    try {
-      this.join();
-    } catch (InterruptedException ex) {
-      LOG.warn("Unexpected exception while stopping data scanner for volume "
-          + volume, ex);
-      Thread.currentThread().interrupt();
-    }
+    super.shutdown();
   }
 
   @VisibleForTesting
-  public ContainerDataScrubberMetrics getMetrics() {
-    return metrics;
+  @Override
+  public ContainerDataScannerMetrics getMetrics() {
+    return this.metrics;

Review Comment:
   Currently `metrics` is stored in both parent and subclasses.  We could eliminate the former by making `getMetrics()` abstract, and using it instead of `metrics` variable.  It would also allow getting rid of the cast in subclass constructors.



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/AbstractContainerScannerMetric.java:
##########
@@ -7,31 +7,31 @@
  * "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
+ *   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.
  *
- * 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.ozoneimpl;
 
 import org.apache.hadoop.hdds.annotation.InterfaceAudience;
 import org.apache.hadoop.metrics2.MetricsSystem;
 import org.apache.hadoop.metrics2.annotation.Metric;
 import org.apache.hadoop.metrics2.annotation.Metrics;
-import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.lib.MutableCounterInt;
 import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
 
 /**
- * This class captures the container meta-data scrubber metrics on the
- * data-node.
- **/
+ * Base class for container scanner metrics.
+ */
 @InterfaceAudience.Private
-@Metrics(about = "DataNode container data scrubber metrics", context = "dfs")
-public final class ContainerMetadataScrubberMetrics {
+@Metrics(about = "Datanode container scanner metrics", context = "dfs")
+public abstract class AbstractContainerScannerMetric {

Review Comment:
   Nit: this is a collection of metrics, should be plural: `AbstractContainerScannerMetrics`.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] errose28 commented on a diff in pull request #3727: HDDS-2476. Share more code between metadata and data scanners.

Posted by GitBox <gi...@apache.org>.
errose28 commented on code in PR #3727:
URL: https://github.com/apache/ozone/pull/3727#discussion_r961181431


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerMetadataScanner.java:
##########
@@ -24,128 +24,46 @@
 
 import java.io.IOException;
 import java.util.Iterator;
-import java.util.concurrent.TimeUnit;
 
 /**
  * This class is responsible to perform metadata verification of the
  * containers.
  */
-public class ContainerMetadataScanner extends Thread {
+public class ContainerMetadataScanner extends AbstractContainerScanner {
   public static final Logger LOG =
       LoggerFactory.getLogger(ContainerMetadataScanner.class);
 
-  private final ContainerController controller;
-  private final long metadataScanInterval;
   private final ContainerMetadataScrubberMetrics metrics;
-  /**
-   * True if the thread is stopping.<p/>
-   * Protected by this object's lock.
-   */
-  private boolean stopping = false;
+  private final ContainerController controller;
 
   public ContainerMetadataScanner(ContainerScrubberConfiguration conf,
                                   ContainerController controller) {
+    super(conf.getMetadataScanInterval(),
+        ContainerMetadataScrubberMetrics.create());
     this.controller = controller;
-    this.metadataScanInterval = conf.getMetadataScanInterval();
-    this.metrics = ContainerMetadataScrubberMetrics.create();
+    this.metrics = (ContainerMetadataScrubberMetrics) super.getMetrics();
     setName("ContainerMetadataScanner");
     setDaemon(true);
   }
 
   @Override
-  public void run() {
-    /*
-     * the outer daemon loop exits on shutdown()
-     */
-    LOG.info("Background ContainerMetadataScanner starting up");
-    try {
-      while (!stopping) {
-        runIteration();
-        if (!stopping) {
-          metrics.resetNumUnhealthyContainers();
-          metrics.resetNumContainersScanned();
-        }
-      }
-    } catch (Exception e) {
-      LOG.error("{} exiting because of exception ", this, e);
-    } finally {
-      if (metrics != null) {
-        metrics.unregister();
-      }
-    }
-  }
-
-  @VisibleForTesting
-  void runIteration() {
-    long start = System.nanoTime();
-    Iterator<Container<?>> containerIt = controller.getContainers();
-    while (!stopping && containerIt.hasNext()) {
-      Container container = containerIt.next();
-      try {
-        scrub(container);
-      } catch (IOException e) {
-        LOG.info("Unexpected error while scrubbing container {}",
-            container.getContainerData().getContainerID());
-      } finally {
-        metrics.incNumContainersScanned();
-      }
-    }
-    long interval = System.nanoTime() - start;
-    if (!stopping) {
-      metrics.incNumScanIterations();
-      LOG.info("Completed an iteration of container metadata scrubber in" +
-              " {} minutes." +
-              " Number of  iterations (since the data-node restart) : {}" +
-              ", Number of containers scanned in this iteration : {}" +
-              ", Number of unhealthy containers found in this iteration : {}",
-          TimeUnit.NANOSECONDS.toMinutes(interval),
-          metrics.getNumScanIterations(),
-          metrics.getNumContainersScanned(),
-          metrics.getNumUnHealthyContainers());
-      long elapsedMillis = TimeUnit.NANOSECONDS.toMillis(interval);
-      long remainingSleep = metadataScanInterval - elapsedMillis;
-      if (remainingSleep > 0) {
-        try {
-          Thread.sleep(remainingSleep);
-        } catch (InterruptedException e) {
-          LOG.info("Background ContainerMetadataScanner interrupted." +
-              " Going to exit");
-          // Restore the interruption flag and the internal `stopping`
-          // variable to prevent the next iteration thus stopping the thread
-          interrupt();
-          this.stopping = true;
-        }
-      }
-    }
+  public Iterator<Container<?>> getContainerIterator() {
+    return controller.getContainers();
   }
 
   @VisibleForTesting
-  public void scrub(Container container) throws IOException {
+  @Override
+  public void scanContainer(Container container) throws IOException {
+    metrics.incNumContainersScanned();

Review Comment:
   nit. Move this after the if statement since that is when the container has finished being scanned. Same for the data scanner.



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerDataScanner.java:
##########
@@ -32,11 +26,16 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
+import java.time.Instant;
+import java.util.Iterator;
+import java.util.Optional;
+
 /**
  * VolumeScanner scans a single volume.  Each VolumeScanner has its own thread.
  * <p>They are all managed by the DataNode's BlockScanner.

Review Comment:
   This comment can be updated. This class is no longer the VolumeScanner, and I think this second line is obsolete. I don't know of a BlockScanner or equivalent class on the datanode.



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerMetadataScanner.java:
##########
@@ -24,128 +24,46 @@
 
 import java.io.IOException;
 import java.util.Iterator;
-import java.util.concurrent.TimeUnit;
 
 /**
  * This class is responsible to perform metadata verification of the
  * containers.
  */
-public class ContainerMetadataScanner extends Thread {
+public class ContainerMetadataScanner extends AbstractContainerScanner {
   public static final Logger LOG =
       LoggerFactory.getLogger(ContainerMetadataScanner.class);
 
-  private final ContainerController controller;
-  private final long metadataScanInterval;
   private final ContainerMetadataScrubberMetrics metrics;
-  /**
-   * True if the thread is stopping.<p/>
-   * Protected by this object's lock.
-   */
-  private boolean stopping = false;
+  private final ContainerController controller;
 
   public ContainerMetadataScanner(ContainerScrubberConfiguration conf,
                                   ContainerController controller) {
+    super(conf.getMetadataScanInterval(),
+        ContainerMetadataScrubberMetrics.create());
     this.controller = controller;
-    this.metadataScanInterval = conf.getMetadataScanInterval();
-    this.metrics = ContainerMetadataScrubberMetrics.create();
+    this.metrics = (ContainerMetadataScrubberMetrics) super.getMetrics();
     setName("ContainerMetadataScanner");
     setDaemon(true);

Review Comment:
   Looks like this setDaemon call can be removed here since the parent is already setting it. Also a minor thing but the thread name can be a parameter to the parent constructor who would call setName instead of each child doing it.



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerMetadataScanner.java:
##########
@@ -24,128 +24,46 @@
 
 import java.io.IOException;
 import java.util.Iterator;
-import java.util.concurrent.TimeUnit;
 
 /**
  * This class is responsible to perform metadata verification of the
  * containers.

Review Comment:
   Let's add to the javadoc that only one thread will be responsible for doing metadata scans across all volumes.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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


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