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