This is an automated email from the ASF dual-hosted git repository. pboado pushed a commit to branch 5.x-cdh6 in repository https://gitbox.apache.org/repos/asf/phoenix.git
commit fe6e767f62c017a48da6fa16a6494c3f4a3b7f72 Author: Thomas D'Silva <tdsi...@apache.org> AuthorDate: Thu Mar 28 23:36:30 2019 +0000 PHOENIX-5219 Fix ConcurrentMutationsIT testLockUntilMVCCAdvanced and testRowLockDuringPreBatchMutateWhenIndexed failures on the master branch --- .../src/main/java/org/apache/phoenix/hbase/index/Indexer.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/Indexer.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/Indexer.java index 8c5184a..c34ffd2 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/Indexer.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/Indexer.java @@ -33,6 +33,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CoprocessorEnvironment; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants.OperationStatusCode; @@ -424,7 +425,6 @@ public class Indexer implements RegionObserver, RegionCoprocessor { ReplayWrite replayWrite = this.builder.getReplayWrite(firstMutation); boolean resetTimeStamp = replayWrite == null; long now = EnvironmentEdgeManager.currentTimeMillis(); - byte[] byteNow = Bytes.toBytes(now); for (int i = 0; i < miniBatchOp.size(); i++) { Mutation m = miniBatchOp.getOperation(i); // skip this mutation if we aren't enabling indexing @@ -436,10 +436,9 @@ public class Indexer implements RegionObserver, RegionCoprocessor { // Unless we're replaying edits to rebuild the index, we update the time stamp // of the data table to prevent overlapping time stamps (which prevents index // inconsistencies as this case isn't handled correctly currently). - for (List<Cell> family : m.getFamilyCellMap().values()) { - List<KeyValue> familyKVs = KeyValueUtil.ensureKeyValues(family); - for (KeyValue kv : familyKVs) { - setTimeStamp(kv, byteNow); + for (List<Cell> cells : m.getFamilyCellMap().values()) { + for (Cell cell : cells) { + CellUtil.setTimestamp(cell, now); } } }