001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.wal;
019
020import java.io.IOException;
021import java.util.ArrayList;
022import java.util.HashMap;
023import java.util.List;
024import java.util.Map;
025import java.util.NavigableMap;
026import java.util.TreeMap;
027import java.util.UUID;
028import org.apache.hadoop.hbase.HBaseInterfaceAudience;
029import org.apache.hadoop.hbase.HConstants;
030import org.apache.hadoop.hbase.TableName;
031import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
032import org.apache.hadoop.hbase.regionserver.SequenceId;
033import org.apache.hadoop.hbase.regionserver.wal.CompressionContext;
034import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
035import org.apache.hadoop.hbase.util.Bytes;
036import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
037import org.apache.yetus.audience.InterfaceAudience;
038
039import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
040
041import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
042import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
043import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FamilyScope;
044import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.ScopeType;
045
046/**
047 * Default implementation of Key for an Entry in the WAL. For internal use only though Replication
048 * needs to have access. The log intermingles edits to many tables and rows, so each log entry
049 * identifies the appropriate table and row. Within a table and row, they're also sorted.
050 * <p>
051 * Some Transactional edits (START, COMMIT, ABORT) will not have an associated row.
052 */
053// TODO: Key and WALEdit are never used separately, or in one-to-many relation, for practical
054// purposes. They need to be merged into WALEntry.
055@InterfaceAudience.LimitedPrivate({ HBaseInterfaceAudience.REPLICATION })
056public class WALKeyImpl implements WALKey {
057  public static final WALKeyImpl EMPTY_WALKEYIMPL = new WALKeyImpl();
058
059  public MultiVersionConcurrencyControl getMvcc() {
060    return mvcc;
061  }
062
063  /**
064   * Use it to complete mvcc transaction. This WALKeyImpl was part of (the transaction is started
065   * when you call append; see the comment on FSHLog#append). To complete call
066   * {@link MultiVersionConcurrencyControl#complete(MultiVersionConcurrencyControl.WriteEntry)} or
067   * {@link MultiVersionConcurrencyControl#complete(MultiVersionConcurrencyControl.WriteEntry)}
068   * @return A WriteEntry gotten from local WAL subsystem.
069   * @see #setWriteEntry(MultiVersionConcurrencyControl.WriteEntry)
070   */
071  public MultiVersionConcurrencyControl.WriteEntry getWriteEntry() {
072    return this.writeEntry;
073  }
074
075  public void setWriteEntry(MultiVersionConcurrencyControl.WriteEntry writeEntry) {
076    assert this.writeEntry == null;
077    this.writeEntry = writeEntry;
078    // Set our sequenceid now using WriteEntry.
079    this.sequenceId = writeEntry.getWriteNumber();
080  }
081
082  private byte[] encodedRegionName;
083
084  private TableName tablename;
085
086  /**
087   * SequenceId for this edit. Set post-construction at write-to-WAL time. Until then it is
088   * NO_SEQUENCE_ID. Change it so multiple threads can read it -- e.g. access is synchronized.
089   */
090  private long sequenceId;
091
092  /**
093   * Used during WAL replay; the sequenceId of the edit when it came into the system.
094   */
095  private long origLogSeqNum = 0;
096
097  /** Time at which this edit was written. */
098  private long writeTime;
099
100  /** The first element in the list is the cluster id on which the change has originated */
101  private List<UUID> clusterIds;
102
103  private NavigableMap<byte[], Integer> replicationScope;
104
105  private long nonceGroup = HConstants.NO_NONCE;
106  private long nonce = HConstants.NO_NONCE;
107  private MultiVersionConcurrencyControl mvcc;
108
109  /**
110   * Set in a way visible to multiple threads; e.g. synchronized getter/setters.
111   */
112  private MultiVersionConcurrencyControl.WriteEntry writeEntry;
113
114  private Map<String, byte[]> extendedAttributes;
115
116  public WALKeyImpl() {
117    init(null, null, 0L, HConstants.LATEST_TIMESTAMP, new ArrayList<>(), HConstants.NO_NONCE,
118      HConstants.NO_NONCE, null, null, null);
119  }
120
121  public WALKeyImpl(final NavigableMap<byte[], Integer> replicationScope) {
122    init(null, null, 0L, HConstants.LATEST_TIMESTAMP, new ArrayList<>(), HConstants.NO_NONCE,
123      HConstants.NO_NONCE, null, replicationScope, null);
124  }
125
126  @InterfaceAudience.Private
127  public WALKeyImpl(final byte[] encodedRegionName, final TableName tablename, long logSeqNum,
128    final long now, UUID clusterId) {
129    List<UUID> clusterIds = new ArrayList<>(1);
130    clusterIds.add(clusterId);
131    init(encodedRegionName, tablename, logSeqNum, now, clusterIds, HConstants.NO_NONCE,
132      HConstants.NO_NONCE, null, null, null);
133  }
134
135  // TODO: Fix being able to pass in sequenceid.
136  public WALKeyImpl(final byte[] encodedRegionName, final TableName tablename, final long now) {
137    init(encodedRegionName, tablename, NO_SEQUENCE_ID, now, EMPTY_UUIDS, HConstants.NO_NONCE,
138      HConstants.NO_NONCE, null, null, null);
139  }
140
141  // TODO: Fix being able to pass in sequenceid.
142  public WALKeyImpl(final byte[] encodedRegionName, final TableName tablename, final long now,
143    final NavigableMap<byte[], Integer> replicationScope) {
144    init(encodedRegionName, tablename, NO_SEQUENCE_ID, now, EMPTY_UUIDS, HConstants.NO_NONCE,
145      HConstants.NO_NONCE, null, replicationScope, null);
146  }
147
148  public WALKeyImpl(final byte[] encodedRegionName, final TableName tablename, final long now,
149    MultiVersionConcurrencyControl mvcc, final NavigableMap<byte[], Integer> replicationScope) {
150    init(encodedRegionName, tablename, NO_SEQUENCE_ID, now, EMPTY_UUIDS, HConstants.NO_NONCE,
151      HConstants.NO_NONCE, mvcc, replicationScope, null);
152  }
153
154  public WALKeyImpl(final byte[] encodedRegionName, final TableName tablename, final long now,
155    MultiVersionConcurrencyControl mvcc, final NavigableMap<byte[], Integer> replicationScope,
156    Map<String, byte[]> extendedAttributes) {
157    init(encodedRegionName, tablename, NO_SEQUENCE_ID, now, EMPTY_UUIDS, HConstants.NO_NONCE,
158      HConstants.NO_NONCE, mvcc, replicationScope, extendedAttributes);
159  }
160
161  public WALKeyImpl(final byte[] encodedRegionName, final TableName tablename, final long now,
162    MultiVersionConcurrencyControl mvcc) {
163    init(encodedRegionName, tablename, NO_SEQUENCE_ID, now, EMPTY_UUIDS, HConstants.NO_NONCE,
164      HConstants.NO_NONCE, mvcc, null, null);
165  }
166
167  /**
168   * Copy constructor that takes in an existing WALKeyImpl plus some extended attributes. Intended
169   * for coprocessors to add annotations to a system-generated WALKey for persistence to the WAL.
170   * @param key                Key to be copied into this new key
171   * @param extendedAttributes Extra attributes to copy into the new key
172   */
173  public WALKeyImpl(WALKeyImpl key, Map<String, byte[]> extendedAttributes) {
174    init(key.getEncodedRegionName(), key.getTableName(), key.getSequenceId(), key.getWriteTime(),
175      key.getClusterIds(), key.getNonceGroup(), key.getNonce(), key.getMvcc(),
176      key.getReplicationScopes(), extendedAttributes);
177
178  }
179
180  /**
181   * Copy constructor that takes in an existing WALKey, the extra WALKeyImpl fields that the parent
182   * interface is missing, plus some extended attributes. Intended for coprocessors to add
183   * annotations to a system-generated WALKey for persistence to the WAL.
184   */
185  public WALKeyImpl(WALKey key, List<UUID> clusterIds, MultiVersionConcurrencyControl mvcc,
186    final NavigableMap<byte[], Integer> replicationScopes, Map<String, byte[]> extendedAttributes) {
187    init(key.getEncodedRegionName(), key.getTableName(), key.getSequenceId(), key.getWriteTime(),
188      clusterIds, key.getNonceGroup(), key.getNonce(), mvcc, replicationScopes, extendedAttributes);
189
190  }
191
192  /**
193   * Create the log key for writing to somewhere. We maintain the tablename mainly for debugging
194   * purposes. A regionName is always a sub-table object.
195   * <p>
196   * Used by log splitting and snapshots.
197   * @param encodedRegionName Encoded name of the region as returned by
198   *                          <code>HRegionInfo#getEncodedNameAsBytes()</code>.
199   * @param tablename         - name of table
200   * @param logSeqNum         - log sequence number
201   * @param now               Time at which this edit was written.
202   * @param clusterIds        the clusters that have consumed the change(used in Replication)
203   * @param nonceGroup        the nonceGroup
204   * @param nonce             the nonce
205   * @param mvcc              the mvcc associate the WALKeyImpl
206   * @param replicationScope  the non-default replication scope associated with the region's column
207   *                          families
208   */
209  // TODO: Fix being able to pass in sequenceid.
210  public WALKeyImpl(final byte[] encodedRegionName, final TableName tablename, long logSeqNum,
211    final long now, List<UUID> clusterIds, long nonceGroup, long nonce,
212    MultiVersionConcurrencyControl mvcc, final NavigableMap<byte[], Integer> replicationScope) {
213    init(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce, mvcc,
214      replicationScope, null);
215  }
216
217  /**
218   * Create the log key for writing to somewhere. We maintain the tablename mainly for debugging
219   * purposes. A regionName is always a sub-table object.
220   * <p>
221   * Used by log splitting and snapshots.
222   * @param encodedRegionName Encoded name of the region as returned by
223   *                          <code>HRegionInfo#getEncodedNameAsBytes()</code>.
224   * @param tablename         - name of table
225   * @param logSeqNum         - log sequence number
226   * @param now               Time at which this edit was written.
227   * @param clusterIds        the clusters that have consumed the change(used in Replication)
228   */
229  // TODO: Fix being able to pass in sequenceid.
230  public WALKeyImpl(final byte[] encodedRegionName, final TableName tablename, long logSeqNum,
231    final long now, List<UUID> clusterIds, long nonceGroup, long nonce,
232    MultiVersionConcurrencyControl mvcc) {
233    init(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce, mvcc, null,
234      null);
235  }
236
237  /**
238   * Create the log key for writing to somewhere. We maintain the tablename mainly for debugging
239   * purposes. A regionName is always a sub-table object.
240   * @param encodedRegionName Encoded name of the region as returned by
241   *                          <code>HRegionInfo#getEncodedNameAsBytes()</code>.
242   * @param tablename         the tablename
243   * @param now               Time at which this edit was written.
244   * @param clusterIds        the clusters that have consumed the change(used in Replication) nn
245   *                          * @param mvcc mvcc control used to generate sequence numbers and
246   *                          control read/write points
247   */
248  public WALKeyImpl(final byte[] encodedRegionName, final TableName tablename, final long now,
249    List<UUID> clusterIds, long nonceGroup, final long nonce,
250    final MultiVersionConcurrencyControl mvcc) {
251    init(encodedRegionName, tablename, NO_SEQUENCE_ID, now, clusterIds, nonceGroup, nonce, mvcc,
252      null, null);
253  }
254
255  /**
256   * Create the log key for writing to somewhere. We maintain the tablename mainly for debugging
257   * purposes. A regionName is always a sub-table object.
258   * @param encodedRegionName Encoded name of the region as returned by
259   *                          <code>HRegionInfo#getEncodedNameAsBytes()</code>. n * @param now Time
260   *                          at which this edit was written.
261   * @param clusterIds        the clusters that have consumed the change(used in Replication)
262   * @param nonceGroup        the nonceGroup
263   * @param nonce             the nonce
264   * @param mvcc              mvcc control used to generate sequence numbers and control read/write
265   *                          points
266   * @param replicationScope  the non-default replication scope of the column families
267   */
268  public WALKeyImpl(final byte[] encodedRegionName, final TableName tablename, final long now,
269    List<UUID> clusterIds, long nonceGroup, final long nonce,
270    final MultiVersionConcurrencyControl mvcc, NavigableMap<byte[], Integer> replicationScope) {
271    init(encodedRegionName, tablename, NO_SEQUENCE_ID, now, clusterIds, nonceGroup, nonce, mvcc,
272      replicationScope, null);
273  }
274
275  /**
276   * Create the log key for writing to somewhere. We maintain the tablename mainly for debugging
277   * purposes. A regionName is always a sub-table object.
278   * @param encodedRegionName Encoded name of the region as returned by
279   *                          <code>HRegionInfo#getEncodedNameAsBytes()</code>. nnnn
280   */
281  // TODO: Fix being able to pass in sequenceid.
282  public WALKeyImpl(final byte[] encodedRegionName, final TableName tablename, long logSeqNum,
283    long nonceGroup, long nonce, final MultiVersionConcurrencyControl mvcc) {
284    init(encodedRegionName, tablename, logSeqNum, EnvironmentEdgeManager.currentTime(), EMPTY_UUIDS,
285      nonceGroup, nonce, mvcc, null, null);
286  }
287
288  public WALKeyImpl(final byte[] encodedRegionName, final TableName tablename, final long now,
289    List<UUID> clusterIds, long nonceGroup, final long nonce,
290    final MultiVersionConcurrencyControl mvcc, NavigableMap<byte[], Integer> replicationScope,
291    Map<String, byte[]> extendedAttributes) {
292    init(encodedRegionName, tablename, NO_SEQUENCE_ID, now, clusterIds, nonceGroup, nonce, mvcc,
293      replicationScope, extendedAttributes);
294  }
295
296  @InterfaceAudience.Private
297  protected void init(final byte[] encodedRegionName, final TableName tablename, long logSeqNum,
298    final long now, List<UUID> clusterIds, long nonceGroup, long nonce,
299    MultiVersionConcurrencyControl mvcc, NavigableMap<byte[], Integer> replicationScope,
300    Map<String, byte[]> extendedAttributes) {
301    this.sequenceId = logSeqNum;
302    this.writeTime = now;
303    this.clusterIds = clusterIds;
304    this.encodedRegionName = encodedRegionName;
305    this.tablename = tablename;
306    this.nonceGroup = nonceGroup;
307    this.nonce = nonce;
308    this.mvcc = mvcc;
309    if (logSeqNum != NO_SEQUENCE_ID) {
310      setSequenceId(logSeqNum);
311    }
312    this.replicationScope = replicationScope;
313    this.extendedAttributes = extendedAttributes;
314  }
315
316  // For deserialization. DO NOT USE. See setWriteEntry below.
317  @InterfaceAudience.Private
318  protected void setSequenceId(long sequenceId) {
319    this.sequenceId = sequenceId;
320  }
321
322  /**
323   * @param compressionContext Compression context to use
324   * @deprecated deparcated since hbase 2.1.0
325   */
326  @Deprecated
327  public void setCompressionContext(CompressionContext compressionContext) {
328    // do nothing
329  }
330
331  /** Returns encoded region name */
332  @Override
333  public byte[] getEncodedRegionName() {
334    return encodedRegionName;
335  }
336
337  /** Returns table name */
338  @Override
339  public TableName getTableName() {
340    return tablename;
341  }
342
343  /**
344   * @return log sequence number
345   * @deprecated Use {@link #getSequenceId()}
346   */
347  @Deprecated
348  public long getLogSeqNum() {
349    return getSequenceId();
350  }
351
352  /**
353   * Used to set original sequenceId for WALKeyImpl during WAL replay
354   */
355  public void setOrigLogSeqNum(final long sequenceId) {
356    this.origLogSeqNum = sequenceId;
357  }
358
359  /**
360   * Return a positive long if current WALKeyImpl is created from a replay edit; a replay edit is an
361   * edit that came in when replaying WALs of a crashed server.
362   * @return original sequence number of the WALEdit
363   */
364  @Override
365  public long getOrigLogSeqNum() {
366    return this.origLogSeqNum;
367  }
368
369  /**
370   * SequenceId is only available post WAL-assign. Calls before this will get you a
371   * {@link SequenceId#NO_SEQUENCE_ID}. See the comment on FSHLog#append and #getWriteNumber in this
372   * method for more on when this sequenceId comes available.
373   * @return long the new assigned sequence number
374   */
375  @Override
376  public long getSequenceId() {
377    return this.sequenceId;
378  }
379
380  /** Returns the write time */
381  @Override
382  public long getWriteTime() {
383    return this.writeTime;
384  }
385
386  public NavigableMap<byte[], Integer> getReplicationScopes() {
387    return replicationScope;
388  }
389
390  /** Returns The nonce group */
391  @Override
392  public long getNonceGroup() {
393    return nonceGroup;
394  }
395
396  /** Returns The nonce */
397  @Override
398  public long getNonce() {
399    return nonce;
400  }
401
402  private void setReplicationScope(NavigableMap<byte[], Integer> replicationScope) {
403    this.replicationScope = replicationScope;
404  }
405
406  public void clearReplicationScope() {
407    setReplicationScope(null);
408  }
409
410  /**
411   * Marks that the cluster with the given clusterId has consumed the change
412   */
413  public void addClusterId(UUID clusterId) {
414    if (!clusterIds.contains(clusterId)) {
415      clusterIds.add(clusterId);
416    }
417  }
418
419  /** Returns the set of cluster Ids that have consumed the change */
420  public List<UUID> getClusterIds() {
421    return clusterIds;
422  }
423
424  /**
425   * @return the cluster id on which the change has originated. It there is no such cluster, it
426   *         returns DEFAULT_CLUSTER_ID (cases where replication is not enabled)
427   */
428  @Override
429  public UUID getOriginatingClusterId() {
430    return clusterIds.isEmpty() ? HConstants.DEFAULT_CLUSTER_ID : clusterIds.get(0);
431  }
432
433  @Override
434  public void addExtendedAttribute(String attributeKey, byte[] attributeValue) {
435    if (extendedAttributes == null) {
436      extendedAttributes = new HashMap<String, byte[]>();
437    }
438    extendedAttributes.put(attributeKey, attributeValue);
439  }
440
441  @Override
442  public byte[] getExtendedAttribute(String attributeKey) {
443    return extendedAttributes != null ? extendedAttributes.get(attributeKey) : null;
444  }
445
446  @Override
447  public Map<String, byte[]> getExtendedAttributes() {
448    return extendedAttributes != null
449      ? new HashMap<String, byte[]>(extendedAttributes)
450      : new HashMap<String, byte[]>();
451  }
452
453  @Override
454  public String toString() {
455    return tablename + "/" + Bytes.toString(encodedRegionName) + "/" + sequenceId;
456  }
457
458  @Override
459  public boolean equals(Object obj) {
460    if (this == obj) {
461      return true;
462    }
463    if (obj == null || getClass() != obj.getClass()) {
464      return false;
465    }
466    return compareTo((WALKey) obj) == 0;
467  }
468
469  @Override
470  public int hashCode() {
471    int result = Bytes.hashCode(this.encodedRegionName);
472    result = (int) (result ^ getSequenceId());
473    result = (int) (result ^ this.writeTime);
474    return result;
475  }
476
477  @Override
478  public int compareTo(WALKey o) {
479    int result = Bytes.compareTo(this.encodedRegionName, o.getEncodedRegionName());
480    if (result == 0) {
481      long sid = getSequenceId();
482      long otherSid = o.getSequenceId();
483      if (sid < otherSid) {
484        result = -1;
485      } else if (sid > otherSid) {
486        result = 1;
487      }
488      if (result == 0) {
489        if (this.writeTime < o.getWriteTime()) {
490          result = -1;
491        } else if (this.writeTime > o.getWriteTime()) {
492          return 1;
493        }
494      }
495    }
496    // why isn't cluster id accounted for?
497    return result;
498  }
499
500  /**
501   * Drop this instance's tablename byte array and instead hold a reference to the provided
502   * tablename. This is not meant to be a general purpose setter - it's only used to collapse
503   * references to conserve memory.
504   */
505  void internTableName(TableName tablename) {
506    // We should not use this as a setter - only to swap
507    // in a new reference to the same table name.
508    assert tablename.equals(this.tablename);
509    this.tablename = tablename;
510  }
511
512  /**
513   * Drop this instance's region name byte array and instead hold a reference to the provided region
514   * name. This is not meant to be a general purpose setter - it's only used to collapse references
515   * to conserve memory.
516   */
517  void internEncodedRegionName(byte[] encodedRegionName) {
518    // We should not use this as a setter - only to swap
519    // in a new reference to the same table name.
520    assert Bytes.equals(this.encodedRegionName, encodedRegionName);
521    this.encodedRegionName = encodedRegionName;
522  }
523
524  public WALProtos.WALKey.Builder getBuilder(WALCellCodec.ByteStringCompressor compressor)
525    throws IOException {
526    WALProtos.WALKey.Builder builder = WALProtos.WALKey.newBuilder();
527    builder.setEncodedRegionName(
528      compressor.compress(this.encodedRegionName, CompressionContext.DictionaryIndex.REGION));
529    builder.setTableName(
530      compressor.compress(this.tablename.getName(), CompressionContext.DictionaryIndex.TABLE));
531    builder.setLogSequenceNumber(getSequenceId());
532    builder.setWriteTime(writeTime);
533    if (this.origLogSeqNum > 0) {
534      builder.setOrigSequenceNumber(this.origLogSeqNum);
535    }
536    if (this.nonce != HConstants.NO_NONCE) {
537      builder.setNonce(nonce);
538    }
539    if (this.nonceGroup != HConstants.NO_NONCE) {
540      builder.setNonceGroup(nonceGroup);
541    }
542    HBaseProtos.UUID.Builder uuidBuilder = HBaseProtos.UUID.newBuilder();
543    for (UUID clusterId : clusterIds) {
544      uuidBuilder.setLeastSigBits(clusterId.getLeastSignificantBits());
545      uuidBuilder.setMostSigBits(clusterId.getMostSignificantBits());
546      builder.addClusterIds(uuidBuilder.build());
547    }
548    if (replicationScope != null) {
549      for (Map.Entry<byte[], Integer> e : replicationScope.entrySet()) {
550        ByteString family =
551          compressor.compress(e.getKey(), CompressionContext.DictionaryIndex.FAMILY);
552        builder.addScopes(FamilyScope.newBuilder().setFamily(family)
553          .setScopeType(ScopeType.forNumber(e.getValue())));
554      }
555    }
556    if (extendedAttributes != null) {
557      for (Map.Entry<String, byte[]> e : extendedAttributes.entrySet()) {
558        WALProtos.Attribute attr = WALProtos.Attribute.newBuilder().setKey(e.getKey())
559          .setValue(compressor.compress(e.getValue(), CompressionContext.DictionaryIndex.TABLE))
560          .build();
561        builder.addExtendedAttributes(attr);
562      }
563    }
564    return builder;
565  }
566
567  public void readFieldsFromPb(WALProtos.WALKey walKey,
568    WALCellCodec.ByteStringUncompressor uncompressor) throws IOException {
569    this.encodedRegionName = uncompressor.uncompress(walKey.getEncodedRegionName(),
570      CompressionContext.DictionaryIndex.REGION);
571    byte[] tablenameBytes =
572      uncompressor.uncompress(walKey.getTableName(), CompressionContext.DictionaryIndex.TABLE);
573    this.tablename = TableName.valueOf(tablenameBytes);
574    clusterIds.clear();
575    for (HBaseProtos.UUID clusterId : walKey.getClusterIdsList()) {
576      clusterIds.add(new UUID(clusterId.getMostSigBits(), clusterId.getLeastSigBits()));
577    }
578    if (walKey.hasNonceGroup()) {
579      this.nonceGroup = walKey.getNonceGroup();
580    }
581    if (walKey.hasNonce()) {
582      this.nonce = walKey.getNonce();
583    }
584    this.replicationScope = null;
585    if (walKey.getScopesCount() > 0) {
586      this.replicationScope = new TreeMap<>(Bytes.BYTES_COMPARATOR);
587      for (FamilyScope scope : walKey.getScopesList()) {
588        byte[] family =
589          uncompressor.uncompress(scope.getFamily(), CompressionContext.DictionaryIndex.FAMILY);
590        this.replicationScope.put(family, scope.getScopeType().getNumber());
591      }
592    }
593    setSequenceId(walKey.getLogSequenceNumber());
594    this.writeTime = walKey.getWriteTime();
595    if (walKey.hasOrigSequenceNumber()) {
596      this.origLogSeqNum = walKey.getOrigSequenceNumber();
597    }
598    if (walKey.getExtendedAttributesCount() > 0) {
599      this.extendedAttributes = new HashMap<>(walKey.getExtendedAttributesCount());
600      for (WALProtos.Attribute attr : walKey.getExtendedAttributesList()) {
601        byte[] value =
602          uncompressor.uncompress(attr.getValue(), CompressionContext.DictionaryIndex.TABLE);
603        extendedAttributes.put(attr.getKey(), value);
604      }
605    }
606  }
607
608  @Override
609  public long estimatedSerializedSizeOf() {
610    long size = encodedRegionName != null ? encodedRegionName.length : 0;
611    size += tablename != null ? tablename.toBytes().length : 0;
612    if (clusterIds != null) {
613      size += 16 * clusterIds.size();
614    }
615    if (nonceGroup != HConstants.NO_NONCE) {
616      size += Bytes.SIZEOF_LONG; // nonce group
617    }
618    if (nonce != HConstants.NO_NONCE) {
619      size += Bytes.SIZEOF_LONG; // nonce
620    }
621    if (replicationScope != null) {
622      for (Map.Entry<byte[], Integer> scope : replicationScope.entrySet()) {
623        size += scope.getKey().length;
624        size += Bytes.SIZEOF_INT;
625      }
626    }
627    size += Bytes.SIZEOF_LONG; // sequence number
628    size += Bytes.SIZEOF_LONG; // write time
629    if (origLogSeqNum > 0) {
630      size += Bytes.SIZEOF_LONG; // original sequence number
631    }
632    return size;
633  }
634}