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