1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.regionserver.wal;
20
21 import java.io.DataInput;
22 import java.io.DataOutput;
23 import java.io.IOException;
24 import java.util.ArrayList;
25 import java.util.NavigableMap;
26 import java.util.TreeMap;
27
28 import org.apache.commons.logging.Log;
29 import org.apache.commons.logging.LogFactory;
30 import org.apache.hadoop.hbase.classification.InterfaceAudience;
31 import org.apache.hadoop.hbase.Cell;
32 import org.apache.hadoop.hbase.CellUtil;
33 import org.apache.hadoop.hbase.HRegionInfo;
34 import org.apache.hadoop.hbase.KeyValue;
35 import org.apache.hadoop.hbase.KeyValueUtil;
36 import org.apache.hadoop.hbase.codec.Codec;
37 import org.apache.hadoop.hbase.io.HeapSize;
38 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
39 import org.apache.hadoop.hbase.util.Bytes;
40 import org.apache.hadoop.hbase.util.ClassSize;
41 import org.apache.hadoop.io.Writable;
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 @InterfaceAudience.Private
79 public class WALEdit implements Writable, HeapSize {
80 public static final Log LOG = LogFactory.getLog(WALEdit.class);
81
82
83 public static final byte [] METAFAMILY = Bytes.toBytes("METAFAMILY");
84 static final byte [] METAROW = Bytes.toBytes("METAROW");
85 static final byte[] COMPLETE_CACHE_FLUSH = Bytes.toBytes("HBASE::CACHEFLUSH");
86 static final byte[] COMPACTION = Bytes.toBytes("HBASE::COMPACTION");
87 private final int VERSION_2 = -1;
88 private final boolean isReplay;
89
90 private final ArrayList<KeyValue> kvs = new ArrayList<KeyValue>(1);
91
92
93 @Deprecated
94 private NavigableMap<byte[], Integer> scopes;
95
96 private CompressionContext compressionContext;
97
98 public WALEdit() {
99 this(false);
100 }
101
102 public WALEdit(boolean isReplay) {
103 this.isReplay = isReplay;
104 }
105
106
107
108
109
110 public static boolean isMetaEditFamily(final byte [] f) {
111 return Bytes.equals(METAFAMILY, f);
112 }
113
114
115
116
117
118 public boolean isReplay() {
119 return this.isReplay;
120 }
121
122 public void setCompressionContext(final CompressionContext compressionContext) {
123 this.compressionContext = compressionContext;
124 }
125
126
127
128
129
130
131
132 @Deprecated
133 public WALEdit add(KeyValue kv) {
134 this.kvs.add(kv);
135 return this;
136 }
137
138
139
140
141
142
143 public WALEdit add(Cell cell) {
144 return add(KeyValueUtil.ensureKeyValue(cell));
145 }
146
147 public boolean isEmpty() {
148 return kvs.isEmpty();
149 }
150
151 public int size() {
152 return kvs.size();
153 }
154
155
156
157
158
159 @Deprecated
160 public ArrayList<KeyValue> getKeyValues() {
161 return kvs;
162 }
163
164
165
166
167 public ArrayList<Cell> getCells() {
168 ArrayList<Cell> cells = new ArrayList<Cell>(kvs.size());
169 cells.addAll(kvs);
170 return cells;
171 }
172
173 public NavigableMap<byte[], Integer> getAndRemoveScopes() {
174 NavigableMap<byte[], Integer> result = scopes;
175 scopes = null;
176 return result;
177 }
178
179 @Override
180 public void readFields(DataInput in) throws IOException {
181 kvs.clear();
182 if (scopes != null) {
183 scopes.clear();
184 }
185 int versionOrLength = in.readInt();
186
187 if (versionOrLength == VERSION_2) {
188
189 int numEdits = in.readInt();
190 for (int idx = 0; idx < numEdits; idx++) {
191 if (compressionContext != null) {
192 this.add(KeyValueCompression.readKV(in, compressionContext));
193 } else {
194 this.add(KeyValue.create(in));
195 }
196 }
197 int numFamilies = in.readInt();
198 if (numFamilies > 0) {
199 if (scopes == null) {
200 scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
201 }
202 for (int i = 0; i < numFamilies; i++) {
203 byte[] fam = Bytes.readByteArray(in);
204 int scope = in.readInt();
205 scopes.put(fam, scope);
206 }
207 }
208 } else {
209
210
211 this.add(KeyValue.create(versionOrLength, in));
212 }
213 }
214
215 @Override
216 public void write(DataOutput out) throws IOException {
217 LOG.warn("WALEdit is being serialized to writable - only expected in test code");
218 out.writeInt(VERSION_2);
219 out.writeInt(kvs.size());
220
221 for (KeyValue kv : kvs) {
222 if (compressionContext != null) {
223 KeyValueCompression.writeKV(out, kv, compressionContext);
224 } else{
225 KeyValue.write(kv, out);
226 }
227 }
228 if (scopes == null) {
229 out.writeInt(0);
230 } else {
231 out.writeInt(scopes.size());
232 for (byte[] key : scopes.keySet()) {
233 Bytes.writeByteArray(out, key);
234 out.writeInt(scopes.get(key));
235 }
236 }
237 }
238
239
240
241
242
243
244
245 public int readFromCells(Codec.Decoder cellDecoder, int expectedCount) throws IOException {
246 kvs.clear();
247 kvs.ensureCapacity(expectedCount);
248 while (kvs.size() < expectedCount && cellDecoder.advance()) {
249 Cell cell = cellDecoder.current();
250 if (!(cell instanceof KeyValue)) {
251 throw new IOException("WAL edit only supports KVs as cells");
252 }
253 kvs.add((KeyValue)cell);
254 }
255 return kvs.size();
256 }
257
258 @Override
259 public long heapSize() {
260 long ret = ClassSize.ARRAYLIST;
261 for (KeyValue kv : kvs) {
262 ret += kv.heapSize();
263 }
264 if (scopes != null) {
265 ret += ClassSize.TREEMAP;
266 ret += ClassSize.align(scopes.size() * ClassSize.MAP_ENTRY);
267
268 }
269 return ret;
270 }
271
272 @Override
273 public String toString() {
274 StringBuilder sb = new StringBuilder();
275
276 sb.append("[#edits: " + kvs.size() + " = <");
277 for (KeyValue kv : kvs) {
278 sb.append(kv.toString());
279 sb.append("; ");
280 }
281 if (scopes != null) {
282 sb.append(" scopes: " + scopes.toString());
283 }
284 sb.append(">]");
285 return sb.toString();
286 }
287
288
289
290
291
292
293 public static WALEdit createCompaction(final HRegionInfo hri, final CompactionDescriptor c) {
294 byte [] pbbytes = c.toByteArray();
295 KeyValue kv = new KeyValue(getRowForRegion(hri), METAFAMILY, COMPACTION, System.currentTimeMillis(), pbbytes);
296 return new WALEdit().add(kv);
297 }
298
299 private static byte[] getRowForRegion(HRegionInfo hri) {
300 byte[] startKey = hri.getStartKey();
301 if (startKey.length == 0) {
302
303
304 return new byte[] {0};
305 }
306 return startKey;
307 }
308
309
310
311
312
313
314 public static CompactionDescriptor getCompaction(Cell kv) throws IOException {
315 if (CellUtil.matchingColumn(kv, METAFAMILY, COMPACTION)) {
316 return CompactionDescriptor.parseFrom(kv.getValue());
317 }
318 return null;
319 }
320 }
321