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
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
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
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
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
327   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Use
328   *             {@link #getTimestamp()} instead
329   */
330  @Deprecated
331  public long getTimeStamp() {
332    return this.getTimestamp();
333  }
334
335  /**
336   * Method for retrieving the timestamp.
337   */
338  public long getTimestamp() {
339    return this.ts;
340  }
341
342  /**
343   * Marks that the clusters with the given clusterIds have consumed the mutation
344   * @param clusterIds of the clusters that have consumed the mutation
345   */
346  public Mutation setClusterIds(List<UUID> clusterIds) {
347    ByteArrayDataOutput out = ByteStreams.newDataOutput();
348    out.writeInt(clusterIds.size());
349    for (UUID clusterId : clusterIds) {
350      out.writeLong(clusterId.getMostSignificantBits());
351      out.writeLong(clusterId.getLeastSignificantBits());
352    }
353    setAttribute(CONSUMED_CLUSTER_IDS, out.toByteArray());
354    return this;
355  }
356
357  /** Returns the set of clusterIds that have consumed the mutation */
358  public List<UUID> getClusterIds() {
359    List<UUID> clusterIds = new ArrayList<>();
360    byte[] bytes = getAttribute(CONSUMED_CLUSTER_IDS);
361    if (bytes != null) {
362      ByteArrayDataInput in = ByteStreams.newDataInput(bytes);
363      int numClusters = in.readInt();
364      for (int i = 0; i < numClusters; i++) {
365        clusterIds.add(new UUID(in.readLong(), in.readLong()));
366      }
367    }
368    return clusterIds;
369  }
370
371  /**
372   * Sets the visibility expression associated with cells in this Mutation.
373   */
374  public Mutation setCellVisibility(CellVisibility expression) {
375    this.setAttribute(VisibilityConstants.VISIBILITY_LABELS_ATTR_KEY,
376      toCellVisibility(expression).toByteArray());
377    return this;
378  }
379
380  /** Returns CellVisibility associated with cells in this Mutation. n */
381  public CellVisibility getCellVisibility() throws DeserializationException {
382    byte[] cellVisibilityBytes = this.getAttribute(VisibilityConstants.VISIBILITY_LABELS_ATTR_KEY);
383    if (cellVisibilityBytes == null) return null;
384    return toCellVisibility(cellVisibilityBytes);
385  }
386
387  /**
388   * Create a protocol buffer CellVisibility based on a client CellVisibility.
389   * @return a protocol buffer CellVisibility
390   */
391  static ClientProtos.CellVisibility toCellVisibility(CellVisibility cellVisibility) {
392    ClientProtos.CellVisibility.Builder builder = ClientProtos.CellVisibility.newBuilder();
393    builder.setExpression(cellVisibility.getExpression());
394    return builder.build();
395  }
396
397  /**
398   * Convert a protocol buffer CellVisibility to a client CellVisibility
399   * @return the converted client CellVisibility
400   */
401  private static CellVisibility toCellVisibility(ClientProtos.CellVisibility proto) {
402    if (proto == null) return null;
403    return new CellVisibility(proto.getExpression());
404  }
405
406  /**
407   * Convert a protocol buffer CellVisibility bytes to a client CellVisibility
408   * @return the converted client CellVisibility
409   */
410  private static CellVisibility toCellVisibility(byte[] protoBytes)
411    throws DeserializationException {
412    if (protoBytes == null) return null;
413    ClientProtos.CellVisibility.Builder builder = ClientProtos.CellVisibility.newBuilder();
414    ClientProtos.CellVisibility proto = null;
415    try {
416      ProtobufUtil.mergeFrom(builder, protoBytes);
417      proto = builder.build();
418    } catch (IOException e) {
419      throw new DeserializationException(e);
420    }
421    return toCellVisibility(proto);
422  }
423
424  /**
425   * Number of KeyValues carried by this Mutation.
426   * @return the total number of KeyValues
427   */
428  public int size() {
429    int size = 0;
430    for (List<Cell> cells : getFamilyCellMap().values()) {
431      size += cells.size();
432    }
433    return size;
434  }
435
436  /** Returns the number of different families */
437  public int numFamilies() {
438    return getFamilyCellMap().size();
439  }
440
441  /** Returns Calculate what Mutation adds to class heap size. */
442  @Override
443  public long heapSize() {
444    long heapsize = MUTATION_OVERHEAD;
445    // Adding row
446    heapsize += ClassSize.align(ClassSize.ARRAY + this.row.length);
447
448    // Adding map overhead
449    heapsize += ClassSize.align(getFamilyCellMap().size() * ClassSize.MAP_ENTRY);
450    for (Map.Entry<byte[], List<Cell>> entry : getFamilyCellMap().entrySet()) {
451      // Adding key overhead
452      heapsize += ClassSize.align(ClassSize.ARRAY + entry.getKey().length);
453
454      // This part is kinds tricky since the JVM can reuse references if you
455      // store the same value, but have a good match with SizeOf at the moment
456      // Adding value overhead
457      heapsize += ClassSize.align(ClassSize.ARRAYLIST);
458      int size = entry.getValue().size();
459      heapsize += ClassSize.align(ClassSize.ARRAY + size * ClassSize.REFERENCE);
460
461      for (Cell cell : entry.getValue()) {
462        heapsize += cell.heapSize();
463      }
464    }
465    heapsize += getAttributeSize();
466    heapsize += extraHeapSize();
467    return ClassSize.align(heapsize);
468  }
469
470  /** Returns The serialized ACL for this operation, or null if none */
471  public byte[] getACL() {
472    return getAttribute(AccessControlConstants.OP_ATTRIBUTE_ACL);
473  }
474
475  /**
476   * Set the ACL for this operation.
477   * @param user  User short name
478   * @param perms Permissions for the user
479   */
480  public Mutation setACL(String user, Permission perms) {
481    setAttribute(AccessControlConstants.OP_ATTRIBUTE_ACL,
482      AccessControlUtil.toUsersAndPermissions(user, perms).toByteArray());
483    return this;
484  }
485
486  /**
487   * Set the ACL for this operation.
488   * @param perms A map of permissions for a user or users
489   */
490  public Mutation setACL(Map<String, Permission> perms) {
491    ListMultimap<String, Permission> permMap = ArrayListMultimap.create();
492    for (Map.Entry<String, Permission> entry : perms.entrySet()) {
493      permMap.put(entry.getKey(), entry.getValue());
494    }
495    setAttribute(AccessControlConstants.OP_ATTRIBUTE_ACL,
496      AccessControlUtil.toUsersAndPermissions(permMap).toByteArray());
497    return this;
498  }
499
500  /**
501   * Return the TTL requested for the result of the mutation, in milliseconds.
502   * @return the TTL requested for the result of the mutation, in milliseconds, or Long.MAX_VALUE if
503   *         unset
504   */
505  public long getTTL() {
506    byte[] ttlBytes = getAttribute(OP_ATTRIBUTE_TTL);
507    if (ttlBytes != null) {
508      return Bytes.toLong(ttlBytes);
509    }
510    return Long.MAX_VALUE;
511  }
512
513  /**
514   * Set the TTL desired for the result of the mutation, in milliseconds.
515   * @param ttl the TTL desired for the result of the mutation, in milliseconds
516   */
517  public Mutation setTTL(long ttl) {
518    setAttribute(OP_ATTRIBUTE_TTL, Bytes.toBytes(ttl));
519    return this;
520  }
521
522  /** Returns current value for returnResults */
523  // Used by Increment and Append only.
524  @InterfaceAudience.Private
525  protected boolean isReturnResults() {
526    byte[] v = getAttribute(RETURN_RESULTS);
527    return v == null ? true : Bytes.toBoolean(v);
528  }
529
530  @InterfaceAudience.Private
531  // Used by Increment and Append only.
532  protected Mutation setReturnResults(boolean returnResults) {
533    setAttribute(RETURN_RESULTS, Bytes.toBytes(returnResults));
534    return this;
535  }
536
537  /**
538   * Subclasses should override this method to add the heap size of their own fields.
539   * @return the heap size to add (will be aligned).
540   */
541  protected long extraHeapSize() {
542    return 0L;
543  }
544
545  /**
546   * Set the timestamp of the delete.
547   */
548  public Mutation setTimestamp(long timestamp) {
549    if (timestamp < 0) {
550      throw new IllegalArgumentException("Timestamp cannot be negative. ts=" + timestamp);
551    }
552    this.ts = timestamp;
553    return this;
554  }
555
556  /**
557   * A convenience method to determine if this object's familyMap contains a value assigned to the
558   * given family &amp; qualifier. Both given arguments must match the KeyValue object to return
559   * true.
560   * @param family    column family
561   * @param qualifier column qualifier
562   * @return returns true if the given family and qualifier already has an existing KeyValue object
563   *         in the family map.
564   */
565  public boolean has(byte[] family, byte[] qualifier) {
566    return has(family, qualifier, this.ts, HConstants.EMPTY_BYTE_ARRAY, true, true);
567  }
568
569  /**
570   * A convenience method to determine if this object's familyMap contains a value assigned to the
571   * given family, qualifier and timestamp. All 3 given arguments must match the KeyValue object to
572   * return true.
573   * @param family    column family
574   * @param qualifier column qualifier
575   * @param ts        timestamp
576   * @return returns true if the given family, qualifier and timestamp already has an existing
577   *         KeyValue object in the family map.
578   */
579  public boolean has(byte[] family, byte[] qualifier, long ts) {
580    return has(family, qualifier, ts, HConstants.EMPTY_BYTE_ARRAY, false, true);
581  }
582
583  /**
584   * A convenience method to determine if this object's familyMap contains a value assigned to the
585   * given family, qualifier and timestamp. All 3 given arguments must match the KeyValue object to
586   * return true.
587   * @param family    column family
588   * @param qualifier column qualifier
589   * @param value     value to check
590   * @return returns true if the given family, qualifier and value already has an existing KeyValue
591   *         object in the family map.
592   */
593  public boolean has(byte[] family, byte[] qualifier, byte[] value) {
594    return has(family, qualifier, this.ts, value, true, false);
595  }
596
597  /**
598   * A convenience method to determine if this object's familyMap contains the given value assigned
599   * to the given family, qualifier and timestamp. All 4 given arguments must match the KeyValue
600   * object to return true.
601   * @param family    column family
602   * @param qualifier column qualifier
603   * @param ts        timestamp
604   * @param value     value to check
605   * @return returns true if the given family, qualifier timestamp and value already has an existing
606   *         KeyValue object in the family map.
607   */
608  public boolean has(byte[] family, byte[] qualifier, long ts, byte[] value) {
609    return has(family, qualifier, ts, value, false, false);
610  }
611
612  /**
613   * Returns a list of all KeyValue objects with matching column family and qualifier.
614   * @param family    column family
615   * @param qualifier column qualifier
616   * @return a list of KeyValue objects with the matching family and qualifier, returns an empty
617   *         list if one doesn't exist for the given family.
618   */
619  public List<Cell> get(byte[] family, byte[] qualifier) {
620    List<Cell> filteredList = new ArrayList<>();
621    for (Cell cell : getCellList(family)) {
622      if (CellUtil.matchingQualifier(cell, qualifier)) {
623        filteredList.add(cell);
624      }
625    }
626    return filteredList;
627  }
628
629  /**
630   * Private method to determine if this object's familyMap contains the given value assigned to the
631   * given family, qualifier and timestamp, respecting the 2 boolean arguments.
632   */
633  protected boolean has(byte[] family, byte[] qualifier, long ts, byte[] value, boolean ignoreTS,
634    boolean ignoreValue) {
635    List<Cell> list = getCellList(family);
636    if (list.isEmpty()) {
637      return false;
638    }
639    // Boolean analysis of ignoreTS/ignoreValue.
640    // T T => 2
641    // T F => 3 (first is always true)
642    // F T => 2
643    // F F => 1
644    if (!ignoreTS && !ignoreValue) {
645      for (Cell cell : list) {
646        if (
647          CellUtil.matchingFamily(cell, family) && CellUtil.matchingQualifier(cell, qualifier)
648            && CellUtil.matchingValue(cell, value) && cell.getTimestamp() == ts
649        ) {
650          return true;
651        }
652      }
653    } else if (ignoreValue && !ignoreTS) {
654      for (Cell cell : list) {
655        if (
656          CellUtil.matchingFamily(cell, family) && CellUtil.matchingQualifier(cell, qualifier)
657            && cell.getTimestamp() == ts
658        ) {
659          return true;
660        }
661      }
662    } else if (!ignoreValue && ignoreTS) {
663      for (Cell cell : list) {
664        if (
665          CellUtil.matchingFamily(cell, family) && CellUtil.matchingQualifier(cell, qualifier)
666            && CellUtil.matchingValue(cell, value)
667        ) {
668          return true;
669        }
670      }
671    } else {
672      for (Cell cell : list) {
673        if (CellUtil.matchingFamily(cell, family) && CellUtil.matchingQualifier(cell, qualifier)) {
674          return true;
675        }
676      }
677    }
678    return false;
679  }
680
681  /**
682   * @param row Row to check
683   * @throws IllegalArgumentException Thrown if <code>row</code> is empty or null or &gt;
684   *                                  {@link HConstants#MAX_ROW_LENGTH}
685   * @return <code>row</code>
686   */
687  static byte[] checkRow(final byte[] row) {
688    return checkRow(row, 0, row == null ? 0 : row.length);
689  }
690
691  /**
692   * @param row Row to check
693   * @throws IllegalArgumentException Thrown if <code>row</code> is empty or null or &gt;
694   *                                  {@link HConstants#MAX_ROW_LENGTH}
695   * @return <code>row</code>
696   */
697  static byte[] checkRow(final byte[] row, final int offset, final int length) {
698    if (row == null) {
699      throw new IllegalArgumentException("Row buffer is null");
700    }
701    if (length == 0) {
702      throw new IllegalArgumentException("Row length is 0");
703    }
704    if (length > HConstants.MAX_ROW_LENGTH) {
705      throw new IllegalArgumentException(
706        "Row length " + length + " is > " + HConstants.MAX_ROW_LENGTH);
707    }
708    return row;
709  }
710
711  static void checkRow(ByteBuffer row) {
712    if (row == null) {
713      throw new IllegalArgumentException("Row buffer is null");
714    }
715    if (row.remaining() == 0) {
716      throw new IllegalArgumentException("Row length is 0");
717    }
718    if (row.remaining() > HConstants.MAX_ROW_LENGTH) {
719      throw new IllegalArgumentException(
720        "Row length " + row.remaining() + " is > " + HConstants.MAX_ROW_LENGTH);
721    }
722  }
723
724  Mutation add(Cell cell) throws IOException {
725    // Checking that the row of the kv is the same as the mutation
726    // TODO: It is fraught with risk if user pass the wrong row.
727    // Throwing the IllegalArgumentException is more suitable I'd say.
728    if (!CellUtil.matchingRows(cell, this.row)) {
729      throw new WrongRowIOException("The row in " + cell.toString()
730        + " doesn't match the original one " + Bytes.toStringBinary(this.row));
731    }
732
733    byte[] family;
734
735    if (cell instanceof IndividualBytesFieldCell) {
736      family = cell.getFamilyArray();
737    } else {
738      family = CellUtil.cloneFamily(cell);
739    }
740
741    if (family == null || family.length == 0) {
742      throw new IllegalArgumentException("Family cannot be null");
743    }
744
745    if (cell instanceof ExtendedCell) {
746      getCellList(family).add(cell);
747    } else {
748      getCellList(family).add(new CellWrapper(cell));
749    }
750    return this;
751  }
752
753  private static final class CellWrapper implements ExtendedCell {
754    private static final long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT // object header
755      + KeyValue.TIMESTAMP_SIZE // timestamp
756      + Bytes.SIZEOF_LONG // sequence id
757      + 1 * ClassSize.REFERENCE); // references to cell
758    private final Cell cell;
759    private long sequenceId;
760    private long timestamp;
761
762    CellWrapper(Cell cell) {
763      assert !(cell instanceof ExtendedCell);
764      this.cell = cell;
765      this.sequenceId = cell.getSequenceId();
766      this.timestamp = cell.getTimestamp();
767    }
768
769    @Override
770    public void setSequenceId(long seqId) {
771      sequenceId = seqId;
772    }
773
774    @Override
775    public void setTimestamp(long ts) {
776      timestamp = ts;
777    }
778
779    @Override
780    public void setTimestamp(byte[] ts) {
781      timestamp = Bytes.toLong(ts);
782    }
783
784    @Override
785    public long getSequenceId() {
786      return sequenceId;
787    }
788
789    @Override
790    public byte[] getValueArray() {
791      return cell.getValueArray();
792    }
793
794    @Override
795    public int getValueOffset() {
796      return cell.getValueOffset();
797    }
798
799    @Override
800    public int getValueLength() {
801      return cell.getValueLength();
802    }
803
804    @Override
805    public byte[] getTagsArray() {
806      return cell.getTagsArray();
807    }
808
809    @Override
810    public int getTagsOffset() {
811      return cell.getTagsOffset();
812    }
813
814    @Override
815    public int getTagsLength() {
816      return cell.getTagsLength();
817    }
818
819    @Override
820    public byte[] getRowArray() {
821      return cell.getRowArray();
822    }
823
824    @Override
825    public int getRowOffset() {
826      return cell.getRowOffset();
827    }
828
829    @Override
830    public short getRowLength() {
831      return cell.getRowLength();
832    }
833
834    @Override
835    public byte[] getFamilyArray() {
836      return cell.getFamilyArray();
837    }
838
839    @Override
840    public int getFamilyOffset() {
841      return cell.getFamilyOffset();
842    }
843
844    @Override
845    public byte getFamilyLength() {
846      return cell.getFamilyLength();
847    }
848
849    @Override
850    public byte[] getQualifierArray() {
851      return cell.getQualifierArray();
852    }
853
854    @Override
855    public int getQualifierOffset() {
856      return cell.getQualifierOffset();
857    }
858
859    @Override
860    public int getQualifierLength() {
861      return cell.getQualifierLength();
862    }
863
864    @Override
865    public long getTimestamp() {
866      return timestamp;
867    }
868
869    @Override
870    public byte getTypeByte() {
871      return cell.getTypeByte();
872    }
873
874    @Override
875    public Optional<Tag> getTag(byte type) {
876      return PrivateCellUtil.getTag(cell, type);
877    }
878
879    @Override
880    public Iterator<Tag> getTags() {
881      return PrivateCellUtil.tagsIterator(cell);
882    }
883
884    @Override
885    public byte[] cloneTags() {
886      return PrivateCellUtil.cloneTags(cell);
887    }
888
889    private long heapOverhead() {
890      return FIXED_OVERHEAD + ClassSize.ARRAY // row
891        + getFamilyLength() == 0
892        ? 0
893        : ClassSize.ARRAY + getQualifierLength() == 0 ? 0
894        : ClassSize.ARRAY + getValueLength() == 0 ? 0
895        : ClassSize.ARRAY + getTagsLength() == 0 ? 0
896        : ClassSize.ARRAY;
897    }
898
899    @Override
900    public long heapSize() {
901      return heapOverhead() + ClassSize.align(getRowLength()) + ClassSize.align(getFamilyLength())
902        + ClassSize.align(getQualifierLength()) + ClassSize.align(getValueLength())
903        + ClassSize.align(getTagsLength());
904    }
905  }
906}