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