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.client;
019
020import java.io.IOException;
021import java.nio.ByteBuffer;
022import java.util.ArrayList;
023import java.util.Arrays;
024import java.util.HashMap;
025import java.util.Iterator;
026import java.util.List;
027import java.util.Map;
028import java.util.NavigableMap;
029import java.util.Optional;
030import java.util.TreeMap;
031import java.util.UUID;
032import java.util.stream.Collectors;
033import org.apache.hadoop.hbase.Cell;
034import org.apache.hadoop.hbase.CellScannable;
035import org.apache.hadoop.hbase.CellScanner;
036import org.apache.hadoop.hbase.CellUtil;
037import org.apache.hadoop.hbase.ExtendedCell;
038import org.apache.hadoop.hbase.HConstants;
039import org.apache.hadoop.hbase.IndividualBytesFieldCell;
040import org.apache.hadoop.hbase.KeyValue;
041import org.apache.hadoop.hbase.PrivateCellUtil;
042import org.apache.hadoop.hbase.Tag;
043import org.apache.hadoop.hbase.exceptions.DeserializationException;
044import org.apache.hadoop.hbase.io.HeapSize;
045import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
046import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
047import org.apache.hadoop.hbase.security.access.AccessControlConstants;
048import org.apache.hadoop.hbase.security.access.AccessControlUtil;
049import org.apache.hadoop.hbase.security.access.Permission;
050import org.apache.hadoop.hbase.security.visibility.CellVisibility;
051import org.apache.hadoop.hbase.security.visibility.VisibilityConstants;
052import org.apache.hadoop.hbase.util.Bytes;
053import org.apache.hadoop.hbase.util.ClassSize;
054import org.apache.yetus.audience.InterfaceAudience;
055
056import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
057import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
058import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap;
059import org.apache.hbase.thirdparty.com.google.common.io.ByteArrayDataInput;
060import org.apache.hbase.thirdparty.com.google.common.io.ByteArrayDataOutput;
061import org.apache.hbase.thirdparty.com.google.common.io.ByteStreams;
062
063@InterfaceAudience.Public
064public abstract class Mutation extends OperationWithAttributes
065  implements Row, CellScannable, HeapSize {
066  public static final long MUTATION_OVERHEAD = ClassSize.align(
067    // This
068    ClassSize.OBJECT +
069    // row + OperationWithAttributes.attributes
070      2 * ClassSize.REFERENCE +
071      // Timestamp
072      1 * Bytes.SIZEOF_LONG +
073      // durability
074      ClassSize.REFERENCE +
075      // familyMap
076      ClassSize.REFERENCE +
077      // familyMap
078      ClassSize.TREEMAP +
079      // priority
080      ClassSize.INTEGER);
081
082  /**
083   * The attribute for storing the list of clusters that have consumed the change.
084   */
085  private static final String CONSUMED_CLUSTER_IDS = "_cs.id";
086
087  /**
088   * The attribute for storing TTL for the result of the mutation.
089   */
090  private static final String OP_ATTRIBUTE_TTL = "_ttl";
091
092  private static final String RETURN_RESULTS = "_rr_";
093
094  // TODO: row should be final
095  protected byte[] row = null;
096  protected long ts = HConstants.LATEST_TIMESTAMP;
097  protected Durability durability = Durability.USE_DEFAULT;
098
099  // TODO: familyMap should be final
100  // A Map sorted by column family.
101  protected NavigableMap<byte[], List<Cell>> familyMap;
102
103  /**
104   * empty construction. We need this empty construction to keep binary compatibility.
105   */
106  protected Mutation() {
107    this.familyMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
108  }
109
110  protected Mutation(Mutation clone) {
111    super(clone);
112    this.row = clone.getRow();
113    this.ts = clone.getTimestamp();
114    this.familyMap = clone.getFamilyCellMap().entrySet().stream()
115      .collect(Collectors.toMap(e -> e.getKey(), e -> new ArrayList<>(e.getValue()), (k, v) -> {
116        throw new RuntimeException("collisions!!!");
117      }, () -> new TreeMap<>(Bytes.BYTES_COMPARATOR)));
118  }
119
120  /**
121   * Construct the mutation with user defined data.
122   * @param row       row. CAN'T be null
123   * @param ts        timestamp
124   * @param familyMap the map to collect all cells internally. CAN'T be null
125   */
126  protected Mutation(byte[] row, long ts, NavigableMap<byte[], List<Cell>> familyMap) {
127    this.row = Preconditions.checkNotNull(row);
128    if (row.length == 0) {
129      throw new IllegalArgumentException("Row can't be empty");
130    }
131    this.ts = ts;
132    this.familyMap = Preconditions.checkNotNull(familyMap);
133  }
134
135  @Override
136  public CellScanner cellScanner() {
137    return CellUtil.createCellScanner(getFamilyCellMap());
138  }
139
140  /**
141   * Creates an empty list if one doesn't exist for the given column family or else it returns the
142   * associated list of Cell objects.
143   * @param family column family
144   * @return a list of Cell objects, returns an empty list if one doesn't exist.
145   */
146  List<Cell> getCellList(byte[] family) {
147    List<Cell> list = getFamilyCellMap().get(family);
148    if (list == null) {
149      list = new ArrayList<>();
150      getFamilyCellMap().put(family, list);
151    }
152    return list;
153  }
154
155  /**
156   * Create a KeyValue with this objects row key and the Put identifier.
157   * @return a KeyValue with this objects row key and the Put identifier.
158   */
159  KeyValue createPutKeyValue(byte[] family, byte[] qualifier, long ts, byte[] value) {
160    return new KeyValue(this.row, family, qualifier, ts, KeyValue.Type.Put, value);
161  }
162
163  /**
164   * Create a KeyValue with this objects row key and the Put identifier.
165   * @return a KeyValue with this objects row key and the Put identifier.
166   */
167  KeyValue createPutKeyValue(byte[] family, byte[] qualifier, long ts, byte[] value, Tag[] tags) {
168    KeyValue kvWithTag = new KeyValue(this.row, family, qualifier, ts, value, tags);
169    return kvWithTag;
170  }
171
172  /**
173   * Create a KeyValue with this objects row key and the Put identifier.
174   * @return a KeyValue with this objects row key and the Put identifier.
175   */
176  KeyValue createPutKeyValue(byte[] family, ByteBuffer qualifier, long ts, ByteBuffer value,
177    Tag[] tags) {
178    return new KeyValue(this.row, 0, this.row == null ? 0 : this.row.length, family, 0,
179      family == null ? 0 : family.length, qualifier, ts, KeyValue.Type.Put, value,
180      tags != null ? Arrays.asList(tags) : null);
181  }
182
183  /**
184   * Compile the column family (i.e. schema) information into a Map. Useful for parsing and
185   * aggregation by debugging, logging, and administration tools.
186   */
187  @Override
188  public Map<String, Object> getFingerprint() {
189    Map<String, Object> map = new HashMap<>();
190    List<String> families = new ArrayList<>(getFamilyCellMap().entrySet().size());
191    // ideally, we would also include table information, but that information
192    // is not stored in each Operation instance.
193    map.put("families", families);
194    for (Map.Entry<byte[], List<Cell>> entry : getFamilyCellMap().entrySet()) {
195      families.add(Bytes.toStringBinary(entry.getKey()));
196    }
197    return map;
198  }
199
200  /**
201   * Compile the details beyond the scope of getFingerprint (row, columns, timestamps, etc.) into a
202   * Map along with the fingerprinted information. Useful for debugging, logging, and administration
203   * tools.
204   * @param maxCols a limit on the number of columns output prior to truncation n
205   */
206  @Override
207  public Map<String, Object> toMap(int maxCols) {
208    // we start with the fingerprint map and build on top of it.
209    Map<String, Object> map = getFingerprint();
210    // replace the fingerprint's simple list of families with a
211    // map from column families to lists of qualifiers and kv details
212    Map<String, List<Map<String, Object>>> columns = new HashMap<>();
213    map.put("families", columns);
214    map.put("row", Bytes.toStringBinary(this.row));
215    int colCount = 0;
216    // iterate through all column families affected
217    for (Map.Entry<byte[], List<Cell>> entry : getFamilyCellMap().entrySet()) {
218      // map from this family to details for each cell affected within the family
219      List<Map<String, Object>> qualifierDetails = new ArrayList<>();
220      columns.put(Bytes.toStringBinary(entry.getKey()), qualifierDetails);
221      colCount += entry.getValue().size();
222      if (maxCols <= 0) {
223        continue;
224      }
225      // add details for each cell
226      for (Cell cell : entry.getValue()) {
227        if (--maxCols <= 0) {
228          continue;
229        }
230        Map<String, Object> cellMap = cellToStringMap(cell);
231        qualifierDetails.add(cellMap);
232      }
233    }
234    map.put("totalColumns", colCount);
235    // add the id if set
236    if (getId() != null) {
237      map.put("id", getId());
238    }
239    // Add the TTL if set
240    // Long.MAX_VALUE is the default, and is interpreted to mean this attribute
241    // has not been set.
242    if (getTTL() != Long.MAX_VALUE) {
243      map.put("ttl", getTTL());
244    }
245    map.put("ts", this.ts);
246    return map;
247  }
248
249  private static Map<String, Object> cellToStringMap(Cell c) {
250    Map<String, Object> stringMap = new HashMap<>();
251    stringMap.put("qualifier",
252      Bytes.toStringBinary(c.getQualifierArray(), c.getQualifierOffset(), c.getQualifierLength()));
253    stringMap.put("timestamp", c.getTimestamp());
254    stringMap.put("vlen", c.getValueLength());
255    List<Tag> tags = PrivateCellUtil.getTags(c);
256    if (tags != null) {
257      List<String> tagsString = new ArrayList<>(tags.size());
258      for (Tag t : tags) {
259        tagsString.add(t.getType() + ":" + Bytes.toStringBinary(Tag.cloneValue(t)));
260      }
261      stringMap.put("tag", tagsString);
262    }
263    return stringMap;
264  }
265
266  /**
267   * Set the durability for this mutation n
268   */
269  public Mutation setDurability(Durability d) {
270    this.durability = d;
271    return this;
272  }
273
274  /** Get the current durability */
275  public Durability getDurability() {
276    return this.durability;
277  }
278
279  /**
280   * Method for retrieving the put's familyMap n
281   */
282  public NavigableMap<byte[], List<Cell>> getFamilyCellMap() {
283    return this.familyMap;
284  }
285
286  /**
287   * Method for setting the mutation's familyMap
288   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Use
289   *             {@link Mutation#Mutation(byte[], long, NavigableMap)} instead
290   */
291  @Deprecated
292  public Mutation setFamilyCellMap(NavigableMap<byte[], List<Cell>> map) {
293    // TODO: Shut this down or move it up to be a Constructor. Get new object rather than change
294    // this internal data member.
295    this.familyMap = map;
296    return this;
297  }
298
299  /**
300   * Method to check if the familyMap is empty
301   * @return true if empty, false otherwise
302   */
303  public boolean isEmpty() {
304    return getFamilyCellMap().isEmpty();
305  }
306
307  /**
308   * Method for retrieving the delete's row n
309   */
310  @Override
311  public byte[] getRow() {
312    return this.row;
313  }
314
315  /**
316   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Use
317   *             {@link Row#COMPARATOR} instead
318   */
319  @Deprecated
320  @Override
321  public int compareTo(final Row d) {
322    return Bytes.compareTo(this.getRow(), d.getRow());
323  }
324
325  /**
326   * Method for retrieving the timestamp n * @deprecated As of release 2.0.0, this will be removed
327   * in HBase 3.0.0. Use {@link #getTimestamp()} instead
328   */
329  @Deprecated
330  public long getTimeStamp() {
331    return this.getTimestamp();
332  }
333
334  /**
335   * Method for retrieving the timestamp. n
336   */
337  public long getTimestamp() {
338    return this.ts;
339  }
340
341  /**
342   * Marks that the clusters with the given clusterIds have consumed the mutation
343   * @param clusterIds of the clusters that have consumed the mutation
344   */
345  public Mutation setClusterIds(List<UUID> clusterIds) {
346    ByteArrayDataOutput out = ByteStreams.newDataOutput();
347    out.writeInt(clusterIds.size());
348    for (UUID clusterId : clusterIds) {
349      out.writeLong(clusterId.getMostSignificantBits());
350      out.writeLong(clusterId.getLeastSignificantBits());
351    }
352    setAttribute(CONSUMED_CLUSTER_IDS, out.toByteArray());
353    return this;
354  }
355
356  /** Returns the set of clusterIds that have consumed the mutation */
357  public List<UUID> getClusterIds() {
358    List<UUID> clusterIds = new ArrayList<>();
359    byte[] bytes = getAttribute(CONSUMED_CLUSTER_IDS);
360    if (bytes != null) {
361      ByteArrayDataInput in = ByteStreams.newDataInput(bytes);
362      int numClusters = in.readInt();
363      for (int i = 0; i < numClusters; i++) {
364        clusterIds.add(new UUID(in.readLong(), in.readLong()));
365      }
366    }
367    return clusterIds;
368  }
369
370  /**
371   * Sets the visibility expression associated with cells in this Mutation. n
372   */
373  public Mutation setCellVisibility(CellVisibility expression) {
374    this.setAttribute(VisibilityConstants.VISIBILITY_LABELS_ATTR_KEY,
375      toCellVisibility(expression).toByteArray());
376    return this;
377  }
378
379  /** Returns CellVisibility associated with cells in this Mutation. n */
380  public CellVisibility getCellVisibility() throws DeserializationException {
381    byte[] cellVisibilityBytes = this.getAttribute(VisibilityConstants.VISIBILITY_LABELS_ATTR_KEY);
382    if (cellVisibilityBytes == null) return null;
383    return toCellVisibility(cellVisibilityBytes);
384  }
385
386  /**
387   * Create a protocol buffer CellVisibility based on a client CellVisibility. n * @return a
388   * protocol buffer CellVisibility
389   */
390  static ClientProtos.CellVisibility toCellVisibility(CellVisibility cellVisibility) {
391    ClientProtos.CellVisibility.Builder builder = ClientProtos.CellVisibility.newBuilder();
392    builder.setExpression(cellVisibility.getExpression());
393    return builder.build();
394  }
395
396  /**
397   * Convert a protocol buffer CellVisibility to a client CellVisibility n * @return the converted
398   * client CellVisibility
399   */
400  private static CellVisibility toCellVisibility(ClientProtos.CellVisibility proto) {
401    if (proto == null) return null;
402    return new CellVisibility(proto.getExpression());
403  }
404
405  /**
406   * Convert a protocol buffer CellVisibility bytes to a client CellVisibility n * @return the
407   * converted client CellVisibility n
408   */
409  private static CellVisibility toCellVisibility(byte[] protoBytes)
410    throws DeserializationException {
411    if (protoBytes == null) return null;
412    ClientProtos.CellVisibility.Builder builder = ClientProtos.CellVisibility.newBuilder();
413    ClientProtos.CellVisibility proto = null;
414    try {
415      ProtobufUtil.mergeFrom(builder, protoBytes);
416      proto = builder.build();
417    } catch (IOException e) {
418      throw new DeserializationException(e);
419    }
420    return toCellVisibility(proto);
421  }
422
423  /**
424   * Number of KeyValues carried by this Mutation.
425   * @return the total number of KeyValues
426   */
427  public int size() {
428    int size = 0;
429    for (List<Cell> cells : getFamilyCellMap().values()) {
430      size += cells.size();
431    }
432    return size;
433  }
434
435  /** Returns the number of different families */
436  public int numFamilies() {
437    return getFamilyCellMap().size();
438  }
439
440  /** Returns Calculate what Mutation adds to class heap size. */
441  @Override
442  public long heapSize() {
443    long heapsize = MUTATION_OVERHEAD;
444    // Adding row
445    heapsize += ClassSize.align(ClassSize.ARRAY + this.row.length);
446
447    // Adding map overhead
448    heapsize += ClassSize.align(getFamilyCellMap().size() * ClassSize.MAP_ENTRY);
449    for (Map.Entry<byte[], List<Cell>> entry : getFamilyCellMap().entrySet()) {
450      // Adding key overhead
451      heapsize += ClassSize.align(ClassSize.ARRAY + entry.getKey().length);
452
453      // This part is kinds tricky since the JVM can reuse references if you
454      // store the same value, but have a good match with SizeOf at the moment
455      // Adding value overhead
456      heapsize += ClassSize.align(ClassSize.ARRAYLIST);
457      int size = entry.getValue().size();
458      heapsize += ClassSize.align(ClassSize.ARRAY + size * ClassSize.REFERENCE);
459
460      for (Cell cell : entry.getValue()) {
461        heapsize += cell.heapSize();
462      }
463    }
464    heapsize += getAttributeSize();
465    heapsize += extraHeapSize();
466    return ClassSize.align(heapsize);
467  }
468
469  /** Returns The serialized ACL for this operation, or null if none */
470  public byte[] getACL() {
471    return getAttribute(AccessControlConstants.OP_ATTRIBUTE_ACL);
472  }
473
474  /**
475   * Set the ACL for this operation.
476   * @param user  User short name
477   * @param perms Permissions for the user
478   */
479  public Mutation setACL(String user, Permission perms) {
480    setAttribute(AccessControlConstants.OP_ATTRIBUTE_ACL,
481      AccessControlUtil.toUsersAndPermissions(user, perms).toByteArray());
482    return this;
483  }
484
485  /**
486   * Set the ACL for this operation.
487   * @param perms A map of permissions for a user or users
488   */
489  public Mutation setACL(Map<String, Permission> perms) {
490    ListMultimap<String, Permission> permMap = ArrayListMultimap.create();
491    for (Map.Entry<String, Permission> entry : perms.entrySet()) {
492      permMap.put(entry.getKey(), entry.getValue());
493    }
494    setAttribute(AccessControlConstants.OP_ATTRIBUTE_ACL,
495      AccessControlUtil.toUsersAndPermissions(permMap).toByteArray());
496    return this;
497  }
498
499  /**
500   * Return the TTL requested for the result of the mutation, in milliseconds.
501   * @return the TTL requested for the result of the mutation, in milliseconds, or Long.MAX_VALUE if
502   *         unset
503   */
504  public long getTTL() {
505    byte[] ttlBytes = getAttribute(OP_ATTRIBUTE_TTL);
506    if (ttlBytes != null) {
507      return Bytes.toLong(ttlBytes);
508    }
509    return Long.MAX_VALUE;
510  }
511
512  /**
513   * Set the TTL desired for the result of the mutation, in milliseconds.
514   * @param ttl the TTL desired for the result of the mutation, in milliseconds n
515   */
516  public Mutation setTTL(long ttl) {
517    setAttribute(OP_ATTRIBUTE_TTL, Bytes.toBytes(ttl));
518    return this;
519  }
520
521  /** Returns current value for returnResults */
522  // Used by Increment and Append only.
523  @InterfaceAudience.Private
524  protected boolean isReturnResults() {
525    byte[] v = getAttribute(RETURN_RESULTS);
526    return v == null ? true : Bytes.toBoolean(v);
527  }
528
529  @InterfaceAudience.Private
530  // Used by Increment and Append only.
531  protected Mutation setReturnResults(boolean returnResults) {
532    setAttribute(RETURN_RESULTS, Bytes.toBytes(returnResults));
533    return this;
534  }
535
536  /**
537   * Subclasses should override this method to add the heap size of their own fields.
538   * @return the heap size to add (will be aligned).
539   */
540  protected long extraHeapSize() {
541    return 0L;
542  }
543
544  /**
545   * Set the timestamp of the delete.
546   */
547  public Mutation setTimestamp(long timestamp) {
548    if (timestamp < 0) {
549      throw new IllegalArgumentException("Timestamp cannot be negative. ts=" + timestamp);
550    }
551    this.ts = timestamp;
552    return this;
553  }
554
555  /**
556   * A convenience method to determine if this object's familyMap contains a value assigned to the
557   * given family &amp; qualifier. Both given arguments must match the KeyValue object to return
558   * true.
559   * @param family    column family
560   * @param qualifier column qualifier
561   * @return returns true if the given family and qualifier already has an existing KeyValue object
562   *         in the family map.
563   */
564  public boolean has(byte[] family, byte[] qualifier) {
565    return has(family, qualifier, this.ts, HConstants.EMPTY_BYTE_ARRAY, true, true);
566  }
567
568  /**
569   * A convenience method to determine if this object's familyMap contains a value assigned to the
570   * given family, qualifier and timestamp. All 3 given arguments must match the KeyValue object to
571   * return true.
572   * @param family    column family
573   * @param qualifier column qualifier
574   * @param ts        timestamp
575   * @return returns true if the given family, qualifier and timestamp already has an existing
576   *         KeyValue object in the family map.
577   */
578  public boolean has(byte[] family, byte[] qualifier, long ts) {
579    return has(family, qualifier, ts, HConstants.EMPTY_BYTE_ARRAY, false, true);
580  }
581
582  /**
583   * A convenience method to determine if this object's familyMap contains a value assigned to the
584   * given family, qualifier and timestamp. All 3 given arguments must match the KeyValue object to
585   * return true.
586   * @param family    column family
587   * @param qualifier column qualifier
588   * @param value     value to check
589   * @return returns true if the given family, qualifier and value already has an existing KeyValue
590   *         object in the family map.
591   */
592  public boolean has(byte[] family, byte[] qualifier, byte[] value) {
593    return has(family, qualifier, this.ts, value, true, false);
594  }
595
596  /**
597   * A convenience method to determine if this object's familyMap contains the given value assigned
598   * to the given family, qualifier and timestamp. All 4 given arguments must match the KeyValue
599   * object to return true.
600   * @param family    column family
601   * @param qualifier column qualifier
602   * @param ts        timestamp
603   * @param value     value to check
604   * @return returns true if the given family, qualifier timestamp and value already has an existing
605   *         KeyValue object in the family map.
606   */
607  public boolean has(byte[] family, byte[] qualifier, long ts, byte[] value) {
608    return has(family, qualifier, ts, value, false, false);
609  }
610
611  /**
612   * Returns a list of all KeyValue objects with matching column family and qualifier.
613   * @param family    column family
614   * @param qualifier column qualifier
615   * @return a list of KeyValue objects with the matching family and qualifier, returns an empty
616   *         list if one doesn't exist for the given family.
617   */
618  public List<Cell> get(byte[] family, byte[] qualifier) {
619    List<Cell> filteredList = new ArrayList<>();
620    for (Cell cell : getCellList(family)) {
621      if (CellUtil.matchingQualifier(cell, qualifier)) {
622        filteredList.add(cell);
623      }
624    }
625    return filteredList;
626  }
627
628  /**
629   * Private method to determine if this object's familyMap contains the given value assigned to the
630   * given family, qualifier and timestamp, respecting the 2 boolean arguments.
631   */
632  protected boolean has(byte[] family, byte[] qualifier, long ts, byte[] value, boolean ignoreTS,
633    boolean ignoreValue) {
634    List<Cell> list = getCellList(family);
635    if (list.isEmpty()) {
636      return false;
637    }
638    // Boolean analysis of ignoreTS/ignoreValue.
639    // T T => 2
640    // T F => 3 (first is always true)
641    // F T => 2
642    // F F => 1
643    if (!ignoreTS && !ignoreValue) {
644      for (Cell cell : list) {
645        if (
646          CellUtil.matchingFamily(cell, family) && CellUtil.matchingQualifier(cell, qualifier)
647            && CellUtil.matchingValue(cell, value) && cell.getTimestamp() == ts
648        ) {
649          return true;
650        }
651      }
652    } else if (ignoreValue && !ignoreTS) {
653      for (Cell cell : list) {
654        if (
655          CellUtil.matchingFamily(cell, family) && CellUtil.matchingQualifier(cell, qualifier)
656            && cell.getTimestamp() == ts
657        ) {
658          return true;
659        }
660      }
661    } else if (!ignoreValue && ignoreTS) {
662      for (Cell cell : list) {
663        if (
664          CellUtil.matchingFamily(cell, family) && CellUtil.matchingQualifier(cell, qualifier)
665            && CellUtil.matchingValue(cell, value)
666        ) {
667          return true;
668        }
669      }
670    } else {
671      for (Cell cell : list) {
672        if (CellUtil.matchingFamily(cell, family) && CellUtil.matchingQualifier(cell, qualifier)) {
673          return true;
674        }
675      }
676    }
677    return false;
678  }
679
680  /**
681   * @param row Row to check
682   * @throws IllegalArgumentException Thrown if <code>row</code> is empty or null or &gt;
683   *                                  {@link HConstants#MAX_ROW_LENGTH}
684   * @return <code>row</code>
685   */
686  static byte[] checkRow(final byte[] row) {
687    return checkRow(row, 0, row == null ? 0 : row.length);
688  }
689
690  /**
691   * @param row Row to check nn * @throws IllegalArgumentException Thrown if <code>row</code> is
692   *            empty or null or &gt; {@link HConstants#MAX_ROW_LENGTH}
693   * @return <code>row</code>
694   */
695  static byte[] checkRow(final byte[] row, final int offset, final int length) {
696    if (row == null) {
697      throw new IllegalArgumentException("Row buffer is null");
698    }
699    if (length == 0) {
700      throw new IllegalArgumentException("Row length is 0");
701    }
702    if (length > HConstants.MAX_ROW_LENGTH) {
703      throw new IllegalArgumentException(
704        "Row length " + length + " is > " + HConstants.MAX_ROW_LENGTH);
705    }
706    return row;
707  }
708
709  static void checkRow(ByteBuffer row) {
710    if (row == null) {
711      throw new IllegalArgumentException("Row buffer is null");
712    }
713    if (row.remaining() == 0) {
714      throw new IllegalArgumentException("Row length is 0");
715    }
716    if (row.remaining() > HConstants.MAX_ROW_LENGTH) {
717      throw new IllegalArgumentException(
718        "Row length " + row.remaining() + " is > " + HConstants.MAX_ROW_LENGTH);
719    }
720  }
721
722  Mutation add(Cell cell) throws IOException {
723    // Checking that the row of the kv is the same as the mutation
724    // TODO: It is fraught with risk if user pass the wrong row.
725    // Throwing the IllegalArgumentException is more suitable I'd say.
726    if (!CellUtil.matchingRows(cell, this.row)) {
727      throw new WrongRowIOException("The row in " + cell.toString()
728        + " doesn't match the original one " + Bytes.toStringBinary(this.row));
729    }
730
731    byte[] family;
732
733    if (cell instanceof IndividualBytesFieldCell) {
734      family = cell.getFamilyArray();
735    } else {
736      family = CellUtil.cloneFamily(cell);
737    }
738
739    if (family == null || family.length == 0) {
740      throw new IllegalArgumentException("Family cannot be null");
741    }
742
743    if (cell instanceof ExtendedCell) {
744      getCellList(family).add(cell);
745    } else {
746      getCellList(family).add(new CellWrapper(cell));
747    }
748    return this;
749  }
750
751  private static final class CellWrapper implements ExtendedCell {
752    private static final long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT // object header
753      + KeyValue.TIMESTAMP_SIZE // timestamp
754      + Bytes.SIZEOF_LONG // sequence id
755      + 1 * ClassSize.REFERENCE); // references to cell
756    private final Cell cell;
757    private long sequenceId;
758    private long timestamp;
759
760    CellWrapper(Cell cell) {
761      assert !(cell instanceof ExtendedCell);
762      this.cell = cell;
763      this.sequenceId = cell.getSequenceId();
764      this.timestamp = cell.getTimestamp();
765    }
766
767    @Override
768    public void setSequenceId(long seqId) {
769      sequenceId = seqId;
770    }
771
772    @Override
773    public void setTimestamp(long ts) {
774      timestamp = ts;
775    }
776
777    @Override
778    public void setTimestamp(byte[] ts) {
779      timestamp = Bytes.toLong(ts);
780    }
781
782    @Override
783    public long getSequenceId() {
784      return sequenceId;
785    }
786
787    @Override
788    public byte[] getValueArray() {
789      return cell.getValueArray();
790    }
791
792    @Override
793    public int getValueOffset() {
794      return cell.getValueOffset();
795    }
796
797    @Override
798    public int getValueLength() {
799      return cell.getValueLength();
800    }
801
802    @Override
803    public byte[] getTagsArray() {
804      return cell.getTagsArray();
805    }
806
807    @Override
808    public int getTagsOffset() {
809      return cell.getTagsOffset();
810    }
811
812    @Override
813    public int getTagsLength() {
814      return cell.getTagsLength();
815    }
816
817    @Override
818    public byte[] getRowArray() {
819      return cell.getRowArray();
820    }
821
822    @Override
823    public int getRowOffset() {
824      return cell.getRowOffset();
825    }
826
827    @Override
828    public short getRowLength() {
829      return cell.getRowLength();
830    }
831
832    @Override
833    public byte[] getFamilyArray() {
834      return cell.getFamilyArray();
835    }
836
837    @Override
838    public int getFamilyOffset() {
839      return cell.getFamilyOffset();
840    }
841
842    @Override
843    public byte getFamilyLength() {
844      return cell.getFamilyLength();
845    }
846
847    @Override
848    public byte[] getQualifierArray() {
849      return cell.getQualifierArray();
850    }
851
852    @Override
853    public int getQualifierOffset() {
854      return cell.getQualifierOffset();
855    }
856
857    @Override
858    public int getQualifierLength() {
859      return cell.getQualifierLength();
860    }
861
862    @Override
863    public long getTimestamp() {
864      return timestamp;
865    }
866
867    @Override
868    public byte getTypeByte() {
869      return cell.getTypeByte();
870    }
871
872    @Override
873    public Optional<Tag> getTag(byte type) {
874      return PrivateCellUtil.getTag(cell, type);
875    }
876
877    @Override
878    public Iterator<Tag> getTags() {
879      return PrivateCellUtil.tagsIterator(cell);
880    }
881
882    @Override
883    public byte[] cloneTags() {
884      return PrivateCellUtil.cloneTags(cell);
885    }
886
887    private long heapOverhead() {
888      return FIXED_OVERHEAD + ClassSize.ARRAY // row
889        + getFamilyLength() == 0
890        ? 0
891        : ClassSize.ARRAY + getQualifierLength() == 0 ? 0
892        : ClassSize.ARRAY + getValueLength() == 0 ? 0
893        : ClassSize.ARRAY + getTagsLength() == 0 ? 0
894        : ClassSize.ARRAY;
895    }
896
897    @Override
898    public long heapSize() {
899      return heapOverhead() + ClassSize.align(getRowLength()) + ClassSize.align(getFamilyLength())
900        + ClassSize.align(getQualifierLength()) + ClassSize.align(getValueLength())
901        + ClassSize.align(getTagsLength());
902    }
903  }
904}