|
23 | 23 | import java.util.Arrays;
|
24 | 24 | import java.util.Comparator;
|
25 | 25 | import java.util.HashMap;
|
26 |
| -import java.util.LinkedList; |
27 | 26 | import java.util.List;
|
28 | 27 | import java.util.Map;
|
29 | 28 | import java.util.concurrent.CompletableFuture;
|
@@ -81,6 +80,8 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
|
81 | 80 | private final AtomicBoolean enabled = new AtomicBoolean(true);
|
82 | 81 | protected List<T> cleanersChain;
|
83 | 82 | protected List<String> excludeDirs;
|
| 83 | + private CompletableFuture<Boolean> future; |
| 84 | + private boolean forceRun; |
84 | 85 |
|
85 | 86 | public CleanerChore(String name, final int sleepPeriod, final Stoppable s, Configuration conf,
|
86 | 87 | FileSystem fs, Path oldFileDir, String confKey, DirScanPool pool) {
|
@@ -168,10 +169,10 @@ static int calculatePoolSize(String poolSize) {
|
168 | 169 | * @param confKey key to get the file cleaner classes from the configuration
|
169 | 170 | */
|
170 | 171 | private void initCleanerChain(String confKey) {
|
171 |
| - this.cleanersChain = new LinkedList<>(); |
172 |
| - String[] logCleaners = conf.getStrings(confKey); |
173 |
| - if (logCleaners != null) { |
174 |
| - for (String className : logCleaners) { |
| 172 | + this.cleanersChain = new ArrayList<>(); |
| 173 | + String[] cleaners = conf.getStrings(confKey); |
| 174 | + if (cleaners != null) { |
| 175 | + for (String className : cleaners) { |
175 | 176 | className = className.trim();
|
176 | 177 | if (className.isEmpty()) {
|
177 | 178 | continue;
|
@@ -208,42 +209,87 @@ private T newFileCleaner(String className, Configuration conf) {
|
208 | 209 | }
|
209 | 210 | }
|
210 | 211 |
|
| 212 | + @Override |
| 213 | + protected boolean initialChore() { |
| 214 | + synchronized (this) { |
| 215 | + if (forceRun) { |
| 216 | + // wake up the threads waiting in triggerCleanerNow, as a triggerNow may triggers the first |
| 217 | + // loop where we will only call initialChore. We need to trigger another run immediately. |
| 218 | + forceRun = false; |
| 219 | + notifyAll(); |
| 220 | + } |
| 221 | + } |
| 222 | + return true; |
| 223 | + } |
| 224 | + |
211 | 225 | @Override
|
212 | 226 | protected void chore() {
|
213 |
| - if (getEnabled()) { |
214 |
| - try { |
215 |
| - pool.latchCountUp(); |
216 |
| - if (runCleaner()) { |
217 |
| - LOG.trace("Cleaned all WALs under {}", oldFileDir); |
| 227 | + CompletableFuture<Boolean> f; |
| 228 | + synchronized (this) { |
| 229 | + if (!enabled.get()) { |
| 230 | + if (!forceRun) { |
| 231 | + LOG.trace("Cleaner chore {} disabled! Not cleaning.", getName()); |
| 232 | + return; |
218 | 233 | } else {
|
219 |
| - LOG.trace("WALs outstanding under {}", oldFileDir); |
| 234 | + LOG.info("Force executing cleaner chore {} when disabled", getName()); |
220 | 235 | }
|
221 |
| - } finally { |
222 |
| - pool.latchCountDown(); |
223 | 236 | }
|
| 237 | + if (future != null) { |
| 238 | + LOG.warn("A cleaner chore {}'s run is in progress, give up running", getName()); |
| 239 | + return; |
| 240 | + } |
| 241 | + f = new CompletableFuture<>(); |
| 242 | + future = f; |
| 243 | + notifyAll(); |
| 244 | + } |
| 245 | + pool.latchCountUp(); |
| 246 | + try { |
| 247 | + preRunCleaner(); |
| 248 | + pool.execute(() -> traverseAndDelete(oldFileDir, true, f)); |
| 249 | + if (f.get()) { |
| 250 | + LOG.trace("Cleaned all files under {}", oldFileDir); |
| 251 | + } else { |
| 252 | + LOG.trace("Files outstanding under {}", oldFileDir); |
| 253 | + } |
| 254 | + } catch (Exception e) { |
| 255 | + LOG.info("Failed to traverse and delete the dir: {}", oldFileDir, e); |
| 256 | + } finally { |
| 257 | + postRunCleaner(); |
| 258 | + synchronized (this) { |
| 259 | + future = null; |
| 260 | + forceRun = false; |
| 261 | + } |
| 262 | + pool.latchCountDown(); |
224 | 263 | // After each cleaner chore, checks if received reconfigure notification while cleaning.
|
225 | 264 | // First in cleaner turns off notification, to avoid another cleaner updating pool again.
|
226 | 265 | // This cleaner is waiting for other cleaners finishing their jobs.
|
227 | 266 | // To avoid missing next chore, only wait 0.8 * period, then shutdown.
|
228 | 267 | pool.tryUpdatePoolSize((long) (0.8 * getTimeUnit().toMillis(getPeriod())));
|
229 |
| - } else { |
230 |
| - LOG.trace("Cleaner chore disabled! Not cleaning."); |
231 | 268 | }
|
232 | 269 | }
|
233 | 270 |
|
234 | 271 | private void preRunCleaner() {
|
235 | 272 | cleanersChain.forEach(FileCleanerDelegate::preClean);
|
236 | 273 | }
|
237 | 274 |
|
238 |
| - public boolean runCleaner() { |
239 |
| - preRunCleaner(); |
240 |
| - try { |
241 |
| - CompletableFuture<Boolean> future = new CompletableFuture<>(); |
242 |
| - pool.execute(() -> traverseAndDelete(oldFileDir, true, future)); |
243 |
| - return future.get(); |
244 |
| - } catch (Exception e) { |
245 |
| - LOG.info("Failed to traverse and delete the dir: {}", oldFileDir, e); |
246 |
| - return false; |
| 275 | + private void postRunCleaner() { |
| 276 | + cleanersChain.forEach(FileCleanerDelegate::postClean); |
| 277 | + } |
| 278 | + |
| 279 | + /** |
| 280 | + * Trigger the cleaner immediately and return a CompletableFuture for getting the result. Return |
| 281 | + * {@code true} means all the old files have been deleted, otherwise {@code false}. |
| 282 | + */ |
| 283 | + public synchronized CompletableFuture<Boolean> triggerCleanerNow() throws InterruptedException { |
| 284 | + for (;;) { |
| 285 | + if (future != null) { |
| 286 | + return future; |
| 287 | + } |
| 288 | + forceRun = true; |
| 289 | + if (!triggerNow()) { |
| 290 | + return CompletableFuture.completedFuture(false); |
| 291 | + } |
| 292 | + wait(); |
247 | 293 | }
|
248 | 294 | }
|
249 | 295 |
|
@@ -396,9 +442,6 @@ int getChorePoolSize() {
|
396 | 442 | return pool.getSize();
|
397 | 443 | }
|
398 | 444 |
|
399 |
| - /** |
400 |
| - * n |
401 |
| - */ |
402 | 445 | public boolean setEnabled(final boolean enabled) {
|
403 | 446 | return this.enabled.getAndSet(enabled);
|
404 | 447 | }
|
@@ -449,7 +492,7 @@ private void traverseAndDelete(Path dir, boolean root, CompletableFuture<Boolean
|
449 | 492 | CompletableFuture.allOf(futures.toArray(new CompletableFuture[futures.size()])),
|
450 | 493 | (voidObj, e) -> {
|
451 | 494 | if (e != null) {
|
452 |
| - result.completeExceptionally(e); |
| 495 | + result.completeExceptionally(FutureUtils.unwrapCompletionException(e)); |
453 | 496 | return;
|
454 | 497 | }
|
455 | 498 | try {
|
|
0 commit comments