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   /**
130    * @return True when current WALEdit is created by log replay. Replication skips WALEdits from
131    *         replay.
132    */
133   public boolean isReplay() {
134     return this.isReplay;
135   }
136 
137   public void setCompressionContext(final CompressionContext compressionContext) {
138     this.compressionContext = compressionContext;
139   }
140 
141   public WALEdit add(Cell cell) {
142     this.cells.add(cell);
143     return this;
144   }
145 
146   public boolean isEmpty() {
147     return cells.isEmpty();
148   }
149 
150   public int size() {
151     return cells.size();
152   }
153 
154   public ArrayList<Cell> getCells() {
155     return cells;
156   }
157 
158   public NavigableMap<byte[], Integer> getAndRemoveScopes() {
159     NavigableMap<byte[], Integer> result = scopes;
160     scopes = null;
161     return result;
162   }
163 
164   @Override
165   public void readFields(DataInput in) throws IOException {
166     cells.clear();
167     if (scopes != null) {
168       scopes.clear();
169     }
170     int versionOrLength = in.readInt();
171     // TODO: Change version when we protobuf.  Also, change way we serialize KV!  Pb it too.
172     if (versionOrLength == VERSION_2) {
173       // this is new style WAL entry containing multiple KeyValues.
174       int numEdits = in.readInt();
175       for (int idx = 0; idx < numEdits; idx++) {
176         if (compressionContext != null) {
177           this.add(KeyValueCompression.readKV(in, compressionContext));
178         } else {
179           this.add(KeyValue.create(in));
180         }
181       }
182       int numFamilies = in.readInt();
183       if (numFamilies > 0) {
184         if (scopes == null) {
185           scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
186         }
187         for (int i = 0; i < numFamilies; i++) {
188           byte[] fam = Bytes.readByteArray(in);
189           int scope = in.readInt();
190           scopes.put(fam, scope);
191         }
192       }
193     } else {
194       // this is an old style WAL entry. The int that we just
195       // read is actually the length of a single KeyValue
196       this.add(KeyValue.create(versionOrLength, in));
197     }
198   }
199 
200   @Override
201   public void write(DataOutput out) throws IOException {
202     LOG.warn("WALEdit is being serialized to writable - only expected in test code");
203     out.writeInt(VERSION_2);
204     out.writeInt(cells.size());
205     // We interleave the two lists for code simplicity
206     for (Cell cell : cells) {
207       // This is not used in any of the core code flows so it is just fine to convert to KV
208       KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
209       if (compressionContext != null) {
210         KeyValueCompression.writeKV(out, kv, compressionContext);
211       } else{
212         KeyValue.write(kv, out);
213       }
214     }
215     if (scopes == null) {
216       out.writeInt(0);
217     } else {
218       out.writeInt(scopes.size());
219       for (byte[] key : scopes.keySet()) {
220         Bytes.writeByteArray(out, key);
221         out.writeInt(scopes.get(key));
222       }
223     }
224   }
225 
226   /**
227    * Reads WALEdit from cells.
228    * @param cellDecoder Cell decoder.
229    * @param expectedCount Expected cell count.
230    * @return Number of KVs read.
231    */
232   public int readFromCells(Codec.Decoder cellDecoder, int expectedCount) throws IOException {
233     cells.clear();
234     cells.ensureCapacity(expectedCount);
235     while (cells.size() < expectedCount && cellDecoder.advance()) {
236       cells.add(cellDecoder.current());
237     }
238     return cells.size();
239   }
240 
241   @Override
242   public long heapSize() {
243     long ret = ClassSize.ARRAYLIST;
244     for (Cell cell : cells) {
245       ret += CellUtil.estimatedHeapSizeOf(cell);
246     }
247     if (scopes != null) {
248       ret += ClassSize.TREEMAP;
249       ret += ClassSize.align(scopes.size() * ClassSize.MAP_ENTRY);
250       // TODO this isn't quite right, need help here
251     }
252     return ret;
253   }
254 
255   @Override
256   public String toString() {
257     StringBuilder sb = new StringBuilder();
258 
259     sb.append("[#edits: " + cells.size() + " = <");
260     for (Cell cell : cells) {
261       sb.append(cell);
262       sb.append("; ");
263     }
264     if (scopes != null) {
265       sb.append(" scopes: " + scopes.toString());
266     }
267     sb.append(">]");
268     return sb.toString();
269   }
270 
271   public static WALEdit createFlushWALEdit(HRegionInfo hri, FlushDescriptor f) {
272     KeyValue kv = new KeyValue(getRowForRegion(hri), METAFAMILY, FLUSH,
273       EnvironmentEdgeManager.currentTime(), f.toByteArray());
274     return new WALEdit().add(kv);
275   }
276 
277   public static FlushDescriptor getFlushDescriptor(Cell cell) throws IOException {
278     if (CellUtil.matchingColumn(cell, METAFAMILY, FLUSH)) {
279       return FlushDescriptor.parseFrom(cell.getValue());
280     }
281     return null;
282   }
283 
284   public static WALEdit createRegionEventWALEdit(HRegionInfo hri,
285       RegionEventDescriptor regionEventDesc) {
286     KeyValue kv = new KeyValue(getRowForRegion(hri), METAFAMILY, REGION_EVENT,
287       EnvironmentEdgeManager.currentTime(), regionEventDesc.toByteArray());
288     return new WALEdit().add(kv);
289   }
290 
291   public static RegionEventDescriptor getRegionEventDescriptor(Cell cell) throws IOException {
292     if (CellUtil.matchingColumn(cell, METAFAMILY, REGION_EVENT)) {
293       return RegionEventDescriptor.parseFrom(cell.getValue());
294     }
295     return null;
296   }
297 
298   /**
299    * Create a compaction WALEdit
300    * @param c
301    * @return A WALEdit that has <code>c</code> serialized as its value
302    */
303   public static WALEdit createCompaction(final HRegionInfo hri, final CompactionDescriptor c) {
304     byte [] pbbytes = c.toByteArray();
305     KeyValue kv = new KeyValue(getRowForRegion(hri), METAFAMILY, COMPACTION,
306       EnvironmentEdgeManager.currentTime(), pbbytes);
307     return new WALEdit().add(kv); //replication scope null so that this won't be replicated
308   }
309 
310   private static byte[] getRowForRegion(HRegionInfo hri) {
311     byte[] startKey = hri.getStartKey();
312     if (startKey.length == 0) {
313       // empty row key is not allowed in mutations because it is both the start key and the end key
314       // we return the smallest byte[] that is bigger (in lex comparison) than byte[0].
315       return new byte[] {0};
316     }
317     return startKey;
318   }
319 
320   /**
321    * Deserialized and returns a CompactionDescriptor is the KeyValue contains one.
322    * @param kv the key value
323    * @return deserialized CompactionDescriptor or null.
324    */
325   public static CompactionDescriptor getCompaction(Cell kv) throws IOException {
326     if (CellUtil.matchingColumn(kv, METAFAMILY, COMPACTION)) {
327       return CompactionDescriptor.parseFrom(kv.getValue());
328     }
329     return null;
330   }
331 
332   /**
333    * Create a bulk loader WALEdit
334    *
335    * @param hri                The HRegionInfo for the region in which we are bulk loading
336    * @param bulkLoadDescriptor The descriptor for the Bulk Loader
337    * @return The WALEdit for the BulkLoad
338    */
339   public static WALEdit createBulkLoadEvent(HRegionInfo hri,
340                                             WALProtos.BulkLoadDescriptor bulkLoadDescriptor) {
341     KeyValue kv = new KeyValue(getRowForRegion(hri),
342         METAFAMILY,
343         BULK_LOAD,
344         EnvironmentEdgeManager.currentTime(),
345         bulkLoadDescriptor.toByteArray());
346     return new WALEdit().add(kv);
347   }
348   
349   /**
350    * Deserialized and returns a BulkLoadDescriptor from the passed in Cell
351    * @param cell the key value
352    * @return deserialized BulkLoadDescriptor or null.
353    */
354   public static WALProtos.BulkLoadDescriptor getBulkLoadDescriptor(Cell cell) throws IOException {
355     if (CellUtil.matchingColumn(cell, METAFAMILY, BULK_LOAD)) {
356       return WALProtos.BulkLoadDescriptor.parseFrom(cell.getValue());
357     }
358     return null;
359   }
360 }