phet commented on code in PR #4047: URL: https://github.com/apache/gobblin/pull/4047#discussion_r1744703943
########## gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/DeleteWorkDirsActivityImpl.java: ########## @@ -0,0 +1,224 @@ +/* + * 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.gobblin.temporal.ddm.activity.impl; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; + +import com.google.common.collect.Maps; +import com.google.common.io.Closer; + +import io.temporal.failure.ApplicationFailure; +import lombok.extern.slf4j.Slf4j; + +import org.apache.gobblin.broker.gobblin_scopes.GobblinScopeTypes; +import org.apache.gobblin.broker.iface.SharedResourcesBroker; +import org.apache.gobblin.commit.DeliverySemantics; +import org.apache.gobblin.configuration.ConfigurationKeys; +import org.apache.gobblin.runtime.JobContext; +import org.apache.gobblin.runtime.JobException; +import org.apache.gobblin.runtime.JobState; +import org.apache.gobblin.runtime.TaskState; +import org.apache.gobblin.temporal.ddm.activity.DeleteWorkDirsActivity; +import org.apache.gobblin.temporal.ddm.util.JobStateUtils; +import org.apache.gobblin.temporal.ddm.work.CleanupResult; +import org.apache.gobblin.temporal.ddm.work.WUProcessingSpec; +import org.apache.gobblin.temporal.ddm.work.assistance.Help; +import org.apache.gobblin.temporal.workflows.metrics.EventSubmitterContext; +import org.apache.gobblin.util.ForkOperatorUtils; +import org.apache.gobblin.util.HadoopUtils; +import org.apache.gobblin.util.JobLauncherUtils; +import org.apache.gobblin.util.ParallelRunner; +import org.apache.gobblin.util.PropertiesUtils; +import org.apache.gobblin.util.WriterUtils; + + +@Slf4j +public class DeleteWorkDirsActivityImpl implements DeleteWorkDirsActivity { + static String UNDEFINED_JOB_NAME = "<job_name_stub>"; + + @Override + public CleanupResult cleanup(WUProcessingSpec workSpec, EventSubmitterContext eventSubmitterContext, Set<String> resourcesToClean) { + //TODO: Emit timers to measure length of cleanup step + Optional<String> optJobName = Optional.empty(); + try { + FileSystem fs = Help.loadFileSystem(workSpec); + JobState jobState = Help.loadJobState(workSpec, fs); + optJobName = Optional.ofNullable(jobState.getJobName()); + + SharedResourcesBroker<GobblinScopeTypes> instanceBroker = JobStateUtils.getSharedResourcesBroker(jobState); + JobContext jobContext = new JobContext(jobState.getProperties(), log, instanceBroker, null); + if (PropertiesUtils.getPropAsBoolean(jobState.getProperties(), ConfigurationKeys.CLEANUP_STAGING_DATA_BY_INITIALIZER, "false")) { + //Clean up will be done by initializer. + return CleanupResult.createEmpty(); + } + try { + if (!canCleanStagingData(jobContext, jobState)) { + log.error("Job " + jobState.getJobName() + " has unfinished commit sequences. Will not clean up staging data."); + return CleanupResult.createEmpty(); + } + } catch (IOException e) { + throw new JobException("Failed to check unfinished commit sequences", e); + } + Map<String, Long> cleanupMetrics = jobState.getPropAsBoolean(ConfigurationKeys.CLEANUP_STAGING_DATA_PER_TASK, ConfigurationKeys.DEFAULT_CLEANUP_STAGING_DATA_PER_TASK) ? + cleanupStagingDataPerTask(jobState, workSpec) : cleanupStagingDataForEntireJob(jobState, resourcesToClean, workSpec); + + return new CleanupResult(cleanupMetrics); + } catch (Exception e) { + throw ApplicationFailure.newNonRetryableFailureWithCause( + String.format("Failed to cleanup temporary folders for job %s", optJobName.orElse(UNDEFINED_JOB_NAME)), + IOException.class.toString(), + new IOException(e) + ); + } + } + + private boolean canCleanStagingData(JobContext jobContext, JobState jobState) + throws IOException { + return jobContext.getSemantics() != DeliverySemantics.EXACTLY_ONCE || !jobContext.getCommitSequenceStore() + .get().exists(jobState.getJobName()); + } + + // Goes through all the task states and cleans them up individually, there is some overlap with the job level cleanup as the job level cleanup will + // also clean up the task level staging data Review Comment: I didn't notice that overlap (at least the impl here) that said, I'm strongly in favor of this activity only deleting paths explicitly given to it. if we also want to consult the TaskStateStore to delete its contents, let's do so in a separate activity. small, single-purpose activities not only bolster observability, but also simplify retry/recovery logic -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
