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