|
@@ -16,7 +16,6 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.ozone.om;
|
|
|
|
|
|
-import com.google.common.annotations.VisibleForTesting;
|
|
|
import com.google.common.base.Preconditions;
|
|
|
import org.apache.hadoop.ozone.OzoneConsts;
|
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
@@ -34,7 +33,6 @@ import org.apache.hadoop.ozone.protocol.proto
|
|
|
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
|
|
|
import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
|
|
|
import org.apache.hadoop.util.Time;
|
|
|
-import org.apache.hadoop.utils.BackgroundService;
|
|
|
import org.apache.hadoop.utils.BatchOperation;
|
|
|
import org.iq80.leveldb.DBException;
|
|
|
import org.slf4j.Logger;
|
|
@@ -91,15 +89,14 @@ public class KeyManagerImpl implements KeyManager {
|
|
|
private final OMMetadataManager metadataManager;
|
|
|
private final long scmBlockSize;
|
|
|
private final boolean useRatis;
|
|
|
- private final BackgroundService keyDeletingService;
|
|
|
- private final BackgroundService openKeyCleanupService;
|
|
|
|
|
|
private final long preallocateMax;
|
|
|
private final Random random;
|
|
|
private final String omId;
|
|
|
|
|
|
public KeyManagerImpl(ScmBlockLocationProtocol scmBlockClient,
|
|
|
- OMMetadataManager metadataManager, OzoneConfiguration conf,
|
|
|
+ OMMetadataManager metadataManager,
|
|
|
+ OzoneConfiguration conf,
|
|
|
String omId) {
|
|
|
this.scmBlockClient = scmBlockClient;
|
|
|
this.metadataManager = metadataManager;
|
|
@@ -107,43 +104,20 @@ public class KeyManagerImpl implements KeyManager {
|
|
|
OZONE_SCM_BLOCK_SIZE_DEFAULT) * OzoneConsts.MB;
|
|
|
this.useRatis = conf.getBoolean(DFS_CONTAINER_RATIS_ENABLED_KEY,
|
|
|
DFS_CONTAINER_RATIS_ENABLED_DEFAULT);
|
|
|
- long blockDeleteInterval = conf.getTimeDuration(
|
|
|
- OZONE_BLOCK_DELETING_SERVICE_INTERVAL,
|
|
|
- OZONE_BLOCK_DELETING_SERVICE_INTERVAL_DEFAULT,
|
|
|
- TimeUnit.MILLISECONDS);
|
|
|
- long serviceTimeout = conf.getTimeDuration(
|
|
|
- OZONE_BLOCK_DELETING_SERVICE_TIMEOUT,
|
|
|
- OZONE_BLOCK_DELETING_SERVICE_TIMEOUT_DEFAULT,
|
|
|
- TimeUnit.MILLISECONDS);
|
|
|
this.preallocateMax = conf.getLong(
|
|
|
OZONE_KEY_PREALLOCATION_MAXSIZE,
|
|
|
OZONE_KEY_PREALLOCATION_MAXSIZE_DEFAULT);
|
|
|
- keyDeletingService = new KeyDeletingService(
|
|
|
- scmBlockClient, this, blockDeleteInterval, serviceTimeout, conf);
|
|
|
- int openkeyCheckInterval = conf.getInt(
|
|
|
- OZONE_OPEN_KEY_CLEANUP_SERVICE_INTERVAL_SECONDS,
|
|
|
- OZONE_OPEN_KEY_CLEANUP_SERVICE_INTERVAL_SECONDS_DEFAULT);
|
|
|
- openKeyCleanupService = new OpenKeyCleanupService(
|
|
|
- scmBlockClient, this, openkeyCheckInterval, serviceTimeout);
|
|
|
random = new Random();
|
|
|
this.omId = omId;
|
|
|
}
|
|
|
|
|
|
- @VisibleForTesting
|
|
|
- public BackgroundService getOpenKeyCleanupService() {
|
|
|
- return openKeyCleanupService;
|
|
|
- }
|
|
|
|
|
|
@Override
|
|
|
public void start() {
|
|
|
- keyDeletingService.start();
|
|
|
- openKeyCleanupService.start();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public void stop() throws IOException {
|
|
|
- keyDeletingService.shutdown();
|
|
|
- openKeyCleanupService.shutdown();
|
|
|
}
|
|
|
|
|
|
private void validateBucket(String volumeName, String bucketName)
|
|
@@ -398,7 +372,7 @@ public class KeyManagerImpl implements KeyManager {
|
|
|
String bucketName = args.getBucketName();
|
|
|
String fromKeyName = args.getKeyName();
|
|
|
if (toKeyName.length() == 0 || fromKeyName.length() == 0) {
|
|
|
- LOG.error("Rename key failed for volume:{} bucket:{} fromKey:{} toKey:{}.",
|
|
|
+ LOG.error("Rename key failed for volume:{} bucket:{} fromKey:{} toKey:{}",
|
|
|
volumeName, bucketName, fromKeyName, toKeyName);
|
|
|
throw new OMException("Key name is empty",
|
|
|
ResultCodes.FAILED_INVALID_KEY_NAME);
|
|
@@ -446,7 +420,7 @@ public class KeyManagerImpl implements KeyManager {
|
|
|
batch.put(toKey, newKeyInfo.getProtobuf().toByteArray());
|
|
|
metadataManager.writeBatch(batch);
|
|
|
} catch (DBException ex) {
|
|
|
- LOG.error("Rename key failed for volume:{} bucket:{} fromKey:{} toKey:{}.",
|
|
|
+ LOG.error("Rename key failed for volume:{} bucket:{} fromKey:{} toKey:{}",
|
|
|
volumeName, bucketName, fromKeyName, toKeyName, ex);
|
|
|
throw new OMException(ex.getMessage(),
|
|
|
ResultCodes.FAILED_KEY_RENAME);
|
|
@@ -487,7 +461,8 @@ public class KeyManagerImpl implements KeyManager {
|
|
|
|
|
|
@Override
|
|
|
public List<OmKeyInfo> listKeys(String volumeName, String bucketName,
|
|
|
- String startKey, String keyPrefix, int maxKeys) throws IOException {
|
|
|
+ String startKey, String keyPrefix,
|
|
|
+ int maxKeys) throws IOException {
|
|
|
Preconditions.checkNotNull(volumeName);
|
|
|
Preconditions.checkNotNull(bucketName);
|
|
|
|