1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.wal;
20
21 import java.io.IOException;
22 import java.io.InterruptedIOException;
23 import java.util.ArrayList;
24 import java.util.Collections;
25 import java.util.HashMap;
26 import java.util.Iterator;
27 import java.util.List;
28 import java.util.Map;
29 import java.util.NavigableMap;
30 import java.util.TreeMap;
31 import java.util.UUID;
32 import java.util.concurrent.CountDownLatch;
33
34
35 import org.apache.hadoop.hbase.util.ByteStringer;
36 import org.apache.commons.logging.Log;
37 import org.apache.commons.logging.LogFactory;
38 import org.apache.hadoop.hbase.classification.InterfaceAudience;
39 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
40 import org.apache.hadoop.hbase.HConstants;
41 import org.apache.hadoop.hbase.TableName;
42 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
43 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FamilyScope;
44 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.ScopeType;
45 import org.apache.hadoop.hbase.regionserver.SequenceId;
46 import org.apache.hadoop.hbase.util.Bytes;
47 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
48
49 import com.google.common.annotations.VisibleForTesting;
50 import com.google.protobuf.ByteString;
51
52
53 import org.apache.hadoop.hbase.regionserver.wal.CompressionContext;
54 import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION)
72 public class WALKey implements SequenceId, Comparable<WALKey> {
73 public static final Log LOG = LogFactory.getLog(WALKey.class);
74
75
76
77
78 @InterfaceAudience.Private
79 protected enum Version {
80 UNVERSIONED(0),
81
82 INITIAL(-1),
83
84
85 COMPRESSED(-2);
86
87 public final int code;
88 static final Version[] byCode;
89 static {
90 byCode = Version.values();
91 for (int i = 0; i < byCode.length; i++) {
92 if (byCode[i].code != -1 * i) {
93 throw new AssertionError("Values in this enum should be descending by one");
94 }
95 }
96 }
97
98 Version(int code) {
99 this.code = code;
100 }
101
102 public boolean atLeast(Version other) {
103 return code <= other.code;
104 }
105
106 public static Version fromCode(int code) {
107 return byCode[code * -1];
108 }
109 }
110
111
112
113
114
115 private static final String PREFIX_CLUSTER_KEY = ".";
116
117
118
119 @InterfaceAudience.Private
120 protected static final Version VERSION = Version.COMPRESSED;
121
122
123 public static final long NO_SEQUENCE_ID = -1;
124
125
126
127 @InterfaceAudience.Private
128 protected byte [] encodedRegionName;
129
130 @InterfaceAudience.Private
131 protected TableName tablename;
132
133 @InterfaceAudience.Private
134 protected long logSeqNum;
135 private long origLogSeqNum = 0;
136 private CountDownLatch seqNumAssignedLatch = new CountDownLatch(1);
137
138
139 @InterfaceAudience.Private
140 protected long writeTime;
141
142
143
144 @InterfaceAudience.Private
145 protected List<UUID> clusterIds;
146
147 private NavigableMap<byte[], Integer> scopes;
148
149 private long nonceGroup = HConstants.NO_NONCE;
150 private long nonce = HConstants.NO_NONCE;
151 static final List<UUID> EMPTY_UUIDS = Collections.unmodifiableList(new ArrayList<UUID>());
152
153
154 @InterfaceAudience.Private
155 protected CompressionContext compressionContext;
156
157 public WALKey() {
158 init(null, null, 0L, HConstants.LATEST_TIMESTAMP,
159 new ArrayList<UUID>(), HConstants.NO_NONCE, HConstants.NO_NONCE);
160 }
161
162 @VisibleForTesting
163 public WALKey(final byte[] encodedRegionName, final TableName tablename, long logSeqNum,
164 final long now, UUID clusterId) {
165 List<UUID> clusterIds = new ArrayList<UUID>();
166 clusterIds.add(clusterId);
167 init(encodedRegionName, tablename, logSeqNum, now, clusterIds,
168 HConstants.NO_NONCE, HConstants.NO_NONCE);
169 }
170
171 public WALKey(final byte[] encodedRegionName, final TableName tablename) {
172 this(encodedRegionName, tablename, System.currentTimeMillis());
173 }
174
175 public WALKey(final byte[] encodedRegionName, final TableName tablename, final long now) {
176 init(encodedRegionName, tablename, NO_SEQUENCE_ID, now,
177 EMPTY_UUIDS, HConstants.NO_NONCE, HConstants.NO_NONCE);
178 }
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193 public WALKey(final byte [] encodedRegionName, final TableName tablename,
194 long logSeqNum, final long now, List<UUID> clusterIds, long nonceGroup, long nonce) {
195 init(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce);
196 }
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211 public WALKey(final byte [] encodedRegionName, final TableName tablename,
212 final long now, List<UUID> clusterIds, long nonceGroup, long nonce) {
213 init(encodedRegionName, tablename, NO_SEQUENCE_ID, now, clusterIds,
214 nonceGroup, nonce);
215 }
216
217
218
219
220
221
222
223
224
225
226
227
228
229 public WALKey(final byte [] encodedRegionName, final TableName tablename, long logSeqNum,
230 long nonceGroup, long nonce) {
231 init(encodedRegionName, tablename, logSeqNum, EnvironmentEdgeManager.currentTime(),
232 EMPTY_UUIDS, nonceGroup, nonce);
233 }
234
235 @InterfaceAudience.Private
236 protected void init(final byte [] encodedRegionName, final TableName tablename,
237 long logSeqNum, final long now, List<UUID> clusterIds, long nonceGroup, long nonce) {
238 this.logSeqNum = logSeqNum;
239 this.writeTime = now;
240 this.clusterIds = clusterIds;
241 this.encodedRegionName = encodedRegionName;
242 this.tablename = tablename;
243 this.nonceGroup = nonceGroup;
244 this.nonce = nonce;
245 }
246
247
248
249
250 public void setCompressionContext(CompressionContext compressionContext) {
251 this.compressionContext = compressionContext;
252 }
253
254
255 public byte [] getEncodedRegionName() {
256 return encodedRegionName;
257 }
258
259
260 public TableName getTablename() {
261 return tablename;
262 }
263
264
265 public long getLogSeqNum() {
266 return this.logSeqNum;
267 }
268
269
270
271
272
273
274
275 @InterfaceAudience.Private
276 public void setLogSeqNum(final long sequence) {
277 this.logSeqNum = sequence;
278 this.seqNumAssignedLatch.countDown();
279 }
280
281
282
283
284
285 public void setOrigLogSeqNum(final long seqId) {
286 this.origLogSeqNum = seqId;
287 }
288
289
290
291
292
293 public long getOrigLogSeqNum() {
294 return this.origLogSeqNum;
295 }
296
297
298
299
300
301
302 @Override
303 public long getSequenceId() throws IOException {
304 try {
305 this.seqNumAssignedLatch.await();
306 } catch (InterruptedException ie) {
307 LOG.warn("Thread interrupted waiting for next log sequence number");
308 InterruptedIOException iie = new InterruptedIOException();
309 iie.initCause(ie);
310 throw iie;
311 }
312 return this.logSeqNum;
313 }
314
315
316
317
318 public long getWriteTime() {
319 return this.writeTime;
320 }
321
322 public NavigableMap<byte[], Integer> getScopes() {
323 return scopes;
324 }
325
326
327 public long getNonceGroup() {
328 return nonceGroup;
329 }
330
331
332 public long getNonce() {
333 return nonce;
334 }
335
336 public void setScopes(NavigableMap<byte[], Integer> scopes) {
337 this.scopes = scopes;
338 }
339
340 public void readOlderScopes(NavigableMap<byte[], Integer> scopes) {
341 if (scopes != null) {
342 Iterator<Map.Entry<byte[], Integer>> iterator = scopes.entrySet()
343 .iterator();
344 while (iterator.hasNext()) {
345 Map.Entry<byte[], Integer> scope = iterator.next();
346 String key = Bytes.toString(scope.getKey());
347 if (key.startsWith(PREFIX_CLUSTER_KEY)) {
348 addClusterId(UUID.fromString(key.substring(PREFIX_CLUSTER_KEY
349 .length())));
350 iterator.remove();
351 }
352 }
353 if (scopes.size() > 0) {
354 this.scopes = scopes;
355 }
356 }
357 }
358
359
360
361
362 public void addClusterId(UUID clusterId) {
363 if (!clusterIds.contains(clusterId)) {
364 clusterIds.add(clusterId);
365 }
366 }
367
368
369
370
371 public List<UUID> getClusterIds() {
372 return clusterIds;
373 }
374
375
376
377
378
379 public UUID getOriginatingClusterId(){
380 return clusterIds.isEmpty() ? HConstants.DEFAULT_CLUSTER_ID : clusterIds.get(0);
381 }
382
383 @Override
384 public String toString() {
385 return tablename + "/" + Bytes.toString(encodedRegionName) + "/" +
386 logSeqNum;
387 }
388
389
390
391
392
393
394
395
396 public Map<String, Object> toStringMap() {
397 Map<String, Object> stringMap = new HashMap<String, Object>();
398 stringMap.put("table", tablename);
399 stringMap.put("region", Bytes.toStringBinary(encodedRegionName));
400 stringMap.put("sequence", logSeqNum);
401 return stringMap;
402 }
403
404 @Override
405 public boolean equals(Object obj) {
406 if (this == obj) {
407 return true;
408 }
409 if (obj == null || getClass() != obj.getClass()) {
410 return false;
411 }
412 return compareTo((WALKey)obj) == 0;
413 }
414
415 @Override
416 public int hashCode() {
417 int result = Bytes.hashCode(this.encodedRegionName);
418 result ^= this.logSeqNum;
419 result ^= this.writeTime;
420 return result;
421 }
422
423 @Override
424 public int compareTo(WALKey o) {
425 int result = Bytes.compareTo(this.encodedRegionName, o.encodedRegionName);
426 if (result == 0) {
427 if (this.logSeqNum < o.logSeqNum) {
428 result = -1;
429 } else if (this.logSeqNum > o.logSeqNum) {
430 result = 1;
431 }
432 if (result == 0) {
433 if (this.writeTime < o.writeTime) {
434 result = -1;
435 } else if (this.writeTime > o.writeTime) {
436 return 1;
437 }
438 }
439 }
440
441 return result;
442 }
443
444
445
446
447
448
449
450 void internTableName(TableName tablename) {
451
452
453 assert tablename.equals(this.tablename);
454 this.tablename = tablename;
455 }
456
457
458
459
460
461
462
463 void internEncodedRegionName(byte []encodedRegionName) {
464
465
466 assert Bytes.equals(this.encodedRegionName, encodedRegionName);
467 this.encodedRegionName = encodedRegionName;
468 }
469
470 public org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey.Builder getBuilder(WALCellCodec.ByteStringCompressor compressor)
471 throws IOException {
472 org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey.Builder builder = org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey.newBuilder();
473 if (compressionContext == null) {
474 builder.setEncodedRegionName(ByteStringer.wrap(this.encodedRegionName));
475 builder.setTableName(ByteStringer.wrap(this.tablename.getName()));
476 } else {
477 builder.setEncodedRegionName(compressor.compress(this.encodedRegionName,
478 compressionContext.regionDict));
479 builder.setTableName(compressor.compress(this.tablename.getName(),
480 compressionContext.tableDict));
481 }
482 builder.setLogSequenceNumber(this.logSeqNum);
483 builder.setWriteTime(writeTime);
484 if(this.origLogSeqNum > 0) {
485 builder.setOrigSequenceNumber(this.origLogSeqNum);
486 }
487 if (this.nonce != HConstants.NO_NONCE) {
488 builder.setNonce(nonce);
489 }
490 if (this.nonceGroup != HConstants.NO_NONCE) {
491 builder.setNonceGroup(nonceGroup);
492 }
493 HBaseProtos.UUID.Builder uuidBuilder = HBaseProtos.UUID.newBuilder();
494 for (UUID clusterId : clusterIds) {
495 uuidBuilder.setLeastSigBits(clusterId.getLeastSignificantBits());
496 uuidBuilder.setMostSigBits(clusterId.getMostSignificantBits());
497 builder.addClusterIds(uuidBuilder.build());
498 }
499 if (scopes != null) {
500 for (Map.Entry<byte[], Integer> e : scopes.entrySet()) {
501 ByteString family = (compressionContext == null) ? ByteStringer.wrap(e.getKey())
502 : compressor.compress(e.getKey(), compressionContext.familyDict);
503 builder.addScopes(FamilyScope.newBuilder()
504 .setFamily(family).setScopeType(ScopeType.valueOf(e.getValue())));
505 }
506 }
507 return builder;
508 }
509
510 public void readFieldsFromPb(
511 org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey walKey, WALCellCodec.ByteStringUncompressor uncompressor) throws IOException {
512 if (this.compressionContext != null) {
513 this.encodedRegionName = uncompressor.uncompress(
514 walKey.getEncodedRegionName(), compressionContext.regionDict);
515 byte[] tablenameBytes = uncompressor.uncompress(
516 walKey.getTableName(), compressionContext.tableDict);
517 this.tablename = TableName.valueOf(tablenameBytes);
518 } else {
519 this.encodedRegionName = walKey.getEncodedRegionName().toByteArray();
520 this.tablename = TableName.valueOf(walKey.getTableName().toByteArray());
521 }
522 clusterIds.clear();
523 if (walKey.hasClusterId()) {
524
525
526 clusterIds.add(new UUID(walKey.getClusterId().getMostSigBits(), walKey.getClusterId()
527 .getLeastSigBits()));
528 }
529 for (HBaseProtos.UUID clusterId : walKey.getClusterIdsList()) {
530 clusterIds.add(new UUID(clusterId.getMostSigBits(), clusterId.getLeastSigBits()));
531 }
532 if (walKey.hasNonceGroup()) {
533 this.nonceGroup = walKey.getNonceGroup();
534 }
535 if (walKey.hasNonce()) {
536 this.nonce = walKey.getNonce();
537 }
538 this.scopes = null;
539 if (walKey.getScopesCount() > 0) {
540 this.scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
541 for (FamilyScope scope : walKey.getScopesList()) {
542 byte[] family = (compressionContext == null) ? scope.getFamily().toByteArray() :
543 uncompressor.uncompress(scope.getFamily(), compressionContext.familyDict);
544 this.scopes.put(family, scope.getScopeType().getNumber());
545 }
546 }
547 this.logSeqNum = walKey.getLogSequenceNumber();
548 this.writeTime = walKey.getWriteTime();
549 if(walKey.hasOrigSequenceNumber()) {
550 this.origLogSeqNum = walKey.getOrigSequenceNumber();
551 }
552 }
553 }