View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
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;
40  import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
41  import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
42  import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
43  import org.apache.hadoop.hbase.util.Bytes;
44  import org.apache.hadoop.hbase.util.ClassSize;
45  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
46  import org.apache.hadoop.io.Writable;
47  
48  
49  /**
50   * WALEdit: Used in HBase's transaction log (WAL) to represent
51   * the collection of edits (KeyValue objects) corresponding to a
52   * single transaction. The class implements "Writable" interface
53   * for serializing/deserializing a set of KeyValue items.
54   *
55   * Previously, if a transaction contains 3 edits to c1, c2, c3 for a row R,
56   * the WAL would have three log entries as follows:
57   *
58   *    <logseq1-for-edit1>:<KeyValue-for-edit-c1>
59   *    <logseq2-for-edit2>:<KeyValue-for-edit-c2>
60   *    <logseq3-for-edit3>:<KeyValue-for-edit-c3>
61   *
62   * This presents problems because row level atomicity of transactions
63   * was not guaranteed. If we crash after few of the above appends make
64   * it, then recovery will restore a partial transaction.
65   *
66   * In the new world, all the edits for a given transaction are written
67   * out as a single record, for example:
68   *
69   *   <logseq#-for-entire-txn>:<WALEdit-for-entire-txn>
70   *
71   * where, the WALEdit is serialized as:
72   *   <-1, # of edits, <KeyValue>, <KeyValue>, ... >
73   * For example:
74   *   <-1, 3, <Keyvalue-for-edit-c1>, <KeyValue-for-edit-c2>, <KeyValue-for-edit-c3>>
75   *
76   * The -1 marker is just a special way of being backward compatible with
77   * an old WAL which would have contained a single <KeyValue>.
78   *
79   * The deserializer for WALEdit backward compatibly detects if the record
80   * is an old style KeyValue or the new style WALEdit.
81   *
82   */
83  @InterfaceAudience.LimitedPrivate({ HBaseInterfaceAudience.REPLICATION,
84      HBaseInterfaceAudience.COPROC })
85  public class WALEdit implements Writable, HeapSize {
86    public static final Log LOG = LogFactory.getLog(WALEdit.class);
87  
88    // TODO: Get rid of this; see HBASE-8457
89    public static final byte [] METAFAMILY = Bytes.toBytes("METAFAMILY");
90    static final byte [] METAROW = Bytes.toBytes("METAROW");
91    static final byte[] COMPACTION = Bytes.toBytes("HBASE::COMPACTION");
92    static final byte [] FLUSH = Bytes.toBytes("HBASE::FLUSH");
93    static final byte [] REGION_EVENT = Bytes.toBytes("HBASE::REGION_EVENT");
94    public static final byte [] BULK_LOAD = Bytes.toBytes("HBASE::BULK_LOAD");
95  
96    private final int VERSION_2 = -1;
97    private final boolean isReplay;
98  
99    private final ArrayList<Cell> cells = new ArrayList<Cell>(1);
100 
101   public static final WALEdit EMPTY_WALEDIT = new WALEdit();
102 
103   // Only here for legacy writable deserialization
104   @Deprecated
105   private NavigableMap<byte[], Integer> scopes;
106 
107   private CompressionContext compressionContext;
108 
109   public WALEdit() {
110     this(false);
111   }
112 
113   public WALEdit(boolean isReplay) {
114     this.isReplay = isReplay;
115   }
116 
117   /**
118    * @param f
119    * @return True is <code>f</code> is {@link #METAFAMILY}
120    */
121   public static boolean isMetaEditFamily(final byte [] f) {
122     return Bytes.equals(METAFAMILY, f);
123   }
124 
125   public static boolean isMetaEditFamily(Cell cell) {
126     return CellUtil.matchingFamily(cell, METAFAMILY);
127   }
128 
129   public boolean isMetaEdit() {
130     for (Cell cell: cells) {
131       if (!isMetaEditFamily(cell)) {
132         return false;
133       }
134     }
135     return true;
136   }
137 
138   /**
139    * @return True when current WALEdit is created by log replay. Replication skips WALEdits from
140    *         replay.
141    */
142   public boolean isReplay() {
143     return this.isReplay;
144   }
145 
146   public void setCompressionContext(final CompressionContext compressionContext) {
147     this.compressionContext = compressionContext;
148   }
149 
150   public WALEdit add(Cell cell) {
151     this.cells.add(cell);
152     return this;
153   }
154 
155   public boolean isEmpty() {
156     return cells.isEmpty();
157   }
158 
159   public int size() {
160     return cells.size();
161   }
162 
163   public ArrayList<Cell> getCells() {
164     return cells;
165   }
166 
167   public NavigableMap<byte[], Integer> getAndRemoveScopes() {
168     NavigableMap<byte[], Integer> result = scopes;
169     scopes = null;
170     return result;
171   }
172 
173   @Override
174   public void readFields(DataInput in) throws IOException {
175     cells.clear();
176     if (scopes != null) {
177       scopes.clear();
178     }
179     int versionOrLength = in.readInt();
180     // TODO: Change version when we protobuf.  Also, change way we serialize KV!  Pb it too.
181     if (versionOrLength == VERSION_2) {
182       // this is new style WAL entry containing multiple KeyValues.
183       int numEdits = in.readInt();
184       for (int idx = 0; idx < numEdits; idx++) {
185         if (compressionContext != null) {
186           this.add(KeyValueCompression.readKV(in, compressionContext));
187         } else {
188           this.add(KeyValue.create(in));
189         }
190       }
191       int numFamilies = in.readInt();
192       if (numFamilies > 0) {
193         if (scopes == null) {
194           scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
195         }
196         for (int i = 0; i < numFamilies; i++) {
197           byte[] fam = Bytes.readByteArray(in);
198           int scope = in.readInt();
199           scopes.put(fam, scope);
200         }
201       }
202     } else {
203       // this is an old style WAL entry. The int that we just
204       // read is actually the length of a single KeyValue
205       this.add(KeyValue.create(versionOrLength, in));
206     }
207   }
208 
209   @Override
210   public void write(DataOutput out) throws IOException {
211     LOG.warn("WALEdit is being serialized to writable - only expected in test code");
212     out.writeInt(VERSION_2);
213     out.writeInt(cells.size());
214     // We interleave the two lists for code simplicity
215     for (Cell cell : cells) {
216       // This is not used in any of the core code flows so it is just fine to convert to KV
217       KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
218       if (compressionContext != null) {
219         KeyValueCompression.writeKV(out, kv, compressionContext);
220       } else{
221         KeyValue.write(kv, out);
222       }
223     }
224     if (scopes == null) {
225       out.writeInt(0);
226     } else {
227       out.writeInt(scopes.size());
228       for (byte[] key : scopes.keySet()) {
229         Bytes.writeByteArray(out, key);
230         out.writeInt(scopes.get(key));
231       }
232     }
233   }
234 
235   /**
236    * Reads WALEdit from cells.
237    * @param cellDecoder Cell decoder.
238    * @param expectedCount Expected cell count.
239    * @return Number of KVs read.
240    */
241   public int readFromCells(Codec.Decoder cellDecoder, int expectedCount) throws IOException {
242     cells.clear();
243     cells.ensureCapacity(expectedCount);
244     while (cells.size() < expectedCount && cellDecoder.advance()) {
245       cells.add(cellDecoder.current());
246     }
247     return cells.size();
248   }
249 
250   @Override
251   public long heapSize() {
252     long ret = ClassSize.ARRAYLIST;
253     for (Cell cell : cells) {
254       ret += CellUtil.estimatedHeapSizeOf(cell);
255     }
256     if (scopes != null) {
257       ret += ClassSize.TREEMAP;
258       ret += ClassSize.align(scopes.size() * ClassSize.MAP_ENTRY);
259       // TODO this isn't quite right, need help here
260     }
261     return ret;
262   }
263 
264   @Override
265   public String toString() {
266     StringBuilder sb = new StringBuilder();
267 
268     sb.append("[#edits: " + cells.size() + " = <");
269     for (Cell cell : cells) {
270       sb.append(cell);
271       sb.append("; ");
272     }
273     if (scopes != null) {
274       sb.append(" scopes: " + scopes.toString());
275     }
276     sb.append(">]");
277     return sb.toString();
278   }
279 
280   public static WALEdit createFlushWALEdit(HRegionInfo hri, FlushDescriptor f) {
281     KeyValue kv = new KeyValue(getRowForRegion(hri), METAFAMILY, FLUSH,
282       EnvironmentEdgeManager.currentTime(), f.toByteArray());
283     return new WALEdit().add(kv);
284   }
285 
286   public static FlushDescriptor getFlushDescriptor(Cell cell) throws IOException {
287     if (CellUtil.matchingColumn(cell, METAFAMILY, FLUSH)) {
288       return FlushDescriptor.parseFrom(cell.getValue());
289     }
290     return null;
291   }
292 
293   public static WALEdit createRegionEventWALEdit(HRegionInfo hri,
294       RegionEventDescriptor regionEventDesc) {
295     KeyValue kv = new KeyValue(getRowForRegion(hri), METAFAMILY, REGION_EVENT,
296       EnvironmentEdgeManager.currentTime(), regionEventDesc.toByteArray());
297     return new WALEdit().add(kv);
298   }
299 
300   public static RegionEventDescriptor getRegionEventDescriptor(Cell cell) throws IOException {
301     if (CellUtil.matchingColumn(cell, METAFAMILY, REGION_EVENT)) {
302       return RegionEventDescriptor.parseFrom(cell.getValue());
303     }
304     return null;
305   }
306 
307   /**
308    * Create a compaction WALEdit
309    * @param c
310    * @return A WALEdit that has <code>c</code> serialized as its value
311    */
312   public static WALEdit createCompaction(final HRegionInfo hri, final CompactionDescriptor c) {
313     byte [] pbbytes = c.toByteArray();
314     KeyValue kv = new KeyValue(getRowForRegion(hri), METAFAMILY, COMPACTION,
315       EnvironmentEdgeManager.currentTime(), pbbytes);
316     return new WALEdit().add(kv); //replication scope null so that this won't be replicated
317   }
318 
319   private static byte[] getRowForRegion(HRegionInfo hri) {
320     byte[] startKey = hri.getStartKey();
321     if (startKey.length == 0) {
322       // empty row key is not allowed in mutations because it is both the start key and the end key
323       // we return the smallest byte[] that is bigger (in lex comparison) than byte[0].
324       return new byte[] {0};
325     }
326     return startKey;
327   }
328 
329   /**
330    * Deserialized and returns a CompactionDescriptor is the KeyValue contains one.
331    * @param kv the key value
332    * @return deserialized CompactionDescriptor or null.
333    */
334   public static CompactionDescriptor getCompaction(Cell kv) throws IOException {
335     if (CellUtil.matchingColumn(kv, METAFAMILY, COMPACTION)) {
336       return CompactionDescriptor.parseFrom(kv.getValue());
337     }
338     return null;
339   }
340 
341   /**
342    * Create a bulk loader WALEdit
343    *
344    * @param hri                The HRegionInfo for the region in which we are bulk loading
345    * @param bulkLoadDescriptor The descriptor for the Bulk Loader
346    * @return The WALEdit for the BulkLoad
347    */
348   public static WALEdit createBulkLoadEvent(HRegionInfo hri,
349                                             WALProtos.BulkLoadDescriptor bulkLoadDescriptor) {
350     KeyValue kv = new KeyValue(getRowForRegion(hri),
351         METAFAMILY,
352         BULK_LOAD,
353         EnvironmentEdgeManager.currentTime(),
354         bulkLoadDescriptor.toByteArray());
355     return new WALEdit().add(kv);
356   }
357 
358   /**
359    * Deserialized and returns a BulkLoadDescriptor from the passed in Cell
360    * @param cell the key value
361    * @return deserialized BulkLoadDescriptor or null.
362    */
363   public static WALProtos.BulkLoadDescriptor getBulkLoadDescriptor(Cell cell) throws IOException {
364     if (CellUtil.matchingColumn(cell, METAFAMILY, BULK_LOAD)) {
365       return WALProtos.BulkLoadDescriptor.parseFrom(cell.getValue());
366     }
367     return null;
368   }
369 }