1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.master.cleaner;
19
20 import static org.apache.hadoop.hbase.HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS;
21
22 import java.io.IOException;
23 import java.util.ArrayList;
24 import java.util.LinkedList;
25 import java.util.List;
26 import java.util.Map;
27 import java.util.concurrent.LinkedBlockingQueue;
28 import java.util.concurrent.TimeUnit;
29
30 import org.apache.commons.logging.Log;
31 import org.apache.commons.logging.LogFactory;
32 import org.apache.hadoop.fs.FileStatus;
33 import org.apache.hadoop.hbase.classification.InterfaceAudience;
34 import org.apache.hadoop.conf.Configuration;
35 import org.apache.hadoop.fs.FileSystem;
36 import org.apache.hadoop.fs.Path;
37 import org.apache.hadoop.hbase.Stoppable;
38 import org.apache.hadoop.hbase.conf.ConfigurationObserver;
39 import org.apache.hadoop.hbase.wal.DefaultWALProvider;
40
41
42
43
44
45
46 @InterfaceAudience.Private
47 public class LogCleaner extends CleanerChore<BaseLogCleanerDelegate>
48 implements ConfigurationObserver {
49 private static final Log LOG = LogFactory.getLog(LogCleaner.class.getName());
50
51 public static final String OLD_WALS_CLEANER_THREAD_SIZE = "hbase.oldwals.cleaner.thread.size";
52 public static final int DEFAULT_OLD_WALS_CLEANER_THREAD_SIZE = 2;
53
54 public static final String OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC =
55 "hbase.oldwals.cleaner.thread.timeout.msec";
56 static final long DEFAULT_OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC = 60 * 1000L;
57
58 public static final String OLD_WALS_CLEANER_THREAD_CHECK_INTERVAL_MSEC =
59 "hbase.oldwals.cleaner.thread.check.interval.msec";
60 static final long DEFAULT_OLD_WALS_CLEANER_THREAD_CHECK_INTERVAL_MSEC = 500L;
61
62 private final LinkedBlockingQueue<CleanerContext> pendingDelete;
63 private List<Thread> oldWALsCleaner;
64 private long cleanerThreadTimeoutMsec;
65 private long cleanerThreadCheckIntervalMsec;
66
67
68
69
70
71
72
73
74 public LogCleaner(final int p, final Stoppable s, Configuration conf, FileSystem fs,
75 Path oldLogDir, DirScanPool pool, Map<String, Object> params) {
76 super("LogsCleaner", p, s, conf, fs, oldLogDir, HBASE_MASTER_LOGCLEANER_PLUGINS, pool, params);
77 this.pendingDelete = new LinkedBlockingQueue<>();
78 int size = conf.getInt(OLD_WALS_CLEANER_THREAD_SIZE, DEFAULT_OLD_WALS_CLEANER_THREAD_SIZE);
79 this.oldWALsCleaner = createOldWalsCleaner(size);
80 this.cleanerThreadTimeoutMsec = conf.getLong(OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC,
81 DEFAULT_OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC);
82 this.cleanerThreadCheckIntervalMsec = conf.getLong(OLD_WALS_CLEANER_THREAD_CHECK_INTERVAL_MSEC,
83 DEFAULT_OLD_WALS_CLEANER_THREAD_CHECK_INTERVAL_MSEC);
84 }
85
86 @Override
87 protected boolean validate(Path file) {
88 return DefaultWALProvider.validateWALFilename(file.getName());
89 }
90
91 @Override
92 public void onConfigurationChange(Configuration conf) {
93 int newSize = conf.getInt(OLD_WALS_CLEANER_THREAD_SIZE, DEFAULT_OLD_WALS_CLEANER_THREAD_SIZE);
94 if (newSize == oldWALsCleaner.size()) {
95 if (LOG.isDebugEnabled()) {
96 LOG.debug("Size from configuration is the same as previous which is " +
97 newSize + ", no need to update.");
98 }
99 return;
100 }
101 interruptOldWALsCleaner();
102 oldWALsCleaner = createOldWalsCleaner(newSize);
103 cleanerThreadTimeoutMsec = conf.getLong(OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC,
104 DEFAULT_OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC);
105 cleanerThreadCheckIntervalMsec = conf.getLong(OLD_WALS_CLEANER_THREAD_CHECK_INTERVAL_MSEC,
106 DEFAULT_OLD_WALS_CLEANER_THREAD_CHECK_INTERVAL_MSEC);
107 }
108
109 @Override
110 protected int deleteFiles(Iterable<FileStatus> filesToDelete) {
111 List<CleanerContext> results = new LinkedList<>();
112 for (FileStatus toDelete : filesToDelete) {
113 CleanerContext context = CleanerContext.createCleanerContext(toDelete,
114 cleanerThreadTimeoutMsec);
115 if (context != null) {
116 pendingDelete.add(context);
117 results.add(context);
118 }
119 }
120
121 int deletedFiles = 0;
122 for (CleanerContext res : results) {
123 deletedFiles += res.getResult(cleanerThreadCheckIntervalMsec) ? 1 : 0;
124 }
125 return deletedFiles;
126 }
127
128 @Override
129 public synchronized void cleanup() {
130 super.cleanup();
131 interruptOldWALsCleaner();
132 }
133
134 int getSizeOfCleaners() {
135 return oldWALsCleaner.size();
136 }
137
138 long getCleanerThreadTimeoutMsec() {
139 return cleanerThreadTimeoutMsec;
140 }
141
142 long getCleanerThreadCheckIntervalMsec() {
143 return cleanerThreadCheckIntervalMsec;
144 }
145
146 private List<Thread> createOldWalsCleaner(int size) {
147 LOG.info("Creating OldWALs cleaners with size=" + size);
148
149 List<Thread> oldWALsCleaner = new ArrayList<>(size);
150 for (int i = 0; i < size; i++) {
151 Thread cleaner = new Thread(new Runnable() {
152 @Override
153 public void run() {
154 deleteFile();
155 }
156 });
157 cleaner.setName("OldWALsCleaner-" + i);
158 cleaner.setDaemon(true);
159 cleaner.start();
160 oldWALsCleaner.add(cleaner);
161 }
162 return oldWALsCleaner;
163 }
164
165 private void interruptOldWALsCleaner() {
166 for (Thread cleaner : oldWALsCleaner) {
167 cleaner.interrupt();
168 }
169 oldWALsCleaner.clear();
170 }
171
172 private void deleteFile() {
173 while (true) {
174 CleanerContext context = null;
175 boolean succeed = false;
176 boolean interrupted = false;
177 try {
178 context = pendingDelete.take();
179 if (context != null) {
180 FileStatus toClean = context.getTargetToClean();
181 succeed = this.fs.delete(toClean.getPath(), false);
182 }
183 } catch (InterruptedException ite) {
184
185 if (context != null) {
186 LOG.warn("Interrupted while cleaning oldWALs " +
187 context.getTargetToClean() + ", try to clean it next round.");
188 }
189 interrupted = true;
190 } catch (IOException e) {
191
192 LOG.warn("Failed to clean oldwals with exception: " + e);
193 succeed = false;
194 } finally {
195 if (context != null) {
196 context.setResult(succeed);
197 }
198 if (interrupted) {
199
200 Thread.currentThread().interrupt();
201 break;
202 }
203 }
204 }
205 if (LOG.isDebugEnabled()) {
206 LOG.debug("Exiting cleaner.");
207 }
208 }
209
210 @Override
211 public synchronized void cancel(boolean mayInterruptIfRunning) {
212 super.cancel(mayInterruptIfRunning);
213 for (Thread t : oldWALsCleaner) {
214 t.interrupt();
215 }
216 }
217
218 private static final class CleanerContext {
219
220 final FileStatus target;
221 volatile boolean result;
222 volatile boolean setFromCleaner = false;
223 long timeoutMsec;
224
225 static CleanerContext createCleanerContext(FileStatus status, long timeoutMsec) {
226 return status != null ? new CleanerContext(status, timeoutMsec) : null;
227 }
228
229 private CleanerContext(FileStatus status, long timeoutMsec) {
230 this.target = status;
231 this.result = false;
232 this.timeoutMsec = timeoutMsec;
233 }
234
235 synchronized void setResult(boolean res) {
236 this.result = res;
237 this.setFromCleaner = true;
238 notify();
239 }
240
241 synchronized boolean getResult(long waitIfNotFinished) {
242 long totalTimeMsec = 0;
243 try {
244 while (!setFromCleaner) {
245 long startTimeNanos = System.nanoTime();
246 wait(waitIfNotFinished);
247 totalTimeMsec += TimeUnit.MILLISECONDS.convert(System.nanoTime() - startTimeNanos,
248 TimeUnit.NANOSECONDS);
249 if (totalTimeMsec >= timeoutMsec) {
250 LOG.warn("Spend too much time " + totalTimeMsec + " ms to delete oldwals " + target);
251 return result;
252 }
253 }
254 } catch (InterruptedException e) {
255 LOG.warn("Interrupted while waiting deletion of " + target);
256 return result;
257 }
258 return result;
259 }
260
261 FileStatus getTargetToClean() {
262 return target;
263 }
264 }
265 }