/** * Increment the counter for the given family and column by the specified amount * * <p>If the family and column already exist in the Increment the counter value is incremented by * the specified amount rather than overridden, as it is in HBase's {@link * Increment#addColumn(byte[], byte[], long)} method * * @param inc The {@link Increment} to update * @param family The column family * @param qualifier The column qualifier * @param amount The amount to increment the counter by */ public static void addIncrement( Increment inc, final byte[] family, final byte[] qualifier, final Long amount) { NavigableMap<byte[], Long> set = inc.getFamilyMap().get(family); if (set == null) { set = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR); } // If qualifier exists, increment amount Long counter = set.get(qualifier); if (counter == null) { counter = 0L; } set.put(qualifier, amount + counter); inc.getFamilyMap().put(family, set); }
/** * Use PreIncrement hook of BaseRegionObserver to overcome deficiencies in Increment * implementation (HBASE-10254): 1) Lack of recognition and identification of when the key value * to increment doesn't exist 2) Lack of the ability to set the timestamp of the updated key * value. Works the same as existing region.increment(), except assumes there is a single column * to increment and uses Phoenix LONG encoding. * * @author jtaylor * @since 3.0.0 */ @Override public Result preIncrement( final ObserverContext<RegionCoprocessorEnvironment> e, final Increment increment) throws IOException { RegionCoprocessorEnvironment env = e.getEnvironment(); // We need to set this to prevent region.increment from being called e.bypass(); e.complete(); HRegion region = env.getRegion(); byte[] row = increment.getRow(); TimeRange tr = increment.getTimeRange(); region.startRegionOperation(); try { Integer lid = region.getLock(null, row, true); try { long maxTimestamp = tr.getMax(); if (maxTimestamp == HConstants.LATEST_TIMESTAMP) { maxTimestamp = EnvironmentEdgeManager.currentTimeMillis(); tr = new TimeRange(tr.getMin(), maxTimestamp); } Get get = new Get(row); get.setTimeRange(tr.getMin(), tr.getMax()); for (Map.Entry<byte[], NavigableMap<byte[], Long>> entry : increment.getFamilyMap().entrySet()) { byte[] cf = entry.getKey(); for (byte[] cq : entry.getValue().keySet()) { get.addColumn(cf, cq); } } Result result = region.get(get); if (result.isEmpty()) { return getErrorResult( row, maxTimestamp, SQLExceptionCode.SEQUENCE_UNDEFINED.getErrorCode()); } KeyValue currentValueKV = Sequence.getCurrentValueKV(result); KeyValue incrementByKV = Sequence.getIncrementByKV(result); KeyValue cacheSizeKV = Sequence.getCacheSizeKV(result); long value = PDataType.LONG .getCodec() .decodeLong(currentValueKV.getBuffer(), currentValueKV.getValueOffset(), null); long incrementBy = PDataType.LONG .getCodec() .decodeLong(incrementByKV.getBuffer(), incrementByKV.getValueOffset(), null); int cacheSize = PDataType.INTEGER .getCodec() .decodeInt(cacheSizeKV.getBuffer(), cacheSizeKV.getValueOffset(), null); value += incrementBy * cacheSize; byte[] valueBuffer = new byte[PDataType.LONG.getByteSize()]; PDataType.LONG.getCodec().encodeLong(value, valueBuffer, 0); Put put = new Put(row, currentValueKV.getTimestamp()); // Hold timestamp constant for sequences, so that clients always only see the latest value // regardless of when they connect. KeyValue newCurrentValueKV = KeyValueUtil.newKeyValue( row, currentValueKV.getFamily(), currentValueKV.getQualifier(), currentValueKV.getTimestamp(), valueBuffer); put.add(newCurrentValueKV); @SuppressWarnings("unchecked") Pair<Mutation, Integer>[] mutations = new Pair[1]; mutations[0] = new Pair<Mutation, Integer>(put, lid); region.batchMutate(mutations); return Sequence.replaceCurrentValueKV(result, newCurrentValueKV); } finally { region.releaseRowLock(lid); } } catch (Throwable t) { ServerUtil.throwIOException("Increment of sequence " + Bytes.toStringBinary(row), t); return null; // Impossible } finally { region.closeRegionOperation(); } }