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