1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.regionserver.compactions;
21
22 import java.io.IOException;
23 import java.util.List;
24 import java.util.concurrent.RejectedExecutionHandler;
25 import java.util.concurrent.ThreadPoolExecutor;
26
27 import org.apache.commons.logging.Log;
28 import org.apache.commons.logging.LogFactory;
29 import org.apache.hadoop.hbase.RemoteExceptionHandler;
30 import org.apache.hadoop.hbase.regionserver.HRegion;
31 import org.apache.hadoop.hbase.regionserver.HRegionServer;
32 import org.apache.hadoop.hbase.regionserver.Store;
33 import org.apache.hadoop.hbase.regionserver.StoreFile;
34 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
35 import org.apache.hadoop.util.StringUtils;
36
37 import com.google.common.base.Function;
38 import com.google.common.base.Joiner;
39 import com.google.common.base.Preconditions;
40 import com.google.common.base.Predicate;
41 import com.google.common.collect.Collections2;
42
43
44
45
46 public class CompactionRequest implements Comparable<CompactionRequest>,
47 Runnable {
48 static final Log LOG = LogFactory.getLog(CompactionRequest.class);
49 private final HRegion r;
50 private final Store s;
51 private final List<StoreFile> files;
52 private final long totalSize;
53 private final boolean isMajor;
54 private int p;
55 private final Long timeInNanos;
56 private HRegionServer server = null;
57
58 public CompactionRequest(HRegion r, Store s,
59 List<StoreFile> files, boolean isMajor, int p) {
60 Preconditions.checkNotNull(r);
61 Preconditions.checkNotNull(files);
62
63 this.r = r;
64 this.s = s;
65 this.files = files;
66 long sz = 0;
67 for (StoreFile sf : files) {
68 sz += sf.getReader().length();
69 }
70 this.totalSize = sz;
71 this.isMajor = isMajor;
72 this.p = p;
73 this.timeInNanos = System.nanoTime();
74 }
75
76
77
78
79
80
81
82
83
84
85
86
87 @Override
88 public int compareTo(CompactionRequest request) {
89
90 if (this.equals(request)) {
91 return 0;
92 }
93 int compareVal;
94
95 compareVal = p - request.p;
96 if (compareVal != 0) {
97 return compareVal;
98 }
99
100 compareVal = timeInNanos.compareTo(request.timeInNanos);
101 if (compareVal != 0) {
102 return compareVal;
103 }
104
105
106 return this.hashCode() - request.hashCode();
107 }
108
109
110 public HRegion getHRegion() {
111 return r;
112 }
113
114
115 public Store getStore() {
116 return s;
117 }
118
119
120 public List<StoreFile> getFiles() {
121 return files;
122 }
123
124
125 public long getSize() {
126 return totalSize;
127 }
128
129 public boolean isMajor() {
130 return this.isMajor;
131 }
132
133
134 public int getPriority() {
135 return p;
136 }
137
138
139 public void setPriority(int p) {
140 this.p = p;
141 }
142
143 public void setServer(HRegionServer hrs) {
144 this.server = hrs;
145 }
146
147 @Override
148 public String toString() {
149 String fsList = Joiner.on(", ").join(
150 Collections2.transform(Collections2.filter(files,
151 new Predicate<StoreFile>() {
152 public boolean apply(StoreFile sf) {
153 return sf.getReader() != null;
154 }
155 }), new Function<StoreFile, String>() {
156 public String apply(StoreFile sf) {
157 return StringUtils.humanReadableInt(sf.getReader().length());
158 }
159 }));
160
161 return "regionName=" + r.getRegionNameAsString() +
162 ", storeName=" + new String(s.getFamily().getName()) +
163 ", fileCount=" + files.size() +
164 ", fileSize=" + StringUtils.humanReadableInt(totalSize) +
165 ((fsList.isEmpty()) ? "" : " (" + fsList + ")") +
166 ", priority=" + p + ", time=" + timeInNanos;
167 }
168
169 @Override
170 public void run() {
171 Preconditions.checkNotNull(server);
172 if (server.isStopped()) {
173 return;
174 }
175 try {
176 long start = EnvironmentEdgeManager.currentTimeMillis();
177 boolean completed = r.compact(this);
178 long now = EnvironmentEdgeManager.currentTimeMillis();
179 LOG.info(((completed) ? "completed" : "aborted") + " compaction: " +
180 this + "; duration=" + StringUtils.formatTimeDiff(now, start));
181 if (completed) {
182 server.getMetrics().addCompaction(now - start, this.totalSize);
183
184 if (s.getCompactPriority() <= 0) {
185 server.compactSplitThread
186 .requestCompaction(r, s, "Recursive enqueue");
187 } else {
188
189 server.compactSplitThread.requestSplit(r);
190 }
191 }
192 } catch (IOException ex) {
193 LOG.error("Compaction failed " + this, RemoteExceptionHandler
194 .checkIOException(ex));
195 server.checkFileSystem();
196 } catch (Exception ex) {
197 LOG.error("Compaction failed " + this, ex);
198 server.checkFileSystem();
199 } finally {
200 s.finishRequest(this);
201 LOG.debug("CompactSplitThread status: " + server.compactSplitThread);
202 }
203 }
204
205
206
207
208 public static class Rejection implements RejectedExecutionHandler {
209
210 @Override
211 public void rejectedExecution(Runnable request, ThreadPoolExecutor pool) {
212 if (request instanceof CompactionRequest) {
213 CompactionRequest cr = (CompactionRequest) request;
214 LOG.debug("Compaction Rejected: " + cr);
215 cr.getStore().finishRequest(cr);
216 }
217 }
218 }
219 }