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