diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java index 9abdf42..87f7f98 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java @@ -92,6 +92,12 @@ public class HTableDescriptor implements Comparable { new Bytes(Bytes.toBytes(OWNER)); /** + * Clock type for the table which defines the semantics of the timestamps used by the table. + */ + public static final String CLOCK_TYPE = "CLOCK_TYPE"; + private static final Bytes CLOCK_TYPE_KEY = new Bytes(Bytes.toBytes(CLOCK_TYPE)); + + /** * INTERNAL Used by rest interface to access this metadata * attribute which denotes if the table is Read Only * @@ -227,6 +233,17 @@ public class HTableDescriptor implements Comparable { public static final boolean DEFAULT_NORMALIZATION_ENABLED = false; /** + * We shall default the tables to System clock until the HLC is completely introduced. + */ + public static final ClockType DEFAULT_CLOCK_TYPE = ClockType.SYSTEM; + + /** + * If clock type is not specified in the HTableDescription, than it means that it is a table + * before Clock type field is introduced. We default to System type for these tables. + */ + public static final ClockType DEFAULT_UNSET_CLOCK_TYPE = ClockType.SYSTEM; + + /** * Constant that denotes the maximum default size of the memstore after which * the contents are flushed to the store files */ @@ -802,6 +819,32 @@ public class HTableDescriptor implements Comparable { } /** + * Sets the Clock type for this table. + * + * @param clockType the timestamp implementation + * @return object of type {@link HTableDescriptor} + * @see ClockType + */ + public HTableDescriptor setClockType(ClockType clockType) { + setValue(CLOCK_TYPE, clockType.name()); + return this; + } + + /** + * Returns the timestamp type for this table. + * + * @return a Timestamp implementation. + * @see ClockType + */ + public ClockType getClockType() { + String name = getValue(CLOCK_TYPE); + if (name == null || name.isEmpty()) { + return DEFAULT_UNSET_CLOCK_TYPE; + } + return ClockType.valueOf(name); + } + + /** * Returns the size of the memstore after which a flush to filesystem is triggered. * * @return memory cache flush size for each hregion, -1 if not set. diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java index 2b50829..5d46431 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java @@ -62,7 +62,6 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.ExceptionUtil; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.PairOfSameType; @@ -1094,8 +1093,6 @@ public class MetaTableAccessor { throws IOException { Table metaHTable = getMetaHTable(conn); Get get = new Get(tableName.getName()).addColumn(getTableFamily(), getStateColumn()); - long time = EnvironmentEdgeManager.currentTime(); - get.setTimeRange(0, time); Result result = metaHTable.get(get); return getTableState(result); @@ -1299,15 +1296,7 @@ public class MetaTableAccessor { */ public static Put makePutFromRegionInfo(HRegionInfo regionInfo) throws IOException { - return makePutFromRegionInfo(regionInfo, EnvironmentEdgeManager.currentTime()); - } - - /** - * Generates and returns a Put containing the region into for the catalog table - */ - public static Put makePutFromRegionInfo(HRegionInfo regionInfo, long ts) - throws IOException { - Put put = new Put(regionInfo.getRegionName(), ts); + Put put = new Put(regionInfo.getRegionName()); addRegionInfo(put, regionInfo); return put; } @@ -1317,20 +1306,8 @@ public class MetaTableAccessor { * table */ public static Delete makeDeleteFromRegionInfo(HRegionInfo regionInfo) { - long now = EnvironmentEdgeManager.currentTime(); - return makeDeleteFromRegionInfo(regionInfo, now); - } - - /** - * Generates and returns a Delete containing the region info for the catalog - * table - */ - public static Delete makeDeleteFromRegionInfo(HRegionInfo regionInfo, long ts) { - if (regionInfo == null) { - throw new IllegalArgumentException("Can't make a delete for null region"); - } Delete delete = new Delete(regionInfo.getRegionName()); - delete.addFamily(getCatalogFamily(), ts); + delete.addFamily(getCatalogFamily()); return delete; } @@ -1454,15 +1431,14 @@ public class MetaTableAccessor { throws IOException { int absoluteIndex = replicaIndexToDeleteFrom + numReplicasToRemove; for (byte[] row : metaRows) { - long now = EnvironmentEdgeManager.currentTime(); Delete deleteReplicaLocations = new Delete(row); for (int i = replicaIndexToDeleteFrom; i < absoluteIndex; i++) { deleteReplicaLocations.addColumns(getCatalogFamily(), - getServerColumn(i), now); + getServerColumn(i)); deleteReplicaLocations.addColumns(getCatalogFamily(), - getSeqNumColumn(i), now); + getSeqNumColumn(i)); deleteReplicaLocations.addColumns(getCatalogFamily(), - getStartCodeColumn(i), now); + getStartCodeColumn(i)); } deleteFromMetaTable(connection, deleteReplicaLocations); } @@ -1572,23 +1548,10 @@ public class MetaTableAccessor { public static void addRegionsToMeta(Connection connection, List regionInfos, int regionReplication) throws IOException { - addRegionsToMeta(connection, regionInfos, regionReplication, HConstants.LATEST_TIMESTAMP); - } - /** - * Adds a hbase:meta row for each of the specified new regions. - * @param connection connection we're using - * @param regionInfos region information list - * @param regionReplication - * @param ts desired timestamp - * @throws IOException if problem connecting or updating meta - */ - public static void addRegionsToMeta(Connection connection, - List regionInfos, int regionReplication, long ts) - throws IOException { List puts = new ArrayList(); for (HRegionInfo regionInfo : regionInfos) { if (RegionReplicaUtil.isDefaultReplica(regionInfo)) { - Put put = makePutFromRegionInfo(regionInfo, ts); + Put put = makePutFromRegionInfo(regionInfo); // Add empty locations for region replicas so that number of replicas can be cached // whenever the primary region is looked up from meta for (int i = 1; i < regionReplication; i++) { @@ -1610,11 +1573,10 @@ public class MetaTableAccessor { public static void addDaughter(final Connection connection, final HRegionInfo regionInfo, final ServerName sn, final long openSeqNum) throws NotAllMetaRegionsOnlineException, IOException { - long now = EnvironmentEdgeManager.currentTime(); - Put put = new Put(regionInfo.getRegionName(), now); + Put put = new Put(regionInfo.getRegionName()); addRegionInfo(put, regionInfo); if (sn != null) { - addLocation(put, sn, openSeqNum, -1, regionInfo.getReplicaId()); + addLocation(put, sn, openSeqNum, regionInfo.getReplicaId()); } putToMetaTable(connection, put); LOG.info("Added daughter " + regionInfo.getEncodedName() + @@ -1630,34 +1592,29 @@ public class MetaTableAccessor { * @param regionA * @param regionB * @param sn the location of the region - * @param masterSystemTime * @param saveBarrier true if need save replication barrier in meta, used for serial replication * @throws IOException */ public static void mergeRegions(final Connection connection, HRegionInfo mergedRegion, - HRegionInfo regionA, HRegionInfo regionB, ServerName sn, int regionReplication, - long masterSystemTime, boolean saveBarrier) + HRegionInfo regionA, HRegionInfo regionB, ServerName sn, int regionReplication, boolean saveBarrier) throws IOException { Table meta = getMetaHTable(connection); try { HRegionInfo copyOfMerged = new HRegionInfo(mergedRegion); - // use the maximum of what master passed us vs local time. - long time = Math.max(EnvironmentEdgeManager.currentTime(), masterSystemTime); - // Put for parent - Put putOfMerged = makePutFromRegionInfo(copyOfMerged, time); + Put putOfMerged = makePutFromRegionInfo(copyOfMerged); putOfMerged.addImmutable(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER, regionA.toByteArray()); putOfMerged.addImmutable(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER, regionB.toByteArray()); // Deletes for merging regions - Delete deleteA = makeDeleteFromRegionInfo(regionA, time); - Delete deleteB = makeDeleteFromRegionInfo(regionB, time); + Delete deleteA = makeDeleteFromRegionInfo(regionA); + Delete deleteB = makeDeleteFromRegionInfo(regionB); // The merged is a new region, openSeqNum = 1 is fine. - addLocation(putOfMerged, sn, 1, -1, mergedRegion.getReplicaId()); + addLocation(putOfMerged, sn, 1, mergedRegion.getReplicaId()); // Add empty locations for region replicas of the merged region so that number of replicas can // be cached whenever the primary region is looked up from meta @@ -1712,8 +1669,8 @@ public class MetaTableAccessor { Put putA = makePutFromRegionInfo(splitA); Put putB = makePutFromRegionInfo(splitB); - addLocation(putA, sn, 1, -1, splitA.getReplicaId()); //new regions, openSeqNum = 1 is fine. - addLocation(putB, sn, 1, -1, splitB.getReplicaId()); + addLocation(putA, sn, 1, splitA.getReplicaId()); //new regions, openSeqNum = 1 is fine. + addLocation(putB, sn, 1, splitB.getReplicaId()); // Add empty locations for region replicas of daughters so that number of replicas can be // cached whenever the primary region is looked up from meta @@ -1757,8 +1714,7 @@ public class MetaTableAccessor { * @param state new state */ public static Put makePutFromTableState(TableState state) { - long time = EnvironmentEdgeManager.currentTime(); - Put put = new Put(state.getTableName().getName(), time); + Put put = new Put(state.getTableName().getName()); put.addColumn(getTableFamily(), getStateColumn(), state.convert().toByteArray()); return put; } @@ -1770,9 +1726,8 @@ public class MetaTableAccessor { */ public static void deleteTableState(Connection connection, TableName table) throws IOException { - long time = EnvironmentEdgeManager.currentTime(); Delete delete = new Delete(table.getName()); - delete.addColumns(getTableFamily(), getStateColumn(), time); + delete.addColumns(getTableFamily(), getStateColumn()); deleteFromMetaTable(connection, delete); LOG.info("Deleted table " + table + " state from META"); } @@ -1821,14 +1776,12 @@ public class MetaTableAccessor { * @param regionInfo region to update location of * @param openSeqNum the latest sequence number obtained when the region was open * @param sn Server name - * @param masterSystemTime wall clock time from master if passed in the open region RPC or -1 * @throws IOException */ public static void updateRegionLocation(Connection connection, - HRegionInfo regionInfo, ServerName sn, long openSeqNum, - long masterSystemTime) + HRegionInfo regionInfo, ServerName sn, long openSeqNum) throws IOException { - updateLocation(connection, regionInfo, sn, openSeqNum, masterSystemTime); + updateLocation(connection, regionInfo, sn, openSeqNum); } /** @@ -1862,21 +1815,15 @@ public class MetaTableAccessor { * @param regionInfo region to update location of * @param sn Server name * @param openSeqNum the latest sequence number obtained when the region was open - * @param masterSystemTime wall clock time from master if passed in the open region RPC or -1 * @throws IOException In particular could throw {@link java.net.ConnectException} * if the server is down on other end. */ private static void updateLocation(final Connection connection, - HRegionInfo regionInfo, ServerName sn, long openSeqNum, - long masterSystemTime) + HRegionInfo regionInfo, ServerName sn, long openSeqNum) throws IOException { - - // use the maximum of what master passed us vs local time. - long time = Math.max(EnvironmentEdgeManager.currentTime(), masterSystemTime); - // region replicas are kept in the primary region's row - Put put = new Put(getMetaKeyForRegion(regionInfo), time); - addLocation(put, sn, openSeqNum, time, regionInfo.getReplicaId()); + Put put = new Put(getMetaKeyForRegion(regionInfo)); + addLocation(put, sn, openSeqNum, regionInfo.getReplicaId()); putToMetaTable(connection, put); LOG.info("Updated row " + regionInfo.getRegionNameAsString() + " with server=" + sn); @@ -1891,9 +1838,8 @@ public class MetaTableAccessor { public static void deleteRegion(Connection connection, HRegionInfo regionInfo) throws IOException { - long time = EnvironmentEdgeManager.currentTime(); Delete delete = new Delete(regionInfo.getRegionName()); - delete.addFamily(getCatalogFamily(), time); + delete.addFamily(getCatalogFamily()); deleteFromMetaTable(connection, delete); LOG.info("Deleted " + regionInfo.getRegionNameAsString()); } @@ -1906,20 +1852,10 @@ public class MetaTableAccessor { */ public static void deleteRegions(Connection connection, List regionsInfo) throws IOException { - deleteRegions(connection, regionsInfo, EnvironmentEdgeManager.currentTime()); - } - /** - * Deletes the specified regions from META. - * @param connection connection we're using - * @param regionsInfo list of regions to be deleted from META - * @throws IOException - */ - public static void deleteRegions(Connection connection, - List regionsInfo, long ts) throws IOException { List deletes = new ArrayList(regionsInfo.size()); for (HRegionInfo hri: regionsInfo) { Delete e = new Delete(hri.getRegionName()); - e.addFamily(getCatalogFamily(), ts); + e.addFamily(getCatalogFamily()); deletes.add(e); } deleteFromMetaTable(connection, deletes); @@ -1966,15 +1902,14 @@ public class MetaTableAccessor { public static void overwriteRegions(Connection connection, List regionInfos, int regionReplication) throws IOException { // use master time for delete marker and the Put - long now = EnvironmentEdgeManager.currentTime(); - deleteRegions(connection, regionInfos, now); + deleteRegions(connection, regionInfos); // Why sleep? This is the easiest way to ensure that the previous deletes does not // eclipse the following puts, that might happen in the same ts from the server. // See HBASE-9906, and HBASE-9879. Once either HBASE-9879, HBASE-8770 is fixed, // or HBASE-9905 is fixed and meta uses seqIds, we do not need the sleep. // // HBASE-13875 uses master timestamp for the mutations. The 20ms sleep is not needed - addRegionsToMeta(connection, regionInfos, regionReplication, now+1); + addRegionsToMeta(connection, regionInfos, regionReplication); LOG.info("Overwritten " + regionInfos); } @@ -1986,10 +1921,9 @@ public class MetaTableAccessor { */ public static void deleteMergeQualifiers(Connection connection, final HRegionInfo mergedRegion) throws IOException { - long time = EnvironmentEdgeManager.currentTime(); Delete delete = new Delete(mergedRegion.getRegionName()); - delete.addColumns(getCatalogFamily(), HConstants.MERGEA_QUALIFIER, time); - delete.addColumns(getCatalogFamily(), HConstants.MERGEB_QUALIFIER, time); + delete.addColumns(getCatalogFamily(), HConstants.MERGEA_QUALIFIER); + delete.addColumns(getCatalogFamily(), HConstants.MERGEB_QUALIFIER); deleteFromMetaTable(connection, delete); LOG.info("Deleted references in merged region " + mergedRegion.getRegionNameAsString() + ", qualifier=" @@ -2004,25 +1938,20 @@ public class MetaTableAccessor { return p; } - public static Put addLocation(final Put p, final ServerName sn, long openSeqNum, - long time, int replicaId){ - if (time <= 0) { - time = EnvironmentEdgeManager.currentTime(); - } - p.addImmutable(getCatalogFamily(), getServerColumn(replicaId), time, + public static Put addLocation(final Put p, final ServerName sn, long openSeqNum, int replicaId){ + p.addImmutable(getCatalogFamily(), getServerColumn(replicaId), Bytes.toBytes(sn.getHostAndPort())); - p.addImmutable(getCatalogFamily(), getStartCodeColumn(replicaId), time, + p.addImmutable(getCatalogFamily(), getStartCodeColumn(replicaId), Bytes.toBytes(sn.getStartcode())); - p.addImmutable(getCatalogFamily(), getSeqNumColumn(replicaId), time, + p.addImmutable(getCatalogFamily(), getSeqNumColumn(replicaId), Bytes.toBytes(openSeqNum)); return p; } public static Put addEmptyLocation(final Put p, int replicaId) { - long now = EnvironmentEdgeManager.currentTime(); - p.addImmutable(getCatalogFamily(), getServerColumn(replicaId), now, null); - p.addImmutable(getCatalogFamily(), getStartCodeColumn(replicaId), now, null); - p.addImmutable(getCatalogFamily(), getSeqNumColumn(replicaId), now, null); + p.addImmutable(getCatalogFamily(), getServerColumn(replicaId), null); + p.addImmutable(getCatalogFamily(), getStartCodeColumn(replicaId), null); + p.addImmutable(getCatalogFamily(), getSeqNumColumn(replicaId), null); return p; } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/Clock.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/Clock.java new file mode 100644 index 0000000..bd5d16b --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/Clock.java @@ -0,0 +1,381 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * http://www.apache.org/licenses/LICENSE-2.0 + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.exceptions.HBaseException; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; + +import static org.apache.hadoop.hbase.util.AtomicUtils.updateMax; + +/** + * A clock is an implementation of an algorithm to get timestamps corresponding to one of the + * {@link TimestampType}s for the current time. Different clock implementations can have + * different semantics associated with them. Every such clock should be able to map its + * representation of time to one of the {link TimestampType}s. + * HBase has traditionally been using the {@link java.lang.System#currentTimeMillis()} to + * timestamp events in HBase. {@link java.lang.System#currentTimeMillis()} does not give any + * guarantees about monotonicity of time. We will keep this implementation of clock in place for + * backward compatibility and call it SYSTEM clock. + * It is easy to provide monotonically non decreasing time semantics by keeping track of the last + * timestamp given by the clock and updating it on receipt of external message. This + * implementation of clock is called SYSTEM_MONOTONIC. + * SYSTEM Clock and SYSTEM_MONOTONIC clock as described above, both being physical clocks, they + * cannot track causality. Hybrid Logical Clocks(HLC), as described in + * HLC Paper, helps tracking + * causality using a + * Logical + * Clock but always keeps the logical time close to the wall time or physical time. It kind + * of has the advantages of both the worlds. One such advantage being getting consistent + * snapshots in physical time as described in the paper. Hybrid Logical Clock has an additional + * advantage that it is always monotonically increasing. + * Note: It is assumed that any physical clock implementation has millisecond resolution else the + * {@link TimestampType} implementation has to changed to accommodate it. It is decided after + * careful discussion to go with millisecond resolution in the HLC design document attached in the + * issue HBASE-14070 . + */ + +@InterfaceAudience.Private +@InterfaceStability.Evolving +public abstract class Clock { + private static final Log LOG = LogFactory.getLog(Clock.class); + + protected PhysicalClock physicalClock; + protected TimestampType timestampType; + public ClockType clockType; + + Clock(PhysicalClock physicalClock) { + this.physicalClock = physicalClock; + } + + /** + * Indicates that Physical Time or Logical Time component has overflowed. This extends + * RuntimeException. + */ + @SuppressWarnings("serial") public static class ClockException extends RuntimeException { + public ClockException(String msg) { + super(msg); + } + } + + /** + * Indicates that clock update cannot be performed because the clock skew is greater than allowed. + */ + @SuppressWarnings("serial") public static class MaxClockSkewException extends HBaseException { + public MaxClockSkewException(String msg) { + super(msg); + } + } + + /** + * This is a method to get the current time. + * + * @return Timestamp of current time in 64 bit representation corresponding to the particular + * clock + */ + public abstract long now() throws RuntimeException; + + /** + * @return true if the clock implementation gives monotonically non decreasing timestamps else + * false. + */ + public abstract boolean isMonotonic(); + + /** + * @return true if the clock implementation gives monotonically increasing timestamps else false. + */ + public abstract boolean isMonotonicallyIncreasing(); + + /** + * @return {@link org.apache.hadoop.hbase.TimestampType} + */ + public TimestampType getTimestampType(){ + return timestampType; + } + + interface Monotonic { + // This is currently equal to the HBase default. + long DEFAULT_MAX_CLOCK_SKEW = 30000; + + /** + * This is a method to update the local clock on receipt of a timestamped message from + * the external world. + * + * @param timestamp The timestamp present in the message received by the node from outside. + */ + long update(long timestamp) throws RuntimeException, HBaseException; + } + + public interface PhysicalClock { + /** + * This is a method to get the current time. + * + * @return Timestamp of current time in 64 bit representation corresponding to the particular + * clock + */ + long now() throws RuntimeException; + + /** + * This is a method to get the unit of the physical time used by the clock + * + * @return A {@link TimeUnit} + */ + TimeUnit getTimeUnit(); + } + + public static class JavaMillisPhysicalClock implements PhysicalClock { + @Override public long now() { + return EnvironmentEdgeManager.currentTime(); + } + + @Override public TimeUnit getTimeUnit() { + return TimeUnit.MILLISECONDS; + } + } + + /** + * Returns the default physical clock used in HBase. It is currently based on + * {@link java.lang.System#currentTimeMillis()} + * + * @return the default PhysicalClock + */ + public static PhysicalClock getDefaultPhysicalClock() { + return new JavaMillisPhysicalClock(); + } + + /** + * System clock is an implementation of clock which doesn't give any monotonic guarantees. + */ + public static class System extends Clock implements PhysicalClock { + + public System() { + super(getDefaultPhysicalClock()); + this.timestampType = TimestampType.PHYSICAL; + this.clockType = ClockType.SYSTEM; + } + + @Override public long now() { + return physicalClock.now(); + } + + @Override public boolean isMonotonic() { + return false; + } + + @Override public boolean isMonotonicallyIncreasing() { + return false; + } + + public TimeUnit getTimeUnit() { + return physicalClock.getTimeUnit(); + } + } + + /** + * System clock is an implementation of clock which guarantees monotonically non-decreasing + * timestamps. + */ + public static class SystemMonotonic extends Clock implements Monotonic, PhysicalClock { + private long maxClockSkew; + private static final long OFFSET = 5000; + AtomicLong physicalTime = new AtomicLong(); + + public SystemMonotonic(PhysicalClock physicalClock, long maxClockSkew) { + super(physicalClock); + this.maxClockSkew = maxClockSkew > 0 ? maxClockSkew : DEFAULT_MAX_CLOCK_SKEW; + this.timestampType = TimestampType.PHYSICAL; + this.clockType = ClockType.SYSTEM_MONOTONIC; + } + + public SystemMonotonic() { + super(getDefaultPhysicalClock()); + this.maxClockSkew = DEFAULT_MAX_CLOCK_SKEW; + this.timestampType = TimestampType.PHYSICAL; + this.clockType = ClockType.SYSTEM_MONOTONIC; + } + + @Override public long now() { + long systemTime = physicalClock.now(); + updateMax(physicalTime, systemTime); + return physicalTime.get(); + } + + public long update(long messageTimestamp) throws MaxClockSkewException { + long systemTime = physicalClock.now(); + if (maxClockSkew > 0 && (messageTimestamp - systemTime) > maxClockSkew) { + throw new MaxClockSkewException( + "Received event with timestamp:" + timestampType.toString(messageTimestamp) + + " which is greater than allowed clock skew "); + } + long physicalTime_ = systemTime > messageTimestamp ? systemTime : messageTimestamp; + updateMax(physicalTime, physicalTime_); + return physicalTime.get(); + } + + @Override public boolean isMonotonic() { + return true; + } + + @Override public boolean isMonotonicallyIncreasing() { + return false; + } + + public TimeUnit getTimeUnit() { + return physicalClock.getTimeUnit(); + } + + @VisibleForTesting void setPhysicalTime(long time) { + physicalTime.set(time); + } + } + + public static class HLC extends Clock implements Monotonic, PhysicalClock { + private long maxClockSkew; + private long physicalTime; + private long logicalTime; + private long maxPhysicalTime; + private long maxLogicalTime; + + public HLC(PhysicalClock physicalClock, long maxClockSkew) { + super(physicalClock); + this.maxClockSkew = maxClockSkew > 0 ? maxClockSkew : DEFAULT_MAX_CLOCK_SKEW; + this.maxPhysicalTime = timestampType.getMaxPhysicalTime(); + this.maxLogicalTime = timestampType.getMaxLogicalTime(); + this.timestampType = TimestampType.HYBRID; + this.physicalTime = 0; + this.logicalTime = 0; + this.clockType = ClockType.HLC; + } + + public HLC() { + super(getDefaultPhysicalClock()); + this.maxClockSkew = DEFAULT_MAX_CLOCK_SKEW; + this.maxPhysicalTime = timestampType.HYBRID.getMaxPhysicalTime(); + this.maxLogicalTime = timestampType.HYBRID.getMaxLogicalTime(); + this.timestampType = TimestampType.HYBRID; + this.physicalTime = 0; + this.logicalTime = 0; + this.clockType = ClockType.HLC; + } + + @Override public synchronized long now() throws ClockException { + long systemTime = physicalClock.now(); + long physicalTime_ = physicalTime; + if (systemTime >= maxPhysicalTime) { + // Extremely unlikely to happen, if this happens upper layers may have to kill the server. + throw new ClockException( + "PT overflowed: " + systemTime + " and max physical time:" + maxPhysicalTime); + } + + if (systemTime > physicalTime_) physicalTime = systemTime; + + if (physicalTime == physicalTime_) { + logicalTime++; + } else { + logicalTime = 0; + } + if (logicalTime >= maxLogicalTime) { + // highly unlikely to happen, when it happens, we throw exception for the above layer to + // handle. + throw new ClockException( + "Logical Time Overflowed: " + logicalTime + "max " + "logical " + "time:" + + maxLogicalTime); + } + return toTimestamp(); + } + + /** + * Updates {@link HLC} with the given timestamp received from elsewhere (possibly + * some other node). Returned timestamp is strict greater than msgTimestamp and local + * timestamp. + * + * @param messageTimestamp timestamp from the external message. + * @return a hybrid timestamp of HLC that is strictly greater than local timestamp and + * msgTimestamp + * @throws ClockException + * @throws MaxClockSkewException + */ + @Override public synchronized long update(long messageTimestamp) + throws ClockException, MaxClockSkewException { + long messagePhysicalTime = timestampType.getPhysicalTime(messageTimestamp); + long messageLogicalTime = timestampType.getLogicalTime(messageTimestamp); + // variable to keep old physical time when we update it. + long physicalTime_ = physicalTime; + long systemTime = physicalClock.now(); + + physicalTime = Math.max(Math.max(physicalTime_, messagePhysicalTime), systemTime); + + if (physicalTime_ >= maxPhysicalTime) { + // Extremely unlikely to happen, if this happens upper layers may have to kill the server. + throw new ClockException( + "Physical Time overflowed: " + systemTime + " and max physical time:" + + maxPhysicalTime); + } else if (messagePhysicalTime - physicalTime_ > maxClockSkew) { + throw new MaxClockSkewException( + "Received event with timestamp:" + timestampType.toString(messageTimestamp) + + " which is greater than allowed clock skew "); + } else if (physicalTime == physicalTime_ && physicalTime_ == messagePhysicalTime) { + logicalTime = Math.max(logicalTime, messageLogicalTime) + 1; + } else if (physicalTime == messagePhysicalTime) { + logicalTime = messageLogicalTime + 1; + } else if (physicalTime == physicalTime_) { + logicalTime++; + } else { + logicalTime = 0; + } + + if (logicalTime >= maxLogicalTime) { + // highly unlikely to happen, when it happens, we throw exception for the above layer to + // handle it the way they wish to. + throw new ClockException( + "Logical Time Overflowed: " + logicalTime + "max " + "logical time: " + maxLogicalTime); + } + return toTimestamp(); + } + + @Override public boolean isMonotonic() { + return true; + } + + @Override public boolean isMonotonicallyIncreasing() { + return true; + } + + public TimeUnit getTimeUnit() { + return physicalClock.getTimeUnit(); + } + + private long toTimestamp() { + return timestampType.toTimestamp(getTimeUnit(), physicalTime, logicalTime); + } + + @VisibleForTesting synchronized void setLogicalTime(long logicalTime) { + this.logicalTime = logicalTime; + } + + @VisibleForTesting synchronized void setPhysicalTime(long physicalTime) { + this.physicalTime = physicalTime; + } + } +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ClockType.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ClockType.java new file mode 100644 index 0000000..4dc53ff --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ClockType.java @@ -0,0 +1,26 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * http://www.apache.org/licenses/LICENSE-2.0 + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase; + +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; + +@InterfaceAudience.Private +@InterfaceStability.Evolving +public enum ClockType { + SYSTEM, SYSTEM_MONOTONIC, HLC +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/TimestampType.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/TimestampType.java new file mode 100644 index 0000000..54059b6 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/TimestampType.java @@ -0,0 +1,314 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * http://www.apache.org/licenses/LICENSE-2.0 + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase; + +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.commons.lang.time.FastDateFormat; + +import java.util.TimeZone; +import java.util.concurrent.TimeUnit; + +/** + * {@link TimestampType} is an enum to represent different ways of encoding time in HBase using + * 64 bits. Time is usually encoded as a 64-bit long in {@link org.apache.hadoop.hbase.Cell} + * timestamps and is used for sorting {@link org.apache.hadoop.hbase.Cell}s, ordering writes etc. + * It has methods which help in constructing or interpreting the 64 bit timestamp and getter + * methods to read the hard coded constants of the particular {@link TimestampType}. + * + *

+ * Enum {@link TimestampType} is dumb in a way. It doesn't have any logic other than interpreting + * the 64 bits. Any monotonically increasing or monotonically non-decreasing semantics of the + * timestamps are the responsibility of the clock implementation generating the particular + * timestamps. There can be several clock implementations, and each such implementation can map + * its representation of the timestamp to one of the available Timestamp types i.e. + * {@link #HYBRID} or {@link #PHYSICAL}. In essence, the {@link TimestampType} is only used + * internally by the Clock implementations and thus never exposed to the user. The user has to + * know only the different available clock types. So, for the user timestamp types do not exist. + *

+ */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public enum TimestampType { + /** + * Hybrid is a Timestamp type used to encode both physical time and logical time components + * into a single. 64 bits long integer. It has methods to decipher the 64 bits hybrid timestamp + * and also to construct the hybrid timestamp. + */ + HYBRID { + /** + * Hard coded 44-bits for physical time, with most significant bit carrying the sign i.e 0 + * as we are dealing with positive integers and the remaining 43 bits are to be interpreted as + * system time in milli seconds. See + * HBASE-14070 for + * understanding the choice of going with the millisecond resolution for physical time. + * Thus allowing us to represent all the dates between unix epoch (1970) and year 2248 with + * signed timestamp comparison with 44 bits for physical time assuming a millisecond + * resolution with signed long integers. Picking 42 bits to represent the physical time has + * the problem of representing time until 2039 only, with signed integers, might cause Y2k39 + * bug hoping HBase to be around till then. The trade-off here is with the year until we can + * represent the physical time vs if we are able capture all the events in the worst case + * (read: leap seconds etc) without the logical component of the timestamp overflowing. With + * 20 bits for logical time, one can represent upto one million events at the same + * millisecond. In case of leap seconds, the no of events happening in the same second is very + * unlikely to exceed one million. + */ + @SuppressWarnings("unused") private static final int BITS_FOR_PHYSICAL_TIME = 44; + + /** + * Remaining 20-bits for logical time, allowing values up to 1,048,576. Logical Time is the + * least significant part of the 64 bit timestamp, so unsigned comparison can be used for LT. + */ + + private static final int BITS_FOR_LOGICAL_TIME = 20; + + /** + * Max value for physical time in the {@link #HYBRID} timestamp representation, inclusive. + * This assumes signed comparison. + */ + private static final long PHYSICAL_TIME_MAX_VALUE = 0x7ffffffffffL; + + /** + * Max value for logical time in the {@link #HYBRID} timestamp representation + */ + static final long LOGICAL_TIME_MAX_VALUE = 0xfffffL; + + public long toEpochTimeMillisFromTimestamp(long timestamp) { + return getPhysicalTime(timestamp); + } + + public long fromEpochTimeMillisToTimestamp(long timestamp) { + return toTimestamp(TimeUnit.MILLISECONDS, timestamp, 0); + } + + long toTimestamp(TimeUnit timeUnit, long physicalTime, long logicalTime) { + physicalTime = TimeUnit.MILLISECONDS.convert(physicalTime, timeUnit); + return (physicalTime << BITS_FOR_LOGICAL_TIME) + logicalTime; + } + + public long getPhysicalTime(long timestamp) { + return timestamp >>> BITS_FOR_LOGICAL_TIME; // assume unsigned timestamp + } + + long getLogicalTime(long timestamp) { + return timestamp & LOGICAL_TIME_MAX_VALUE; + } + + long getMaxPhysicalTime() { + return PHYSICAL_TIME_MAX_VALUE; + } + + long getMaxLogicalTime() { + return LOGICAL_TIME_MAX_VALUE; + } + + int getBitsForLogicalTime() { + return BITS_FOR_LOGICAL_TIME; + } + + /** + * Returns whether the given timestamp is "likely" of {@link #HYBRID} {@link TimestampType}. + * Timestamp implementations can use the full range of 64bits long to represent physical and + * logical components of time. However, this method returns whether the given timestamp is a + * likely representation depending on heuristics for the clock implementation. + * + * Hybrid timestamps are checked whether they belong to Hybrid range assuming + * that Hybrid timestamps will only have > 0 logical time component for timestamps + * corresponding to years after 2016. This method will return false if lt > 0 and year is + * before 2016. Due to left shifting for Hybrid time, all millisecond-since-epoch timestamps + * from years 1970-10K fall into + * year 1970 when interpreted as Hybrid timestamps. Thus, {@link #isLikelyOfType(long, boolean)} will + * return false for timestamps which are in the year 1970 and logical time = 0 when + * interpreted as of type Hybrid Time. + * + *

+ * Note that this method uses heuristics which may not hold + * if system timestamps are intermixed from client side and server side or timestamp + * sources other than system clock are used. + *

+ * @param timestamp {@link #HYBRID} Timestamp + * @param isClockMonotonic if the clock that generated this timestamp is monotonic + * @return true if the timestamp is likely to be of the corresponding {@link TimestampType} + * else false + */ + boolean isLikelyOfType(long timestamp, boolean isClockMonotonic) { + long physicalTime = getPhysicalTime(timestamp); + long logicalTime = getLogicalTime(timestamp); + + // heuristic 1: Up until year 2016 (1451635200000), lt component cannot be non-zero. + if (physicalTime < 1451635200000L && logicalTime != 0) { + return false; + } else if (physicalTime < 31536000000L) { + // heuristic 2: Even if logical time = 0, physical time after left shifting by 20 bits, + // will be before year 1971(31536000000L), as after left shifting by 20, all epoch ms + // timestamps from wall time end up in year less than 1971, even for epoch time for the + // year 10000. This assumes Hybrid time is not used to represent timestamps for year 1970 + // UTC. + return false; + } + return true; + } + + /** + * Returns a string representation for Physical Time and Logical Time components. The format is: + * yyyy-MM-dd HH:mm:ss:SSS(Physical Time),Logical Time + * Physical Time is converted to UTC time and not to local time for uniformity. + * Example: 2015-07-17 16:56:35:891(1437177395891), 0 + * @param timestamp A {@link #HYBRID} Timestamp + * @return A date time string formatted as mentioned in the method description + */ + String toString(long timestamp) { + long physicalTime = getPhysicalTime(timestamp); + long logicalTime = getLogicalTime(timestamp); + return new StringBuilder().append(dateFormat.format(physicalTime)).append("(") + .append(physicalTime).append(")").append(", ").append(logicalTime).toString(); + } + }, + + /** + * Physical is a Timestamp type used to encode the physical time in 64 bits. + * It has helper methods to decipher the 64 bit encoding of physical time. + */ + PHYSICAL { + public long toEpochTimeMillisFromTimestamp(long timestamp) { + return timestamp; + } + + public long fromEpochTimeMillisToTimestamp(long timestamp) { + return timestamp; + } + + long toTimestamp(TimeUnit timeUnit, long physicalTime, long logicalTime) { + return TimeUnit.MILLISECONDS.convert(physicalTime, timeUnit); + } + + public long getPhysicalTime(long timestamp) { + return timestamp; + } + + long getLogicalTime(long timestamp) { + return 0; + } + + long getMaxPhysicalTime() { + return Long.MAX_VALUE; + } + + long getMaxLogicalTime() { + return 0; + } + + int getBitsForLogicalTime() { + return 0; + } + + boolean isLikelyOfType(long timestamp, boolean isClockMonotonic) { + // heuristic: the timestamp should be up to year 3K (32503680000000L). + if (!isClockMonotonic) { + return true; + } + return timestamp < 32503680000000L; + } + + /** + * Returns a string representation for Physical Time and Logical Time components. The format is: + * yyyy-MM-dd HH:mm:ss:SSS(Physical Time) + * Physical Time is converted to UTC time and not to local time for uniformity. + * Example: 2015-07-17 16:56:35:891(1437177395891), 0 + * @param timestamp epoch time in milliseconds + * @return A date time string formatted as mentioned in the method description + */ + String toString(long timestamp) { + long physicalTime = timestamp; + return new StringBuilder().append(dateFormat.format(physicalTime)).append("(") + .append(physicalTime).append(")").append(", ").append("0").toString(); + } + }; + + /** + * This is used internally by the enum methods of Hybrid and Physical Timestamp types to + * convert the + * timestamp to the format set here. UTC timezone instead of local time zone for convenience + * and uniformity + */ + private static final FastDateFormat dateFormat = + FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss:SSS", TimeZone.getTimeZone("UTC")); + + /** + * Converts the given timestamp to the unix epoch timestamp with millisecond resolution. + * Returned timestamp is compatible with System.currentTimeMillis(). + * @param timestamp {@link #HYBRID} or {@link #PHYSICAL} Timestamp + * @return number of milliseconds from epoch + */ + abstract public long toEpochTimeMillisFromTimestamp(long timestamp); + + /** + * Converts the given time in milliseconds to the corresponding {@link TimestampType} + * representation. + * @param timeInMillis epoch time in {@link TimeUnit#MILLISECONDS} + * @return a timestamp representation corresponding to {@link TimestampType}. + */ + abstract public long fromEpochTimeMillisToTimestamp(long timeInMillis); + + /** + * Converts the given physical clock in the given {@link TimeUnit} to a 64-bit timestamp + * @param timeUnit a time unit as in the enum {@link TimeUnit} + * @param physicalTime physical time + * @param logicalTime logical time + * @return a timestamp in 64 bits + */ + abstract long toTimestamp(TimeUnit timeUnit, long physicalTime, long logicalTime); + + /** + * Extracts and returns the physical time from the timestamp + * @param timestamp {@link #HYBRID} or {@link #PHYSICAL} Timestamp + * @return physical time in {@link TimeUnit#MILLISECONDS} + */ + abstract public long getPhysicalTime(long timestamp); + + /** + * Extracts and returns the logical time from the timestamp + * @param timestamp {@link #HYBRID} or {@link #PHYSICAL} Timestamp + * @return logical time + */ + abstract long getLogicalTime(long timestamp); + + /** + * @return the maximum possible physical time in {@link TimeUnit#MILLISECONDS} + */ + abstract long getMaxPhysicalTime(); + + /** + * @return the maximum possible logical time + */ + abstract long getMaxLogicalTime(); + + /** + * @return number of least significant bits allocated for logical time + */ + abstract int getBitsForLogicalTime(); + + /** + * @param timestamp epoch time in milliseconds + * @param isClockMonotonic if the clock that generated this timestamp is monotonic + * @return True if the timestamp generated by the clock is of type {@link #PHYSICAL} else False + */ + abstract boolean isLikelyOfType(long timestamp, boolean isClockMonotonic); + + abstract String toString(long timestamp); + +} diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestClock.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestClock.java new file mode 100644 index 0000000..05c0865 --- /dev/null +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestClock.java @@ -0,0 +1,292 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase; + +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.apache.hadoop.hbase.testclassification.SmallTests; + +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +@Category(SmallTests.class) +public class TestClock { + + private static final long OFFSET = 5000; + + // All Clocks Tests + + /** + * Remove this test if moving away from millis resolution for physical time. Be sure to change + * {@link TimestampType} methods which assume millisecond resolution. + */ + @Test public void TestClocksPhysicalTimeResolution() { + Clock.System systemClock = new Clock.System(); + Clock.SystemMonotonic systemMonotonicClock = new Clock.SystemMonotonic(); + Clock.HLC hybridLogicalClock = new Clock.HLC(); + assertTrue(systemClock.getTimeUnit() == systemMonotonicClock.getTimeUnit() + && systemClock.getTimeUnit() == hybridLogicalClock.getTimeUnit() + && TimeUnit.MILLISECONDS == systemClock.getTimeUnit()); + } + + // All System Clock Tests + @Test public void TestSystemClockIsMonotonic() { + Clock.System systemClock = new Clock.System(); + assertFalse(systemClock.isMonotonic()); + } + + @Test public void testSystemClockIsMonotonicallyIncreasing() { + Clock.System systemClock = new Clock.System(); + assertFalse(systemClock.isMonotonicallyIncreasing()); + } + + // All System Monotonic Clock Tests + + @Test public void testSystemMonotonicClockNow() { + assertTrue(true); + } + + @Test public void testSystemMonotonicClockUpdate() { + assertTrue(true); + } + + @Test public void testSystemMonotonicClockIsMonotonic() { + Clock.SystemMonotonic systemMonotonicClock = new Clock.SystemMonotonic(); + assertTrue(systemMonotonicClock.isMonotonic()); + } + + @Test public void testSystemMonotonicClockIsMonotonicallyIncreasing() { + Clock.SystemMonotonic systemMonotonicClock = new Clock.SystemMonotonic(); + assertFalse(systemMonotonicClock.isMonotonicallyIncreasing()); + } + + // All Hybrid Logical Clock Tests + @Test public void testHLCIsMonotonic() { + Clock.HLC hybridLogicalClock = new Clock.HLC(); + assertTrue(hybridLogicalClock.isMonotonic()); + } + + @Test public void testHLCIsMonotonicallyIncreasing() { + Clock.HLC hybridLogicalClock = new Clock.HLC(); + assertTrue(hybridLogicalClock.isMonotonicallyIncreasing()); + } + + @Test public void testSystemMonotonicNow() { + Clock.SystemMonotonic systemMonotonicClock = new Clock.SystemMonotonic(); + final long currentTime = System.currentTimeMillis(); + systemMonotonicClock.setPhysicalTime(currentTime + OFFSET); + assertTrue(systemMonotonicClock.now() == currentTime + OFFSET); + + systemMonotonicClock.setPhysicalTime(currentTime - OFFSET); + assertTrue(systemMonotonicClock.now() >= currentTime); + + } + + @Test public void testSystemMonotonicUpdate() { + Clock.SystemMonotonic systemMonotonicClock = new Clock.SystemMonotonic(); + final long currentTime = System.currentTimeMillis(); + long messageTimestamp; + + // case 1: Message Timestamp ahead of current time and last physical timestamp. + messageTimestamp = currentTime + 2*OFFSET; + systemMonotonicClock.setPhysicalTime(currentTime + OFFSET); + try { + assertTrue(systemMonotonicClock.update(messageTimestamp) == messageTimestamp); + } catch (Clock.MaxClockSkewException e) { + assertFalse(true); + } + + messageTimestamp = currentTime + 2*OFFSET; + systemMonotonicClock.setPhysicalTime(currentTime - OFFSET); + try { + assertTrue(systemMonotonicClock.update(messageTimestamp) == messageTimestamp); + } catch (Clock.MaxClockSkewException e) { + assertFalse(true); + } + + messageTimestamp = currentTime + 10*OFFSET; + systemMonotonicClock.setPhysicalTime(currentTime + OFFSET); + try { + systemMonotonicClock.update(messageTimestamp); + assertTrue(false); + } catch (Clock.MaxClockSkewException e) { + assertTrue(true); + } + + // case 2: last physical timestamp is greater than current time and message timestamp + messageTimestamp = currentTime + OFFSET; + systemMonotonicClock.setPhysicalTime(currentTime + 2*OFFSET); + try { + assertTrue(systemMonotonicClock.update(messageTimestamp) == currentTime + 2*OFFSET); + } catch (Clock.MaxClockSkewException e) { + assertFalse(true); + } + + messageTimestamp = currentTime - OFFSET; + systemMonotonicClock.setPhysicalTime(currentTime + 2*OFFSET); + try { + assertTrue(systemMonotonicClock.update(messageTimestamp) == currentTime + 2*OFFSET); + } catch (Clock.MaxClockSkewException e) { + assertFalse(true); + } + + // case 3: current time is greater than last physical timestamp and message timestamp + messageTimestamp = currentTime - OFFSET; + systemMonotonicClock.setPhysicalTime(currentTime - 2*OFFSET); + try { + assertTrue(systemMonotonicClock.update(messageTimestamp) >= currentTime); + } catch (Clock.MaxClockSkewException e) { + assertFalse(true); + } + + messageTimestamp = currentTime - 2*OFFSET; + systemMonotonicClock.setPhysicalTime(currentTime - OFFSET); + try { + assertTrue(systemMonotonicClock.update(messageTimestamp) >= currentTime); + } catch (Clock.MaxClockSkewException e) { + assertFalse(true); + } + } + + @Test public void testHLCNow() { + Clock.HLC hybridLogicalClock = new Clock.HLC(); + final long currentTime = System.currentTimeMillis(); + // case 1: Test Physical Time updates. + hybridLogicalClock.setPhysicalTime(currentTime + OFFSET); + hybridLogicalClock.setLogicalTime(1); + assertTrue(hybridLogicalClock.now() == TimestampType.HYBRID + .toTimestamp(TimeUnit.MILLISECONDS, currentTime + OFFSET, 2)); + + // case 2: current physical time is greater than last physical time component of HLC. + hybridLogicalClock.setPhysicalTime(currentTime - OFFSET); + hybridLogicalClock.setLogicalTime(1); + assertTrue(hybridLogicalClock.now() > TimestampType.HYBRID + .toTimestamp(TimeUnit.MILLISECONDS, currentTime - OFFSET, 1)); + } + + @Test public void testHLCUpdate() { + Clock.HLC hybridLogicalClock = new Clock.HLC(); + final long currentTime = System.currentTimeMillis(); + long messageTimestamp, timestamp; + /* case 1: Message timestamp is ahead of the current timestamp and the last given timestamp. + Has two sub cases: a) last physical timestamp ahead of current time. + b) current time ahead of last physical timestamp. + c) Skew Exception check. + */ + // 1a. + hybridLogicalClock.setPhysicalTime(currentTime + OFFSET); + hybridLogicalClock.setLogicalTime(1); + messageTimestamp = + TimestampType.HYBRID.toTimestamp(TimeUnit.MILLISECONDS, currentTime + 2 * OFFSET, 1); + try { + timestamp = hybridLogicalClock.update(messageTimestamp); + assertTrue(timestamp == TimestampType.HYBRID + .toTimestamp(TimeUnit.MILLISECONDS, currentTime + 2 * OFFSET, 2)); + } catch (Clock.MaxClockSkewException maxClockSkew) { + System.out.println(maxClockSkew.getMessage()); + } + //1b. + hybridLogicalClock.setPhysicalTime(currentTime - OFFSET); + hybridLogicalClock.setLogicalTime(1); + messageTimestamp = + TimestampType.HYBRID.toTimestamp(TimeUnit.MILLISECONDS, currentTime + 2 * OFFSET, 1); + try { + timestamp = hybridLogicalClock.update(messageTimestamp); + assertTrue(timestamp == TimestampType.HYBRID + .toTimestamp(TimeUnit.MILLISECONDS, currentTime + 2 * OFFSET, 2)); + } catch (Clock.MaxClockSkewException maxClockSkew) { + System.out.println(maxClockSkew.getMessage()); + } + + //1c. + hybridLogicalClock.setPhysicalTime(currentTime - OFFSET); + hybridLogicalClock.setLogicalTime(1); + messageTimestamp = + TimestampType.HYBRID.toTimestamp(TimeUnit.MILLISECONDS, currentTime + 10 * OFFSET, 1); + try { + timestamp = hybridLogicalClock.update(messageTimestamp); + assertTrue(false); + } catch (Clock.MaxClockSkewException maxClockSkew) { + assertTrue(true); + } + + // Case 2: Last Physical timestamp is ahead of message timestamp and current time + // Has two sub cases: a) message timestamp ahead of current time. + // b) current time is ahead of message timestamp. + + // 2a. + messageTimestamp = + TimestampType.HYBRID.toTimestamp(TimeUnit.MILLISECONDS, currentTime + 2 * OFFSET, 1); + hybridLogicalClock.setPhysicalTime(currentTime + 3 * OFFSET); + hybridLogicalClock.setLogicalTime(1); + try { + timestamp = hybridLogicalClock.update(messageTimestamp); + assertTrue(timestamp == TimestampType.HYBRID + .toTimestamp(TimeUnit.MILLISECONDS, currentTime + 3 * OFFSET, 2)); + } catch (Clock.MaxClockSkewException maxClockSkew) { + System.out.println(maxClockSkew.getMessage()); + } + + //2b. + messageTimestamp = + TimestampType.HYBRID.toTimestamp(TimeUnit.MILLISECONDS, currentTime - 2 * OFFSET, 1); + hybridLogicalClock.setPhysicalTime(currentTime + 3 * OFFSET); + hybridLogicalClock.setLogicalTime(1); + try { + timestamp = hybridLogicalClock.update(messageTimestamp); + assertTrue(timestamp == TimestampType.HYBRID + .toTimestamp(TimeUnit.MILLISECONDS, currentTime + 3 * OFFSET, 2)); + } catch (Clock.MaxClockSkewException maxClockSkew) { + System.out.println(maxClockSkew.getMessage()); + } + + // Case 3: Current time is ahead of message timestamp and last physical Timestamp. + // Has two sub cases: a) last physical timestamp ahead of message timestamp. + // b) message timestamp ahead of last physical timestamp + + //3a. + messageTimestamp = + TimestampType.HYBRID.toTimestamp(TimeUnit.MILLISECONDS, currentTime - 2 * OFFSET, 1); + hybridLogicalClock.setPhysicalTime(currentTime - OFFSET); + hybridLogicalClock.setLogicalTime(10); + try { + timestamp = hybridLogicalClock.update(messageTimestamp); + assertTrue( + timestamp > TimestampType.HYBRID.toTimestamp(TimeUnit.MILLISECONDS, currentTime, 0)); + } catch (Clock.MaxClockSkewException maxClockSkew) { + System.out.println(maxClockSkew.getMessage()); + } + + //3b. + messageTimestamp = + TimestampType.HYBRID.toTimestamp(TimeUnit.MILLISECONDS, currentTime - OFFSET, 1); + hybridLogicalClock.setPhysicalTime(currentTime - 2 * OFFSET); + hybridLogicalClock.setLogicalTime(10); + try { + timestamp = hybridLogicalClock.update(messageTimestamp); + assertTrue( + timestamp > TimestampType.HYBRID.toTimestamp(TimeUnit.MILLISECONDS, currentTime, 0)); + } catch (Clock.MaxClockSkewException maxClockSkew) { + System.out.println(maxClockSkew.getMessage()); + } + + } +} diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTimestampType.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTimestampType.java new file mode 100644 index 0000000..01c8314 --- /dev/null +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTimestampType.java @@ -0,0 +1,237 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase; + +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.*; + +@Category(SmallTests.class) +public class TestTimestampType { + + private static long testPhysicalTime = 1234567890123L; + private static long testLogicalTime = 12; + + /* + * Tests for TimestampType enum + */ + + @Test + public void testFromToEpoch() { + for (TimestampType timestamp : TimestampType.values()) { + long wallTime = System.currentTimeMillis(); + long converted = timestamp.toEpochTimeMillisFromTimestamp( + timestamp.fromEpochTimeMillisToTimestamp(wallTime)); + + assertEquals(wallTime, converted); + } + } + + /* Tests for HL Clock */ + @Test + public void testHybridMaxValues() { + // assert 44-bit Physical Time with signed comparison (actual 43 bits) + assertEquals( + (1L << (63-TimestampType.HYBRID.getBitsForLogicalTime())) - 1, + TimestampType.HYBRID.getMaxPhysicalTime()); + + // assert 20-bit Logical Time + assertEquals( + (1L << TimestampType.HYBRID.getBitsForLogicalTime()) - 1, + TimestampType.HYBRID.getMaxLogicalTime()); + + // assert that maximum representable timestamp is Long.MAX_VALUE (assuming signed comparison). + assertEquals( + Long.MAX_VALUE, + TimestampType.HYBRID.toTimestamp(TimeUnit.MILLISECONDS, + TimestampType.HYBRID.getMaxPhysicalTime(), + TimestampType.HYBRID.getMaxLogicalTime()) + ); + } + + @Test + public void testHybridGetPhysicalTime() { + long ts = TimestampType.HYBRID.toTimestamp(TimeUnit.MILLISECONDS, testPhysicalTime, testLogicalTime); + assertEquals(testPhysicalTime, TimestampType.HYBRID.getPhysicalTime(ts)); + } + + @Test + public void testHybridGetLogicalTime() { + long ts = TimestampType.HYBRID.toTimestamp(TimeUnit.MILLISECONDS, testPhysicalTime, testLogicalTime); + assertEquals(testLogicalTime, TimestampType.HYBRID.getLogicalTime(ts)); + } + + @Test + public void testHybridToString() { + long ts = TimestampType.HYBRID.toTimestamp(TimeUnit.MILLISECONDS, testPhysicalTime, testLogicalTime); + + assertEquals("2009-02-13T23:31:30:123(1234567890123), 12", TimestampType.HYBRID.toString(ts)); + } + + @Test + public void testHybridToTimestamp() { + long expected = (testPhysicalTime << TimestampType.HYBRID.getBitsForLogicalTime()) + testLogicalTime; + // test millisecond + long ts = TimestampType.HYBRID.toTimestamp(TimeUnit.MILLISECONDS, testPhysicalTime, testLogicalTime); + assertEquals(ts, expected); + + // test nanosecond + ts = TimestampType.HYBRID.toTimestamp(TimeUnit.NANOSECONDS, TimeUnit.MILLISECONDS.toNanos(testPhysicalTime), testLogicalTime); + assertEquals(ts, expected); + } + + @Test + public void testHybridIsLikelyOfType() throws ParseException { + SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss:SSS Z"); + + // test timestamps of Hybrid type from year 1971 to 2248 where lt = 0 + for (int year = 1971; year <= 2248; year += 1) { + Date date = dateFormat.parse(year + "-01-01T11:22:33:444 UTC"); + + // Hybrid type ts with pt = date and lt = 0 + long ts = TimestampType.HYBRID.toTimestamp(TimeUnit.MILLISECONDS, date.getTime(), 0); + System.out.println(TimestampType.HYBRID.toString(ts)); + + assertTrue(TimestampType.HYBRID.isLikelyOfType(ts, true)); + } + + // test timestamps of Hybrid type from year 2016 to 2348 where lt > 0 + for (int year = 2016; year <= 2248; year += 1) { + Date date = dateFormat.parse(year + "-01-01T11:22:33:444 UTC"); + + // Hybrid type ts with pt = date and lt = 123 + long ts = TimestampType.HYBRID.toTimestamp(TimeUnit.MILLISECONDS, date.getTime(), 123); + System.out.println(TimestampType.HYBRID.toString(ts)); + + assertTrue(TimestampType.HYBRID.isLikelyOfType(ts, true)); + } + + // test that timestamps from different years are not Hybrid type + for (int year = 1970; year <= 10000 ;year += 10) { + // Stardate 1970 to 10000 + Date date = dateFormat.parse(year + "-01-01T00:00:00:000 UTC"); + long ts = date.getTime(); + System.out.println(TimestampType.PHYSICAL.toString(ts)); + System.out.println(TimestampType.PHYSICAL.toString(TimestampType.HYBRID.getPhysicalTime(ts))); + + assertFalse(TimestampType.HYBRID.isLikelyOfType(ts, true)); + } + + // test that timestamps up to 2016 are not Hybrid even if lt = 0 + for (int year = 1970; year <= 2016; year += 1) { + Date date = dateFormat.parse(year + "-01-01T11:22:33:444 UTC"); + + // reset lt = 0 + long ts = ((date.getTime() + >> TimestampType.HYBRID.getBitsForLogicalTime()) << TimestampType.HYBRID.getBitsForLogicalTime()); + System.out.println(Long.toHexString(ts)); + + System.out.println(TimestampType.PHYSICAL.toString(ts)); + System.out.println(TimestampType.PHYSICAL.toString(TimestampType.HYBRID.getPhysicalTime(ts))); + + assertFalse(TimestampType.HYBRID.isLikelyOfType(ts, true)); + } + + // test that timestamps from currentTime epoch are not Hybrid type + long systemTimeNow = System.currentTimeMillis(); + System.out.println(TimestampType.PHYSICAL.toString(systemTimeNow)); + System.out.println(TimestampType.PHYSICAL.toString((TimestampType.HYBRID.getPhysicalTime(systemTimeNow)))); + assertFalse(TimestampType.HYBRID.isLikelyOfType(systemTimeNow, true)); + } + + + @Test + public void testPhysicalMaxValues() { + assertEquals( + (1L << 63) - 1, + TimestampType.PHYSICAL.getMaxPhysicalTime()); + + assertEquals(0, TimestampType.PHYSICAL.getMaxLogicalTime()); + } + + @Test + public void testPhysicalGetPhysicalTime() { + long ts = TimestampType.PHYSICAL.toTimestamp(TimeUnit.MILLISECONDS, testPhysicalTime, testLogicalTime); + assertEquals(testPhysicalTime, TimestampType.PHYSICAL.getPhysicalTime(ts)); + } + + @Test + public void testPhysicalGetLogicalTime() { + long ts = TimestampType.PHYSICAL.toTimestamp(TimeUnit.MILLISECONDS, testPhysicalTime, testLogicalTime); + assertEquals(0, TimestampType.PHYSICAL.getLogicalTime(ts)); + } + + @Test + public void testPhysicalToString() { + long ts = TimestampType.PHYSICAL.toTimestamp(TimeUnit.MILLISECONDS, testPhysicalTime, testLogicalTime); + + assertEquals("2009-02-13T23:31:30:123(1234567890123), 0", TimestampType.PHYSICAL.toString(ts)); + } + + @Test + public void testPhysicalToTimestamp() { + // test millisecond + long ts = TimestampType.PHYSICAL.toTimestamp(TimeUnit.MILLISECONDS, testPhysicalTime, testLogicalTime); + assertEquals(ts, testPhysicalTime); + + // test nanosecond + ts = TimestampType.PHYSICAL.toTimestamp(TimeUnit.NANOSECONDS, TimeUnit.MILLISECONDS.toNanos(testPhysicalTime), testLogicalTime); + assertEquals(ts, testPhysicalTime); + } + + @Test + public void testPhysicalIsLikelyOfType() throws ParseException { + SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss:SSS Z"); + + // test that timestamps from 1970 to 3K epoch are of Physical type + for (int year = 1970; year < 3000 ;year += 10) { + // Start date 1970 to 10000 + Date date = dateFormat.parse(year + "-01-01T00:00:00:000 UTC"); + long ts = date.getTime(); + System.out.println(TimestampType.PHYSICAL.toString(ts)); + System.out.println(TimestampType.PHYSICAL.toString(TimestampType.HYBRID.getPhysicalTime(ts))); + + assertTrue(TimestampType.PHYSICAL.isLikelyOfType(ts, true)); + } + + // test that timestamps from currentTime epoch are of Physical type + long systemTimeNow = System.currentTimeMillis(); + System.out.println(TimestampType.PHYSICAL.toString(systemTimeNow)); + assertTrue(TimestampType.PHYSICAL.isLikelyOfType(systemTimeNow, true)); + + // test timestamps of Hybrid type from year 1970 to 2248 are not of Physical type + for (int year = 1970; year <= 2248; year += 1) { + Date date = dateFormat.parse(year + "-01-01T11:22:33:444 UTC"); + + // Hybrid type ts with pt = date and lt = 0 + long ts = TimestampType.HYBRID.toTimestamp(TimeUnit.MILLISECONDS, date.getTime(), 0); + System.out.println(TimestampType.HYBRID.toString(ts)); + System.out.println(TimestampType.PHYSICAL.toString(ts)); + + assertFalse(TimestampType.PHYSICAL.isLikelyOfType(ts, true)); + } + } +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index 2022c5e..9bb406f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -1450,6 +1450,14 @@ public class HMaster extends HRegionServer implements MasterServices { String namespace = hTableDescriptor.getTableName().getNamespaceAsString(); this.clusterSchemaService.getNamespace(namespace); + // set the Clock type to default clock type if not set. As old tables also may not have + // the clock type set. To differentiate between old tables with no clock type set and + // new tables with no clock type set, we need to ensure to set the default value of + // clock type. + if (hTableDescriptor.getValue(hTableDescriptor.CLOCK_TYPE) == null) { + hTableDescriptor.setClockType(hTableDescriptor.DEFAULT_CLOCK_TYPE); + } + HRegionInfo[] newRegions = ModifyRegionUtils.createHRegionInfos(hTableDescriptor, splitKeys); checkInitialized(); sanityCheckTableDescriptor(hTableDescriptor); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java index 2dbc087..c29882c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java @@ -199,7 +199,7 @@ public class RegionStateStore { if (openSeqNum >= 0) { Preconditions.checkArgument(state == State.OPEN && serverName != null, "Open region should be on a server"); - MetaTableAccessor.addLocation(metaPut, serverName, openSeqNum, -1, replicaId); + MetaTableAccessor.addLocation(metaPut, serverName, openSeqNum, replicaId); info.append(", openSeqNum=").append(openSeqNum); info.append(", server=").append(serverName); } @@ -260,8 +260,7 @@ public class RegionStateStore { void mergeRegions(HRegionInfo p, HRegionInfo a, HRegionInfo b, ServerName sn, int regionReplication) throws IOException { - MetaTableAccessor.mergeRegions(server.getConnection(), p, a, b, sn, regionReplication, - EnvironmentEdgeManager.currentTime(), - server.getTableDescriptors().get(p.getTable()).hasSerialReplicationScope()); + MetaTableAccessor.mergeRegions(server.getConnection(), p, a, b, sn, regionReplication, server + .getTableDescriptors().get(p.getTable()).hasSerialReplicationScope()); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 86c02ea..21fbb12 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -105,6 +105,7 @@ import org.apache.hadoop.hbase.RegionTooBusyException; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.TagUtil; +import org.apache.hadoop.hbase.Clock; import org.apache.hadoop.hbase.UnknownScannerException; import org.apache.hadoop.hbase.backup.HFileArchiver; import org.apache.hadoop.hbase.classification.InterfaceAudience; @@ -370,6 +371,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi return minimumReadPoint; } + @Override + public Clock getClock() { + return this.clock; + } + /* * Data structure of write state flags used coordinating flushes, * compactions and closes. @@ -557,6 +563,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi new ConcurrentHashMap(); final RegionServerServices rsServices; + final private Clock clock; private RegionServerAccounting rsAccounting; private long flushCheckInterval; // flushPerChanges is to prevent too many changes in memstore @@ -713,6 +720,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi ? DEFAULT_DURABILITY : htd.getDurability(); if (rsServices != null) { + this.clock = rsServices.getRegionServerClock(htd.getClockType()); this.rsAccounting = this.rsServices.getRegionServerAccounting(); // don't initialize coprocessors if not running within a regionserver // TODO: revisit if coprocessors should load in other cases @@ -727,6 +735,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi recoveringRegions.put(encodedName, this); } } else { + Clock systemClock = new Clock.System(); + this.clock = systemClock; this.metricsRegionWrapper = null; this.metricsRegion = null; } @@ -3048,7 +3058,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi try { // STEP 1. Try to acquire as many locks as we can, and ensure we acquire at least one. int numReadyToWrite = 0; - long now = EnvironmentEdgeManager.currentTime(); + long now = this.clock.now(); while (lastIndexExclusive < batchOp.operations.length) { if (checkBatchOp(batchOp, lastIndexExclusive, familyMaps, now)) { lastIndexExclusive++; @@ -3101,7 +3111,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // STEP 2. Update any LATEST_TIMESTAMP timestamps // We should record the timestamp only after we have acquired the rowLock, // otherwise, newer puts/deletes are not guaranteed to have a newer timestamp - now = EnvironmentEdgeManager.currentTime(); + now = clock.now(); byte[] byteNow = Bytes.toBytes(now); // Nothing to put/delete -- an exception in the above such as NoSuchColumnFamily? @@ -3549,9 +3559,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // non-decreasing (see HBASE-14070) we should make sure that the mutation has a // larger timestamp than what was observed via Get. doBatchMutate already does this, but // there is no way to pass the cellTs. See HBASE-14054. - long now = EnvironmentEdgeManager.currentTime(); - long ts = Math.max(now, cellTs); // ensure write is not eclipsed - byte[] byteTs = Bytes.toBytes(ts); + long now = clock.now(); + if(!clock.isMonotonic()) { + now = Math.max(now, cellTs); // ensure write is not eclipsed + } + byte[] byteTs = Bytes.toBytes(now); if (mutation != null) { if (mutation instanceof Put) { updateCellTimestamps(mutation.getFamilyCellMap().values(), byteTs); @@ -3856,7 +3868,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi if (timestampSlop == HConstants.LATEST_TIMESTAMP) { return; } - long maxTs = now + timestampSlop; + long maxTs = clock.getTimestampType().getPhysicalTime(now) + timestampSlop; for (List kvs : familyMap.values()) { assert kvs instanceof RandomAccess; int listSize = kvs.size(); @@ -6966,7 +6978,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // Short circuit the read only case if (processor.readOnly()) { try { - long now = EnvironmentEdgeManager.currentTime(); + long now = clock.now(); doProcessRowWithTimeout(processor, now, this, null, null, timeout); processor.postProcess(this, walEdit, true); } finally { @@ -6995,7 +7007,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi lock(this.updatesLock.readLock(), acquiredRowLocks.size() == 0 ? 1 : acquiredRowLocks.size()); locked = true; boolean success = false; - long now = EnvironmentEdgeManager.currentTime(); + long now = clock.now(); try { // STEP 4. Let the processor scan the rows, generate mutations and add waledits doProcessRowWithTimeout(processor, now, this, mutations, walEdit, timeout); @@ -7304,7 +7316,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi final List results) throws IOException { WALEdit walEdit = null; - long now = EnvironmentEdgeManager.currentTime(); + long now = clock.now(); final boolean writeToWAL = effectiveDurability != Durability.SKIP_WAL; // Process a Store/family at a time. for (Map.Entry> entry: mutation.getFamilyCellMap().entrySet()) { @@ -7471,7 +7483,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } } else { // Append's KeyValue.Type==Put and ts==HConstants.LATEST_TIMESTAMP - CellUtil.updateLatestStamp(delta, now); + CellUtil.updateLatestStamp(delta, clock.now()); newCell = delta; tags = TagUtil.carryForwardTTLTag(tags, mutation.getTTL()); if (tags != null) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 89bfbf3..5ad04d9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -69,6 +69,8 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.Clock; +import org.apache.hadoop.hbase.ClockType; import org.apache.hadoop.hbase.HealthCheckChore; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.NotServingRegionException; @@ -311,6 +313,10 @@ public class HRegionServer extends HasThread implements // debugging and unit tests. private volatile boolean abortRequested; + final protected Clock hybridLogicalClock; + final protected Clock systemMonotonicClock; + final protected Clock systemClock; + ConcurrentMap rowlocks = new ConcurrentHashMap(); // A state before we go into stopped state. At this stage we're closing user @@ -554,6 +560,10 @@ public class HRegionServer extends HasThread implements this.abortRequested = false; this.stopped = false; + this.hybridLogicalClock = new Clock.HLC(); + this.systemMonotonicClock = new Clock.SystemMonotonic(); + this.systemClock = new Clock.System(); + rpcServices = createRpcServices(); this.startcode = System.currentTimeMillis(); if (this instanceof HMaster) { @@ -1925,6 +1935,19 @@ public class HRegionServer extends HasThread implements } @Override + public Clock getRegionServerClock(ClockType clockType) { + Clock systemClock = new Clock.System(); + return systemClock; + /*if(clockType.equals(ClockType.HLC)){ + return new Clock.System(); + } else if(clockType.equals(ClockType.SYSTEM_MONOTONIC)) { + return new Clock.System(); + } else { + return new Clock.System(); + }*/ + } + + @Override public Connection getConnection() { return getClusterConnection(); } @@ -2027,7 +2050,6 @@ public class HRegionServer extends HasThread implements public boolean reportRegionStateTransition(final RegionStateTransitionContext context) { TransitionCode code = context.getCode(); long openSeqNum = context.getOpenSeqNum(); - long masterSystemTime = context.getMasterSystemTime(); HRegionInfo[] hris = context.getHris(); if (TEST_SKIP_REPORTING_TRANSITION) { @@ -2046,7 +2068,7 @@ public class HRegionServer extends HasThread implements } else { try { MetaTableAccessor.updateRegionLocation(clusterConnection, - hris[0], serverName, openSeqNum, masterSystemTime); + hris[0], serverName, openSeqNum); } catch (IOException e) { LOG.info("Failed to update meta", e); return false; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java index c4bd849..1abc67e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java @@ -62,6 +62,8 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.TimestampType; +import org.apache.hadoop.hbase.Clock; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.TagType; import org.apache.hadoop.hbase.TagUtil; @@ -303,10 +305,10 @@ public class HStore implements Store { /** * @param family - * @return TTL in seconds of the specified family + * @return TTL in milli seconds of the specified family */ public static long determineTTLFromFamily(final HColumnDescriptor family) { - // HCD.getTimeToLive returns ttl in seconds. Convert to milliseconds. + // HColumnDescriptor.getTimeToLive returns ttl in seconds. Convert to milliseconds. long ttl = family.getTimeToLive(); if (ttl == HConstants.FOREVER) { // Default is unlimited ttl. @@ -358,6 +360,11 @@ public class HStore implements Store { } @Override + public Clock getClock() { + return region.getClock(); + } + + @Override public long getSnapshotSize() { return this.memstore.getSnapshotSize(); } @@ -1750,6 +1757,36 @@ public class HStore implements Store { return wantedVersions > maxVersions ? maxVersions: wantedVersions; } + /** + * @param cell + * @param oldestTimestamp + * @return true if the cell is expired + */ + static boolean isCellTTLExpired(final Cell cell, final long oldestTimestamp, final long now, + TimestampType timestampType) { + // Look for a TTL tag first. Use it instead of the family setting if + // found. If a cell has multiple TTLs, resolve the conflict by using the + // first tag encountered. + Iterator i = CellUtil.tagsIterator(cell); + while (i.hasNext()) { + Tag t = i.next(); + if (TagType.TTL_TAG_TYPE == t.getType()) { + // Unlike in schema cell TTLs are stored in milliseconds, no need + // to convert + long ts = cell.getTimestamp(); + assert t.getValueLength() == Bytes.SIZEOF_LONG; + long ttl = TagUtil.getValueAsLong(t); + if (timestampType.fromEpochTimeMillisToTimestamp(ts) + ttl < now) { + return true; + } + // Per cell TTLs cannot extend lifetime beyond family settings, so + // fall through to check that + break; + } + } + return false; + } + @Override public boolean canSplit() { this.lock.readLock().lock(); @@ -2048,7 +2085,7 @@ public class HStore implements Store { this.lock.readLock().lock(); try { - long now = EnvironmentEdgeManager.currentTime(); + long now = this.getHRegion().getClock().now(); return this.memstore.updateColumnValue(row, f, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java index efd68b8..786993a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java @@ -27,6 +27,7 @@ import java.util.List; import java.util.Map; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.Clock; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HDFSBlocksDistribution; @@ -81,6 +82,11 @@ public interface Region extends ConfigurationObserver { /** @return table descriptor for this region */ HTableDescriptor getTableDesc(); + /** @return clock of the Region Server corresponding the clock type used by the + * table contained in this region. + */ + Clock getClock(); + /** @return true if region is available (not closed and not closing) */ boolean isAvailable(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransactionImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransactionImpl.java index 9e7f97b..823d30a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransactionImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransactionImpl.java @@ -341,19 +341,16 @@ public class RegionMergeTransactionImpl implements RegionMergeTransaction { HRegionInfo regionB, ServerName serverName, List mutations) throws IOException { HRegionInfo copyOfMerged = new HRegionInfo(mergedRegion); - // use the maximum of what master passed us vs local time. - long time = Math.max(EnvironmentEdgeManager.currentTime(), masterSystemTime); - // Put for parent - Put putOfMerged = MetaTableAccessor.makePutFromRegionInfo(copyOfMerged, time); + Put putOfMerged = MetaTableAccessor.makePutFromRegionInfo(copyOfMerged); putOfMerged.addColumn(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER, regionA.toByteArray()); putOfMerged.addColumn(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER, regionB.toByteArray()); mutations.add(putOfMerged); // Deletes for merging regions - Delete deleteA = MetaTableAccessor.makeDeleteFromRegionInfo(regionA, time); - Delete deleteB = MetaTableAccessor.makeDeleteFromRegionInfo(regionB, time); + Delete deleteA = MetaTableAccessor.makeDeleteFromRegionInfo(regionA); + Delete deleteB = MetaTableAccessor.makeDeleteFromRegionInfo(regionB); mutations.add(deleteA); mutations.add(deleteB); // The merged is a new region, openSeqNum = 1 is fine. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java index 356a88b..cde287d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java @@ -36,6 +36,8 @@ import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.Regio import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.Clock; +import org.apache.hadoop.hbase.ClockType; import org.apache.zookeeper.KeeperException; import com.google.protobuf.Service; @@ -55,6 +57,8 @@ public interface RegionServerServices extends OnlineRegions, FavoredNodesForRegi * default (common) WAL */ WAL getWAL(HRegionInfo regionInfo) throws IOException; + Clock getRegionServerClock(ClockType clockType); + /** * @return Implementation of {@link CompactionRequestor} or null. */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java index 853a4cf..d8fdb99 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.Clock; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.client.Scan; @@ -321,6 +322,12 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf long getFlushableSize(); /** + * @return clock of the Region Server corresponding the clock type used by the + * table referred to by this store. + */ + Clock getClock(); + + /** * Returns the memstore snapshot size * @return size of the memstore snapshot */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java index e008a40..17869ae 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java @@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.TimestampType; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.IsolationLevel; import org.apache.hadoop.hbase.client.Scan; @@ -54,6 +55,8 @@ import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher.MatchC import org.apache.hadoop.hbase.regionserver.querymatcher.UserScanQueryMatcher; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import com.google.common.annotations.VisibleForTesting; + /** * Scanner scans both the memstore and the Store. Coalesce KeyValue stream * into List<KeyValue> for a single row. @@ -90,6 +93,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner protected final int minVersions; protected final long maxRowSize; protected final long cellsPerHeartbeatCheck; + private final TimestampType timestampType; // Collects all the KVHeap that are eagerly getting closed during the // course of a scan @@ -166,19 +170,20 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner // the seek operation. However, we also look the row-column Bloom filter // for multi-row (non-"get") scans because this is not done in // StoreFile.passesBloomFilter(Scan, SortedSet). - this.useRowColBloom = numCol > 1 || (!get && numCol == 1); - - this.maxRowSize = scanInfo.getTableMaxRowSize(); - this.scanUsePread = scan.isSmall()? true: scanInfo.isUsePread(); - this.cellsPerHeartbeatCheck = scanInfo.getCellsPerTimeoutCheck(); - // Parallel seeking is on if the config allows and more there is more than one store file. - if (this.store != null && this.store.getStorefilesCount() > 1) { - RegionServerServices rsService = ((HStore)store).getHRegion().getRegionServerServices(); - if (rsService != null && scanInfo.isParallelSeekEnabled()) { - this.parallelSeekEnabled = true; - this.executor = rsService.getExecutorService(); - } - } + this.useRowColBloom = numCol > 1 || (!get && numCol == 1); + + this.maxRowSize = scanInfo.getTableMaxRowSize(); + this.scanUsePread = scan.isSmall()? true: scanInfo.isUsePread(); + this.timestampType = store == null? TimestampType.PHYSICAL:store.getClock().getTimestampType(); + this.cellsPerHeartbeatCheck = scanInfo.getCellsPerTimeoutCheck(); + // Parallel seeking is on if the config allows and more there is more than one store file. + if (this.store != null && this.store.getStorefilesCount() > 1) { + RegionServerServices rsService = ((HStore)store).getHRegion().getRegionServerServices(); + if (rsService != null && scanInfo.isParallelSeekEnabled()) { + this.parallelSeekEnabled = true; + this.executor = rsService.getExecutorService(); + } + } } protected void addCurrentScanners(List scanners) { @@ -202,7 +207,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner throw new DoNotRetryIOException("Cannot specify any column for a raw scan"); } matcher = UserScanQueryMatcher.create(scan, scanInfo, columns, oldestUnexpiredTS, now, - store.getCoprocessorHost()); + timestampType, store.getCoprocessorHost()); this.store.addChangedReaderObserver(this); @@ -278,12 +283,12 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner // use legacy query matcher since we do not consider the scan object in our code. Only used to // keep compatibility for coprocessor. matcher = LegacyScanQueryMatcher.create(scan, scanInfo, null, scanType, smallestReadPoint, - earliestPutTs, oldestUnexpiredTS, now, dropDeletesFromRow, dropDeletesToRow, - store.getCoprocessorHost()); + earliestPutTs, oldestUnexpiredTS, now, timestampType, dropDeletesFromRow, + dropDeletesToRow, store.getCoprocessorHost()); } else { matcher = CompactionScanQueryMatcher.create(scanInfo, scanType, smallestReadPoint, - earliestPutTs, oldestUnexpiredTS, now, dropDeletesFromRow, dropDeletesToRow, - store.getCoprocessorHost()); + earliestPutTs, oldestUnexpiredTS, now, timestampType, dropDeletesFromRow, + dropDeletesToRow, store.getCoprocessorHost()); } // Filter the list of scanners using Bloom filters, time range, TTL, etc. @@ -319,11 +324,10 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner public StoreScanner(final Scan scan, ScanInfo scanInfo, ScanType scanType, final NavigableSet columns, final List scanners, long earliestPutTs, long readPt) throws IOException { - this(null, scan, scanInfo, columns, readPt, - scanType == ScanType.USER_SCAN ? scan.getCacheBlocks() : false); + this(null, scan, scanInfo, columns, readPt, scan.getCacheBlocks()); if (scanType == ScanType.USER_SCAN) { this.matcher = UserScanQueryMatcher.create(scan, scanInfo, columns, oldestUnexpiredTS, now, - null); + timestampType, null); } else { if (scan.hasFilter() || (scan.getStartRow() != null && scan.getStartRow().length > 0) || (scan.getStopRow() != null && scan.getStopRow().length > 0) @@ -331,10 +335,12 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner // use legacy query matcher since we do not consider the scan object in our code. Only used // to keep compatibility for coprocessor. matcher = LegacyScanQueryMatcher.create(scan, scanInfo, columns, scanType, Long.MAX_VALUE, - earliestPutTs, oldestUnexpiredTS, now, null, null, store.getCoprocessorHost()); + earliestPutTs, oldestUnexpiredTS, now, timestampType, null, null, + store.getCoprocessorHost()); } else { this.matcher = CompactionScanQueryMatcher.create(scanInfo, scanType, Long.MAX_VALUE, - earliestPutTs, oldestUnexpiredTS, now, null, null, null); + earliestPutTs, oldestUnexpiredTS, now, timestampType, null, null, + null); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/CompactionScanQueryMatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/CompactionScanQueryMatcher.java index d3224dc..cdc6b59 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/CompactionScanQueryMatcher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/CompactionScanQueryMatcher.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver.querymatcher; import java.io.IOException; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.TimestampType; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeepDeletedCells; import org.apache.hadoop.hbase.classification.InterfaceAudience; @@ -44,11 +45,10 @@ public abstract class CompactionScanQueryMatcher extends ScanQueryMatcher { protected final KeepDeletedCells keepDeletedCells; protected CompactionScanQueryMatcher(ScanInfo scanInfo, DeleteTracker deletes, - long readPointToUse, long oldestUnexpiredTS, long now) { + long readPointToUse, long oldestUnexpiredTS, long now, TimestampType timestampType) { super(HConstants.EMPTY_START_ROW, scanInfo, new ScanWildcardColumnTracker(scanInfo.getMinVersions(), scanInfo.getMaxVersions(), - oldestUnexpiredTS), - oldestUnexpiredTS, now); + oldestUnexpiredTS), oldestUnexpiredTS, now, timestampType); this.maxReadPointToTrackVersions = readPointToUse; this.deletes = deletes; this.keepDeletedCells = scanInfo.getKeepDeletedCells(); @@ -99,21 +99,22 @@ public abstract class CompactionScanQueryMatcher extends ScanQueryMatcher { } public static CompactionScanQueryMatcher create(ScanInfo scanInfo, ScanType scanType, - long readPointToUse, long earliestPutTs, long oldestUnexpiredTS, long now, - byte[] dropDeletesFromRow, byte[] dropDeletesToRow, + long readPointToUse, long earliestPutTs, long oldestUnexpiredTS, long now, TimestampType + timestampType, byte[] dropDeletesFromRow, byte[] dropDeletesToRow, RegionCoprocessorHost regionCoprocessorHost) throws IOException { DeleteTracker deleteTracker = instantiateDeleteTracker(regionCoprocessorHost); if (dropDeletesFromRow == null) { if (scanType == ScanType.COMPACT_RETAIN_DELETES) { return new MinorCompactionScanQueryMatcher(scanInfo, deleteTracker, readPointToUse, - oldestUnexpiredTS, now); + oldestUnexpiredTS, now, timestampType); } else { return new MajorCompactionScanQueryMatcher(scanInfo, deleteTracker, readPointToUse, - earliestPutTs, oldestUnexpiredTS, now); + earliestPutTs, oldestUnexpiredTS, now, timestampType); } } else { return new StripeCompactionScanQueryMatcher(scanInfo, deleteTracker, readPointToUse, - earliestPutTs, oldestUnexpiredTS, now, dropDeletesFromRow, dropDeletesToRow); + earliestPutTs, oldestUnexpiredTS, now, timestampType, dropDeletesFromRow, + dropDeletesToRow); } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/DropDeletesCompactionScanQueryMatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/DropDeletesCompactionScanQueryMatcher.java index 89725fe..06aa9a6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/DropDeletesCompactionScanQueryMatcher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/DropDeletesCompactionScanQueryMatcher.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.regionserver.querymatcher; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.TimestampType; import org.apache.hadoop.hbase.KeepDeletedCells; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.regionserver.ScanInfo; @@ -53,8 +54,9 @@ public abstract class DropDeletesCompactionScanQueryMatcher extends CompactionSc protected final long earliestPutTs; protected DropDeletesCompactionScanQueryMatcher(ScanInfo scanInfo, DeleteTracker deletes, - long readPointToUse, long earliestPutTs, long oldestUnexpiredTS, long now) { - super(scanInfo, deletes, readPointToUse, oldestUnexpiredTS, now); + long readPointToUse, long earliestPutTs, long oldestUnexpiredTS, long now, TimestampType + timestampType) { + super(scanInfo, deletes, readPointToUse, oldestUnexpiredTS, now, timestampType); this.timeToPurgeDeletes = scanInfo.getTimeToPurgeDeletes(); this.earliestPutTs = earliestPutTs; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ExplicitColumnTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ExplicitColumnTracker.java index da65c78..1093449 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ExplicitColumnTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ExplicitColumnTracker.java @@ -65,7 +65,6 @@ public class ExplicitColumnTracker implements ColumnTracker { */ private long latestTSOfCurrentColumn; private long oldestStamp; - /** * Default constructor. * @param columns columns specified user in query diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/LegacyScanQueryMatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/LegacyScanQueryMatcher.java index ea4bd97..1c1d195 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/LegacyScanQueryMatcher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/LegacyScanQueryMatcher.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.Filter.ReturnCode; import org.apache.hadoop.hbase.io.TimeRange; +import org.apache.hadoop.hbase.TimestampType; import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost; import org.apache.hadoop.hbase.regionserver.ScanInfo; import org.apache.hadoop.hbase.regionserver.ScanType; @@ -115,8 +116,8 @@ public class LegacyScanQueryMatcher extends ScanQueryMatcher { private LegacyScanQueryMatcher(Scan scan, ScanInfo scanInfo, ColumnTracker columns, boolean hasNullColumn, DeleteTracker deletes, ScanType scanType, long readPointToUse, - long earliestPutTs, long oldestUnexpiredTS, long now) { - super(scan.getStartRow(), scanInfo, columns, oldestUnexpiredTS, now); + long earliestPutTs, long oldestUnexpiredTS, long now, TimestampType timestampType) { + super(scan.getStartRow(), scanInfo, columns, oldestUnexpiredTS, now, timestampType); TimeRange timeRange = scan.getColumnFamilyTimeRange().get(scanInfo.getFamily()); if (timeRange == null) { this.tr = scan.getTimeRange(); @@ -138,10 +139,11 @@ public class LegacyScanQueryMatcher extends ScanQueryMatcher { private LegacyScanQueryMatcher(Scan scan, ScanInfo scanInfo, ColumnTracker columns, boolean hasNullColumn, DeleteTracker deletes, ScanType scanType, long readPointToUse, - long earliestPutTs, long oldestUnexpiredTS, long now, byte[] dropDeletesFromRow, + long earliestPutTs, long oldestUnexpiredTS, long now, TimestampType timestampType, byte[] + dropDeletesFromRow, byte[] dropDeletesToRow) { this(scan, scanInfo, columns, hasNullColumn, deletes, scanType, readPointToUse, earliestPutTs, - oldestUnexpiredTS, now); + oldestUnexpiredTS, now, timestampType); this.dropDeletesFromRow = Preconditions.checkNotNull(dropDeletesFromRow); this.dropDeletesToRow = Preconditions.checkNotNull(dropDeletesToRow); } @@ -365,7 +367,8 @@ public class LegacyScanQueryMatcher extends ScanQueryMatcher { public static LegacyScanQueryMatcher create(Scan scan, ScanInfo scanInfo, NavigableSet columns, ScanType scanType, long readPointToUse, long earliestPutTs, - long oldestUnexpiredTS, long now, byte[] dropDeletesFromRow, byte[] dropDeletesToRow, + long oldestUnexpiredTS, long now, TimestampType timestampType, byte[] dropDeletesFromRow, + byte[] dropDeletesToRow, RegionCoprocessorHost regionCoprocessorHost) throws IOException { int maxVersions = Math.min(scan.getMaxVersions(), scanInfo.getMaxVersions()); boolean hasNullColumn; @@ -387,11 +390,11 @@ public class LegacyScanQueryMatcher extends ScanQueryMatcher { DeleteTracker deletes = instantiateDeleteTracker(regionCoprocessorHost); if (dropDeletesFromRow == null) { return new LegacyScanQueryMatcher(scan, scanInfo, columnTracker, hasNullColumn, deletes, - scanType, readPointToUse, earliestPutTs, oldestUnexpiredTS, now); + scanType, readPointToUse, earliestPutTs, oldestUnexpiredTS, now, timestampType); } else { return new LegacyScanQueryMatcher(scan, scanInfo, columnTracker, hasNullColumn, deletes, - scanType, readPointToUse, earliestPutTs, oldestUnexpiredTS, now, dropDeletesFromRow, - dropDeletesToRow); + scanType, readPointToUse, earliestPutTs, oldestUnexpiredTS, now, timestampType, + dropDeletesFromRow, dropDeletesToRow); } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/MajorCompactionScanQueryMatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/MajorCompactionScanQueryMatcher.java index 6a2ed40..9df86ca 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/MajorCompactionScanQueryMatcher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/MajorCompactionScanQueryMatcher.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver.querymatcher; import java.io.IOException; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.TimestampType; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.regionserver.ScanInfo; @@ -31,8 +32,9 @@ import org.apache.hadoop.hbase.regionserver.ScanInfo; public class MajorCompactionScanQueryMatcher extends DropDeletesCompactionScanQueryMatcher { public MajorCompactionScanQueryMatcher(ScanInfo scanInfo, DeleteTracker deletes, - long readPointToUse, long earliestPutTs, long oldestUnexpiredTS, long now) { - super(scanInfo, deletes, readPointToUse, earliestPutTs, oldestUnexpiredTS, now); + long readPointToUse, long earliestPutTs, long oldestUnexpiredTS, long now, TimestampType + timestampType) { + super(scanInfo, deletes, readPointToUse, earliestPutTs, oldestUnexpiredTS, now, timestampType); } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/MinorCompactionScanQueryMatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/MinorCompactionScanQueryMatcher.java index 3b6acde..493774a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/MinorCompactionScanQueryMatcher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/MinorCompactionScanQueryMatcher.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver.querymatcher; import java.io.IOException; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.TimestampType; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.regionserver.ScanInfo; @@ -31,8 +32,8 @@ import org.apache.hadoop.hbase.regionserver.ScanInfo; public class MinorCompactionScanQueryMatcher extends CompactionScanQueryMatcher { public MinorCompactionScanQueryMatcher(ScanInfo scanInfo, DeleteTracker deletes, - long readPointToUse, long oldestUnexpiredTS, long now) { - super(scanInfo, deletes, readPointToUse, oldestUnexpiredTS, now); + long readPointToUse, long oldestUnexpiredTS, long now, TimestampType timestampType) { + super(scanInfo, deletes, readPointToUse, oldestUnexpiredTS, now, timestampType); } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/NormalUserScanQueryMatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/NormalUserScanQueryMatcher.java index 3942f04..9b660af 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/NormalUserScanQueryMatcher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/NormalUserScanQueryMatcher.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver.querymatcher; import java.io.IOException; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.TimestampType; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.KeepDeletedCells; import org.apache.hadoop.hbase.classification.InterfaceAudience; @@ -43,8 +44,9 @@ public class NormalUserScanQueryMatcher extends UserScanQueryMatcher { private final boolean seePastDeleteMarkers; protected NormalUserScanQueryMatcher(Scan scan, ScanInfo scanInfo, ColumnTracker columns, - boolean hasNullColumn, DeleteTracker deletes, long oldestUnexpiredTS, long now) { - super(scan, scanInfo, columns, hasNullColumn, oldestUnexpiredTS, now); + boolean hasNullColumn, DeleteTracker deletes, long oldestUnexpiredTS, long now, TimestampType + timestampType) { + super(scan, scanInfo, columns, hasNullColumn, oldestUnexpiredTS, now, timestampType); this.deletes = deletes; this.get = scan.isGetScan(); this.seePastDeleteMarkers = scanInfo.getKeepDeletedCells() != KeepDeletedCells.FALSE; @@ -87,11 +89,11 @@ public class NormalUserScanQueryMatcher extends UserScanQueryMatcher { public static NormalUserScanQueryMatcher create(Scan scan, ScanInfo scanInfo, ColumnTracker columns, boolean hasNullColumn, long oldestUnexpiredTS, long now, - RegionCoprocessorHost regionCoprocessorHost) throws IOException { + TimestampType timestampType, RegionCoprocessorHost regionCoprocessorHost) throws IOException { DeleteTracker deletes = instantiateDeleteTracker(regionCoprocessorHost); if (scan.isReversed()) { return new NormalUserScanQueryMatcher(scan, scanInfo, columns, hasNullColumn, deletes, - oldestUnexpiredTS, now) { + oldestUnexpiredTS, now, timestampType) { @Override protected boolean moreRowsMayExistsAfter(int cmpToStopRow) { @@ -100,7 +102,7 @@ public class NormalUserScanQueryMatcher extends UserScanQueryMatcher { }; } else { return new NormalUserScanQueryMatcher(scan, scanInfo, columns, hasNullColumn, deletes, - oldestUnexpiredTS, now); + oldestUnexpiredTS, now, timestampType); } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/RawScanQueryMatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/RawScanQueryMatcher.java index acdae90..7c2d873 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/RawScanQueryMatcher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/RawScanQueryMatcher.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver.querymatcher; import java.io.IOException; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.TimestampType; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.regionserver.ScanInfo; @@ -31,8 +32,8 @@ import org.apache.hadoop.hbase.regionserver.ScanInfo; public class RawScanQueryMatcher extends UserScanQueryMatcher { protected RawScanQueryMatcher(Scan scan, ScanInfo scanInfo, ColumnTracker columns, - boolean hasNullColumn, long oldestUnexpiredTS, long now) { - super(scan, scanInfo, columns, hasNullColumn, oldestUnexpiredTS, now); + boolean hasNullColumn, long oldestUnexpiredTS, long now, TimestampType timestampType) { + super(scan, scanInfo, columns, hasNullColumn, oldestUnexpiredTS, now, timestampType); } @Override @@ -59,10 +60,10 @@ public class RawScanQueryMatcher extends UserScanQueryMatcher { } public static RawScanQueryMatcher create(Scan scan, ScanInfo scanInfo, ColumnTracker columns, - boolean hasNullColumn, long oldestUnexpiredTS, long now) { + boolean hasNullColumn, long oldestUnexpiredTS, long now, TimestampType timestampType) { if (scan.isReversed()) { return new RawScanQueryMatcher(scan, scanInfo, columns, hasNullColumn, oldestUnexpiredTS, - now) { + now, timestampType) { @Override protected boolean moreRowsMayExistsAfter(int cmpToStopRow) { @@ -71,7 +72,7 @@ public class RawScanQueryMatcher extends UserScanQueryMatcher { }; } else { return new RawScanQueryMatcher(scan, scanInfo, columns, hasNullColumn, oldestUnexpiredTS, - now); + now, timestampType); } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ScanQueryMatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ScanQueryMatcher.java index b5469d3..b1ea046 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ScanQueryMatcher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ScanQueryMatcher.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.TagType; import org.apache.hadoop.hbase.TagUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.TimestampType; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost; import org.apache.hadoop.hbase.regionserver.ScanInfo; @@ -121,18 +122,21 @@ public abstract class ScanQueryMatcher { protected final long now; + protected final TimestampType timestampType; + /** Row the query is on */ protected Cell currentRow; protected boolean stickyNextRow; protected ScanQueryMatcher(byte[] startRow, ScanInfo scanInfo, ColumnTracker columns, - long oldestUnexpiredTS, long now) { + long oldestUnexpiredTS, long now, TimestampType timestampType) { this.rowComparator = scanInfo.getComparator(); this.startKey = CellUtil.createFirstDeleteFamilyCellOnRow(startRow, scanInfo.getFamily()); this.oldestUnexpiredTS = oldestUnexpiredTS; this.now = now; this.columns = columns; + this.timestampType = timestampType; } /** @@ -140,8 +144,8 @@ public abstract class ScanQueryMatcher { * @param oldestTimestamp * @return true if the cell is expired */ - private static boolean isCellTTLExpired(final Cell cell, final long oldestTimestamp, - final long now) { + private static boolean isCellTTLExpired(final Cell cell, final long oldestTimestamp, final long + now, TimestampType timestampType) { // Look for a TTL tag first. Use it instead of the family setting if // found. If a cell has multiple TTLs, resolve the conflict by using the // first tag encountered. @@ -154,7 +158,7 @@ public abstract class ScanQueryMatcher { long ts = cell.getTimestamp(); assert t.getValueLength() == Bytes.SIZEOF_LONG; long ttl = TagUtil.getValueAsLong(t); - if (ts + ttl < now) { + if (ts < timestampType.fromEpochTimeMillisToTimestamp(Math.max(0, now - ttl))) { return true; } // Per cell TTLs cannot extend lifetime beyond family settings, so @@ -197,7 +201,7 @@ public abstract class ScanQueryMatcher { return columns.getNextRowOrNextColumn(cell); } // check if the cell is expired by cell TTL - if (isCellTTLExpired(cell, this.oldestUnexpiredTS, this.now)) { + if (isCellTTLExpired(cell, this.oldestUnexpiredTS, this.now, this.timestampType)) { return MatchCode.SKIP; } return null; @@ -209,16 +213,16 @@ public abstract class ScanQueryMatcher { } DeleteResult deleteResult = deletes.isDeleted(cell); switch (deleteResult) { - case FAMILY_DELETED: - case COLUMN_DELETED: - return columns.getNextRowOrNextColumn(cell); - case VERSION_DELETED: - case FAMILY_VERSION_DELETED: - return MatchCode.SKIP; - case NOT_DELETED: - return null; - default: - throw new RuntimeException("Unexpected delete result: " + deleteResult); + case FAMILY_DELETED: + case COLUMN_DELETED: + return columns.getNextRowOrNextColumn(cell); + case VERSION_DELETED: + case FAMILY_VERSION_DELETED: + return MatchCode.SKIP; + case NOT_DELETED: + return null; + default: + throw new RuntimeException("Unexpected delete result: " + deleteResult); } } @@ -292,7 +296,7 @@ public abstract class ScanQueryMatcher { return CellUtil.createLastOnRowCol(cell); } else { return CellUtil.createFirstOnRowCol(cell, nextColumn.getBuffer(), nextColumn.getOffset(), - nextColumn.getLength()); + nextColumn.getLength()); } } @@ -303,7 +307,7 @@ public abstract class ScanQueryMatcher { */ public int compareKeyForNextRow(Cell nextIndexed, Cell currentCell) { return rowComparator.compareKeyBasedOnColHint(nextIndexed, currentCell, 0, 0, null, 0, 0, - HConstants.OLDEST_TIMESTAMP, Type.Minimum.getCode()); + HConstants.OLDEST_TIMESTAMP, Type.Minimum.getCode()); } /** @@ -315,12 +319,12 @@ public abstract class ScanQueryMatcher { ColumnCount nextColumn = columns.getColumnHint(); if (nextColumn == null) { return rowComparator.compareKeyBasedOnColHint(nextIndexed, currentCell, 0, 0, null, 0, 0, - HConstants.OLDEST_TIMESTAMP, Type.Minimum.getCode()); + HConstants.OLDEST_TIMESTAMP, Type.Minimum.getCode()); } else { return rowComparator.compareKeyBasedOnColHint(nextIndexed, currentCell, - currentCell.getFamilyOffset(), currentCell.getFamilyLength(), nextColumn.getBuffer(), - nextColumn.getOffset(), nextColumn.getLength(), HConstants.LATEST_TIMESTAMP, - Type.Maximum.getCode()); + currentCell.getFamilyOffset(), currentCell.getFamilyLength(), nextColumn.getBuffer(), + nextColumn.getOffset(), nextColumn.getLength(), HConstants.LATEST_TIMESTAMP, + Type.Maximum.getCode()); } } @@ -347,11 +351,11 @@ public abstract class ScanQueryMatcher { static MatchCode checkColumn(ColumnTracker columnTracker, byte[] bytes, int offset, int length, long ttl, byte type, boolean ignoreCount) throws IOException { KeyValue kv = KeyValueUtil.createFirstOnRow(HConstants.EMPTY_BYTE_ARRAY, 0, 0, - HConstants.EMPTY_BYTE_ARRAY, 0, 0, bytes, offset, length); + HConstants.EMPTY_BYTE_ARRAY, 0, 0, bytes, offset, length); MatchCode matchCode = columnTracker.checkColumn(kv, type); if (matchCode == MatchCode.INCLUDE) { return columnTracker.checkVersions(kv, ttl, type, ignoreCount); } return matchCode; } -} +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/StripeCompactionScanQueryMatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/StripeCompactionScanQueryMatcher.java index c1e63b4..fbab233 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/StripeCompactionScanQueryMatcher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/StripeCompactionScanQueryMatcher.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver.querymatcher; import java.io.IOException; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.TimestampType; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.regionserver.ScanInfo; @@ -41,9 +42,9 @@ public class StripeCompactionScanQueryMatcher extends DropDeletesCompactionScanQ private DropDeletesInOutput dropDeletesInOutput = DropDeletesInOutput.BEFORE; public StripeCompactionScanQueryMatcher(ScanInfo scanInfo, DeleteTracker deletes, - long readPointToUse, long earliestPutTs, long oldestUnexpiredTS, long now, - byte[] dropDeletesFromRow, byte[] dropDeletesToRow) { - super(scanInfo, deletes, readPointToUse, earliestPutTs, oldestUnexpiredTS, now); + long readPointToUse, long earliestPutTs, long oldestUnexpiredTS, long now, TimestampType + timestampType, byte[] dropDeletesFromRow, byte[] dropDeletesToRow) { + super(scanInfo, deletes, readPointToUse, earliestPutTs, oldestUnexpiredTS, now, timestampType); this.dropDeletesFromRow = dropDeletesFromRow; this.dropDeletesToRow = dropDeletesToRow; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/UserScanQueryMatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/UserScanQueryMatcher.java index ec7fc11..bdc001c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/UserScanQueryMatcher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/UserScanQueryMatcher.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.NavigableSet; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.TimestampType; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.filter.Filter; @@ -51,8 +52,8 @@ public abstract class UserScanQueryMatcher extends ScanQueryMatcher { protected final TimeRange tr; protected UserScanQueryMatcher(Scan scan, ScanInfo scanInfo, ColumnTracker columns, - boolean hasNullColumn, long oldestUnexpiredTS, long now) { - super(scan.getStartRow(), scanInfo, columns, oldestUnexpiredTS, now); + boolean hasNullColumn, long oldestUnexpiredTS, long now, TimestampType timestampType) { + super(scan.getStartRow(), scanInfo, columns, oldestUnexpiredTS, now, timestampType); this.hasNullColumn = hasNullColumn; this.filter = scan.getFilter(); this.stopRow = scan.getStopRow(); @@ -186,8 +187,8 @@ public abstract class UserScanQueryMatcher extends ScanQueryMatcher { } public static UserScanQueryMatcher create(Scan scan, ScanInfo scanInfo, - NavigableSet columns, long oldestUnexpiredTS, long now, - RegionCoprocessorHost regionCoprocessorHost) throws IOException { + NavigableSet columns, long oldestUnexpiredTS, long now, TimestampType + timestampType, RegionCoprocessorHost regionCoprocessorHost) throws IOException { int maxVersions = scan.isRaw() ? scan.getMaxVersions() : Math.min(scan.getMaxVersions(), scanInfo.getMaxVersions()); boolean hasNullColumn; @@ -208,10 +209,10 @@ public abstract class UserScanQueryMatcher extends ScanQueryMatcher { } if (scan.isRaw()) { return RawScanQueryMatcher.create(scan, scanInfo, columnTracker, hasNullColumn, - oldestUnexpiredTS, now); + oldestUnexpiredTS, now, timestampType); } else { return NormalUserScanQueryMatcher.create(scan, scanInfo, columnTracker, hasNullColumn, - oldestUnexpiredTS, now, regionCoprocessorHost); + oldestUnexpiredTS, now, timestampType, regionCoprocessorHost); } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java index 81dadd9..fa213d6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java @@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableInfoMissingException; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.ClockType; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.regionserver.BloomType; @@ -184,6 +185,7 @@ public class FSTableDescriptors implements TableDescriptors { metaDescriptor.addCoprocessor( "org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint", null, Coprocessor.PRIORITY_SYSTEM, null); + metaDescriptor.setClockType(ClockType.HLC); return metaDescriptor; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java index eaf8d54..f849106 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java @@ -176,7 +176,7 @@ public class HBaseFsckRepair { // see the additional replicas when it is asked to assign. The // final value of these columns will be different and will be updated // by the actual regionservers that start hosting the respective replicas - MetaTableAccessor.addLocation(put, sn, sn.getStartcode(), -1, i); + MetaTableAccessor.addLocation(put, sn, sn.getStartcode(), i); } } meta.put(put); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java index 6f225d6..a09189b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java @@ -250,6 +250,11 @@ public class MockRegionServerServices implements RegionServerServices { return null; } + @Override public Clock getRegionServerClock(ClockType clockType) { + Clock systemClock = new Clock.System(); + return systemClock; + } + @Override public ExecutorService getExecutorService() { return null; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java index d750faf..44fe721 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java @@ -361,20 +361,20 @@ public class TestMetaTableAccessor { Table meta = MetaTableAccessor.getMetaHTable(connection); try { - MetaTableAccessor.updateRegionLocation(connection, primary, serverName0, seqNum0, -1); + MetaTableAccessor.updateRegionLocation(connection, primary, serverName0, seqNum0); // assert that the server, startcode and seqNum columns are there for the primary region assertMetaLocation(meta, primary.getRegionName(), serverName0, seqNum0, 0, true); // add replica = 1 - MetaTableAccessor.updateRegionLocation(connection, replica1, serverName1, seqNum1, -1); + MetaTableAccessor.updateRegionLocation(connection, replica1, serverName1, seqNum1); // check whether the primary is still there assertMetaLocation(meta, primary.getRegionName(), serverName0, seqNum0, 0, true); // now check for replica 1 assertMetaLocation(meta, primary.getRegionName(), serverName1, seqNum1, 1, true); // add replica = 1 - MetaTableAccessor.updateRegionLocation(connection, replica100, serverName100, seqNum100, -1); + MetaTableAccessor.updateRegionLocation(connection, replica100, serverName100, seqNum100); // check whether the primary is still there assertMetaLocation(meta, primary.getRegionName(), serverName0, seqNum0, 0, true); // check whether the replica 1 is still there @@ -480,8 +480,7 @@ public class TestMetaTableAccessor { List regionInfos = Lists.newArrayList(parentA, parentB); MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 3); - MetaTableAccessor.mergeRegions(connection, merged, parentA, parentB, serverName0, 3, - HConstants.LATEST_TIMESTAMP, false); + MetaTableAccessor.mergeRegions(connection, merged, parentA, parentB, serverName0, 3, false); assertEmptyMetaLocation(meta, merged.getRegionName(), 1); assertEmptyMetaLocation(meta, merged.getRegionName(), 2); @@ -534,98 +533,6 @@ public class TestMetaTableAccessor { table.close(); } - /** - * Tests whether maximum of masters system time versus RSs local system time is used - */ - @Test - public void testMastersSystemTimeIsUsedInUpdateLocations() throws IOException { - long regionId = System.currentTimeMillis(); - HRegionInfo regionInfo = new HRegionInfo(TableName.valueOf("table_foo"), - HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, regionId, 0); - - ServerName sn = ServerName.valueOf("bar", 0, 0); - Table meta = MetaTableAccessor.getMetaHTable(connection); - try { - List regionInfos = Lists.newArrayList(regionInfo); - MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 1); - - long masterSystemTime = EnvironmentEdgeManager.currentTime() + 123456789; - MetaTableAccessor.updateRegionLocation(connection, regionInfo, sn, 1, masterSystemTime); - - Get get = new Get(regionInfo.getRegionName()); - Result result = meta.get(get); - Cell serverCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY, - MetaTableAccessor.getServerColumn(0)); - Cell startCodeCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY, - MetaTableAccessor.getStartCodeColumn(0)); - Cell seqNumCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY, - MetaTableAccessor.getSeqNumColumn(0)); - assertNotNull(serverCell); - assertNotNull(startCodeCell); - assertNotNull(seqNumCell); - assertTrue(serverCell.getValueLength() > 0); - assertTrue(startCodeCell.getValueLength() > 0); - assertTrue(seqNumCell.getValueLength() > 0); - assertEquals(masterSystemTime, serverCell.getTimestamp()); - assertEquals(masterSystemTime, startCodeCell.getTimestamp()); - assertEquals(masterSystemTime, seqNumCell.getTimestamp()); - } finally { - meta.close(); - } - } - - @Test - public void testMastersSystemTimeIsUsedInMergeRegions() throws IOException { - long regionId = System.currentTimeMillis(); - HRegionInfo regionInfoA = new HRegionInfo(TableName.valueOf("table_foo"), - HConstants.EMPTY_START_ROW, new byte[] {'a'}, false, regionId, 0); - HRegionInfo regionInfoB = new HRegionInfo(TableName.valueOf("table_foo"), - new byte[] {'a'}, HConstants.EMPTY_END_ROW, false, regionId, 0); - HRegionInfo mergedRegionInfo = new HRegionInfo(TableName.valueOf("table_foo"), - HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, regionId, 0); - - ServerName sn = ServerName.valueOf("bar", 0, 0); - Table meta = MetaTableAccessor.getMetaHTable(connection); - try { - List regionInfos = Lists.newArrayList(regionInfoA, regionInfoB); - MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 1); - - // write the serverName column with a big current time, but set the masters time as even - // bigger. When region merge deletes the rows for regionA and regionB, the serverName columns - // should not be seen by the following get - long serverNameTime = EnvironmentEdgeManager.currentTime() + 100000000; - long masterSystemTime = EnvironmentEdgeManager.currentTime() + 123456789; - - // write the serverName columns - MetaTableAccessor.updateRegionLocation(connection, regionInfoA, sn, 1, serverNameTime); - - // assert that we have the serverName column with expected ts - Get get = new Get(mergedRegionInfo.getRegionName()); - Result result = meta.get(get); - Cell serverCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY, - MetaTableAccessor.getServerColumn(0)); - assertNotNull(serverCell); - assertEquals(serverNameTime, serverCell.getTimestamp()); - - // now merge the regions, effectively deleting the rows for region a and b. - MetaTableAccessor.mergeRegions(connection, mergedRegionInfo, - regionInfoA, regionInfoB, sn, 1, masterSystemTime, false); - - result = meta.get(get); - serverCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY, - MetaTableAccessor.getServerColumn(0)); - Cell startCodeCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY, - MetaTableAccessor.getStartCodeColumn(0)); - Cell seqNumCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY, - MetaTableAccessor.getSeqNumColumn(0)); - assertNull(serverCell); - assertNull(startCodeCell); - assertNull(seqNumCell); - } finally { - meta.close(); - } - } - public static class SpyingRpcSchedulerFactory extends SimpleRpcSchedulerFactory { @Override public RpcScheduler create(Configuration conf, PriorityFunction priority, Abortable server) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java index b59a583..7187ef5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java @@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.ClockType; import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotEnabledException; import org.apache.hadoop.hbase.TableNotFoundException; @@ -420,6 +421,7 @@ public class TestAdmin1 { htd.addFamily(fam1); htd.addFamily(fam2); htd.addFamily(fam3); + htd.setClockType(ClockType.SYSTEM); this.admin.createTable(htd); Table table = TEST_UTIL.getConnection().getTable(htd.getTableName()); HTableDescriptor confirmedHtd = table.getTableDescriptor(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java index 2927023..b06c72b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java @@ -32,6 +32,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hbase.CellScannable; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.ClockType; +import org.apache.hadoop.hbase.Clock; import org.apache.hadoop.hbase.ChoreService; import org.apache.hadoop.hbase.CoordinatedStateManager; import org.apache.hadoop.hbase.HRegionInfo; @@ -574,6 +576,12 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices { } @Override + public Clock getRegionServerClock(ClockType clockType) { + Clock systemClock = new Clock.System(); + return systemClock; + } + + @Override public ExecutorService getExecutorService() { return null; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java index 36f505b..80abcc6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java @@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.InterProcessLock; import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.ScheduledChore; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.ClockType; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.Waiter; @@ -325,6 +326,7 @@ public class TestTableLockManager { final HTableDescriptor desc = new HTableDescriptor(tableName); final byte[] family = Bytes.toBytes("test_cf"); desc.addFamily(new HColumnDescriptor(family)); + desc.setClockType(ClockType.SYSTEM); admin.createTable(desc); // create with one region // write some data, not much diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java index a3804dd..e86b5ac 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java @@ -61,6 +61,8 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.Clock; +import org.apache.hadoop.hbase.ClockType; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; @@ -171,6 +173,7 @@ public class TestHRegionReplayEvents { when(rss.getServerName()).thenReturn(ServerName.valueOf("foo", 1, 1)); when(rss.getConfiguration()).thenReturn(CONF); when(rss.getRegionServerAccounting()).thenReturn(new RegionServerAccounting()); + when(rss.getRegionServerClock((ClockType)any())).thenReturn(new Clock.System()); String string = org.apache.hadoop.hbase.executor.EventType.RS_COMPACTED_FILES_DISCHARGER .toString(); ExecutorService es = new ExecutorService(string); @@ -1671,4 +1674,4 @@ public class TestHRegionReplayEvents { return TEST_UTIL.createLocalHRegion(tableName, startKey, stopKey, callingMethod, conf, isReadOnly, durability, wal, families); } -} \ No newline at end of file +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java index 2cae887..a9483fc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; import java.io.IOException; import java.util.ArrayList; @@ -30,6 +31,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.Clock; +import org.apache.hadoop.hbase.ClockType; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.testclassification.RegionServerTests; @@ -101,6 +104,7 @@ public class TestRegionSplitPolicy { final List regions = new ArrayList(); Mockito.when(rss.getOnlineRegions(TABLENAME)).thenReturn(regions); Mockito.when(mockRegion.getRegionServerServices()).thenReturn(rss); + Mockito.when(rss.getRegionServerClock(ClockType.SYSTEM)).thenReturn(new Clock.System()); // Set max size for this 'table'. long maxSplitSize = 1024L; htd.setMaxFileSize(maxSplitSize); @@ -162,6 +166,7 @@ public class TestRegionSplitPolicy { Mockito.when(mockRegion.getRegionServerServices()).thenReturn(rss); Mockito.when(mockRegion.getBlockedRequestsCount()).thenReturn(0L); Mockito.when(mockRegion.getWriteRequestsCount()).thenReturn(0L); + Mockito.when(rss.getRegionServerClock(ClockType.SYSTEM)).thenReturn(new Clock.System()); BusyRegionSplitPolicy policy = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java index e9bb468..4f5196f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver; import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; import java.io.IOException; import java.util.NavigableMap; @@ -33,6 +34,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.Clock; +import org.apache.hadoop.hbase.ClockType; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; @@ -195,6 +198,7 @@ public class TestWALLockup { Mockito.when(server.isStopped()).thenReturn(false); Mockito.when(server.isAborted()).thenReturn(false); RegionServerServices services = Mockito.mock(RegionServerServices.class); + Mockito.when(services.getRegionServerClock(ClockType.SYSTEM)).thenReturn(new Clock.System()); // OK. Now I have my mocked up Server & RegionServerServices and dodgy WAL, go ahead with test. FileSystem fs = FileSystem.get(CONF); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestCompactionScanQueryMatcher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestCompactionScanQueryMatcher.java index 055fe1c..10e88a0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestCompactionScanQueryMatcher.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestCompactionScanQueryMatcher.java @@ -30,6 +30,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeepDeletedCells; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.TimestampType; import org.apache.hadoop.hbase.KeyValue.Type; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.regionserver.ScanInfo; @@ -78,7 +79,7 @@ public class TestCompactionScanQueryMatcher extends AbstractTestScanQueryMatcher CompactionScanQueryMatcher qm = CompactionScanQueryMatcher.create(scanInfo, ScanType.COMPACT_RETAIN_DELETES, Long.MAX_VALUE, HConstants.OLDEST_TIMESTAMP, - HConstants.OLDEST_TIMESTAMP, now, from, to, null); + HConstants.OLDEST_TIMESTAMP, now, TimestampType.PHYSICAL, from, to, null); List actual = new ArrayList( rows.length); byte[] prevRow = null; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestUserScanQueryMatcher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestUserScanQueryMatcher.java index 04c3611..165a673 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestUserScanQueryMatcher.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestUserScanQueryMatcher.java @@ -27,6 +27,7 @@ import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.TimestampType; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeepDeletedCells; @@ -55,7 +56,7 @@ public class TestUserScanQueryMatcher extends AbstractTestScanQueryMatcher { // Do with fam2 which has a col2 qualifier. UserScanQueryMatcher qm = UserScanQueryMatcher.create(scan, new ScanInfo(this.conf, fam2, 10, 1, ttl, KeepDeletedCells.FALSE, 0, rowComparator), - get.getFamilyMap().get(fam2), now - ttl, now, null); + get.getFamilyMap().get(fam2), now - ttl, now, TimestampType.PHYSICAL, null); Cell kv = new KeyValue(row1, fam2, col2, 1, data); Cell cell = CellUtil.createLastOnRowCol(kv); qm.setToNewRow(kv); @@ -81,7 +82,7 @@ public class TestUserScanQueryMatcher extends AbstractTestScanQueryMatcher { // 2,4,5 UserScanQueryMatcher qm = UserScanQueryMatcher.create(scan, new ScanInfo(this.conf, fam2, 0, 1, ttl, KeepDeletedCells.FALSE, 0, rowComparator), - get.getFamilyMap().get(fam2), now - ttl, now, null); + get.getFamilyMap().get(fam2), now - ttl, now, TimestampType.PHYSICAL, null); List memstore = new ArrayList(); memstore.add(new KeyValue(row1, fam2, col1, 1, data)); @@ -124,7 +125,7 @@ public class TestUserScanQueryMatcher extends AbstractTestScanQueryMatcher { long now = EnvironmentEdgeManager.currentTime(); UserScanQueryMatcher qm = UserScanQueryMatcher.create(scan, new ScanInfo(this.conf, fam2, 0, 1, ttl, KeepDeletedCells.FALSE, 0, rowComparator), null, - now - ttl, now, null); + now - ttl, now, TimestampType.PHYSICAL, null); List memstore = new ArrayList(); memstore.add(new KeyValue(row1, fam2, col1, 1, data)); @@ -169,7 +170,7 @@ public class TestUserScanQueryMatcher extends AbstractTestScanQueryMatcher { long now = EnvironmentEdgeManager.currentTime(); UserScanQueryMatcher qm = UserScanQueryMatcher.create(scan, new ScanInfo(this.conf, fam2, 0, 1, testTTL, KeepDeletedCells.FALSE, 0, rowComparator), - get.getFamilyMap().get(fam2), now - testTTL, now, null); + get.getFamilyMap().get(fam2), now - testTTL, now, TimestampType.PHYSICAL, null); KeyValue[] kvs = new KeyValue[] { new KeyValue(row1, fam2, col1, now - 100, data), new KeyValue(row1, fam2, col2, now - 50, data), @@ -211,7 +212,7 @@ public class TestUserScanQueryMatcher extends AbstractTestScanQueryMatcher { long now = EnvironmentEdgeManager.currentTime(); UserScanQueryMatcher qm = UserScanQueryMatcher.create(scan, new ScanInfo(this.conf, fam2, 0, 1, testTTL, KeepDeletedCells.FALSE, 0, rowComparator), null, - now - testTTL, now, null); + now - testTTL, now, TimestampType.PHYSICAL, null); KeyValue[] kvs = new KeyValue[] { new KeyValue(row1, fam2, col1, now - 100, data), new KeyValue(row1, fam2, col2, now - 50, data), diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java index a974639..d126bad 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java @@ -52,6 +52,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.Clock; +import org.apache.hadoop.hbase.ClockType; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -704,6 +706,8 @@ public abstract class AbstractTestWALReplay { RegionServerServices rsServices = Mockito.mock(RegionServerServices.class); Mockito.doReturn(false).when(rsServices).isAborted(); when(rsServices.getServerName()).thenReturn(ServerName.valueOf("foo", 10, 10)); + when(rsServices.getRegionServerClock(ClockType.SYSTEM)).thenReturn(new Clock.System()); + Configuration customConf = new Configuration(this.conf); customConf.set(DefaultStoreEngine.DEFAULT_STORE_FLUSHER_CLASS_KEY, CustomStoreFlusher.class.getName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplicas.java index 2aa436c..17dbde7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplicas.java @@ -173,7 +173,7 @@ public class TestHBaseFsckReplicas extends BaseTestHBaseFsck { Collection var = admin.getClusterStatus().getServers(); ServerName sn = var.toArray(new ServerName[var.size()])[0]; //add a location with replicaId as 2 (since we already have replicas with replicaid 0 and 1) - MetaTableAccessor.addLocation(put, sn, sn.getStartcode(), -1, 2); + MetaTableAccessor.addLocation(put, sn, sn.getStartcode(), 2); meta.put(put); // assign the new replica HBaseFsckRepair.fixUnassigned(admin, newHri);