bharatviswa504 commented on a change in pull request #1511:
URL: https://github.com/apache/ozone/pull/1511#discussion_r532879677
##########
File path:
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmAcls.java
##########
@@ -80,7 +82,7 @@ public static void init() throws Exception {
scmId = UUID.randomUUID().toString();
omId = UUID.randomUUID().toString();
conf.setBoolean(OZONE_ACL_ENABLED, true);
- conf.setInt(OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS, 2);
+ conf.setTimeDuration(OZONE_OPEN_KEY_EXPIRE_THRESHOLD, 2, TimeUnit.SECONDS);
Review comment:
Do we need to set this in this test?
##########
File path:
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java
##########
@@ -87,6 +87,21 @@ private OMConfigKeys() {
"ozone.key.deleting.limit.per.task";
public static final int OZONE_KEY_DELETING_LIMIT_PER_TASK_DEFAULT = 1000;
+ public static final String OZONE_OPEN_KEY_CLEANUP_SERVICE_INTERVAL =
Review comment:
Minor: Do we want to append om to these configs?
Generally, most OM configs have **"ozone.om"**
##########
File path:
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OpenKeyCleanupService.java
##########
@@ -1,88 +1,251 @@
/**
- * 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
+ * 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.
+ * 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.om;
-import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.google.protobuf.ServiceException;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OpenKey;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OpenKeyBucket;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteOpenKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.hadoop.util.Time;
import org.apache.hadoop.hdds.utils.BackgroundService;
import org.apache.hadoop.hdds.utils.BackgroundTask;
import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult.EmptyTaskResult;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.util.Preconditions;
+import org.apache.ratis.util.TimeDuration;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import java.util.concurrent.TimeUnit;
-
/**
- * This is the background service to delete hanging open keys.
- * Scan the metadata of om periodically to get
- * the keys with prefix "#open#" and ask scm to
- * delete metadata accordingly, if scm returns
- * success for keys, then clean up those keys.
+ * Background service to move keys whose creation time is past a given
+ * threshold from the open key table to the deleted table, where they will
+ * later be purged by the {@link KeyDeletingService}.
*/
public class OpenKeyCleanupService extends BackgroundService {
-
private static final Logger LOG =
- LoggerFactory.getLogger(OpenKeyCleanupService.class);
+ LoggerFactory.getLogger(KeyDeletingService.class);
- private final static int OPEN_KEY_DELETING_CORE_POOL_SIZE = 2;
+ // Use only a single thread for open key deletion. Multiple threads would
read
+ // from the same table and can send deletion requests for same key multiple
+ // times.
+ private final static int KEY_DELETING_CORE_POOL_SIZE = 1;
+ private final OzoneManager ozoneManager;
private final KeyManager keyManager;
- private final ScmBlockLocationProtocol scmClient;
+ private final ClientId clientId = ClientId.randomId();
+ private final TimeDuration expireThreshold;
+ private final int cleanupLimitPerTask;
+ private final AtomicLong submittedOpenKeyCount;
+ private final AtomicLong runCount;
- public OpenKeyCleanupService(ScmBlockLocationProtocol scmClient,
- KeyManager keyManager, int serviceInterval,
- long serviceTimeout) {
- super("OpenKeyCleanupService", serviceInterval, TimeUnit.SECONDS,
- OPEN_KEY_DELETING_CORE_POOL_SIZE, serviceTimeout);
+ OpenKeyCleanupService(OzoneManager ozoneManager, KeyManager keyManager,
+ long serviceInterval, ConfigurationSource conf) {
+
+ super("OpenKeyCleanupService", serviceInterval, TimeUnit.MILLISECONDS,
+ KEY_DELETING_CORE_POOL_SIZE);
+ this.ozoneManager = ozoneManager;
this.keyManager = keyManager;
- this.scmClient = scmClient;
+
+ long expireDuration = conf.getTimeDuration(
+ OMConfigKeys.OZONE_OPEN_KEY_EXPIRE_THRESHOLD,
+ OMConfigKeys.OZONE_OPEN_KEY_EXPIRE_THRESHOLD_DEFAULT.getDuration(),
+ TimeUnit.MILLISECONDS);
+
+ this.expireThreshold =
+ TimeDuration.valueOf(expireDuration, TimeUnit.MILLISECONDS);
+
+ this.cleanupLimitPerTask = conf.getInt(
+ OMConfigKeys.OZONE_OPEN_KEY_CLEANUP_LIMIT_PER_TASK,
+ OMConfigKeys.OZONE_OPEN_KEY_CLEANUP_LIMIT_PER_TASK_DEFAULT);
+
+ this.submittedOpenKeyCount = new AtomicLong(0);
+ this.runCount = new AtomicLong(0);
+ }
+
+ /**
+ * Returns the number of times this Background service has run.
+ *
+ * @return Long, run count.
+ */
+ @VisibleForTesting
+ public AtomicLong getRunCount() {
+ return runCount;
+ }
+
+ /**
+ * Returns the number of open keys that were submitted for deletion by this
+ * service. If these keys were committed from the open key table between
+ * being submitted for deletion and the actual delete operation, they will
+ * not be deleted.
+ *
+ * @return Long count.
+ */
+ @VisibleForTesting
+ public AtomicLong getSubmittedOpenKeyCount() {
+ return submittedOpenKeyCount;
}
@Override
public BackgroundTaskQueue getTasks() {
BackgroundTaskQueue queue = new BackgroundTaskQueue();
- queue.add(new OpenKeyDeletingTask());
+ queue.add(new OpenKeyCleanupTask());
return queue;
}
- private class OpenKeyDeletingTask implements BackgroundTask {
+ private boolean shouldRun() {
+ if (ozoneManager == null) {
+ // OzoneManager can be null for testing
+ return true;
+ }
+ return ozoneManager.isLeader();
Review comment:
Need to use isLeaderReady to avoid retry of the request and getting the
keys from the delete table.
##########
File path:
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OpenKeyCleanupService.java
##########
@@ -1,88 +1,251 @@
/**
- * 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
+ * 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.
+ * 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.om;
-import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.google.protobuf.ServiceException;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OpenKey;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OpenKeyBucket;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteOpenKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.hadoop.util.Time;
import org.apache.hadoop.hdds.utils.BackgroundService;
import org.apache.hadoop.hdds.utils.BackgroundTask;
import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult.EmptyTaskResult;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.util.Preconditions;
+import org.apache.ratis.util.TimeDuration;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import java.util.concurrent.TimeUnit;
-
/**
- * This is the background service to delete hanging open keys.
- * Scan the metadata of om periodically to get
- * the keys with prefix "#open#" and ask scm to
- * delete metadata accordingly, if scm returns
- * success for keys, then clean up those keys.
+ * Background service to move keys whose creation time is past a given
+ * threshold from the open key table to the deleted table, where they will
+ * later be purged by the {@link KeyDeletingService}.
*/
public class OpenKeyCleanupService extends BackgroundService {
-
private static final Logger LOG =
- LoggerFactory.getLogger(OpenKeyCleanupService.class);
+ LoggerFactory.getLogger(KeyDeletingService.class);
Review comment:
Minor: KeyDeletingService -> OpenKeyCleanupService
##########
File path:
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMDbCheckpointServlet.java
##########
@@ -88,7 +89,7 @@ public void init() throws Exception {
scmId = UUID.randomUUID().toString();
omId = UUID.randomUUID().toString();
conf.setBoolean(OZONE_ACL_ENABLED, true);
- conf.setInt(OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS, 2);
+ conf.setTimeDuration(OZONE_OPEN_KEY_EXPIRE_THRESHOLD, 2, TimeUnit.SECONDS);
Review comment:
Do we need to set this in this test?
##########
File path:
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OpenKeyCleanupService.java
##########
@@ -1,88 +1,251 @@
/**
- * 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
+ * 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.
+ * 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.om;
-import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.google.protobuf.ServiceException;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OpenKey;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OpenKeyBucket;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteOpenKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.hadoop.util.Time;
import org.apache.hadoop.hdds.utils.BackgroundService;
import org.apache.hadoop.hdds.utils.BackgroundTask;
import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult.EmptyTaskResult;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.util.Preconditions;
+import org.apache.ratis.util.TimeDuration;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import java.util.concurrent.TimeUnit;
-
/**
- * This is the background service to delete hanging open keys.
- * Scan the metadata of om periodically to get
- * the keys with prefix "#open#" and ask scm to
- * delete metadata accordingly, if scm returns
- * success for keys, then clean up those keys.
+ * Background service to move keys whose creation time is past a given
+ * threshold from the open key table to the deleted table, where they will
+ * later be purged by the {@link KeyDeletingService}.
*/
public class OpenKeyCleanupService extends BackgroundService {
-
private static final Logger LOG =
- LoggerFactory.getLogger(OpenKeyCleanupService.class);
+ LoggerFactory.getLogger(KeyDeletingService.class);
- private final static int OPEN_KEY_DELETING_CORE_POOL_SIZE = 2;
+ // Use only a single thread for open key deletion. Multiple threads would
read
+ // from the same table and can send deletion requests for same key multiple
+ // times.
+ private final static int KEY_DELETING_CORE_POOL_SIZE = 1;
+ private final OzoneManager ozoneManager;
private final KeyManager keyManager;
- private final ScmBlockLocationProtocol scmClient;
+ private final ClientId clientId = ClientId.randomId();
+ private final TimeDuration expireThreshold;
+ private final int cleanupLimitPerTask;
+ private final AtomicLong submittedOpenKeyCount;
+ private final AtomicLong runCount;
- public OpenKeyCleanupService(ScmBlockLocationProtocol scmClient,
- KeyManager keyManager, int serviceInterval,
- long serviceTimeout) {
- super("OpenKeyCleanupService", serviceInterval, TimeUnit.SECONDS,
- OPEN_KEY_DELETING_CORE_POOL_SIZE, serviceTimeout);
+ OpenKeyCleanupService(OzoneManager ozoneManager, KeyManager keyManager,
+ long serviceInterval, ConfigurationSource conf) {
+
+ super("OpenKeyCleanupService", serviceInterval, TimeUnit.MILLISECONDS,
+ KEY_DELETING_CORE_POOL_SIZE);
+ this.ozoneManager = ozoneManager;
this.keyManager = keyManager;
- this.scmClient = scmClient;
+
+ long expireDuration = conf.getTimeDuration(
+ OMConfigKeys.OZONE_OPEN_KEY_EXPIRE_THRESHOLD,
+ OMConfigKeys.OZONE_OPEN_KEY_EXPIRE_THRESHOLD_DEFAULT.getDuration(),
+ TimeUnit.MILLISECONDS);
+
+ this.expireThreshold =
+ TimeDuration.valueOf(expireDuration, TimeUnit.MILLISECONDS);
+
+ this.cleanupLimitPerTask = conf.getInt(
+ OMConfigKeys.OZONE_OPEN_KEY_CLEANUP_LIMIT_PER_TASK,
+ OMConfigKeys.OZONE_OPEN_KEY_CLEANUP_LIMIT_PER_TASK_DEFAULT);
+
+ this.submittedOpenKeyCount = new AtomicLong(0);
+ this.runCount = new AtomicLong(0);
+ }
+
+ /**
+ * Returns the number of times this Background service has run.
+ *
+ * @return Long, run count.
+ */
+ @VisibleForTesting
+ public AtomicLong getRunCount() {
+ return runCount;
+ }
+
+ /**
+ * Returns the number of open keys that were submitted for deletion by this
+ * service. If these keys were committed from the open key table between
+ * being submitted for deletion and the actual delete operation, they will
+ * not be deleted.
+ *
+ * @return Long count.
+ */
+ @VisibleForTesting
+ public AtomicLong getSubmittedOpenKeyCount() {
+ return submittedOpenKeyCount;
}
@Override
public BackgroundTaskQueue getTasks() {
BackgroundTaskQueue queue = new BackgroundTaskQueue();
- queue.add(new OpenKeyDeletingTask());
+ queue.add(new OpenKeyCleanupTask());
return queue;
}
- private class OpenKeyDeletingTask implements BackgroundTask {
+ private boolean shouldRun() {
+ if (ozoneManager == null) {
+ // OzoneManager can be null for testing
+ return true;
+ }
+ return ozoneManager.isLeader();
+ }
+ private class OpenKeyCleanupTask implements BackgroundTask {
@Override
public int getPriority() {
return 0;
}
@Override
public BackgroundTaskResult call() throws Exception {
- // This method is currently never used. It will be implemented in
- // HDDS-4122, and integrated into the rest of the code base in HDDS-4123.
+ // Check if this is the Leader OM. If not leader, no need to execute this
+ // task.
+ if (shouldRun()) {
+ runCount.incrementAndGet();
+
+ try {
+ long startTime = Time.monotonicNow();
+ List<String> expiredOpenKeys = keyManager.getExpiredOpenKeys(
+ expireThreshold, cleanupLimitPerTask);
+
+ if (expiredOpenKeys != null && !expiredOpenKeys.isEmpty()) {
+ submitOpenKeysDeleteRequest(expiredOpenKeys);
+ LOG.debug("Number of expired keys submitted for deletion: {}, " +
+ "elapsed time: {}ms",
+ expiredOpenKeys.size(), Time.monotonicNow() - startTime);
+ submittedOpenKeyCount.addAndGet(expiredOpenKeys.size());
+ }
+ } catch (IOException e) {
+ LOG.error("Error while running delete keys background task. Will " +
+ "retry at next run.", e);
+ }
+ }
+ // By design, no one cares about the results of this call back.
+ return EmptyTaskResult.newResult();
+ }
+
+ /**
+ * Submits a Ratis request to move the keys in {@code expiredOpenKeys}
+ * out of the open key table and into the delete table.
+ */
+ public void submitOpenKeysDeleteRequest(List<String> expiredOpenKeys) {
+ Map<Pair<String, String>, List<OpenKey>> openKeysPerBucket =
+ new HashMap<>();
+
+ for (String keyName: expiredOpenKeys) {
+ // Separate volume, bucket, key name, and client ID, and add to the
+ // bucket grouping map.
+ addToMap(openKeysPerBucket, keyName);
+ LOG.debug("Open Key {} has been marked as expired and is being " +
+ "submitted for deletion", keyName);
+ }
+
+ DeleteOpenKeysRequest.Builder requestBuilder =
+ DeleteOpenKeysRequest.newBuilder();
+
+ // Add keys to open key delete request by bucket.
+ for (Map.Entry<Pair<String, String>, List<OpenKey>> entry:
+ openKeysPerBucket.entrySet()) {
+
+ Pair<String, String> volumeBucketPair = entry.getKey();
+ OpenKeyBucket openKeyBucket = OpenKeyBucket.newBuilder()
+ .setVolumeName(volumeBucketPair.getLeft())
+ .setBucketName(volumeBucketPair.getRight())
+ .addAllKeys(entry.getValue())
+ .build();
+ requestBuilder.addOpenKeysPerBucket(openKeyBucket);
+ }
+
+ OMRequest omRequest = OMRequest.newBuilder()
+ .setCmdType(Type.DeleteOpenKeys)
+ .setDeleteOpenKeysRequest(requestBuilder)
+ .setClientId(clientId.toString())
+ .build();
+
try {
- // The new API for deleting expired open keys in OM HA will differ
- // significantly from the old implementation.
- // The old implementation has been removed so the code compiles.
- keyManager.getExpiredOpenKeys(0);
- } catch (IOException e) {
- LOG.error("Unable to get hanging open keys, retry in"
- + " next interval", e);
+ ozoneManager.getOmServerProtocol().submitRequest(null, omRequest);
Review comment:
Need to use new API added by HDDS-4481
```
/**
* API used internally from OzoneManager Server when requests needs to be
* submitted to ratis, where the crafted RaftClientRequest is passed along.
* @param omRequest
* @param raftClientRequest
* @return OMResponse
* @throws ServiceException
*/
public OMResponse submitRequest(OMRequest omRequest,
RaftClientRequest raftClientRequest) throws ServiceException {
RaftClientReply raftClientReply =
submitRequestToRatis(raftClientRequest);
return processReply(omRequest, raftClientReply);
}
```
##########
File path:
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OpenKeyCleanupService.java
##########
@@ -1,88 +1,251 @@
/**
- * 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
+ * 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.
+ * 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.om;
-import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.google.protobuf.ServiceException;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OpenKey;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OpenKeyBucket;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteOpenKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.hadoop.util.Time;
import org.apache.hadoop.hdds.utils.BackgroundService;
import org.apache.hadoop.hdds.utils.BackgroundTask;
import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult.EmptyTaskResult;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.util.Preconditions;
+import org.apache.ratis.util.TimeDuration;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import java.util.concurrent.TimeUnit;
-
/**
- * This is the background service to delete hanging open keys.
- * Scan the metadata of om periodically to get
- * the keys with prefix "#open#" and ask scm to
- * delete metadata accordingly, if scm returns
- * success for keys, then clean up those keys.
+ * Background service to move keys whose creation time is past a given
+ * threshold from the open key table to the deleted table, where they will
+ * later be purged by the {@link KeyDeletingService}.
*/
public class OpenKeyCleanupService extends BackgroundService {
-
private static final Logger LOG =
- LoggerFactory.getLogger(OpenKeyCleanupService.class);
+ LoggerFactory.getLogger(KeyDeletingService.class);
- private final static int OPEN_KEY_DELETING_CORE_POOL_SIZE = 2;
+ // Use only a single thread for open key deletion. Multiple threads would
read
+ // from the same table and can send deletion requests for same key multiple
+ // times.
+ private final static int KEY_DELETING_CORE_POOL_SIZE = 1;
+ private final OzoneManager ozoneManager;
private final KeyManager keyManager;
- private final ScmBlockLocationProtocol scmClient;
+ private final ClientId clientId = ClientId.randomId();
+ private final TimeDuration expireThreshold;
+ private final int cleanupLimitPerTask;
+ private final AtomicLong submittedOpenKeyCount;
+ private final AtomicLong runCount;
- public OpenKeyCleanupService(ScmBlockLocationProtocol scmClient,
- KeyManager keyManager, int serviceInterval,
- long serviceTimeout) {
- super("OpenKeyCleanupService", serviceInterval, TimeUnit.SECONDS,
- OPEN_KEY_DELETING_CORE_POOL_SIZE, serviceTimeout);
+ OpenKeyCleanupService(OzoneManager ozoneManager, KeyManager keyManager,
+ long serviceInterval, ConfigurationSource conf) {
+
+ super("OpenKeyCleanupService", serviceInterval, TimeUnit.MILLISECONDS,
+ KEY_DELETING_CORE_POOL_SIZE);
+ this.ozoneManager = ozoneManager;
this.keyManager = keyManager;
- this.scmClient = scmClient;
+
+ long expireDuration = conf.getTimeDuration(
+ OMConfigKeys.OZONE_OPEN_KEY_EXPIRE_THRESHOLD,
+ OMConfigKeys.OZONE_OPEN_KEY_EXPIRE_THRESHOLD_DEFAULT.getDuration(),
+ TimeUnit.MILLISECONDS);
+
+ this.expireThreshold =
+ TimeDuration.valueOf(expireDuration, TimeUnit.MILLISECONDS);
+
+ this.cleanupLimitPerTask = conf.getInt(
+ OMConfigKeys.OZONE_OPEN_KEY_CLEANUP_LIMIT_PER_TASK,
+ OMConfigKeys.OZONE_OPEN_KEY_CLEANUP_LIMIT_PER_TASK_DEFAULT);
+
+ this.submittedOpenKeyCount = new AtomicLong(0);
+ this.runCount = new AtomicLong(0);
+ }
+
+ /**
+ * Returns the number of times this Background service has run.
+ *
+ * @return Long, run count.
+ */
+ @VisibleForTesting
+ public AtomicLong getRunCount() {
+ return runCount;
+ }
+
+ /**
+ * Returns the number of open keys that were submitted for deletion by this
+ * service. If these keys were committed from the open key table between
+ * being submitted for deletion and the actual delete operation, they will
+ * not be deleted.
+ *
+ * @return Long count.
+ */
+ @VisibleForTesting
+ public AtomicLong getSubmittedOpenKeyCount() {
+ return submittedOpenKeyCount;
}
@Override
public BackgroundTaskQueue getTasks() {
BackgroundTaskQueue queue = new BackgroundTaskQueue();
- queue.add(new OpenKeyDeletingTask());
+ queue.add(new OpenKeyCleanupTask());
return queue;
}
- private class OpenKeyDeletingTask implements BackgroundTask {
+ private boolean shouldRun() {
+ if (ozoneManager == null) {
+ // OzoneManager can be null for testing
+ return true;
+ }
+ return ozoneManager.isLeader();
Review comment:
isLeaderReady API is added as part of HDDS-4484
##########
File path:
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OpenKeyCleanupService.java
##########
@@ -1,88 +1,251 @@
/**
- * 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
+ * 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.
+ * 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.om;
-import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.google.protobuf.ServiceException;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OpenKey;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OpenKeyBucket;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteOpenKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.hadoop.util.Time;
import org.apache.hadoop.hdds.utils.BackgroundService;
import org.apache.hadoop.hdds.utils.BackgroundTask;
import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult.EmptyTaskResult;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.util.Preconditions;
+import org.apache.ratis.util.TimeDuration;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import java.util.concurrent.TimeUnit;
-
/**
- * This is the background service to delete hanging open keys.
- * Scan the metadata of om periodically to get
- * the keys with prefix "#open#" and ask scm to
- * delete metadata accordingly, if scm returns
- * success for keys, then clean up those keys.
+ * Background service to move keys whose creation time is past a given
+ * threshold from the open key table to the deleted table, where they will
+ * later be purged by the {@link KeyDeletingService}.
*/
public class OpenKeyCleanupService extends BackgroundService {
-
private static final Logger LOG =
- LoggerFactory.getLogger(OpenKeyCleanupService.class);
+ LoggerFactory.getLogger(KeyDeletingService.class);
- private final static int OPEN_KEY_DELETING_CORE_POOL_SIZE = 2;
+ // Use only a single thread for open key deletion. Multiple threads would
read
+ // from the same table and can send deletion requests for same key multiple
+ // times.
+ private final static int KEY_DELETING_CORE_POOL_SIZE = 1;
Review comment:
KEY_DELETING_CORE_POOL_SIZE -> OPEN_KEY_DELETING_CORE_POOL_SIZE
##########
File path:
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OpenKeyCleanupService.java
##########
@@ -1,88 +1,251 @@
/**
- * 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
+ * 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.
+ * 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.om;
-import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.google.protobuf.ServiceException;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OpenKey;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OpenKeyBucket;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteOpenKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.hadoop.util.Time;
import org.apache.hadoop.hdds.utils.BackgroundService;
import org.apache.hadoop.hdds.utils.BackgroundTask;
import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult.EmptyTaskResult;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.util.Preconditions;
+import org.apache.ratis.util.TimeDuration;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import java.util.concurrent.TimeUnit;
-
/**
- * This is the background service to delete hanging open keys.
- * Scan the metadata of om periodically to get
- * the keys with prefix "#open#" and ask scm to
- * delete metadata accordingly, if scm returns
- * success for keys, then clean up those keys.
+ * Background service to move keys whose creation time is past a given
+ * threshold from the open key table to the deleted table, where they will
+ * later be purged by the {@link KeyDeletingService}.
*/
public class OpenKeyCleanupService extends BackgroundService {
-
private static final Logger LOG =
- LoggerFactory.getLogger(OpenKeyCleanupService.class);
+ LoggerFactory.getLogger(KeyDeletingService.class);
- private final static int OPEN_KEY_DELETING_CORE_POOL_SIZE = 2;
+ // Use only a single thread for open key deletion. Multiple threads would
read
+ // from the same table and can send deletion requests for same key multiple
+ // times.
+ private final static int KEY_DELETING_CORE_POOL_SIZE = 1;
+ private final OzoneManager ozoneManager;
private final KeyManager keyManager;
- private final ScmBlockLocationProtocol scmClient;
+ private final ClientId clientId = ClientId.randomId();
+ private final TimeDuration expireThreshold;
+ private final int cleanupLimitPerTask;
+ private final AtomicLong submittedOpenKeyCount;
+ private final AtomicLong runCount;
- public OpenKeyCleanupService(ScmBlockLocationProtocol scmClient,
- KeyManager keyManager, int serviceInterval,
- long serviceTimeout) {
- super("OpenKeyCleanupService", serviceInterval, TimeUnit.SECONDS,
- OPEN_KEY_DELETING_CORE_POOL_SIZE, serviceTimeout);
+ OpenKeyCleanupService(OzoneManager ozoneManager, KeyManager keyManager,
+ long serviceInterval, ConfigurationSource conf) {
+
+ super("OpenKeyCleanupService", serviceInterval, TimeUnit.MILLISECONDS,
+ KEY_DELETING_CORE_POOL_SIZE);
+ this.ozoneManager = ozoneManager;
this.keyManager = keyManager;
- this.scmClient = scmClient;
+
+ long expireDuration = conf.getTimeDuration(
+ OMConfigKeys.OZONE_OPEN_KEY_EXPIRE_THRESHOLD,
+ OMConfigKeys.OZONE_OPEN_KEY_EXPIRE_THRESHOLD_DEFAULT.getDuration(),
+ TimeUnit.MILLISECONDS);
+
+ this.expireThreshold =
+ TimeDuration.valueOf(expireDuration, TimeUnit.MILLISECONDS);
+
+ this.cleanupLimitPerTask = conf.getInt(
+ OMConfigKeys.OZONE_OPEN_KEY_CLEANUP_LIMIT_PER_TASK,
+ OMConfigKeys.OZONE_OPEN_KEY_CLEANUP_LIMIT_PER_TASK_DEFAULT);
+
+ this.submittedOpenKeyCount = new AtomicLong(0);
+ this.runCount = new AtomicLong(0);
+ }
+
+ /**
+ * Returns the number of times this Background service has run.
+ *
+ * @return Long, run count.
+ */
+ @VisibleForTesting
+ public AtomicLong getRunCount() {
+ return runCount;
+ }
+
+ /**
+ * Returns the number of open keys that were submitted for deletion by this
+ * service. If these keys were committed from the open key table between
+ * being submitted for deletion and the actual delete operation, they will
+ * not be deleted.
+ *
+ * @return Long count.
+ */
+ @VisibleForTesting
+ public AtomicLong getSubmittedOpenKeyCount() {
+ return submittedOpenKeyCount;
}
@Override
public BackgroundTaskQueue getTasks() {
BackgroundTaskQueue queue = new BackgroundTaskQueue();
- queue.add(new OpenKeyDeletingTask());
+ queue.add(new OpenKeyCleanupTask());
return queue;
}
- private class OpenKeyDeletingTask implements BackgroundTask {
+ private boolean shouldRun() {
+ if (ozoneManager == null) {
+ // OzoneManager can be null for testing
+ return true;
+ }
+ return ozoneManager.isLeader();
+ }
+ private class OpenKeyCleanupTask implements BackgroundTask {
@Override
public int getPriority() {
return 0;
}
@Override
public BackgroundTaskResult call() throws Exception {
- // This method is currently never used. It will be implemented in
- // HDDS-4122, and integrated into the rest of the code base in HDDS-4123.
+ // Check if this is the Leader OM. If not leader, no need to execute this
+ // task.
+ if (shouldRun()) {
+ runCount.incrementAndGet();
+
+ try {
+ long startTime = Time.monotonicNow();
+ List<String> expiredOpenKeys = keyManager.getExpiredOpenKeys(
+ expireThreshold, cleanupLimitPerTask);
+
+ if (expiredOpenKeys != null && !expiredOpenKeys.isEmpty()) {
+ submitOpenKeysDeleteRequest(expiredOpenKeys);
+ LOG.debug("Number of expired keys submitted for deletion: {}, " +
+ "elapsed time: {}ms",
+ expiredOpenKeys.size(), Time.monotonicNow() - startTime);
+ submittedOpenKeyCount.addAndGet(expiredOpenKeys.size());
+ }
+ } catch (IOException e) {
+ LOG.error("Error while running delete keys background task. Will " +
+ "retry at next run.", e);
+ }
+ }
+ // By design, no one cares about the results of this call back.
+ return EmptyTaskResult.newResult();
+ }
+
+ /**
+ * Submits a Ratis request to move the keys in {@code expiredOpenKeys}
+ * out of the open key table and into the delete table.
+ */
+ public void submitOpenKeysDeleteRequest(List<String> expiredOpenKeys) {
+ Map<Pair<String, String>, List<OpenKey>> openKeysPerBucket =
+ new HashMap<>();
+
+ for (String keyName: expiredOpenKeys) {
+ // Separate volume, bucket, key name, and client ID, and add to the
+ // bucket grouping map.
+ addToMap(openKeysPerBucket, keyName);
+ LOG.debug("Open Key {} has been marked as expired and is being " +
+ "submitted for deletion", keyName);
+ }
+
+ DeleteOpenKeysRequest.Builder requestBuilder =
+ DeleteOpenKeysRequest.newBuilder();
+
+ // Add keys to open key delete request by bucket.
+ for (Map.Entry<Pair<String, String>, List<OpenKey>> entry:
+ openKeysPerBucket.entrySet()) {
+
+ Pair<String, String> volumeBucketPair = entry.getKey();
+ OpenKeyBucket openKeyBucket = OpenKeyBucket.newBuilder()
+ .setVolumeName(volumeBucketPair.getLeft())
+ .setBucketName(volumeBucketPair.getRight())
+ .addAllKeys(entry.getValue())
+ .build();
+ requestBuilder.addOpenKeysPerBucket(openKeyBucket);
+ }
+
+ OMRequest omRequest = OMRequest.newBuilder()
+ .setCmdType(Type.DeleteOpenKeys)
+ .setDeleteOpenKeysRequest(requestBuilder)
+ .setClientId(clientId.toString())
+ .build();
+
try {
- // The new API for deleting expired open keys in OM HA will differ
- // significantly from the old implementation.
- // The old implementation has been removed so the code compiles.
- keyManager.getExpiredOpenKeys(0);
- } catch (IOException e) {
- LOG.error("Unable to get hanging open keys, retry in"
- + " next interval", e);
+ ozoneManager.getOmServerProtocol().submitRequest(null, omRequest);
+ } catch (ServiceException ex) {
+ LOG.error("Open key delete request failed. Will retry at next run.",
+ ex);
}
- return BackgroundTaskResult.EmptyTaskResult.newResult();
+ }
+ }
+
+ /**
+ * Separates {@code openKeyName} into its volume, bucket, key, and client ID.
+ * Creates an {@link OpenKey} object from {@code openKeyName}'s key and
+ * client ID, and maps {@code openKeyName}'s volume and bucket to this
+ * {@link OpenKey}.
+ */
+ private void addToMap(Map<Pair<String, String>, List<OpenKey>>
+ openKeysPerBucket, String openKeyName) {
+ // Parse volume and bucket name
+ String[] split = openKeyName.split(OM_KEY_PREFIX);
+
+ // First element of the split is an empty string since the key begins
+ // with the separator.
+ Preconditions.assertTrue(split.length == 5, "Unable to separate volume, " +
+ "bucket, key, and client ID from open key {}.", openKeyName);
+
+ Pair<String, String> volumeBucketPair = Pair.of(split[1], split[2]);
+ if (!openKeysPerBucket.containsKey(volumeBucketPair)) {
+ openKeysPerBucket.put(volumeBucketPair, new ArrayList<>());
+ }
+
+ try {
+ OpenKey openKey = OpenKey.newBuilder()
+ .setName(split[3])
+ .setClientID(Long.parseLong(split[4]))
Review comment:
clientID will be split.length -1 right?
Example:
/vol/buck/dir1/dir2/dir3/file1/10000
split[4] in this case will be dir3, which is not clientID.
----------------------------------------------------------------
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:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]