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