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/05/23 10:45:11 UTC

[GitHub] [ozone] sodonnel commented on a diff in pull request #3445: HDDS-6771. EC: ReplicationManager - make ContainerReplicaPendingOps into a SCM service

sodonnel commented on code in PR #3445:
URL: https://github.com/apache/ozone/pull/3445#discussion_r879300197


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ContainerReplicaPendingOps.java:
##########
@@ -48,10 +50,19 @@ public class ContainerReplicaPendingOps {
       pendingOps = new ConcurrentHashMap<>();
   private final Striped<ReadWriteLock> stripedLock = Striped.readWriteLock(64);
 
-  public ContainerReplicaPendingOps(final ConfigurationSource conf,
-      Clock clock) {
+  private final ExpiredContainerReplicaOpScrubber
+      expiredContainerReplicaOpScrubber;
+
+  public ContainerReplicaPendingOps(
+      final ConfigurationSource conf,
+      final Clock clock,
+      final SCMServiceManager serviceManager,
+      final SCMContext scmContext) {

Review Comment:
   If we are going to have a separate "scrubber" class, then I don't think ContainerReplicaPendingOps needs a dependency on SCMServiceManager or SCMContext. It also doesn't need to keep a reference to the Scrubber thread.
   
   The scrubber thread will be the "service" that needs registered with ServiceManager, and it will need a reference to the ContainerReplicaPendingOps instance. Even the scrubber does not need a dependency on ServiceManager - we can register it like:
   
   ```
   Scrubber scrubber = new Scrubber(containerReplicaPendingOps);
   serviceManager.register(scrubber);
   ```
   
   That reduces the dependencies all the class have.



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ExpiredContainerReplicaOpScrubber.java:
##########
@@ -0,0 +1,180 @@
+/*
+ * 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.hdds.scm.container.replication;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.ha.SCMContext;
+import org.apache.hadoop.hdds.scm.ha.SCMService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.time.Clock;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * Background service to clean up expired container replica operations.
+ */
+public class ExpiredContainerReplicaOpScrubber implements SCMService {

Review Comment:
   I have seen a few of these "service" classes, and there is a lot of repeated code in them, some of it tricky concurrent code with `wait()` and `notify()` etc. I think this is an opportunity to create an abstract class that that implements the service interfaces and provides a common set of code that other services can use going forward. If we create that, perhaps we can make the new replicationManager use it, and there was another service added recently for EC - BackGroundPipelineScrubber - its a very simple class like this one. I'm sure we can create a common "service" class these classes can extend.



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