|
@@ -1,52 +1,54 @@
|
|
|
/**
|
|
|
- * 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 com.google.common.annotations.VisibleForTesting;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
-import org.apache.hadoop.ozone.common.DeleteBlockGroupResult;
|
|
|
-import org.apache.hadoop.ozone.common.BlockGroup;
|
|
|
import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
|
|
|
-import org.apache.hadoop.util.StringUtils;
|
|
|
+import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
+import org.apache.hadoop.ozone.common.BlockGroup;
|
|
|
+import org.apache.hadoop.ozone.common.DeleteBlockGroupResult;
|
|
|
import org.apache.hadoop.util.Time;
|
|
|
import org.apache.hadoop.utils.BackgroundService;
|
|
|
import org.apache.hadoop.utils.BackgroundTask;
|
|
|
import org.apache.hadoop.utils.BackgroundTaskQueue;
|
|
|
import org.apache.hadoop.utils.BackgroundTaskResult;
|
|
|
import org.apache.hadoop.utils.BackgroundTaskResult.EmptyTaskResult;
|
|
|
+import org.apache.hadoop.utils.db.Table;
|
|
|
+import org.rocksdb.RocksDBException;
|
|
|
+import org.rocksdb.WriteBatch;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.util.List;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
+import java.util.concurrent.atomic.AtomicLong;
|
|
|
|
|
|
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_KEY_DELETING_LIMIT_PER_TASK;
|
|
|
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_KEY_DELETING_LIMIT_PER_TASK_DEFAULT;
|
|
|
|
|
|
/**
|
|
|
- * This is the background service to delete keys.
|
|
|
- * Scan the metadata of om periodically to get
|
|
|
- * the keys with prefix "#deleting" and ask scm to
|
|
|
- * delete metadata accordingly, if scm returns
|
|
|
- * success for keys, then clean up those keys.
|
|
|
+ * This is the background service to delete keys. Scan the metadata of om
|
|
|
+ * periodically to get the keys from DeletedTable and ask scm to delete
|
|
|
+ * metadata accordingly, if scm returns success for keys, then clean up those
|
|
|
+ * keys.
|
|
|
*/
|
|
|
public class KeyDeletingService extends BackgroundService {
|
|
|
-
|
|
|
private static final Logger LOG =
|
|
|
LoggerFactory.getLogger(KeyDeletingService.class);
|
|
|
|
|
@@ -56,6 +58,8 @@ public class KeyDeletingService extends BackgroundService {
|
|
|
private final ScmBlockLocationProtocol scmClient;
|
|
|
private final KeyManager manager;
|
|
|
private final int keyLimitPerTask;
|
|
|
+ private final AtomicLong deletedKeyCount;
|
|
|
+ private final AtomicLong runCount;
|
|
|
|
|
|
public KeyDeletingService(ScmBlockLocationProtocol scmClient,
|
|
|
KeyManager manager, long serviceInterval,
|
|
@@ -66,6 +70,28 @@ public class KeyDeletingService extends BackgroundService {
|
|
|
this.manager = manager;
|
|
|
this.keyLimitPerTask = conf.getInt(OZONE_KEY_DELETING_LIMIT_PER_TASK,
|
|
|
OZONE_KEY_DELETING_LIMIT_PER_TASK_DEFAULT);
|
|
|
+ this.deletedKeyCount = 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 keys deleted by the background service.
|
|
|
+ *
|
|
|
+ * @return Long count.
|
|
|
+ */
|
|
|
+ @VisibleForTesting
|
|
|
+ public AtomicLong getDeletedKeyCount() {
|
|
|
+ return deletedKeyCount;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -76,11 +102,11 @@ public class KeyDeletingService extends BackgroundService {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * A key deleting task scans OM DB and looking for a certain number
|
|
|
- * of pending-deletion keys, sends these keys along with their associated
|
|
|
- * blocks to SCM for deletion. Once SCM confirms keys are deleted (once
|
|
|
- * SCM persisted the blocks info in its deletedBlockLog), it removes
|
|
|
- * these keys from the DB.
|
|
|
+ * A key deleting task scans OM DB and looking for a certain number of
|
|
|
+ * pending-deletion keys, sends these keys along with their associated blocks
|
|
|
+ * to SCM for deletion. Once SCM confirms keys are deleted (once SCM persisted
|
|
|
+ * the blocks info in its deletedBlockLog), it removes these keys from the
|
|
|
+ * DB.
|
|
|
*/
|
|
|
private class KeyDeletingTask implements
|
|
|
BackgroundTask<BackgroundTaskResult> {
|
|
@@ -92,51 +118,55 @@ public class KeyDeletingService extends BackgroundService {
|
|
|
|
|
|
@Override
|
|
|
public BackgroundTaskResult call() throws Exception {
|
|
|
+ runCount.incrementAndGet();
|
|
|
try {
|
|
|
long startTime = Time.monotonicNow();
|
|
|
List<BlockGroup> keyBlocksList = manager
|
|
|
.getPendingDeletionKeys(keyLimitPerTask);
|
|
|
- if (keyBlocksList.size() > 0) {
|
|
|
- LOG.info("Found {} to-delete keys in OM", keyBlocksList.size());
|
|
|
+ if (keyBlocksList != null && keyBlocksList.size() > 0) {
|
|
|
List<DeleteBlockGroupResult> results =
|
|
|
scmClient.deleteKeyBlocks(keyBlocksList);
|
|
|
- for (DeleteBlockGroupResult result : results) {
|
|
|
- if (result.isSuccess()) {
|
|
|
- try {
|
|
|
- // Purge key from OM DB.
|
|
|
- manager.deletePendingDeletionKey(result.getObjectKey());
|
|
|
- LOG.debug("Key {} deleted from OM DB", result.getObjectKey());
|
|
|
- } catch (IOException e) {
|
|
|
- // if a pending deletion key is failed to delete,
|
|
|
- // print a warning here and retain it in this state,
|
|
|
- // so that it can be attempt to delete next time.
|
|
|
- LOG.warn("Failed to delete pending-deletion key {}",
|
|
|
- result.getObjectKey(), e);
|
|
|
- }
|
|
|
- } else {
|
|
|
- // Key deletion failed, retry in next interval.
|
|
|
- LOG.warn("Key {} deletion failed because some of the blocks"
|
|
|
- + " were failed to delete, failed blocks: {}",
|
|
|
- result.getObjectKey(),
|
|
|
- StringUtils.join(",", result.getFailedBlocks()));
|
|
|
- }
|
|
|
+ if (results != null) {
|
|
|
+ int delCount = deleteAllKeys(results);
|
|
|
+ LOG.debug("Number of keys deleted: {}, elapsed time: {}ms",
|
|
|
+ delCount, Time.monotonicNow() - startTime);
|
|
|
+ deletedKeyCount.addAndGet(delCount);
|
|
|
}
|
|
|
-
|
|
|
- if (!results.isEmpty()) {
|
|
|
- LOG.info("Number of key deleted from OM DB: {},"
|
|
|
- + " task elapsed time: {}ms",
|
|
|
- results.size(), Time.monotonicNow() - startTime);
|
|
|
- }
|
|
|
-
|
|
|
- return results::size;
|
|
|
- } else {
|
|
|
- LOG.debug("No pending deletion key found in OM");
|
|
|
}
|
|
|
} catch (IOException e) {
|
|
|
- LOG.error("Unable to get pending deletion keys, retry in"
|
|
|
- + " next interval", e);
|
|
|
+ LOG.error("Error while running delete keys background task. Will " +
|
|
|
+ "retry at next run.", e);
|
|
|
}
|
|
|
+ // By desing, no one cares about the results of this call back.
|
|
|
return EmptyTaskResult.newResult();
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Deletes all the keys that SCM has acknowledged and queued for delete.
|
|
|
+ *
|
|
|
+ * @param results DeleteBlockGroups returned by SCM.
|
|
|
+ * @throws RocksDBException on Error.
|
|
|
+ * @throws IOException on Error
|
|
|
+ */
|
|
|
+ private int deleteAllKeys(List<DeleteBlockGroupResult> results)
|
|
|
+ throws RocksDBException, IOException {
|
|
|
+ Table deletedTable = manager.getMetadataManager().getDeletedTable();
|
|
|
+ // Put all keys to delete in a single transaction and call for delete.
|
|
|
+ int deletedCount = 0;
|
|
|
+ try (WriteBatch writeBatch = new WriteBatch()) {
|
|
|
+ for (DeleteBlockGroupResult result : results) {
|
|
|
+ if (result.isSuccess()) {
|
|
|
+ // Purge key from OM DB.
|
|
|
+ writeBatch.delete(deletedTable.getHandle(),
|
|
|
+ DFSUtil.string2Bytes(result.getObjectKey()));
|
|
|
+ LOG.debug("Key {} deleted from OM DB", result.getObjectKey());
|
|
|
+ deletedCount++;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ // Write a single transaction for delete.
|
|
|
+ manager.getMetadataManager().getStore().write(writeBatch);
|
|
|
+ }
|
|
|
+ return deletedCount;
|
|
|
+ }
|
|
|
}
|
|
|
}
|