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