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.util.List;
022import java.util.Map;
023import java.util.NavigableMap;
024import java.util.TreeMap;
025import java.util.UUID;
026import org.apache.hadoop.hbase.Cell;
027import org.apache.hadoop.hbase.CellBuilder;
028import org.apache.hadoop.hbase.CellBuilderType;
029import org.apache.hadoop.hbase.CellUtil;
030import org.apache.hadoop.hbase.ExtendedCell;
031import org.apache.hadoop.hbase.KeyValue;
032import org.apache.hadoop.hbase.io.TimeRange;
033import org.apache.hadoop.hbase.security.access.Permission;
034import org.apache.hadoop.hbase.security.visibility.CellVisibility;
035import org.apache.hadoop.hbase.util.Bytes;
036import org.apache.hadoop.hbase.util.ClassSize;
037import org.apache.yetus.audience.InterfaceAudience;
038
039import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
040
041/**
042 * Used to perform Increment operations on a single row.
043 * <p>
044 * This operation ensures atomicity to readers. Increments are done under a single row lock, so
045 * write operations to a row are synchronized, and readers are guaranteed to see this operation
046 * fully completed.
047 * <p>
048 * To increment columns of a row, instantiate an Increment object with the row to increment. At
049 * least one column to increment must be specified using the
050 * {@link #addColumn(byte[], byte[], long)} method.
051 */
052@InterfaceAudience.Public
053public class Increment extends Mutation {
054  private static final int HEAP_OVERHEAD = ClassSize.REFERENCE + ClassSize.TIMERANGE;
055  private TimeRange tr = TimeRange.allTime();
056
057  /**
058   * Create a Increment operation for the specified row.
059   * <p>
060   * At least one column must be incremented.
061   * @param row row key (we will make a copy of this).
062   */
063  public Increment(byte[] row) {
064    this(row, 0, row.length);
065  }
066
067  /**
068   * Create a Increment operation for the specified row.
069   * <p>
070   * At least one column must be incremented.
071   * @param row row key (we will make a copy of this).
072   */
073  public Increment(final byte[] row, final int offset, final int length) {
074    checkRow(row, offset, length);
075    this.row = Bytes.copy(row, offset, length);
076  }
077
078  /**
079   * Copy constructor
080   * @param incrementToCopy increment to copy
081   */
082  public Increment(Increment incrementToCopy) {
083    super(incrementToCopy);
084    this.tr = incrementToCopy.getTimeRange();
085  }
086
087  /**
088   * Construct the Increment with user defined data. NOTED: 1) all cells in the familyMap must have
089   * the Type.Put 2) the row of each cell must be same with passed row.
090   * @param row       row. CAN'T be null
091   * @param ts        timestamp
092   * @param familyMap the map to collect all cells internally. CAN'T be null
093   */
094  public Increment(byte[] row, long ts, NavigableMap<byte[], List<Cell>> familyMap) {
095    super(row, ts, familyMap);
096  }
097
098  /**
099   * Add the specified KeyValue to this operation.
100   * @param cell individual Cell
101   * @throws java.io.IOException e
102   */
103  @Override
104  public Increment add(Cell cell) throws IOException {
105    super.add(cell);
106    return this;
107  }
108
109  /**
110   * Increment the column from the specific family with the specified qualifier by the specified
111   * amount.
112   * <p>
113   * Overrides previous calls to addColumn for this family and qualifier.
114   * @param family    family name
115   * @param qualifier column qualifier
116   * @param amount    amount to increment by
117   * @return the Increment object
118   */
119  public Increment addColumn(byte[] family, byte[] qualifier, long amount) {
120    Preconditions.checkArgument(family != null, "family cannot be null");
121    List<ExtendedCell> list = getCellList(family);
122    KeyValue kv = createPutKeyValue(family, qualifier, ts, Bytes.toBytes(amount));
123    list.add(kv);
124    return this;
125  }
126
127  /**
128   * Gets the TimeRange used for this increment.
129   */
130  public TimeRange getTimeRange() {
131    return this.tr;
132  }
133
134  /**
135   * Sets the TimeRange to be used on the Get for this increment.
136   * <p>
137   * This is useful for when you have counters that only last for specific periods of time (ie.
138   * counters that are partitioned by time). By setting the range of valid times for this increment,
139   * you can potentially gain some performance with a more optimal Get operation. Be careful adding
140   * the time range to this class as you will update the old cell if the time range doesn't include
141   * the latest cells.
142   * <p>
143   * This range is used as [minStamp, maxStamp).
144   * @param minStamp minimum timestamp value, inclusive
145   * @param maxStamp maximum timestamp value, exclusive
146   * @throws IOException if invalid time range
147   */
148  public Increment setTimeRange(long minStamp, long maxStamp) throws IOException {
149    tr = TimeRange.between(minStamp, maxStamp);
150    return this;
151  }
152
153  @Override
154  public Increment setTimestamp(long timestamp) {
155    super.setTimestamp(timestamp);
156    return this;
157  }
158
159  /**
160   * @param returnResults True (default) if the increment operation should return the results. A
161   *                      client that is not interested in the result can save network bandwidth
162   *                      setting this to false.
163   */
164  @Override
165  public Increment setReturnResults(boolean returnResults) {
166    super.setReturnResults(returnResults);
167    return this;
168  }
169
170  /** Returns current setting for returnResults */
171  // This method makes public the superclasses's protected method.
172  @Override
173  public boolean isReturnResults() {
174    return super.isReturnResults();
175  }
176
177  /**
178   * Method for retrieving the number of families to increment from
179   * @return number of families
180   */
181  @Override
182  public int numFamilies() {
183    return this.familyMap.size();
184  }
185
186  /**
187   * Method for checking if any families have been inserted into this Increment
188   * @return true if familyMap is non empty false otherwise
189   */
190  public boolean hasFamilies() {
191    return !this.familyMap.isEmpty();
192  }
193
194  /**
195   * Before 0.95, when you called Increment#getFamilyMap(), you got back a map of families to a list
196   * of Longs. Now, {@link #getFamilyCellMap()} returns families by list of Cells. This method has
197   * been added so you can have the old behavior.
198   * @return Map of families to a Map of qualifiers and their Long increments.
199   * @since 0.95.0
200   */
201  public Map<byte[], NavigableMap<byte[], Long>> getFamilyMapOfLongs() {
202    NavigableMap<byte[], List<Cell>> map = super.getFamilyCellMap();
203    Map<byte[], NavigableMap<byte[], Long>> results = new TreeMap<>(Bytes.BYTES_COMPARATOR);
204    for (Map.Entry<byte[], List<Cell>> entry : map.entrySet()) {
205      NavigableMap<byte[], Long> longs = new TreeMap<>(Bytes.BYTES_COMPARATOR);
206      for (Cell cell : entry.getValue()) {
207        longs.put(CellUtil.cloneQualifier(cell),
208          Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
209      }
210      results.put(entry.getKey(), longs);
211    }
212    return results;
213  }
214
215  /**
216   *   */
217  @Override
218  public String toString() {
219    StringBuilder sb = new StringBuilder();
220    sb.append("row=");
221    sb.append(Bytes.toStringBinary(this.row));
222    if (this.familyMap.isEmpty()) {
223      sb.append(", no columns set to be incremented");
224      return sb.toString();
225    }
226    sb.append(", families=");
227    boolean moreThanOne = false;
228    for (Map.Entry<byte[], List<ExtendedCell>> entry : this.familyMap.entrySet()) {
229      if (moreThanOne) {
230        sb.append("), ");
231      } else {
232        moreThanOne = true;
233        sb.append("{");
234      }
235      sb.append("(family=");
236      sb.append(Bytes.toString(entry.getKey()));
237      sb.append(", columns=");
238      if (entry.getValue() == null) {
239        sb.append("NONE");
240      } else {
241        sb.append("{");
242        boolean moreThanOneB = false;
243        for (Cell cell : entry.getValue()) {
244          if (moreThanOneB) {
245            sb.append(", ");
246          } else {
247            moreThanOneB = true;
248          }
249          sb.append(CellUtil.getCellKeyAsString(cell) + "+="
250            + Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
251        }
252        sb.append("}");
253      }
254    }
255    sb.append("}");
256    return sb.toString();
257  }
258
259  @Override
260  protected long extraHeapSize() {
261    return HEAP_OVERHEAD;
262  }
263
264  @Override
265  public Increment setAttribute(String name, byte[] value) {
266    return (Increment) super.setAttribute(name, value);
267  }
268
269  @Override
270  public Increment setId(String id) {
271    return (Increment) super.setId(id);
272  }
273
274  @Override
275  public Increment setDurability(Durability d) {
276    return (Increment) super.setDurability(d);
277  }
278
279  @Override
280  public Increment setClusterIds(List<UUID> clusterIds) {
281    return (Increment) super.setClusterIds(clusterIds);
282  }
283
284  @Override
285  public Increment setCellVisibility(CellVisibility expression) {
286    return (Increment) super.setCellVisibility(expression);
287  }
288
289  @Override
290  public Increment setACL(String user, Permission perms) {
291    return (Increment) super.setACL(user, perms);
292  }
293
294  @Override
295  public Increment setACL(Map<String, Permission> perms) {
296    return (Increment) super.setACL(perms);
297  }
298
299  @Override
300  public Increment setTTL(long ttl) {
301    return (Increment) super.setTTL(ttl);
302  }
303
304  @Override
305  public Increment setPriority(int priority) {
306    return (Increment) super.setPriority(priority);
307  }
308
309  @Override
310  public CellBuilder getCellBuilder(CellBuilderType type) {
311    return getCellBuilder(type, Cell.Type.Put);
312  }
313}