-
Notifications
You must be signed in to change notification settings - Fork 2.4k
/
CleanActionExecutor.java
272 lines (243 loc) · 12.8 KB
/
CleanActionExecutor.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
/*
* 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.hudi.table.action.clean;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.model.HoodieActionInstant;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
import org.apache.hudi.avro.model.HoodieCleanerPlan;
import org.apache.hudi.client.transaction.TransactionManager;
import org.apache.hudi.common.HoodieCleanStat;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.CleanFileInfo;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
import org.apache.hudi.common.util.CleanerUtils;
import org.apache.hudi.common.util.HoodieTimer;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.common.util.collection.ImmutablePair;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.internal.schema.io.FileBasedInternalSchemaStorageManager;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.action.BaseActionExecutor;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import java.util.stream.Stream;
public class CleanActionExecutor<T extends HoodieRecordPayload, I, K, O> extends BaseActionExecutor<T, I, K, O, HoodieCleanMetadata> {
private static final long serialVersionUID = 1L;
private static final Logger LOG = LogManager.getLogger(CleanActionExecutor.class);
private final TransactionManager txnManager;
private final boolean skipLocking;
public CleanActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable<T, I, K, O> table, String instantTime) {
this(context, config, table, instantTime, false);
}
public CleanActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable<T, I, K, O> table, String instantTime, boolean skipLocking) {
super(context, config, table, instantTime);
this.txnManager = new TransactionManager(config, table.getMetaClient().getFs());
this.skipLocking = skipLocking;
}
private static Boolean deleteFileAndGetResult(FileSystem fs, String deletePathStr) throws IOException {
Path deletePath = new Path(deletePathStr);
LOG.debug("Working on delete path :" + deletePath);
try {
boolean isDirectory = fs.isDirectory(deletePath);
boolean deleteResult = fs.delete(deletePath, isDirectory);
if (deleteResult) {
LOG.debug("Cleaned file at path :" + deletePath);
}
return deleteResult;
} catch (FileNotFoundException fio) {
// With cleanPlan being used for retried cleaning operations, its possible to clean a file twice
return false;
}
}
private static Stream<Pair<String, PartitionCleanStat>> deleteFilesFunc(Iterator<Pair<String, CleanFileInfo>> cleanFileInfo, HoodieTable table) {
Map<String, PartitionCleanStat> partitionCleanStatMap = new HashMap<>();
FileSystem fs = table.getMetaClient().getFs();
cleanFileInfo.forEachRemaining(partitionDelFileTuple -> {
String partitionPath = partitionDelFileTuple.getLeft();
Path deletePath = new Path(partitionDelFileTuple.getRight().getFilePath());
String deletePathStr = deletePath.toString();
Boolean deletedFileResult = null;
try {
deletedFileResult = deleteFileAndGetResult(fs, deletePathStr);
} catch (IOException e) {
LOG.error("Delete file failed: " + deletePathStr);
}
final PartitionCleanStat partitionCleanStat =
partitionCleanStatMap.computeIfAbsent(partitionPath, k -> new PartitionCleanStat(partitionPath));
boolean isBootstrapBasePathFile = partitionDelFileTuple.getRight().isBootstrapBaseFile();
if (isBootstrapBasePathFile) {
// For Bootstrap Base file deletions, store the full file path.
partitionCleanStat.addDeleteFilePatterns(deletePath.toString(), true);
partitionCleanStat.addDeletedFileResult(deletePath.toString(), deletedFileResult, true);
} else {
partitionCleanStat.addDeleteFilePatterns(deletePath.getName(), false);
partitionCleanStat.addDeletedFileResult(deletePath.getName(), deletedFileResult, false);
}
});
return partitionCleanStatMap.entrySet().stream().map(e -> Pair.of(e.getKey(), e.getValue()));
}
/**
* Performs cleaning of partition paths according to cleaning policy and returns the number of files cleaned. Handles
* skews in partitions to clean by making files to clean as the unit of task distribution.
*
* @throws IllegalArgumentException if unknown cleaning policy is provided
*/
List<HoodieCleanStat> clean(HoodieEngineContext context, HoodieCleanerPlan cleanerPlan) {
int cleanerParallelism = Math.min(
(int) (cleanerPlan.getFilePathsToBeDeletedPerPartition().values().stream().mapToInt(List::size).count()),
config.getCleanerParallelism());
LOG.info("Using cleanerParallelism: " + cleanerParallelism);
context.setJobStatus(this.getClass().getSimpleName(), "Perform cleaning of partitions: " + config.getTableName());
Stream<Pair<String, CleanFileInfo>> filesToBeDeletedPerPartition =
cleanerPlan.getFilePathsToBeDeletedPerPartition().entrySet().stream()
.flatMap(x -> x.getValue().stream().map(y -> new ImmutablePair<>(x.getKey(),
new CleanFileInfo(y.getFilePath(), y.getIsBootstrapBaseFile()))));
Stream<ImmutablePair<String, PartitionCleanStat>> partitionCleanStats =
context.mapPartitionsToPairAndReduceByKey(filesToBeDeletedPerPartition,
iterator -> deleteFilesFunc(iterator, table), PartitionCleanStat::merge, cleanerParallelism);
Map<String, PartitionCleanStat> partitionCleanStatsMap = partitionCleanStats
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
List<String> partitionsToBeDeleted = cleanerPlan.getPartitionsToBeDeleted() != null ? cleanerPlan.getPartitionsToBeDeleted() : new ArrayList<>();
partitionsToBeDeleted.forEach(entry -> {
try {
deleteFileAndGetResult(table.getMetaClient().getFs(), table.getMetaClient().getBasePath() + "/" + entry);
} catch (IOException e) {
LOG.warn("Partition deletion failed " + entry);
}
});
// Return PartitionCleanStat for each partition passed.
return cleanerPlan.getFilePathsToBeDeletedPerPartition().keySet().stream().map(partitionPath -> {
PartitionCleanStat partitionCleanStat = partitionCleanStatsMap.containsKey(partitionPath)
? partitionCleanStatsMap.get(partitionPath)
: new PartitionCleanStat(partitionPath);
HoodieActionInstant actionInstant = cleanerPlan.getEarliestInstantToRetain();
return HoodieCleanStat.newBuilder().withPolicy(config.getCleanerPolicy()).withPartitionPath(partitionPath)
.withEarliestCommitRetained(Option.ofNullable(
actionInstant != null
? new HoodieInstant(HoodieInstant.State.valueOf(actionInstant.getState()),
actionInstant.getAction(), actionInstant.getTimestamp())
: null))
.withDeletePathPattern(partitionCleanStat.deletePathPatterns())
.withSuccessfulDeletes(partitionCleanStat.successDeleteFiles())
.withFailedDeletes(partitionCleanStat.failedDeleteFiles())
.withDeleteBootstrapBasePathPatterns(partitionCleanStat.getDeleteBootstrapBasePathPatterns())
.withSuccessfulDeleteBootstrapBaseFiles(partitionCleanStat.getSuccessfulDeleteBootstrapBaseFiles())
.withFailedDeleteBootstrapBaseFiles(partitionCleanStat.getFailedDeleteBootstrapBaseFiles())
.isPartitionDeleted(partitionsToBeDeleted.contains(partitionPath))
.build();
}).collect(Collectors.toList());
}
/**
* Executes the Cleaner plan stored in the instant metadata.
*/
HoodieCleanMetadata runPendingClean(HoodieTable<T, I, K, O> table, HoodieInstant cleanInstant) {
try {
HoodieCleanerPlan cleanerPlan = CleanerUtils.getCleanerPlan(table.getMetaClient(), cleanInstant);
return runClean(table, cleanInstant, cleanerPlan);
} catch (IOException e) {
throw new HoodieIOException(e.getMessage(), e);
}
}
private HoodieCleanMetadata runClean(HoodieTable<T, I, K, O> table, HoodieInstant cleanInstant, HoodieCleanerPlan cleanerPlan) {
ValidationUtils.checkArgument(cleanInstant.getState().equals(HoodieInstant.State.REQUESTED)
|| cleanInstant.getState().equals(HoodieInstant.State.INFLIGHT));
try {
final HoodieInstant inflightInstant;
final HoodieTimer timer = new HoodieTimer();
timer.startTimer();
if (cleanInstant.isRequested()) {
inflightInstant = table.getActiveTimeline().transitionCleanRequestedToInflight(cleanInstant,
TimelineMetadataUtils.serializeCleanerPlan(cleanerPlan));
} else {
inflightInstant = cleanInstant;
}
List<HoodieCleanStat> cleanStats = clean(context, cleanerPlan);
if (cleanStats.isEmpty()) {
return HoodieCleanMetadata.newBuilder().build();
}
table.getMetaClient().reloadActiveTimeline();
HoodieCleanMetadata metadata = CleanerUtils.convertCleanMetadata(
inflightInstant.getTimestamp(),
Option.of(timer.endTimer()),
cleanStats
);
if (!skipLocking) {
this.txnManager.beginTransaction(Option.empty(), Option.empty());
}
writeTableMetadata(metadata, inflightInstant.getTimestamp());
table.getActiveTimeline().transitionCleanInflightToComplete(inflightInstant,
TimelineMetadataUtils.serializeCleanMetadata(metadata));
LOG.info("Marked clean started on " + inflightInstant.getTimestamp() + " as complete");
return metadata;
} catch (IOException e) {
throw new HoodieIOException("Failed to clean up after commit", e);
} finally {
if (!skipLocking) {
this.txnManager.endTransaction(Option.empty());
}
}
}
@Override
public HoodieCleanMetadata execute() {
List<HoodieCleanMetadata> cleanMetadataList = new ArrayList<>();
// If there are inflight(failed) or previously requested clean operation, first perform them
List<HoodieInstant> pendingCleanInstants = table.getCleanTimeline()
.filterInflightsAndRequested().getInstants().collect(Collectors.toList());
if (pendingCleanInstants.size() > 0) {
// try to clean old history schema.
try {
FileBasedInternalSchemaStorageManager fss = new FileBasedInternalSchemaStorageManager(table.getMetaClient());
fss.cleanOldFiles(pendingCleanInstants.stream().map(is -> is.getTimestamp()).collect(Collectors.toList()));
} catch (Exception e) {
// we should not affect original clean logic. Swallow exception and log warn.
LOG.warn("failed to clean old history schema");
}
pendingCleanInstants.forEach(hoodieInstant -> {
if (table.getCleanTimeline().isEmpty(hoodieInstant)) {
table.getActiveTimeline().deleteEmptyInstantIfExists(hoodieInstant);
} else {
LOG.info("Finishing previously unfinished cleaner instant=" + hoodieInstant);
try {
cleanMetadataList.add(runPendingClean(table, hoodieInstant));
} catch (Exception e) {
LOG.warn("Failed to perform previous clean operation, instant: " + hoodieInstant, e);
}
}
table.getMetaClient().reloadActiveTimeline();
});
}
// return the last clean metadata for now
// TODO (NA) : Clean only the earliest pending clean just like how we do for other table services
// This requires the CleanActionExecutor to be refactored as BaseCommitActionExecutor
return cleanMetadataList.size() > 0 ? cleanMetadataList.get(cleanMetadataList.size() - 1) : null;
}
}