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