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