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