ivandika3 commented on code in PR #8681: URL: https://github.com/apache/ozone/pull/8681#discussion_r2206451628
########## hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/KeyLifecycleService.java: ########## @@ -0,0 +1,634 @@ +/* + * 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.om.service; + +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_KEY_LIFECYCLE_SERVICE_DELETE_BATCH_SIZE; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_KEY_LIFECYCLE_SERVICE_DELETE_BATCH_SIZE_DEFAULT; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_KEY_LIFECYCLE_SERVICE_ENABLED; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_KEY_LIFECYCLE_SERVICE_ENABLED_DEFAULT; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.protobuf.ServiceException; +import java.io.IOException; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdds.conf.ConfigurationSource; +import org.apache.hadoop.hdds.conf.StorageUnit; +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 org.apache.hadoop.hdds.utils.FaultInjector; +import org.apache.hadoop.hdds.utils.db.Table; +import org.apache.hadoop.hdds.utils.db.TableIterator; +import org.apache.hadoop.ozone.ClientVersion; +import org.apache.hadoop.ozone.OzoneConsts; +import org.apache.hadoop.ozone.om.KeyManager; +import org.apache.hadoop.ozone.om.OMConfigKeys; +import org.apache.hadoop.ozone.om.OMMetadataManager; +import org.apache.hadoop.ozone.om.OzoneManager; +import org.apache.hadoop.ozone.om.OzoneTrash; +import org.apache.hadoop.ozone.om.helpers.BucketLayout; +import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; +import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.helpers.OmLCRule; +import org.apache.hadoop.ozone.om.helpers.OmLifecycleConfiguration; +import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs; +import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteKeyArgs; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteKeysRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest; +import org.apache.ratis.protocol.ClientId; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This is the background service to manage object lifecycle based on bucket lifecycle configuration. + */ +public class KeyLifecycleService extends BackgroundService { + public static final Logger LOG = + LoggerFactory.getLogger(KeyLifecycleService.class); + + private final OzoneManager ozoneManager; + //TODO: honor this parameter in next patch + private int keyLimitPerIterator; + private final AtomicBoolean suspended; + private KeyLifecycleServiceMetrics metrics; + private boolean isServiceEnabled; + // A set of bucket name that have LifecycleActionTask scheduled + private final ConcurrentHashMap<String, LifecycleActionTask> inFlight; + private OMMetadataManager omMetadataManager; + private int ratisByteLimit; + private ClientId clientId = ClientId.randomId(); + private AtomicLong callId = new AtomicLong(0); + private OzoneTrash ozoneTrash; + private static List<FaultInjector> injectors; + + public KeyLifecycleService(OzoneManager ozoneManager, + KeyManager manager, long serviceInterval, + long serviceTimeout, int poolSize, + ConfigurationSource conf) { + super(KeyLifecycleService.class.getSimpleName(), serviceInterval, TimeUnit.MILLISECONDS, + poolSize, serviceTimeout, ozoneManager.getThreadNamePrefix()); + this.ozoneManager = ozoneManager; + this.keyLimitPerIterator = conf.getInt(OZONE_KEY_LIFECYCLE_SERVICE_DELETE_BATCH_SIZE, + OZONE_KEY_LIFECYCLE_SERVICE_DELETE_BATCH_SIZE_DEFAULT); + Preconditions.checkArgument(keyLimitPerIterator >= 0, + OZONE_KEY_LIFECYCLE_SERVICE_DELETE_BATCH_SIZE + " cannot be negative."); + this.suspended = new AtomicBoolean(false); + this.metrics = KeyLifecycleServiceMetrics.create(); + this.isServiceEnabled = conf.getBoolean(OZONE_KEY_LIFECYCLE_SERVICE_ENABLED, + OZONE_KEY_LIFECYCLE_SERVICE_ENABLED_DEFAULT); + this.inFlight = new ConcurrentHashMap(); + this.omMetadataManager = ozoneManager.getMetadataManager(); + int limit = (int) conf.getStorageSize( + OMConfigKeys.OZONE_OM_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT, + OMConfigKeys.OZONE_OM_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT_DEFAULT, + StorageUnit.BYTES); + // always go to 90% of max limit for request as other header will be added + this.ratisByteLimit = (int) (limit * 0.9); + this.ozoneTrash = ozoneManager.getOzoneTrash(); + } + + @Override + public BackgroundTaskQueue getTasks() { + BackgroundTaskQueue queue = new BackgroundTaskQueue(); + if (!shouldRun()) { + return queue; + } + + List<OmLifecycleConfiguration> lifecycleConfigurationList = + omMetadataManager.listLifecycleConfigurations(); + for (OmLifecycleConfiguration lifecycleConfiguration : lifecycleConfigurationList) { + if (lifecycleConfiguration.getRules().stream().anyMatch(r -> r.isEnabled())) { + LifecycleActionTask task = new LifecycleActionTask(lifecycleConfiguration); + if (this.inFlight.putIfAbsent(lifecycleConfiguration.getFormattedKey(), task) == null) { + queue.add(task); + LOG.info("LifecycleActionTask of {} is scheduled", lifecycleConfiguration.getFormattedKey()); + } else { + metrics.incrNumSkippedTask(); + LOG.info("LifecycleActionTask of {} is already running", lifecycleConfiguration.getFormattedKey()); + } + } else { + LOG.info("LifecycleConfiguration of {} is not enabled", lifecycleConfiguration.getFormattedKey()); + } + } + LOG.info("{} LifecycleActionTasks scheduled", queue.size()); + return queue; + } + + private boolean shouldRun() { + if (getOzoneManager() == null) { + // OzoneManager can be null for testing + return true; + } + return isServiceEnabled && !suspended.get() && getOzoneManager().isLeaderReady(); + } + + public KeyLifecycleServiceMetrics getMetrics() { + return metrics; + } + + public OzoneManager getOzoneManager() { + return ozoneManager; + } + + /** + * Suspend the service. + */ + @VisibleForTesting + public void suspend() { + suspended.set(true); + } + + /** + * Resume the service if suspended. + */ + @VisibleForTesting + public void resume() { + suspended.set(false); + } + + @Override + public void shutdown() { + super.shutdown(); + KeyLifecycleServiceMetrics.unregister(); + } + + /** + * A lifecycle action task for one specific bucket, scanning OM DB and evaluating if any existing + * object/key qualified for expiration according to bucket's lifecycle configuration, and sending + * key delete command respectively. + */ + public final class LifecycleActionTask implements BackgroundTask { + private final OmLifecycleConfiguration policy; + private long taskStartTime; + private long numKeyIterated = 0; + private long numDirIterated = 0; + private long numDirDeleted = 0; + private long numKeyDeleted = 0; + private long sizeKeyDeleted = 0; + + public LifecycleActionTask(OmLifecycleConfiguration lcConfig) { + this.policy = lcConfig; + } + + @Override + public int getPriority() { + return 0; + } + + @Override + public BackgroundTaskResult call() { + EmptyTaskResult result = EmptyTaskResult.newResult(); + String bucketName = policy.getFormattedKey(); + // Check if this is the Leader OM. If not leader, no need to execute this task. + if (shouldRun()) { + LOG.info("Running LifecycleActionTask {}", bucketName); + taskStartTime = System.currentTimeMillis(); + OmBucketInfo bucket; + try { + if (getInjector(0) != null) { + getInjector(0).pause(); + } + bucket = omMetadataManager.getBucketTable().get(bucketName); + if (bucket == null) { + LOG.warn("Bucket {} cannot be found, might be deleted during this task's execution", bucketName); + onFailure(bucketName); + return result; + } + } catch (IOException e) { + LOG.warn("Failed to get Bucket {}", bucketName, e); + onFailure(bucketName); + return result; + } + + List<OmLCRule> originRuleList = policy.getRules(); + // remove disabled rules + List<OmLCRule> ruleList = originRuleList.stream().filter(r -> r.isEnabled()).collect(Collectors.toList()); + + boolean tagEnabled = ruleList.stream().anyMatch(r -> r.isTagEnable()); + // TODO, set a rule with tag on FSO bucket should fail at creation time. + if (bucket.getBucketLayout() == BucketLayout.FILE_SYSTEM_OPTIMIZED && tagEnabled) { + LOG.info("Fail the task as rule with tag is not supported on FSO bucket {}", bucketName); + onFailure(bucketName); + return result; + } Review Comment: Should have already been supported so we can allow object tags based rule in FSO bucket. -- 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-unsubscr...@ozone.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@ozone.apache.org For additional commands, e-mail: issues-h...@ozone.apache.org