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