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.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.codec.Codec;
37  import org.apache.hadoop.hbase.io.HeapSize;
38  import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
39  import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
40  import org.apache.hadoop.hbase.util.Bytes;
41  import org.apache.hadoop.hbase.util.ClassSize;
42  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
43  import org.apache.hadoop.io.Writable;
44  
45  
46  /**
47   * WALEdit: Used in HBase's transaction log (WAL) to represent
48   * the collection of edits (KeyValue objects) corresponding to a
49   * single transaction. The class implements "Writable" interface
50   * for serializing/deserializing a set of KeyValue items.
51   *
52   * Previously, if a transaction contains 3 edits to c1, c2, c3 for a row R,
53   * the HLog would have three log entries as follows:
54   *
55   *    <logseq1-for-edit1>:<KeyValue-for-edit-c1>
56   *    <logseq2-for-edit2>:<KeyValue-for-edit-c2>
57   *    <logseq3-for-edit3>:<KeyValue-for-edit-c3>
58   *
59   * This presents problems because row level atomicity of transactions
60   * was not guaranteed. If we crash after few of the above appends make
61   * it, then recovery will restore a partial transaction.
62   *
63   * In the new world, all the edits for a given transaction are written
64   * out as a single record, for example:
65   *
66   *   <logseq#-for-entire-txn>:<WALEdit-for-entire-txn>
67   *
68   * where, the WALEdit is serialized as:
69   *   <-1, # of edits, <KeyValue>, <KeyValue>, ... >
70   * For example:
71   *   <-1, 3, <Keyvalue-for-edit-c1>, <KeyValue-for-edit-c2>, <KeyValue-for-edit-c3>>
72   *
73   * The -1 marker is just a special way of being backward compatible with
74   * an old HLog which would have contained a single <KeyValue>.
75   *
76   * The deserializer for WALEdit backward compatibly detects if the record
77   * is an old style KeyValue or the new style WALEdit.
78   *
79   */
80  @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION)
81  public class WALEdit implements Writable, HeapSize {
82    public static final Log LOG = LogFactory.getLog(WALEdit.class);
83  
84    // TODO: Get rid of this; see HBASE-8457
85    public static final byte [] METAFAMILY = Bytes.toBytes("METAFAMILY");
86    static final byte [] METAROW = Bytes.toBytes("METAROW");
87    static final byte[] COMPACTION = Bytes.toBytes("HBASE::COMPACTION");
88    static final byte [] FLUSH = Bytes.toBytes("HBASE::FLUSH");
89  
90    private final int VERSION_2 = -1;
91    private final boolean isReplay;
92  
93    private final ArrayList<KeyValue> kvs = new ArrayList<KeyValue>(1);
94  
95    public static final WALEdit EMPTY_WALEDIT = new WALEdit();
96  
97    // Only here for legacy writable deserialization
98    @Deprecated
99    private NavigableMap<byte[], Integer> scopes;
100 
101   private CompressionContext compressionContext;
102 
103   public WALEdit() {
104     this(false);
105   }
106 
107   public WALEdit(boolean isReplay) {
108     this.isReplay = isReplay;
109   }
110 
111   /**
112    * @param f
113    * @return True is <code>f</code> is {@link #METAFAMILY}
114    */
115   public static boolean isMetaEditFamily(final byte [] f) {
116     return Bytes.equals(METAFAMILY, f);
117   }
118 
119   public static boolean isMetaEditFamily(Cell cell) {
120     return CellUtil.matchingFamily(cell, METAFAMILY);
121   }
122 
123   /**
124    * @return True when current WALEdit is created by log replay. Replication skips WALEdits from
125    *         replay.
126    */
127   public boolean isReplay() {
128     return this.isReplay;
129   }
130 
131   public void setCompressionContext(final CompressionContext compressionContext) {
132     this.compressionContext = compressionContext;
133   }
134 
135   public WALEdit add(KeyValue kv) {
136     this.kvs.add(kv);
137     return this;
138   }
139 
140   public boolean isEmpty() {
141     return kvs.isEmpty();
142   }
143 
144   public int size() {
145     return kvs.size();
146   }
147 
148   public ArrayList<KeyValue> getKeyValues() {
149     return kvs;
150   }
151 
152   public NavigableMap<byte[], Integer> getAndRemoveScopes() {
153     NavigableMap<byte[], Integer> result = scopes;
154     scopes = null;
155     return result;
156   }
157 
158   @Override
159   public void readFields(DataInput in) throws IOException {
160     kvs.clear();
161     if (scopes != null) {
162       scopes.clear();
163     }
164     int versionOrLength = in.readInt();
165     // TODO: Change version when we protobuf.  Also, change way we serialize KV!  Pb it too.
166     if (versionOrLength == VERSION_2) {
167       // this is new style HLog entry containing multiple KeyValues.
168       int numEdits = in.readInt();
169       for (int idx = 0; idx < numEdits; idx++) {
170         if (compressionContext != null) {
171           this.add(KeyValueCompression.readKV(in, compressionContext));
172         } else {
173           this.add(KeyValue.create(in));
174         }
175       }
176       int numFamilies = in.readInt();
177       if (numFamilies > 0) {
178         if (scopes == null) {
179           scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
180         }
181         for (int i = 0; i < numFamilies; i++) {
182           byte[] fam = Bytes.readByteArray(in);
183           int scope = in.readInt();
184           scopes.put(fam, scope);
185         }
186       }
187     } else {
188       // this is an old style HLog entry. The int that we just
189       // read is actually the length of a single KeyValue
190       this.add(KeyValue.create(versionOrLength, in));
191     }
192   }
193 
194   @Override
195   public void write(DataOutput out) throws IOException {
196     LOG.warn("WALEdit is being serialized to writable - only expected in test code");
197     out.writeInt(VERSION_2);
198     out.writeInt(kvs.size());
199     // We interleave the two lists for code simplicity
200     for (KeyValue kv : kvs) {
201       if (compressionContext != null) {
202         KeyValueCompression.writeKV(out, kv, compressionContext);
203       } else{
204         KeyValue.write(kv, out);
205       }
206     }
207     if (scopes == null) {
208       out.writeInt(0);
209     } else {
210       out.writeInt(scopes.size());
211       for (byte[] key : scopes.keySet()) {
212         Bytes.writeByteArray(out, key);
213         out.writeInt(scopes.get(key));
214       }
215     }
216   }
217 
218   /**
219    * Reads WALEdit from cells.
220    * @param cellDecoder Cell decoder.
221    * @param expectedCount Expected cell count.
222    * @return Number of KVs read.
223    */
224   public int readFromCells(Codec.Decoder cellDecoder, int expectedCount) throws IOException {
225     kvs.clear();
226     kvs.ensureCapacity(expectedCount);
227     while (kvs.size() < expectedCount && cellDecoder.advance()) {
228       Cell cell = cellDecoder.current();
229       if (!(cell instanceof KeyValue)) {
230         throw new IOException("WAL edit only supports KVs as cells");
231       }
232       kvs.add((KeyValue)cell);
233     }
234     return kvs.size();
235   }
236 
237   @Override
238   public long heapSize() {
239     long ret = ClassSize.ARRAYLIST;
240     for (KeyValue kv : kvs) {
241       ret += kv.heapSize();
242     }
243     if (scopes != null) {
244       ret += ClassSize.TREEMAP;
245       ret += ClassSize.align(scopes.size() * ClassSize.MAP_ENTRY);
246       // TODO this isn't quite right, need help here
247     }
248     return ret;
249   }
250 
251   @Override
252   public String toString() {
253     StringBuilder sb = new StringBuilder();
254 
255     sb.append("[#edits: " + kvs.size() + " = <");
256     for (KeyValue kv : kvs) {
257       sb.append(kv.toString());
258       sb.append("; ");
259     }
260     if (scopes != null) {
261       sb.append(" scopes: " + scopes.toString());
262     }
263     sb.append(">]");
264     return sb.toString();
265   }
266 
267   public static WALEdit createFlushWALEdit(HRegionInfo hri, FlushDescriptor f) {
268     KeyValue kv = new KeyValue(getRowForRegion(hri), METAFAMILY, FLUSH,
269       EnvironmentEdgeManager.currentTimeMillis(), f.toByteArray());
270     return new WALEdit().add(kv);
271   }
272 
273   public static FlushDescriptor getFlushDescriptor(Cell cell) throws IOException {
274     if (CellUtil.matchingColumn(cell, METAFAMILY, FLUSH)) {
275       return FlushDescriptor.parseFrom(cell.getValue());
276     }
277     return null;
278   }
279 
280   /**
281    * Create a compacion WALEdit
282    * @param c
283    * @return A WALEdit that has <code>c</code> serialized as its value
284    */
285   public static WALEdit createCompaction(final HRegionInfo hri, final CompactionDescriptor c) {
286     byte [] pbbytes = c.toByteArray();
287     KeyValue kv = new KeyValue(getRowForRegion(hri), METAFAMILY, COMPACTION,
288       EnvironmentEdgeManager.currentTimeMillis(), pbbytes);
289     return new WALEdit().add(kv); //replication scope null so that this won't be replicated
290   }
291 
292   private static byte[] getRowForRegion(HRegionInfo hri) {
293     byte[] startKey = hri.getStartKey();
294     if (startKey.length == 0) {
295       // empty row key is not allowed in mutations because it is both the start key and the end key
296       // we return the smallest byte[] that is bigger (in lex comparison) than byte[0].
297       return new byte[] {0};
298     }
299     return startKey;
300   }
301 
302   /**
303    * Deserialized and returns a CompactionDescriptor is the KeyValue contains one.
304    * @param kv the key value
305    * @return deserialized CompactionDescriptor or null.
306    */
307   public static CompactionDescriptor getCompaction(Cell kv) throws IOException {
308     if (CellUtil.matchingColumn(kv, METAFAMILY, COMPACTION)) {
309       return CompactionDescriptor.parseFrom(kv.getValue());
310     }
311     return null;
312   }
313 }