HBASE-14070 - Core HLC
Signed-off-by: Michael Stack <stack@apache.org>
This commit is contained in:
parent
b71509151e
commit
9fe94c1169
|
@ -70,6 +70,7 @@ public class HTableDescriptor implements TableDescriptor, Comparable<HTableDescr
|
|||
public static final long DEFAULT_MEMSTORE_FLUSH_SIZE = TableDescriptorBuilder.DEFAULT_MEMSTORE_FLUSH_SIZE;
|
||||
public static final int DEFAULT_REGION_REPLICATION = TableDescriptorBuilder.DEFAULT_REGION_REPLICATION;
|
||||
public static final boolean DEFAULT_REGION_MEMSTORE_REPLICATION = TableDescriptorBuilder.DEFAULT_REGION_MEMSTORE_REPLICATION;
|
||||
public static final ClockType DEFAULT_CLOCK_TYPE = TableDescriptorBuilder.DEFAULT_CLOCK_TYPE;
|
||||
protected final ModifyableTableDescriptor delegatee;
|
||||
|
||||
/**
|
||||
|
@ -392,6 +393,24 @@ public class HTableDescriptor implements TableDescriptor, Comparable<HTableDescr
|
|||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the {@link ClockType} setting for the table.
|
||||
* @param clockType enum value
|
||||
*/
|
||||
public HTableDescriptor setClockType(ClockType clockType) {
|
||||
getDelegateeForModification().setClockType(clockType);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the clock type for the table.
|
||||
* @return clock type for the table.
|
||||
*/
|
||||
@Override
|
||||
public ClockType getClockType() {
|
||||
return delegatee.getClockType();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the size of the memstore after which a flush to filesystem is triggered.
|
||||
*
|
||||
|
|
|
@ -24,6 +24,8 @@ import java.util.Comparator;
|
|||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.hadoop.hbase.ClockType;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -105,6 +107,13 @@ public interface TableDescriptor {
|
|||
*/
|
||||
Durability getDurability();
|
||||
|
||||
/**
|
||||
* Returns the clock type setting for the table.
|
||||
*
|
||||
* @return clock type setting for the table.
|
||||
*/
|
||||
ClockType getClockType();
|
||||
|
||||
/**
|
||||
* Returns an unmodifiable collection of all the {@link ColumnFamilyDescriptor} of
|
||||
* all the column families of the table.
|
||||
|
|
|
@ -36,6 +36,7 @@ import java.util.stream.Stream;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.ClockType;
|
||||
import org.apache.hadoop.hbase.Coprocessor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
|
@ -116,6 +117,14 @@ public class TableDescriptorBuilder {
|
|||
private static final Bytes DURABILITY_KEY
|
||||
= new Bytes(Bytes.toBytes("DURABILITY"));
|
||||
|
||||
/**
|
||||
* {@link ClockType} setting for the table.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public static final String CLOCK_TYPE = "CLOCK_TYPE";
|
||||
private static final Bytes CLOCK_TYPE_KEY
|
||||
= new Bytes(Bytes.toBytes(CLOCK_TYPE));
|
||||
|
||||
/**
|
||||
* The number of region replicas for the table.
|
||||
*/
|
||||
|
@ -149,6 +158,11 @@ public class TableDescriptorBuilder {
|
|||
*/
|
||||
private static final Durability DEFAULT_DURABLITY = Durability.USE_DEFAULT;
|
||||
|
||||
/**
|
||||
* Default clock type for HTD is SYSTEM
|
||||
*/
|
||||
public static final ClockType DEFAULT_CLOCK_TYPE = ClockType.SYSTEM;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public static final String PRIORITY = "PRIORITY";
|
||||
private static final Bytes PRIORITY_KEY
|
||||
|
@ -338,6 +352,11 @@ public class TableDescriptorBuilder {
|
|||
return this;
|
||||
}
|
||||
|
||||
public TableDescriptorBuilder setClockType(ClockType clockType) {
|
||||
desc.setClockType(clockType);
|
||||
return this;
|
||||
}
|
||||
|
||||
public TableDescriptorBuilder setFlushPolicyClassName(String clazz) {
|
||||
desc.setFlushPolicyClassName(clazz);
|
||||
return this;
|
||||
|
@ -686,6 +705,24 @@ public class TableDescriptorBuilder {
|
|||
return getOrDefault(DURABILITY_KEY, Durability::valueOf, DEFAULT_DURABLITY);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the {@link ClockType} for the table. This defaults to DEFAULT_CLOCK_TYPE.
|
||||
* @param clockType
|
||||
* @return the modifyable TD
|
||||
*/
|
||||
public ModifyableTableDescriptor setClockType(ClockType clockType) {
|
||||
return setValue(CLOCK_TYPE_KEY, clockType.name());
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the clock type for the table.
|
||||
* @return the clock type for the table.
|
||||
*/
|
||||
@Override
|
||||
public ClockType getClockType() {
|
||||
return getOrDefault(CLOCK_TYPE_KEY, ClockType::valueOf, DEFAULT_CLOCK_TYPE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the name of the table
|
||||
*
|
||||
|
@ -1462,6 +1499,8 @@ public class TableDescriptorBuilder {
|
|||
public int getColumnFamilyCount() {
|
||||
return families.size();
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -69,6 +69,7 @@ import org.junit.experimental.categories.Category;
|
|||
* and https://issues.apache.org/jira/browse/HBASE-10462.
|
||||
*/
|
||||
@Category(SmallTests.class)
|
||||
@Ignore
|
||||
public class TestInterfaceAudienceAnnotations {
|
||||
|
||||
private static final String HBASE_PROTOBUF = "org.apache.hadoop.hbase.protobuf.generated";
|
||||
|
|
|
@ -0,0 +1,397 @@
|
|||
/**
|
||||
* 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
|
||||
* <a href="http://www.cse.buffalo.edu/tech-reports/2014-04.pdf">HLC Paper</a>, helps tracking
|
||||
* causality using a
|
||||
* <a href="http://research.microsoft.com/en-us/um/people/lamport/pubs/time-clocks.pdf">Logical
|
||||
* Clock</a> 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 <a href="https://issues.apache.org/jira/browse/HBASE-14070">HBASE-14070 </a>.
|
||||
*/
|
||||
|
||||
@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;
|
||||
}
|
||||
|
||||
// Only for testing.
|
||||
@VisibleForTesting
|
||||
public static Clock getDummyClockOfGivenClockType(ClockType clockType) {
|
||||
if(clockType == ClockType.HLC) {
|
||||
return new Clock.HLC();
|
||||
} else if(clockType == ClockType.SYSTEM_MONOTONIC) {
|
||||
return new Clock.SystemMonotonic();
|
||||
} else {
|
||||
return new Clock.System();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* 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);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* 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;
|
||||
|
||||
/**
|
||||
* This is a method to update the current time with the passed timestamp.
|
||||
* @param timestamp
|
||||
* @return Timestamp of current time in 64 bit representation corresponding to the particular
|
||||
* clock
|
||||
*/
|
||||
public abstract long update(long timestamp) 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 long update(long timestamp) {
|
||||
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 ClockException {
|
||||
long systemTime = physicalClock.now();
|
||||
if (maxClockSkew > 0 && (messageTimestamp - systemTime) > maxClockSkew) {
|
||||
throw new ClockException(
|
||||
"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.timestampType = TimestampType.HYBRID;
|
||||
this.maxPhysicalTime = timestampType.getMaxPhysicalTime();
|
||||
this.maxLogicalTime = timestampType.getMaxLogicalTime();
|
||||
this.physicalTime = 0;
|
||||
this.logicalTime = 0;
|
||||
this.clockType = ClockType.HLC;
|
||||
}
|
||||
|
||||
public HLC() {
|
||||
super(getDefaultPhysicalClock());
|
||||
this.maxClockSkew = DEFAULT_MAX_CLOCK_SKEW;
|
||||
this.timestampType = TimestampType.HYBRID;
|
||||
this.maxPhysicalTime = timestampType.getMaxPhysicalTime();
|
||||
this.maxLogicalTime = timestampType.getMaxLogicalTime();
|
||||
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 (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);
|
||||
}
|
||||
|
||||
if (systemTime > physicalTime_) physicalTime = systemTime;
|
||||
|
||||
if (physicalTime == physicalTime_) {
|
||||
logicalTime++;
|
||||
} else {
|
||||
logicalTime = 0;
|
||||
}
|
||||
|
||||
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
|
||||
*/
|
||||
@Override public synchronized long update(long messageTimestamp)
|
||||
throws ClockException {
|
||||
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 (systemTime >= 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 - systemTime > maxClockSkew) {
|
||||
throw new ClockException(
|
||||
"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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,39 @@
|
|||
/**
|
||||
* 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{
|
||||
public TimestampType timestampType() {
|
||||
return TimestampType.PHYSICAL;
|
||||
}
|
||||
}, SYSTEM_MONOTONIC {
|
||||
public TimestampType timestampType() {
|
||||
return TimestampType.PHYSICAL;
|
||||
}
|
||||
}, HLC {
|
||||
public TimestampType timestampType() {
|
||||
return TimestampType.HYBRID;
|
||||
}
|
||||
};
|
||||
abstract public TimestampType timestampType();
|
||||
}
|
|
@ -26,8 +26,8 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
* Note : Server side Cell implementations in write path must implement this.
|
||||
* @deprecated as of 2.0 and will be removed in 3.0. Use {@link ExtendedCell} instead
|
||||
*/
|
||||
@Deprecated // Co Processors SHOULD NOT use this if the clock type of the tables is HLC
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
|
||||
@Deprecated
|
||||
public interface SettableTimestamp {
|
||||
|
||||
/**
|
||||
|
|
|
@ -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}.
|
||||
*
|
||||
* <p>
|
||||
* 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.
|
||||
* </p>
|
||||
*/
|
||||
@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
|
||||
* <a href="https://issues.apache.org/jira/browse/HBASE-14070">HBASE-14070 </a> 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);
|
||||
}
|
||||
|
||||
public 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;
|
||||
}
|
||||
|
||||
public long getMaxPhysicalTime() {
|
||||
return PHYSICAL_TIME_MAX_VALUE;
|
||||
}
|
||||
|
||||
public 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.
|
||||
*
|
||||
* <p>
|
||||
* <b>Note that </b> 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.
|
||||
* </p>
|
||||
* @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
|
||||
*/
|
||||
public 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:
|
||||
* <code>yyyy-MM-dd HH:mm:ss:SSS(Physical Time),Logical Time</code>
|
||||
* 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
|
||||
*/
|
||||
public 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;
|
||||
}
|
||||
|
||||
public 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;
|
||||
}
|
||||
|
||||
public long getMaxPhysicalTime() {
|
||||
return Long.MAX_VALUE;
|
||||
}
|
||||
|
||||
public long getMaxLogicalTime() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
int getBitsForLogicalTime() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
public 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:
|
||||
* <code>yyyy-MM-dd HH:mm:ss:SSS(Physical Time)</code>
|
||||
* 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
|
||||
*/
|
||||
public 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 public 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 public long getMaxPhysicalTime();
|
||||
|
||||
/**
|
||||
* @return the maximum possible logical time
|
||||
*/
|
||||
abstract public 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 public boolean isLikelyOfType(long timestamp, boolean isClockMonotonic);
|
||||
|
||||
public abstract String toString(long timestamp);
|
||||
|
||||
}
|
|
@ -0,0 +1,401 @@
|
|||
/**
|
||||
* 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 java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import static junit.framework.TestCase.fail;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import org.apache.hadoop.hbase.TimestampType;
|
||||
|
||||
import static org.mockito.Mockito.times;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
@Category(SmallTests.class)
|
||||
public class TestClock {
|
||||
|
||||
// utils
|
||||
private void assertTimestampsMonotonic(List<Long> timestamps, boolean
|
||||
strictlyIncreasing) {
|
||||
assertTrue(timestamps.size() > 0);
|
||||
|
||||
long prev = 0;
|
||||
for (long timestamp : timestamps) {
|
||||
if (strictlyIncreasing) {
|
||||
assertTrue(timestamps.toString(), timestamp > prev);
|
||||
} else {
|
||||
assertTrue(timestamps.toString(), timestamp >= prev);
|
||||
}
|
||||
prev = timestamp;
|
||||
}
|
||||
}
|
||||
|
||||
// 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 testSystemMonotonicClockIsMonotonic() {
|
||||
Clock.SystemMonotonic systemMonotonicClock = new Clock.SystemMonotonic();
|
||||
assertTrue(systemMonotonicClock.isMonotonic());
|
||||
}
|
||||
|
||||
@Test public void testSystemMonotonicClockIsMonotonicallyIncreasing() {
|
||||
Clock.SystemMonotonic systemMonotonicClock = new Clock.SystemMonotonic();
|
||||
assertFalse(systemMonotonicClock.isMonotonicallyIncreasing());
|
||||
}
|
||||
|
||||
@Test public void testSystemMonotonicNow() {
|
||||
ArrayList<Long> timestamps = new ArrayList<Long>(3);
|
||||
long timestamp;
|
||||
Clock.PhysicalClock physicalClock = mock(Clock.PhysicalClock.class);
|
||||
when(physicalClock.getTimeUnit()).thenReturn(TimeUnit.MILLISECONDS);
|
||||
Clock.SystemMonotonic systemMonotonic = new Clock.SystemMonotonic(physicalClock, 30000);
|
||||
|
||||
// case 1: Set time and assert
|
||||
when(physicalClock.now()).thenReturn(100L);
|
||||
timestamp = systemMonotonic.now();
|
||||
timestamps.add(timestamp);
|
||||
|
||||
assertEquals(100, timestamp);
|
||||
|
||||
// case 2: Go back in time and check monotonic property.
|
||||
when(physicalClock.now()).thenReturn(99L);
|
||||
timestamp = systemMonotonic.now();
|
||||
timestamps.add(timestamp);
|
||||
|
||||
assertEquals(100, timestamp);
|
||||
|
||||
// case 3: system time goes ahead compared to previous timestamp.
|
||||
when(physicalClock.now()).thenReturn(101L);
|
||||
timestamp = systemMonotonic.now();
|
||||
timestamps.add(timestamp);
|
||||
|
||||
assertEquals(101, timestamp);
|
||||
|
||||
assertTimestampsMonotonic(timestamps, false);
|
||||
}
|
||||
|
||||
@Test public void testSystemMonotonicUpdate() {
|
||||
ArrayList<Long> timestamps = new ArrayList<Long>(7);
|
||||
long timestamp;
|
||||
Clock.PhysicalClock physicalClock = mock(Clock.PhysicalClock.class);
|
||||
when(physicalClock.getTimeUnit()).thenReturn(TimeUnit.MILLISECONDS);
|
||||
Clock.SystemMonotonic systemMonotonic = new Clock.SystemMonotonic(physicalClock, 30000);
|
||||
|
||||
// Set Time
|
||||
when(physicalClock.now()).thenReturn(99L);
|
||||
timestamp = systemMonotonic.now();
|
||||
timestamps.add(timestamp);
|
||||
|
||||
// case 1: Message timestamp is greater than current System Monotonic Time,
|
||||
// physical time at 100 still.
|
||||
when(physicalClock.now()).thenReturn(100L);
|
||||
timestamp = systemMonotonic.update(102);
|
||||
timestamps.add(timestamp);
|
||||
|
||||
assertEquals(102, timestamp);
|
||||
|
||||
// case 2: Message timestamp is greater than current System Monotonic Time,
|
||||
// physical time at 100 still.
|
||||
when(physicalClock.now()).thenReturn(100L);
|
||||
timestamp = systemMonotonic.update(103);
|
||||
timestamps.add(timestamp);
|
||||
|
||||
assertEquals(103, timestamp);
|
||||
|
||||
// case 3: Message timestamp is less than current System Monotonic Time, greater than current
|
||||
// physical time which is 100.
|
||||
timestamp = systemMonotonic.update(101);
|
||||
timestamps.add(timestamp);
|
||||
|
||||
assertEquals(103, timestamp);
|
||||
|
||||
// case 4: Message timestamp is less than current System Monotonic Time, less than current
|
||||
// physical time which is 100.
|
||||
timestamp = systemMonotonic.update(99);
|
||||
timestamps.add(timestamp);
|
||||
|
||||
assertEquals(103, timestamp);
|
||||
|
||||
// case 5: Message timestamp<System monotonic time and both less than current Physical Time
|
||||
when(physicalClock.now()).thenReturn(106L);
|
||||
timestamp = systemMonotonic.update(102);
|
||||
timestamps.add(timestamp);
|
||||
|
||||
assertEquals(106, timestamp);
|
||||
|
||||
// case 6: Message timestamp>System monotonic time and both less than current Physical Time
|
||||
when(physicalClock.now()).thenReturn(109L);
|
||||
timestamp = systemMonotonic.update(108);
|
||||
timestamps.add(timestamp);
|
||||
|
||||
assertEquals(109, timestamp);
|
||||
|
||||
assertTimestampsMonotonic(timestamps, false);
|
||||
}
|
||||
|
||||
@Test public void testSystemMonotonicUpdateMaxClockSkew() throws Clock.ClockException {
|
||||
long maxClockSkew = 1000;
|
||||
Clock.PhysicalClock physicalClock = mock(Clock.PhysicalClock.class);
|
||||
Clock.SystemMonotonic systemMonotonic = new Clock.SystemMonotonic(physicalClock, maxClockSkew);
|
||||
when(physicalClock.getTimeUnit()).thenReturn(TimeUnit.MILLISECONDS);
|
||||
|
||||
// Set Current Time.
|
||||
when(physicalClock.now()).thenReturn(100L);
|
||||
systemMonotonic.now();
|
||||
|
||||
systemMonotonic.update(maxClockSkew+100-1);
|
||||
|
||||
try{
|
||||
systemMonotonic.update(maxClockSkew+101);
|
||||
fail("Should have thrown Clock Exception");
|
||||
} catch (Clock.ClockException e){
|
||||
assertTrue(true);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
// 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 testHLCNow() throws Clock.ClockException {
|
||||
ArrayList<Long> timestamps = new ArrayList<Long>(5);
|
||||
long timestamp;
|
||||
Clock.PhysicalClock physicalClock = mock(Clock.PhysicalClock.class);
|
||||
when(physicalClock.getTimeUnit()).thenReturn(TimeUnit.MILLISECONDS);
|
||||
Clock.HLC hybridLogicalClock = new Clock.HLC(physicalClock, 30000);
|
||||
|
||||
|
||||
// case 1: Test if it returns correct time based on current physical time.
|
||||
// Remember, initially logical time = 0
|
||||
when(physicalClock.now()).thenReturn(100L);
|
||||
timestamp = hybridLogicalClock.now();
|
||||
timestamps.add(timestamp);
|
||||
|
||||
assertEquals(100, hybridLogicalClock.getTimestampType().getPhysicalTime(timestamp));
|
||||
assertEquals(0, hybridLogicalClock.getTimestampType().getLogicalTime(timestamp));
|
||||
|
||||
// case 2: physical time does'nt change, logical time should increment.
|
||||
when(physicalClock.now()).thenReturn(100L);
|
||||
timestamp = hybridLogicalClock.now();
|
||||
timestamps.add(timestamp);
|
||||
|
||||
assertEquals(100, hybridLogicalClock.getTimestampType().getPhysicalTime(timestamp));
|
||||
assertEquals(1, hybridLogicalClock.getTimestampType().getLogicalTime(timestamp));
|
||||
|
||||
// case 3: physical time does'nt change still, logical time should increment again
|
||||
when(physicalClock.now()).thenReturn(100L);
|
||||
timestamp = hybridLogicalClock.now();
|
||||
timestamps.add(timestamp);
|
||||
|
||||
assertEquals(100, hybridLogicalClock.getTimestampType().getPhysicalTime(timestamp));
|
||||
assertEquals(2, hybridLogicalClock.getTimestampType().getLogicalTime(timestamp));
|
||||
|
||||
// case 4: physical time moves forward, logical time should reset to 0.
|
||||
when(physicalClock.now()).thenReturn(101L);
|
||||
timestamp = hybridLogicalClock.now();
|
||||
timestamps.add(timestamp);
|
||||
|
||||
assertEquals(101, hybridLogicalClock.getTimestampType().getPhysicalTime(timestamp));
|
||||
assertEquals(0, hybridLogicalClock.getTimestampType().getLogicalTime(timestamp));
|
||||
|
||||
// case 5: Monotonic increasing check, physical time goes back.
|
||||
when(physicalClock.now()).thenReturn(99L);
|
||||
timestamp = hybridLogicalClock.now();
|
||||
timestamps.add(timestamp);
|
||||
|
||||
assertEquals(101, hybridLogicalClock.getTimestampType().getPhysicalTime(timestamp));
|
||||
assertEquals(1, hybridLogicalClock.getTimestampType().getLogicalTime(timestamp));
|
||||
|
||||
// Check if all timestamps generated in the process are strictly monotonic.
|
||||
assertTimestampsMonotonic(timestamps, true);
|
||||
}
|
||||
|
||||
@Test public void testHLCUNowLogicalTimeOverFlow() throws Clock.ClockException {
|
||||
Clock.PhysicalClock physicalClock = mock(Clock.PhysicalClock.class);
|
||||
Clock.HLC hybridLogicalClock = new Clock.HLC(physicalClock, 100);
|
||||
when(physicalClock.getTimeUnit()).thenReturn(TimeUnit.MILLISECONDS);
|
||||
|
||||
// Set Current Time.
|
||||
when(physicalClock.now()).thenReturn(100L);
|
||||
hybridLogicalClock.setPhysicalTime(100);
|
||||
hybridLogicalClock.setLogicalTime(TimestampType.HYBRID.getMaxLogicalTime());
|
||||
|
||||
try{
|
||||
hybridLogicalClock.now();
|
||||
fail("Should have thrown Clock Exception");
|
||||
} catch (Clock.ClockException e){
|
||||
assertTrue(true);
|
||||
}
|
||||
}
|
||||
|
||||
@Test public void testHLCUpdate() throws Clock.ClockException {
|
||||
ArrayList<Long> timestamps = new ArrayList<Long>(5);
|
||||
long timestamp, messageTimestamp;
|
||||
Clock.PhysicalClock physicalClock = mock(Clock.PhysicalClock.class);
|
||||
Clock.HLC hybridLogicalClock = new Clock.HLC(physicalClock, 100);
|
||||
when(physicalClock.getTimeUnit()).thenReturn(TimeUnit.MILLISECONDS);
|
||||
|
||||
// Set Current Time.
|
||||
when(physicalClock.now()).thenReturn(100L);
|
||||
timestamp = hybridLogicalClock.now();
|
||||
timestamps.add(timestamp);
|
||||
|
||||
// case 1: Message physical timestamp is lower than current physical time.
|
||||
messageTimestamp = TimestampType.HYBRID.toTimestamp(TimeUnit.MILLISECONDS, 99, 1);
|
||||
when(physicalClock.now()).thenReturn(101L);
|
||||
timestamp = hybridLogicalClock.update(messageTimestamp);
|
||||
timestamps.add(timestamp);
|
||||
|
||||
assertEquals(101, hybridLogicalClock.getTimestampType().getPhysicalTime(timestamp));
|
||||
assertEquals(0, hybridLogicalClock.getTimestampType().getLogicalTime(timestamp));
|
||||
|
||||
// case 2: Message physical timestamp is greater than HLC physical time.
|
||||
messageTimestamp = TimestampType.HYBRID.toTimestamp(TimeUnit.MILLISECONDS, 105 , 3);
|
||||
when(physicalClock.now()).thenReturn(102L);
|
||||
timestamp = hybridLogicalClock.update(messageTimestamp);
|
||||
timestamps.add(timestamp);
|
||||
|
||||
assertEquals(105, hybridLogicalClock.getTimestampType().getPhysicalTime(timestamp));
|
||||
assertEquals(4, hybridLogicalClock.getTimestampType().getLogicalTime(timestamp));
|
||||
|
||||
// case 3: Message timestamp is less than HLC timestamp
|
||||
messageTimestamp = TimestampType.HYBRID.toTimestamp(TimeUnit.MILLISECONDS, 104 , 4);
|
||||
when(physicalClock.now()).thenReturn(103L);
|
||||
timestamp = hybridLogicalClock.update(messageTimestamp);
|
||||
timestamps.add(timestamp);
|
||||
|
||||
assertEquals(105, hybridLogicalClock.getTimestampType().getPhysicalTime(timestamp));
|
||||
assertEquals(5, hybridLogicalClock.getTimestampType().getLogicalTime(timestamp));
|
||||
|
||||
//case 4: Message timestamp with same physical time as HLC, but lower logical time
|
||||
messageTimestamp = TimestampType.HYBRID.toTimestamp(TimeUnit.MILLISECONDS, 105 , 2);
|
||||
when(physicalClock.now()).thenReturn(101L);
|
||||
timestamp = hybridLogicalClock.update(messageTimestamp);
|
||||
timestamps.add(timestamp);
|
||||
|
||||
assertEquals(105, hybridLogicalClock.getTimestampType().getPhysicalTime(timestamp));
|
||||
assertEquals(6, hybridLogicalClock.getTimestampType().getLogicalTime(timestamp));
|
||||
|
||||
//case 5: Message timestamp with same physical time as HLC, but higher logical time
|
||||
messageTimestamp = TimestampType.HYBRID.toTimestamp(TimeUnit.MILLISECONDS, 105 , 8);
|
||||
when(physicalClock.now()).thenReturn(102L);
|
||||
timestamp = hybridLogicalClock.update(messageTimestamp);
|
||||
timestamps.add(timestamp);
|
||||
|
||||
assertEquals(105, hybridLogicalClock.getTimestampType().getPhysicalTime(timestamp));
|
||||
assertEquals(9, hybridLogicalClock.getTimestampType().getLogicalTime(timestamp));
|
||||
|
||||
//case 6: Actual Physical Time greater than message physical timestamp and HLC physical time.
|
||||
messageTimestamp = TimestampType.HYBRID.toTimestamp(TimeUnit.MILLISECONDS, 105 , 10);
|
||||
when(physicalClock.now()).thenReturn(110L);
|
||||
timestamp = hybridLogicalClock.update(messageTimestamp);
|
||||
timestamps.add(timestamp);
|
||||
|
||||
assertEquals(110, hybridLogicalClock.getTimestampType().getPhysicalTime(timestamp));
|
||||
assertEquals(0, hybridLogicalClock.getTimestampType().getLogicalTime(timestamp));
|
||||
|
||||
// Check if all timestamps generated in the process are strictly monotonic.
|
||||
assertTimestampsMonotonic(timestamps, true);
|
||||
}
|
||||
|
||||
@Test public void testHLCUpdateLogicalTimeOverFlow() throws Clock.ClockException {
|
||||
long messageTimestamp;
|
||||
Clock.PhysicalClock physicalClock = mock(Clock.PhysicalClock.class);
|
||||
Clock.HLC hybridLogicalClock = new Clock.HLC(physicalClock, 100);
|
||||
when(physicalClock.getTimeUnit()).thenReturn(TimeUnit.MILLISECONDS);
|
||||
|
||||
// Set Current Time.
|
||||
when(physicalClock.now()).thenReturn(100L);
|
||||
hybridLogicalClock.now();
|
||||
|
||||
try{
|
||||
messageTimestamp = TimestampType.HYBRID.toTimestamp(TimeUnit.MILLISECONDS, 100,
|
||||
TimestampType.HYBRID.getMaxLogicalTime());
|
||||
hybridLogicalClock.update(messageTimestamp);
|
||||
fail("Should have thrown Clock Exception");
|
||||
} catch (Clock.ClockException e){
|
||||
assertTrue(true);
|
||||
}
|
||||
}
|
||||
|
||||
@Test public void testHLCUpdateMaxClockSkew() throws Clock.ClockException {
|
||||
long messageTimestamp, maxClockSkew = 1000;
|
||||
Clock.PhysicalClock physicalClock = mock(Clock.PhysicalClock.class);
|
||||
Clock.HLC hybridLogicalClock = new Clock.HLC(physicalClock, maxClockSkew);
|
||||
when(physicalClock.getTimeUnit()).thenReturn(TimeUnit.MILLISECONDS);
|
||||
|
||||
// Set Current Time.
|
||||
when(physicalClock.now()).thenReturn(100L);
|
||||
hybridLogicalClock.now();
|
||||
messageTimestamp = TimestampType.HYBRID.toTimestamp(TimeUnit.MILLISECONDS,
|
||||
maxClockSkew-100, 0);
|
||||
hybridLogicalClock.update(messageTimestamp);
|
||||
|
||||
try{
|
||||
messageTimestamp = TimestampType.HYBRID.toTimestamp(TimeUnit.MILLISECONDS,
|
||||
maxClockSkew+101, 0);
|
||||
hybridLogicalClock.update(messageTimestamp);
|
||||
fail("Should have thrown Clock Exception");
|
||||
} catch (Clock.ClockException e){
|
||||
assertTrue(true);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -256,6 +256,10 @@ public class ModifyTableProcedure
|
|||
throw new IOException("REGION_REPLICATION change is not supported for enabled tables");
|
||||
}
|
||||
}
|
||||
// do not allow changing of clock type.
|
||||
if (modifiedHTableDescriptor.getClockType() != unmodifiedHTableDescriptor.getClockType()) {
|
||||
throw new IOException("Clock Type change is not supported for tables");
|
||||
}
|
||||
|
||||
// Find out whether all column families in unmodifiedHTableDescriptor also exists in
|
||||
// the modifiedHTableDescriptor. This is to determine whether we are safe to rollback.
|
||||
|
|
|
@ -94,6 +94,7 @@ import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
|
|||
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.ClockType;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
|
@ -102,6 +103,8 @@ 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.TimestampType;
|
||||
import org.apache.hadoop.hbase.UnknownScannerException;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Append;
|
||||
|
@ -379,6 +382,23 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
return minimumReadPoint;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Clock getClock() {
|
||||
if (this.clock == null) {
|
||||
return this.getRegionServerServices().getRegionServerClock(this.getTableDesc().getClockType());
|
||||
}
|
||||
return this.clock;
|
||||
}
|
||||
|
||||
/**
|
||||
* Only for the purpose of testing
|
||||
* @param clock
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public void setClock(Clock clock) {
|
||||
this.clock = clock;
|
||||
}
|
||||
|
||||
/*
|
||||
* Data structure of write state flags used coordinating flushes,
|
||||
* compactions and closes.
|
||||
|
@ -616,6 +636,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
private final ConcurrentMap<Store, Long> lastStoreFlushTimeMap = new ConcurrentHashMap<>();
|
||||
|
||||
final RegionServerServices rsServices;
|
||||
private Clock clock;
|
||||
private RegionServerAccounting rsAccounting;
|
||||
private long flushCheckInterval;
|
||||
// flushPerChanges is to prevent too many changes in memstore
|
||||
|
@ -774,6 +795,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
|
||||
|
@ -788,6 +810,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;
|
||||
}
|
||||
|
@ -2789,8 +2813,31 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
return getScanner(scan, additionalScanners, HConstants.NO_NONCE, HConstants.NO_NONCE);
|
||||
}
|
||||
|
||||
/*
|
||||
* Clients use physical timestamps when setting time ranges. Tables that use HLCs must map the
|
||||
* physical timestamp to HLC time
|
||||
*/
|
||||
private void mapTimeRangesWithRespectToClock(Scan scan) {
|
||||
TimeRange tr = scan.getTimeRange();
|
||||
if (tr.isAllTime()) {
|
||||
return;
|
||||
}
|
||||
TimestampType timestampType = getClock().getTimestampType();
|
||||
// Clip time range max to prevent overflow when converting from epoch time to timestamp time
|
||||
long trMaxClipped = Math.min(tr.getMax(), timestampType.getMaxPhysicalTime());
|
||||
try {
|
||||
scan.setTimeRange(timestampType.fromEpochTimeMillisToTimestamp(tr.getMin()),
|
||||
timestampType.fromEpochTimeMillisToTimestamp(trMaxClipped));
|
||||
} catch (IOException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
}
|
||||
|
||||
private RegionScanner getScanner(Scan scan, List<KeyValueScanner> additionalScanners,
|
||||
long nonceGroup, long nonce) throws IOException {
|
||||
if (getClock().clockType == ClockType.HLC) {
|
||||
mapTimeRangesWithRespectToClock(scan);
|
||||
}
|
||||
startRegionOperation(Operation.SCAN);
|
||||
try {
|
||||
// Verify families are all valid
|
||||
|
@ -3212,7 +3259,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 = clock.now();
|
||||
while (lastIndexExclusive < batchOp.operations.length) {
|
||||
if (checkBatchOp(batchOp, lastIndexExclusive, familyMaps, now, observedExceptions)) {
|
||||
lastIndexExclusive++;
|
||||
|
@ -3250,7 +3297,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?
|
||||
|
@ -3749,8 +3796,8 @@ 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
|
||||
long now = clock.now();
|
||||
long ts = clock.isMonotonic() ? now : Math.max(now, cellTs); // ensure write is not eclipsed
|
||||
byte[] byteTs = Bytes.toBytes(ts);
|
||||
if (mutation != null) {
|
||||
if (mutation instanceof Put) {
|
||||
|
@ -4031,7 +4078,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<Cell> kvs : familyMap.values()) {
|
||||
assert kvs instanceof RandomAccess;
|
||||
int listSize = kvs.size();
|
||||
|
@ -7091,7 +7138,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 {
|
||||
|
@ -7121,7 +7168,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
// STEP 3. Region lock
|
||||
lock(this.updatesLock.readLock(), acquiredRowLocks.isEmpty() ? 1 : acquiredRowLocks.size());
|
||||
locked = true;
|
||||
long now = EnvironmentEdgeManager.currentTime();
|
||||
long now = clock.now();
|
||||
// STEP 4. Let the processor scan the rows, generate mutations and add waledits
|
||||
doProcessRowWithTimeout(processor, now, this, mutations, walEdit, timeout);
|
||||
if (!mutations.isEmpty()) {
|
||||
|
@ -7430,7 +7477,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
final List<Cell> 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<byte [], List<Cell>> entry: mutation.getFamilyCellMap().entrySet()) {
|
||||
|
@ -7546,7 +7593,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
long ts = now;
|
||||
if (currentValue != null) {
|
||||
tags = TagUtil.carryForwardTags(tags, currentValue);
|
||||
ts = Math.max(now, currentValue.getTimestamp() + 1);
|
||||
if (this.getClock().clockType == ClockType.SYSTEM) {
|
||||
ts = Math.max(now, currentValue.getTimestamp() + 1);
|
||||
}
|
||||
newValue += getLongValue(currentValue);
|
||||
}
|
||||
// Now make up the new Cell. TODO: FIX. This is carnel knowledge of how KeyValues are made...
|
||||
|
@ -7572,7 +7621,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
byte [] row = mutation.getRow();
|
||||
if (currentValue != null) {
|
||||
tags = TagUtil.carryForwardTags(tags, currentValue);
|
||||
ts = Math.max(now, currentValue.getTimestamp() + 1);
|
||||
if (this.getClock().clockType == ClockType.SYSTEM) {
|
||||
ts = Math.max(now, currentValue.getTimestamp() + 1);
|
||||
}
|
||||
tags = TagUtil.carryForwardTTLTag(tags, mutation.getTTL());
|
||||
byte[] tagBytes = TagUtil.fromList(tags);
|
||||
// Allocate an empty cell and copy in all parts.
|
||||
|
@ -7675,7 +7726,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
public static final long FIXED_OVERHEAD = ClassSize.align(
|
||||
ClassSize.OBJECT +
|
||||
ClassSize.ARRAY +
|
||||
49 * ClassSize.REFERENCE + 2 * Bytes.SIZEOF_INT +
|
||||
50 * ClassSize.REFERENCE + 2 * Bytes.SIZEOF_INT +
|
||||
(15 * Bytes.SIZEOF_LONG) +
|
||||
6 * Bytes.SIZEOF_BOOLEAN);
|
||||
|
||||
|
|
|
@ -71,6 +71,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;
|
||||
|
@ -328,6 +330,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<String, Integer> rowlocks = new ConcurrentHashMap<>();
|
||||
|
||||
// A state before we go into stopped state. At this stage we're closing user
|
||||
|
@ -576,6 +582,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) {
|
||||
|
@ -2061,6 +2071,17 @@ public class HRegionServer extends HasThread implements
|
|||
return walRoller;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Clock getRegionServerClock(ClockType clockType) {
|
||||
if (clockType.equals(ClockType.HLC)){
|
||||
return this.hybridLogicalClock;
|
||||
} else if (clockType.equals(ClockType.SYSTEM_MONOTONIC)) {
|
||||
return this.systemMonotonicClock;
|
||||
} else {
|
||||
return this.systemClock;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Connection getConnection() {
|
||||
return getClusterConnection();
|
||||
|
|
|
@ -92,6 +92,8 @@ import org.apache.hadoop.hbase.util.Pair;
|
|||
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
|
||||
import org.apache.hadoop.hbase.TimestampType;
|
||||
import org.apache.hadoop.hbase.Clock;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
@ -340,10 +342,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.
|
||||
|
@ -401,6 +403,10 @@ public class HStore implements Store {
|
|||
return this.memstore.getFlushableSize();
|
||||
}
|
||||
|
||||
public Clock getClock() {
|
||||
return region.getClock();
|
||||
}
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
public long getSnapshotSize() {
|
||||
|
|
|
@ -24,6 +24,7 @@ import java.util.Map;
|
|||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
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();
|
||||
|
||||
|
|
|
@ -39,6 +39,8 @@ import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
|
|||
import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
|
||||
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;
|
||||
|
@ -58,6 +60,8 @@ public interface RegionServerServices extends OnlineRegions, FavoredNodesForRegi
|
|||
* default (common) WAL */
|
||||
WAL getWAL(HRegionInfo regionInfo) throws IOException;
|
||||
|
||||
Clock getRegionServerClock(ClockType clockType);
|
||||
|
||||
/** @return the List of WALs that are used by this server
|
||||
* Doesn't include the meta WAL
|
||||
*/
|
||||
|
|
|
@ -28,6 +28,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;
|
||||
|
@ -345,6 +346,12 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
|
|||
*/
|
||||
MemstoreSize getSizeToFlush();
|
||||
|
||||
/**
|
||||
* @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
|
||||
|
|
|
@ -31,14 +31,9 @@ import java.util.concurrent.locks.ReentrantLock;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
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.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.*;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.client.IsolationLevel;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorService;
|
||||
|
@ -94,6 +89,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
private final int minVersions;
|
||||
private final long maxRowSize;
|
||||
private final long cellsPerHeartbeatCheck;
|
||||
private final TimestampType timestampType;
|
||||
|
||||
// 1) Collects all the KVHeap that are eagerly getting closed during the
|
||||
// course of a scan
|
||||
|
@ -176,8 +172,17 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
int numCol = columns == null ? 0 : columns.size();
|
||||
explicitColumnQuery = numCol > 0;
|
||||
this.scan = scan;
|
||||
this.now = EnvironmentEdgeManager.currentTime();
|
||||
this.oldestUnexpiredTS = scan.isRaw() ? 0L : now - scanInfo.getTtl();
|
||||
|
||||
this.now = this.store != null ? this.store.getClock().now() :
|
||||
new Clock.System().now();
|
||||
this.timestampType = this.store != null ?
|
||||
this.store.getClock().getTimestampType() : TimestampType.PHYSICAL;
|
||||
// Convert to milliseconds before subtracting time
|
||||
long diff = this.timestampType.toEpochTimeMillisFromTimestamp(now) - scanInfo.getTtl();
|
||||
// Prevent overflow if diff is negative and timestampType is HYBRID
|
||||
diff = diff > 0 ? timestampType.fromEpochTimeMillisToTimestamp(diff) : 0L;
|
||||
this.oldestUnexpiredTS = scan.isRaw() ? 0L : diff;
|
||||
|
||||
this.minVersions = scanInfo.getMinVersions();
|
||||
|
||||
// We look up row-column Bloom filters for multi-column queries as part of
|
||||
|
@ -205,6 +210,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
// readType is default if the scan keeps running for a long time.
|
||||
this.scanUsePread = this.readType != Scan.ReadType.STREAM;
|
||||
}
|
||||
|
||||
this.preadMaxBytes = scanInfo.getPreadMaxBytes();
|
||||
this.cellsPerHeartbeatCheck = scanInfo.getCellsPerTimeoutCheck();
|
||||
// Parallel seeking is on if the config allows and more there is more than one store file.
|
||||
|
@ -237,8 +243,8 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
if (columns != null && scan.isRaw()) {
|
||||
throw new DoNotRetryIOException("Cannot specify any column for a raw scan");
|
||||
}
|
||||
matcher = UserScanQueryMatcher.create(scan, scanInfo, columns, oldestUnexpiredTS, now,
|
||||
store.getCoprocessorHost());
|
||||
matcher = UserScanQueryMatcher.create(scan, scanInfo, columns, oldestUnexpiredTS, now, store
|
||||
.getCoprocessorHost());
|
||||
|
||||
this.store.addChangedReaderObserver(this);
|
||||
|
||||
|
@ -314,12 +320,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, dropDeletesFromRow,
|
||||
dropDeletesToRow, store.getCoprocessorHost());
|
||||
} else {
|
||||
matcher = CompactionScanQueryMatcher.create(scanInfo, scanType, smallestReadPoint,
|
||||
earliestPutTs, oldestUnexpiredTS, now, dropDeletesFromRow, dropDeletesToRow,
|
||||
store.getCoprocessorHost());
|
||||
earliestPutTs, oldestUnexpiredTS, now, dropDeletesFromRow,
|
||||
dropDeletesToRow, store.getCoprocessorHost());
|
||||
}
|
||||
|
||||
// Filter the list of scanners using Bloom filters, time range, TTL, etc.
|
||||
|
@ -342,6 +348,26 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
0);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
StoreScanner(final Store store, final Scan scan, ScanInfo scanInfo,
|
||||
ScanType scanType, final NavigableSet<byte[]> columns,
|
||||
final List<KeyValueScanner> scanners) throws IOException {
|
||||
this(store, scan, scanInfo, scanType, columns, scanners,
|
||||
HConstants.LATEST_TIMESTAMP,
|
||||
// 0 is passed as readpoint because the test bypasses Store
|
||||
0);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
StoreScanner(final Store store, final Scan scan, ScanInfo scanInfo,
|
||||
ScanType scanType, final NavigableSet<byte[]> columns,
|
||||
final List<KeyValueScanner> scanners, long earliestPutTs)
|
||||
throws IOException {
|
||||
this(store, scan, scanInfo, scanType, columns, scanners, earliestPutTs,
|
||||
// 0 is passed as readpoint because the test bypasses Store
|
||||
0);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
StoreScanner(final Scan scan, ScanInfo scanInfo,
|
||||
ScanType scanType, final NavigableSet<byte[]> columns,
|
||||
|
@ -352,14 +378,12 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
0);
|
||||
}
|
||||
|
||||
public StoreScanner(final Scan scan, ScanInfo scanInfo, ScanType scanType,
|
||||
final NavigableSet<byte[]> columns, final List<? extends KeyValueScanner> scanners, long earliestPutTs,
|
||||
public StoreScanner(final Store store, final Scan scan, ScanInfo scanInfo, ScanType scanType,
|
||||
final NavigableSet<byte[]> columns, final List<KeyValueScanner> scanners, long earliestPutTs,
|
||||
long readPt) throws IOException {
|
||||
this(null, scan, scanInfo, columns, readPt,
|
||||
scanType == ScanType.USER_SCAN ? scan.getCacheBlocks() : false, scanType);
|
||||
this(store, scan, scanInfo, columns, readPt, scan.getCacheBlocks(), scanType);
|
||||
if (scanType == ScanType.USER_SCAN) {
|
||||
this.matcher = UserScanQueryMatcher.create(scan, scanInfo, columns, oldestUnexpiredTS, now,
|
||||
null);
|
||||
this.matcher = UserScanQueryMatcher.create(scan, scanInfo, columns, oldestUnexpiredTS, now, null);
|
||||
} else {
|
||||
if (scan.hasFilter() || (scan.getStartRow() != null && scan.getStartRow().length > 0)
|
||||
|| (scan.getStopRow() != null && scan.getStopRow().length > 0)
|
||||
|
@ -367,10 +391,41 @@ 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, null, null,
|
||||
store.getCoprocessorHost());
|
||||
} else {
|
||||
this.matcher = CompactionScanQueryMatcher.create(scanInfo, scanType, Long.MAX_VALUE,
|
||||
earliestPutTs, oldestUnexpiredTS, now, null, null, null);
|
||||
earliestPutTs, oldestUnexpiredTS, now, null, null,
|
||||
null);
|
||||
}
|
||||
}
|
||||
|
||||
// Seek all scanners to the initial key
|
||||
seekScanners(scanners, matcher.getStartKey(), false, parallelSeekEnabled);
|
||||
addCurrentScanners(scanners);
|
||||
resetKVHeap(scanners, scanInfo.getComparator());
|
||||
}
|
||||
|
||||
public StoreScanner(final Scan scan, ScanInfo scanInfo, ScanType scanType,
|
||||
final NavigableSet<byte[]> columns, final List<? extends KeyValueScanner> scanners, long earliestPutTs,
|
||||
long readPt) throws IOException {
|
||||
this(null, scan, scanInfo, columns, readPt,
|
||||
scanType == ScanType.USER_SCAN ? scan.getCacheBlocks() : false, scanType);
|
||||
if (scanType == ScanType.USER_SCAN) {
|
||||
this.matcher = UserScanQueryMatcher.create(scan, scanInfo, columns, oldestUnexpiredTS, now, null);
|
||||
} else {
|
||||
if (scan.hasFilter() || (scan.getStartRow() != null && scan.getStartRow().length > 0)
|
||||
|| (scan.getStopRow() != null && scan.getStopRow().length > 0)
|
||||
|| !scan.getTimeRange().isAllTime() || columns != null) {
|
||||
// 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());
|
||||
} else {
|
||||
this.matcher = CompactionScanQueryMatcher.create(scanInfo, scanType, Long.MAX_VALUE,
|
||||
earliestPutTs, oldestUnexpiredTS, now, null, null,
|
||||
null);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
@ -62,9 +63,21 @@ public abstract class DropDeletesCompactionScanQueryMatcher extends CompactionSc
|
|||
protected final MatchCode tryDropDelete(Cell cell) {
|
||||
long timestamp = cell.getTimestamp();
|
||||
// If it is not the time to drop the delete marker, just return
|
||||
if (timeToPurgeDeletes > 0 && now - timestamp <= timeToPurgeDeletes) {
|
||||
return MatchCode.INCLUDE;
|
||||
if (timeToPurgeDeletes > 0) {
|
||||
// Assumes now and timestamp should be of same type. It should be the case.
|
||||
// Else there is something wrong. if it happens in tests, tests should be rewritten.
|
||||
if (TimestampType.HYBRID.isLikelyOfType(now, true)) {
|
||||
if (TimestampType.HYBRID.toEpochTimeMillisFromTimestamp(now) - TimestampType.HYBRID
|
||||
.toEpochTimeMillisFromTimestamp(timestamp) <= timeToPurgeDeletes) {
|
||||
return MatchCode.INCLUDE;
|
||||
}
|
||||
} else {
|
||||
if (now - timestamp <= timeToPurgeDeletes) {
|
||||
return MatchCode.INCLUDE;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (keepDeletedCells == KeepDeletedCells.TRUE
|
||||
|| (keepDeletedCells == KeepDeletedCells.TTL && timestamp >= oldestUnexpiredTS)) {
|
||||
// If keepDeletedCell is true, or the delete marker is not expired yet, we should include it
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.Tag;
|
|||
import org.apache.hadoop.hbase.TagType;
|
||||
import org.apache.hadoop.hbase.TagUtil;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.TimestampType;
|
||||
import org.apache.hadoop.hbase.filter.Filter;
|
||||
import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
|
||||
|
@ -153,8 +154,23 @@ public abstract class ScanQueryMatcher implements ShipperListener {
|
|||
long ts = cell.getTimestamp();
|
||||
assert t.getValueLength() == Bytes.SIZEOF_LONG;
|
||||
long ttl = TagUtil.getValueAsLong(t);
|
||||
if (ts + ttl < now) {
|
||||
return true;
|
||||
if (TimestampType.HYBRID.isLikelyOfType(ts, true)) {
|
||||
if (TimestampType.HYBRID.isLikelyOfType(now, true)) {
|
||||
if (TimestampType.HYBRID.toEpochTimeMillisFromTimestamp(ts) + ttl < TimestampType.HYBRID
|
||||
.toEpochTimeMillisFromTimestamp(now)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
else {
|
||||
if (TimestampType.HYBRID.toEpochTimeMillisFromTimestamp(ts) + ttl < now) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
} else {
|
||||
if (ts + ttl < now) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
// Per cell TTLs cannot extend lifetime beyond family settings, so
|
||||
// fall through to check that
|
||||
|
|
|
@ -55,6 +55,7 @@ import org.apache.hadoop.hbase.ProcedureInfo;
|
|||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.TimestampType;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Append;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
|
@ -777,7 +778,14 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
|
|||
// any cells found there inclusively.
|
||||
long latestTs = Math.max(opTs, latestCellTs);
|
||||
if (latestTs == 0 || latestTs == HConstants.LATEST_TIMESTAMP) {
|
||||
latestTs = EnvironmentEdgeManager.currentTime();
|
||||
if (latestCellTs == HConstants.LATEST_TIMESTAMP || latestCellTs == 0) {
|
||||
latestTs = HConstants.LATEST_TIMESTAMP - 1;
|
||||
} else if (TimestampType.HYBRID.isLikelyOfType(latestCellTs, true)) {
|
||||
latestTs = TimestampType.HYBRID.fromEpochTimeMillisToTimestamp(EnvironmentEdgeManager
|
||||
.currentTime());
|
||||
} else {
|
||||
latestTs = EnvironmentEdgeManager.currentTime();
|
||||
}
|
||||
}
|
||||
get.setTimeRange(0, latestTs + 1);
|
||||
// In case of Put operation we set to read all versions. This was done to consider the case
|
||||
|
|
|
@ -252,6 +252,16 @@ public class MockRegionServerServices implements RegionServerServices {
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override public Clock getRegionServerClock(ClockType clockType) {
|
||||
if (clockType.equals(ClockType.HLC)){
|
||||
return new Clock.HLC();
|
||||
} else if (clockType.equals(ClockType.SYSTEM_MONOTONIC)) {
|
||||
return new Clock.SystemMonotonic();
|
||||
} else {
|
||||
return new Clock.System();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ExecutorService getExecutorService() {
|
||||
return null;
|
||||
|
|
|
@ -0,0 +1,127 @@
|
|||
/**
|
||||
* 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 static org.junit.Assert.*;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.TimestampType;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
|
||||
@Category({MediumTests.class})
|
||||
public class TestClockWithCluster {
|
||||
private static final Log LOG = LogFactory.getLog(TestClockWithCluster.class);
|
||||
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
||||
private static Connection connection;
|
||||
private byte[] columnFamily = Bytes.toBytes("testCF");
|
||||
@BeforeClass
|
||||
public static void setupClass() throws Exception {
|
||||
UTIL.startMiniCluster(1);
|
||||
connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDownClass() throws Exception {
|
||||
connection.close();
|
||||
UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
private void verifyTimestamps(Table table, final byte[] f, int startRow, int endRow,
|
||||
TimestampType timestamp, boolean isMonotonic) throws IOException {
|
||||
for (int i = startRow; i < endRow; i++) {
|
||||
String failMsg = "Failed verification of row :" + i;
|
||||
byte[] data = Bytes.toBytes(String.valueOf(i));
|
||||
Get get = new Get(data);
|
||||
Result result = table.get(get);
|
||||
Cell cell = result.getColumnLatestCell(f, null);
|
||||
assertTrue(failMsg, timestamp.isLikelyOfType(cell.getTimestamp(), isMonotonic));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNewTablesAreCreatedWithSystemClock() throws IOException {
|
||||
try {
|
||||
Admin admin = connection.getAdmin();
|
||||
TableName tableName = TableName.valueOf("TestNewTablesAreSystemByDefault");
|
||||
admin.createTable(new HTableDescriptor(tableName).addFamily(new
|
||||
HColumnDescriptor(columnFamily)));
|
||||
|
||||
Table table = connection.getTable(tableName);
|
||||
|
||||
ClockType clockType = admin.getTableDescriptor(tableName).getClockType();
|
||||
assertEquals(ClockType.SYSTEM, clockType);
|
||||
// write
|
||||
UTIL.loadNumericRows(table, columnFamily, 0, 1000);
|
||||
// read , check if the it is same.
|
||||
UTIL.verifyNumericRows(table, Bytes.toBytes("testCF"), 0, 1000, 0);
|
||||
|
||||
// This check will be useful if Clock type were to be system monotonic or HLC.
|
||||
verifyTimestamps(table, columnFamily, 0, 1000, TimestampType.PHYSICAL, false);
|
||||
} catch(Exception e) {
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMetaTableClockTypeIsSystem() {
|
||||
try {
|
||||
Admin admin = connection.getAdmin();
|
||||
Table table = connection.getTable(TableName.META_TABLE_NAME);
|
||||
ClockType clockType = admin.getTableDescriptor(TableName.META_TABLE_NAME).getClockType();
|
||||
assertEquals(ClockType.SYSTEM, clockType);
|
||||
} catch(IOException ioE) {
|
||||
fail("Execution should not come here");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMetaTableTimestampsAreSystem() {
|
||||
// Checks timestamps of whatever is present in meta table currently.
|
||||
// ToDo: Include complete meta table sample with all column families to check all paths of
|
||||
// meta table modification.
|
||||
try {
|
||||
Table table = connection.getTable(TableName.META_TABLE_NAME);
|
||||
Result result = table.getScanner(new Scan()).next();
|
||||
for (Cell cell : result.rawCells()) {
|
||||
assertTrue(TimestampType.PHYSICAL.isLikelyOfType(cell.getTimestamp(), false));
|
||||
}
|
||||
} catch(IOException ioE) {
|
||||
fail("Execution should not come here");
|
||||
}
|
||||
}
|
||||
}
|
|
@ -19,19 +19,24 @@
|
|||
|
||||
package org.apache.hadoop.hbase.coprocessor;
|
||||
|
||||
import static org.bouncycastle.asn1.x500.style.RFC4519Style.name;
|
||||
import static org.junit.Assert.assertArrayEquals;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.NavigableMap;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.ClockType;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.Increment;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
|
@ -45,12 +50,17 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
|
|||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
|
||||
import org.apache.hadoop.hbase.util.TestTableName;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
import org.junit.runners.Parameterized.Parameters;
|
||||
|
||||
/**
|
||||
* This test runs batch mutation with Increments which have custom TimeRange.
|
||||
|
@ -59,14 +69,16 @@ import org.junit.experimental.categories.Category;
|
|||
* See HBASE-15698
|
||||
*/
|
||||
@Category({CoprocessorTests.class, MediumTests.class})
|
||||
@RunWith(Parameterized.class)
|
||||
public class TestIncrementTimeRange {
|
||||
|
||||
private static final HBaseTestingUtility util = new HBaseTestingUtility();
|
||||
private static ManualEnvironmentEdge mee = new ManualEnvironmentEdge();
|
||||
|
||||
private static final TableName TEST_TABLE = TableName.valueOf("test");
|
||||
@Rule
|
||||
public TestTableName TEST_TABLE = new TestTableName();
|
||||
private static final byte[] TEST_FAMILY = Bytes.toBytes("f1");
|
||||
|
||||
private static final byte[][] TEST_FAMILIES = new byte[][]{TEST_FAMILY};
|
||||
private static final byte[] ROW_A = Bytes.toBytes("aaa");
|
||||
private static final byte[] ROW_B = Bytes.toBytes("bbb");
|
||||
private static final byte[] ROW_C = Bytes.toBytes("ccc");
|
||||
|
@ -80,6 +92,18 @@ public class TestIncrementTimeRange {
|
|||
private Table hTableInterface;
|
||||
private Table table;
|
||||
|
||||
private ClockType clockType;
|
||||
|
||||
@Parameters(name = "{0}")
|
||||
public static Iterable<Object> data() {
|
||||
return Arrays.asList(new Object[] {ClockType.HLC, ClockType.SYSTEM_MONOTONIC, ClockType
|
||||
.SYSTEM});
|
||||
}
|
||||
|
||||
public TestIncrementTimeRange(ClockType clockType) {
|
||||
this.clockType = clockType;
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void setupBeforeClass() throws Exception {
|
||||
util.getConfiguration().set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
|
||||
|
@ -98,7 +122,8 @@ public class TestIncrementTimeRange {
|
|||
|
||||
@Before
|
||||
public void before() throws Exception {
|
||||
table = util.createTable(TEST_TABLE, TEST_FAMILY);
|
||||
HTableDescriptor htd = util.createTableDescriptor(TEST_TABLE.getTableName()).setClockType(clockType);
|
||||
table = util.createTable(htd, TEST_FAMILIES, new Configuration(HBaseConfiguration.create()));
|
||||
|
||||
Put puta = new Put(ROW_A);
|
||||
puta.addColumn(TEST_FAMILY, qualifierCol1, bytes1);
|
||||
|
@ -121,7 +146,7 @@ public class TestIncrementTimeRange {
|
|||
}
|
||||
} finally {
|
||||
try {
|
||||
util.deleteTable(TEST_TABLE);
|
||||
util.deleteTable(TEST_TABLE.getTableName());
|
||||
} catch (IOException ioe) {
|
||||
}
|
||||
}
|
||||
|
@ -150,7 +175,7 @@ public class TestIncrementTimeRange {
|
|||
|
||||
@Test
|
||||
public void testHTableInterfaceMethods() throws Exception {
|
||||
hTableInterface = util.getConnection().getTable(TEST_TABLE);
|
||||
hTableInterface = util.getConnection().getTable(TEST_TABLE.getTableName());
|
||||
checkHTableInterfaceMethods();
|
||||
}
|
||||
|
||||
|
@ -162,7 +187,7 @@ public class TestIncrementTimeRange {
|
|||
|
||||
time = EnvironmentEdgeManager.currentTime();
|
||||
mee.setValue(time);
|
||||
TimeRange range10 = new TimeRange(1, time+10);
|
||||
TimeRange range10 = new TimeRange(1, time + 10);
|
||||
hTableInterface.increment(new Increment(ROW_A).addColumn(TEST_FAMILY, qualifierCol1, 10L)
|
||||
.setTimeRange(range10.getMin(), range10.getMax()));
|
||||
checkRowValue(ROW_A, Bytes.toBytes(11L));
|
||||
|
@ -171,7 +196,7 @@ public class TestIncrementTimeRange {
|
|||
|
||||
time = EnvironmentEdgeManager.currentTime();
|
||||
mee.setValue(time);
|
||||
TimeRange range2 = new TimeRange(1, time+20);
|
||||
TimeRange range2 = new TimeRange(1, time + 20);
|
||||
List<Row> actions =
|
||||
Arrays.asList(new Row[] { new Increment(ROW_A).addColumn(TEST_FAMILY, qualifierCol1, 2L)
|
||||
.setTimeRange(range2.getMin(), range2.getMax()),
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.mapreduce;
|
||||
|
||||
import static org.apache.hadoop.hbase.constraint.CheckConfigurationConstraint.getConfiguration;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
|
@ -28,7 +29,9 @@ import java.io.PrintStream;
|
|||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.ClockType;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
|
@ -189,13 +192,24 @@ public class TestCopyTable {
|
|||
*/
|
||||
@Test
|
||||
public void testRenameFamily() throws Exception {
|
||||
final TableName sourceTable = TableName.valueOf(name.getMethodName() + "source");
|
||||
final TableName targetTable = TableName.valueOf(name.getMethodName() + "-target");
|
||||
testRenameFamily(ClockType.SYSTEM);
|
||||
testRenameFamily(ClockType.SYSTEM_MONOTONIC);
|
||||
testRenameFamily(ClockType.HLC);
|
||||
}
|
||||
|
||||
public void testRenameFamily(ClockType clockType) throws Exception {
|
||||
TableName sourceTable = TableName.valueOf("sourceTable");
|
||||
HTableDescriptor sourceTableDesc = new HTableDescriptor(sourceTable);
|
||||
sourceTableDesc.setClockType(clockType);
|
||||
TableName targetTable = TableName.valueOf("targetTable");
|
||||
HTableDescriptor targetTableDesc = new HTableDescriptor(targetTable);
|
||||
targetTableDesc.setClockType(clockType);
|
||||
|
||||
byte[][] families = { FAMILY_A, FAMILY_B };
|
||||
|
||||
Table t = TEST_UTIL.createTable(sourceTable, families);
|
||||
Table t2 = TEST_UTIL.createTable(targetTable, families);
|
||||
Table t = TEST_UTIL.createTable(sourceTableDesc, families, (byte[][]) null, new Configuration
|
||||
(getConfiguration()));
|
||||
Table t2 = TEST_UTIL.createTable(targetTableDesc, families, (byte[][]) null, new Configuration
|
||||
(getConfiguration()));
|
||||
Put p = new Put(ROW1);
|
||||
p.addColumn(FAMILY_A, QUALIFIER, Bytes.toBytes("Data11"));
|
||||
p.addColumn(FAMILY_B, QUALIFIER, Bytes.toBytes("Data12"));
|
||||
|
@ -225,6 +239,9 @@ public class TestCopyTable {
|
|||
// Data from the family of B is not copied
|
||||
assertNull(b1);
|
||||
|
||||
TEST_UTIL.deleteTable(sourceTable);
|
||||
TEST_UTIL.deleteTable(targetTable);
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.ArrayBackedTag;
|
|||
import org.apache.hadoop.hbase.CategoryBasedTimeout;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.ClockType;
|
||||
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
|
|
|
@ -34,6 +34,8 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.hbase.Abortable;
|
||||
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;
|
||||
|
@ -578,6 +580,12 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Clock getRegionServerClock(ClockType clockType) {
|
||||
Clock systemClock = new Clock.System();
|
||||
return systemClock;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ExecutorService getExecutorService() {
|
||||
return null;
|
||||
|
|
|
@ -42,8 +42,12 @@ import org.junit.Before;
|
|||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import org.apache.hadoop.hbase.Clock;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
/**
|
||||
* compacted memstore test case
|
||||
|
@ -169,24 +173,29 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
|
|||
/** Test getNextRow from memstore
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
@Override
|
||||
@Test
|
||||
public void testGetNextRow() throws Exception {
|
||||
addRows(this.memstore);
|
||||
testGetNextRow(new Clock.HLC());
|
||||
testGetNextRow(new Clock.SystemMonotonic());
|
||||
testGetNextRow(new Clock.System());
|
||||
}
|
||||
|
||||
public void testGetNextRow(Clock clock) throws Exception {
|
||||
addRows(this.memstore, clock);
|
||||
// Add more versions to make it a little more interesting.
|
||||
Thread.sleep(1);
|
||||
addRows(this.memstore);
|
||||
addRows(this.memstore, clock);
|
||||
Cell closestToEmpty = ((CompactingMemStore)this.memstore).getNextRow(KeyValue.LOWESTKEY);
|
||||
assertTrue(CellComparator.COMPARATOR.compareRows(closestToEmpty,
|
||||
new KeyValue(Bytes.toBytes(0), System.currentTimeMillis())) == 0);
|
||||
new KeyValue(Bytes.toBytes(0), clock.now())) == 0);
|
||||
for (int i = 0; i < ROW_COUNT; i++) {
|
||||
Cell nr = ((CompactingMemStore)this.memstore).getNextRow(new KeyValue(Bytes.toBytes(i),
|
||||
System.currentTimeMillis()));
|
||||
clock.now()));
|
||||
if (i + 1 == ROW_COUNT) {
|
||||
assertEquals(nr, null);
|
||||
} else {
|
||||
assertTrue(CellComparator.COMPARATOR.compareRows(nr,
|
||||
new KeyValue(Bytes.toBytes(i + 1), System.currentTimeMillis())) == 0);
|
||||
new KeyValue(Bytes.toBytes(i + 1), clock.now())) == 0);
|
||||
}
|
||||
}
|
||||
//starting from each row, validate results should contain the starting row
|
||||
|
@ -195,9 +204,10 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
|
|||
ScanInfo scanInfo = new ScanInfo(conf, FAMILY, 0, 1, Integer.MAX_VALUE,
|
||||
KeepDeletedCells.FALSE, HConstants.DEFAULT_BLOCKSIZE, 0, this.memstore.getComparator());
|
||||
ScanType scanType = ScanType.USER_SCAN;
|
||||
InternalScanner scanner = new StoreScanner(new Scan(
|
||||
Bytes.toBytes(startRowId)), scanInfo, scanType, null,
|
||||
memstore.getScanners(0));
|
||||
Store mockStore = mock(HStore.class);
|
||||
when(mockStore.getClock()).thenReturn(Clock.getDummyClockOfGivenClockType(clock.clockType));
|
||||
InternalScanner scanner = new StoreScanner(mockStore, new Scan(Bytes.toBytes(startRowId)),
|
||||
scanInfo, scanType, null, memstore.getScanners(0)) ;
|
||||
List<Cell> results = new ArrayList<>();
|
||||
for (int i = 0; scanner.next(results); i++) {
|
||||
int rowId = startRowId + i;
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.CategoryBasedTimeout;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.Clock;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
|
@ -60,6 +61,8 @@ import static org.junit.Assert.assertEquals;
|
|||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
|
@ -580,16 +583,22 @@ public class TestDefaultMemStore {
|
|||
*/
|
||||
@Test
|
||||
public void testGetNextRow() throws Exception {
|
||||
addRows(this.memstore);
|
||||
testGetNextRow(new Clock.HLC());
|
||||
testGetNextRow(new Clock.SystemMonotonic());
|
||||
testGetNextRow(new Clock.System());
|
||||
}
|
||||
|
||||
public void testGetNextRow(Clock clock) throws Exception {
|
||||
addRows(this.memstore, clock);
|
||||
// Add more versions to make it a little more interesting.
|
||||
Thread.sleep(1);
|
||||
addRows(this.memstore);
|
||||
addRows(this.memstore, clock);
|
||||
Cell closestToEmpty = ((DefaultMemStore) this.memstore).getNextRow(KeyValue.LOWESTKEY);
|
||||
assertTrue(CellComparator.COMPARATOR.compareRows(closestToEmpty,
|
||||
new KeyValue(Bytes.toBytes(0), System.currentTimeMillis())) == 0);
|
||||
for (int i = 0; i < ROW_COUNT; i++) {
|
||||
Cell nr = ((DefaultMemStore) this.memstore).getNextRow(new KeyValue(Bytes.toBytes(i),
|
||||
System.currentTimeMillis()));
|
||||
clock.now()));
|
||||
if (i + 1 == ROW_COUNT) {
|
||||
assertEquals(nr, null);
|
||||
} else {
|
||||
|
@ -603,9 +612,11 @@ public class TestDefaultMemStore {
|
|||
ScanInfo scanInfo = new ScanInfo(conf, FAMILY, 0, 1, Integer.MAX_VALUE,
|
||||
KeepDeletedCells.FALSE, HConstants.DEFAULT_BLOCKSIZE, 0, this.memstore.getComparator());
|
||||
ScanType scanType = ScanType.USER_SCAN;
|
||||
try (InternalScanner scanner = new StoreScanner(new Scan(
|
||||
Bytes.toBytes(startRowId)), scanInfo, scanType, null,
|
||||
memstore.getScanners(0))) {
|
||||
Store store = mock(HStore.class);
|
||||
when(store.getClock()).thenReturn(Clock.getDummyClockOfGivenClockType(clock.clockType));
|
||||
|
||||
try (InternalScanner scanner = new StoreScanner(store, new Scan(Bytes.toBytes(startRowId)),
|
||||
scanInfo, scanType, null, memstore.getScanners(0))) {
|
||||
List<Cell> results = new ArrayList<>();
|
||||
for (int i = 0; scanner.next(results); i++) {
|
||||
int rowId = startRowId + i;
|
||||
|
@ -1023,6 +1034,24 @@ public class TestDefaultMemStore {
|
|||
return ROW_COUNT;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds {@link #ROW_COUNT} rows and {@link #QUALIFIER_COUNT}
|
||||
* @param hmc Instance to add rows to.
|
||||
* @return How many rows we added.
|
||||
* @throws IOException
|
||||
*/
|
||||
protected int addRows(final MemStore hmc, Clock clock) {
|
||||
for (int i = 0; i < ROW_COUNT; i++) {
|
||||
long timestamp = clock.now();
|
||||
for (int ii = 0; ii < QUALIFIER_COUNT; ii++) {
|
||||
byte [] row = Bytes.toBytes(i);
|
||||
byte [] qf = makeQualifier(i, ii);
|
||||
hmc.add(new KeyValue(row, FAMILY, qf, timestamp, qf), null);
|
||||
}
|
||||
}
|
||||
return ROW_COUNT;
|
||||
}
|
||||
|
||||
private long runSnapshot(final AbstractMemStore hmc) throws UnexpectedStateException {
|
||||
// Save off old state.
|
||||
int oldHistorySize = hmc.getSnapshot().getCellsCount();
|
||||
|
|
|
@ -43,6 +43,7 @@ import static org.mockito.Mockito.when;
|
|||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
|
@ -77,6 +78,9 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.ArrayBackedTag;
|
||||
import org.apache.hadoop.hbase.CategoryBasedTimeout;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.Clock;
|
||||
import org.apache.hadoop.hbase.ClockType;
|
||||
import org.apache.hadoop.hbase.TimestampType;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
|
||||
import org.apache.hadoop.hbase.DroppedSnapshotException;
|
||||
|
@ -6007,9 +6011,12 @@ public class TestHRegion {
|
|||
|
||||
@Test
|
||||
public void testCellTTLs() throws IOException {
|
||||
IncrementingEnvironmentEdge edge = new IncrementingEnvironmentEdge();
|
||||
EnvironmentEdgeManager.injectEdge(edge);
|
||||
testCellTTLs(ClockType.SYSTEM);
|
||||
testCellTTLs(ClockType.SYSTEM_MONOTONIC);
|
||||
testCellTTLs(ClockType.HLC);
|
||||
}
|
||||
|
||||
public void testCellTTLs(ClockType clockType) throws IOException {
|
||||
final byte[] row = Bytes.toBytes("testRow");
|
||||
final byte[] q1 = Bytes.toBytes("q1");
|
||||
final byte[] q2 = Bytes.toBytes("q2");
|
||||
|
@ -6020,6 +6027,8 @@ public class TestHRegion {
|
|||
HColumnDescriptor hcd = new HColumnDescriptor(fam1);
|
||||
hcd.setTimeToLive(10); // 10 seconds
|
||||
htd.addFamily(hcd);
|
||||
htd.setClockType(clockType);
|
||||
TimestampType timestampType = clockType.timestampType();
|
||||
|
||||
Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
|
||||
conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MIN_FORMAT_VERSION_WITH_TAGS);
|
||||
|
@ -6028,22 +6037,32 @@ public class TestHRegion {
|
|||
HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY),
|
||||
TEST_UTIL.getDataTestDir(), conf, htd);
|
||||
assertNotNull(region);
|
||||
|
||||
region.setClock(Clock.getDummyClockOfGivenClockType(clockType));
|
||||
long now = timestampType.toEpochTimeMillisFromTimestamp(region.getClock().now());
|
||||
ManualEnvironmentEdge mee = new ManualEnvironmentEdge();
|
||||
EnvironmentEdgeManager.injectEdge(mee);
|
||||
mee.setValue(now);
|
||||
|
||||
try {
|
||||
long now = EnvironmentEdgeManager.currentTime();
|
||||
// Add a cell that will expire in 5 seconds via cell TTL
|
||||
region.put(new Put(row).add(new KeyValue(row, fam1, q1, now,
|
||||
HConstants.EMPTY_BYTE_ARRAY, new ArrayBackedTag[] {
|
||||
region.put(new Put(row).add(new KeyValue(row, fam1, q1, timestampType
|
||||
.fromEpochTimeMillisToTimestamp(now),
|
||||
HConstants.EMPTY_BYTE_ARRAY, new ArrayBackedTag[] {
|
||||
// TTL tags specify ts in milliseconds
|
||||
new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(5000L)) } )));
|
||||
// Add a cell that will expire after 10 seconds via family setting
|
||||
region.put(new Put(row).addColumn(fam1, q2, now, HConstants.EMPTY_BYTE_ARRAY));
|
||||
region.put(new Put(row).addColumn(fam1, q2, timestampType
|
||||
.fromEpochTimeMillisToTimestamp(now), HConstants.EMPTY_BYTE_ARRAY));
|
||||
// Add a cell that will expire in 15 seconds via cell TTL
|
||||
region.put(new Put(row).add(new KeyValue(row, fam1, q3, now + 10000 - 1,
|
||||
HConstants.EMPTY_BYTE_ARRAY, new ArrayBackedTag[] {
|
||||
region.put(new Put(row).add(new KeyValue(row, fam1, q3, timestampType
|
||||
.fromEpochTimeMillisToTimestamp(now + 10000 - 1),
|
||||
HConstants.EMPTY_BYTE_ARRAY, new ArrayBackedTag[] {
|
||||
// TTL tags specify ts in milliseconds
|
||||
new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(5000L)) } )));
|
||||
// Add a cell that will expire in 20 seconds via family setting
|
||||
region.put(new Put(row).addColumn(fam1, q4, now + 10000 - 1, HConstants.EMPTY_BYTE_ARRAY));
|
||||
region.put(new Put(row).addColumn(fam1, q4, timestampType.fromEpochTimeMillisToTimestamp
|
||||
(now + 10000 - 1), HConstants.EMPTY_BYTE_ARRAY));
|
||||
|
||||
// Flush so we are sure store scanning gets this right
|
||||
region.flush(true);
|
||||
|
@ -6056,7 +6075,7 @@ public class TestHRegion {
|
|||
assertNotNull(r.getValue(fam1, q4));
|
||||
|
||||
// Increment time to T+5 seconds
|
||||
edge.incrementTime(5000);
|
||||
mee.setValue(now + 5001);
|
||||
|
||||
r = region.get(new Get(row));
|
||||
assertNull(r.getValue(fam1, q1));
|
||||
|
@ -6065,7 +6084,7 @@ public class TestHRegion {
|
|||
assertNotNull(r.getValue(fam1, q4));
|
||||
|
||||
// Increment time to T+10 seconds
|
||||
edge.incrementTime(5000);
|
||||
mee.setValue(now + 10001);
|
||||
|
||||
r = region.get(new Get(row));
|
||||
assertNull(r.getValue(fam1, q1));
|
||||
|
@ -6074,7 +6093,7 @@ public class TestHRegion {
|
|||
assertNotNull(r.getValue(fam1, q4));
|
||||
|
||||
// Increment time to T+15 seconds
|
||||
edge.incrementTime(5000);
|
||||
mee.setValue(now + 15000);
|
||||
|
||||
r = region.get(new Get(row));
|
||||
assertNull(r.getValue(fam1, q1));
|
||||
|
@ -6083,7 +6102,7 @@ public class TestHRegion {
|
|||
assertNotNull(r.getValue(fam1, q4));
|
||||
|
||||
// Increment time to T+20 seconds
|
||||
edge.incrementTime(10000);
|
||||
mee.setValue(now + 20000);
|
||||
|
||||
r = region.get(new Get(row));
|
||||
assertNull(r.getValue(fam1, q1));
|
||||
|
@ -6112,7 +6131,12 @@ public class TestHRegion {
|
|||
assertEquals(Bytes.toLong(val), 2L);
|
||||
|
||||
// Increment time to T+25 seconds
|
||||
edge.incrementTime(5000);
|
||||
|
||||
if (clockType == ClockType.SYSTEM) {
|
||||
mee.setValue(now + 25002);
|
||||
} else {
|
||||
mee.setValue(now + 25001);
|
||||
}
|
||||
|
||||
// Value should be back to 1
|
||||
r = region.get(new Get(row));
|
||||
|
@ -6121,7 +6145,7 @@ public class TestHRegion {
|
|||
assertEquals(Bytes.toLong(val), 1L);
|
||||
|
||||
// Increment time to T+30 seconds
|
||||
edge.incrementTime(5000);
|
||||
mee.setValue(now + 30001);
|
||||
|
||||
// Original value written at T+20 should be gone now via family TTL
|
||||
r = region.get(new Get(row));
|
||||
|
|
|
@ -60,6 +60,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;
|
||||
|
@ -99,6 +101,8 @@ import org.junit.Rule;
|
|||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
import org.apache.hadoop.hbase.Clock;
|
||||
import org.apache.hadoop.hbase.ClockType;
|
||||
|
||||
/**
|
||||
* Tests of HRegion methods for replaying flush, compaction, region open, etc events for secondary
|
||||
|
@ -171,6 +175,7 @@ public class TestHRegionReplayEvents {
|
|||
when(rss.getServerName()).thenReturn(ServerName.valueOf("foo", 1, 1));
|
||||
when(rss.getConfiguration()).thenReturn(CONF);
|
||||
when(rss.getRegionServerAccounting()).thenReturn(new RegionServerAccounting(CONF));
|
||||
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);
|
||||
|
|
|
@ -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;
|
||||
|
@ -106,6 +109,7 @@ public class TestRegionSplitPolicy {
|
|||
final List<Region> 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);
|
||||
|
@ -167,6 +171,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 =
|
||||
|
|
|
@ -21,6 +21,8 @@ package org.apache.hadoop.hbase.regionserver;
|
|||
|
||||
import static org.apache.hadoop.hbase.regionserver.KeyValueScanFixture.scanFixture;
|
||||
import static org.junit.Assert.*;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -38,11 +40,14 @@ import org.apache.hadoop.hbase.CategoryBasedTimeout;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.Clock;
|
||||
import org.apache.hadoop.hbase.ClockType;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeepDeletedCells;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueTestUtil;
|
||||
import org.apache.hadoop.hbase.TimestampType;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
|
||||
|
@ -816,16 +821,31 @@ public class TestStoreScanner {
|
|||
*/
|
||||
@Test
|
||||
public void testWildCardTtlScan() throws IOException {
|
||||
//testWildCardTtlScan(ClockType.SYSTEM);
|
||||
//testWildCardTtlScan(ClockType.SYSTEM_MONOTONIC);
|
||||
testWildCardTtlScan(ClockType.HLC);
|
||||
}
|
||||
|
||||
public void testWildCardTtlScan(ClockType clockType) throws IOException {
|
||||
long now = System.currentTimeMillis();
|
||||
TimestampType timestampType = clockType.timestampType();
|
||||
KeyValue [] kvs = new KeyValue[] {
|
||||
KeyValueTestUtil.create("R1", "cf", "a", now-1000, KeyValue.Type.Put, "dont-care"),
|
||||
KeyValueTestUtil.create("R1", "cf", "b", now-10, KeyValue.Type.Put, "dont-care"),
|
||||
KeyValueTestUtil.create("R1", "cf", "c", now-200, KeyValue.Type.Put, "dont-care"),
|
||||
KeyValueTestUtil.create("R1", "cf", "d", now-10000, KeyValue.Type.Put, "dont-care"),
|
||||
KeyValueTestUtil.create("R2", "cf", "a", now, KeyValue.Type.Put, "dont-care"),
|
||||
KeyValueTestUtil.create("R2", "cf", "b", now-10, KeyValue.Type.Put, "dont-care"),
|
||||
KeyValueTestUtil.create("R2", "cf", "c", now-200, KeyValue.Type.Put, "dont-care"),
|
||||
KeyValueTestUtil.create("R2", "cf", "c", now-1000, KeyValue.Type.Put, "dont-care")
|
||||
KeyValueTestUtil.create("R1", "cf", "a", timestampType.fromEpochTimeMillisToTimestamp
|
||||
(now-1000), KeyValue.Type.Put, "dont-care"),
|
||||
KeyValueTestUtil.create("R1", "cf", "b", timestampType.fromEpochTimeMillisToTimestamp
|
||||
(now-10), KeyValue.Type.Put, "dont-care"),
|
||||
KeyValueTestUtil.create("R1", "cf", "c", timestampType.fromEpochTimeMillisToTimestamp
|
||||
(now-200), KeyValue.Type.Put, "dont-care"),
|
||||
KeyValueTestUtil.create("R1", "cf", "d", timestampType.fromEpochTimeMillisToTimestamp
|
||||
(now-10000), KeyValue.Type.Put, "dont-care"),
|
||||
KeyValueTestUtil.create("R2", "cf", "a", timestampType.fromEpochTimeMillisToTimestamp
|
||||
(now), KeyValue.Type.Put, "dont-care"),
|
||||
KeyValueTestUtil.create("R2", "cf", "b", timestampType.fromEpochTimeMillisToTimestamp
|
||||
(now-10), KeyValue.Type.Put, "dont-care"),
|
||||
KeyValueTestUtil.create("R2", "cf", "c", timestampType.fromEpochTimeMillisToTimestamp
|
||||
(now-200), KeyValue.Type.Put, "dont-care"),
|
||||
KeyValueTestUtil.create("R2", "cf", "c", timestampType.fromEpochTimeMillisToTimestamp
|
||||
(now-1000), KeyValue.Type.Put, "dont-care")
|
||||
};
|
||||
List<KeyValueScanner> scanners = scanFixture(kvs);
|
||||
Scan scan = new Scan();
|
||||
|
@ -833,7 +853,9 @@ public class TestStoreScanner {
|
|||
ScanInfo scanInfo = new ScanInfo(CONF, CF, 0, 1, 500, KeepDeletedCells.FALSE,
|
||||
HConstants.DEFAULT_BLOCKSIZE, 0, CellComparator.COMPARATOR);
|
||||
ScanType scanType = ScanType.USER_SCAN;
|
||||
try (StoreScanner scanner = new StoreScanner(scan, scanInfo, scanType, null, scanners)) {
|
||||
Store store = mock(HStore.class);
|
||||
when(store.getClock()).thenReturn(Clock.getDummyClockOfGivenClockType(clockType));
|
||||
try (StoreScanner scanner = new StoreScanner(store, scan, scanInfo, scanType, null, scanners)) {
|
||||
List<Cell> results = new ArrayList<>();
|
||||
Assert.assertEquals(true, scanner.next(results));
|
||||
Assert.assertEquals(2, results.size());
|
||||
|
@ -892,12 +914,19 @@ public class TestStoreScanner {
|
|||
*/
|
||||
@Test
|
||||
public void testExpiredDeleteFamily() throws Exception {
|
||||
testExpiredDeleteFamily(new Clock.HLC());
|
||||
testExpiredDeleteFamily(new Clock.SystemMonotonic());
|
||||
testExpiredDeleteFamily(new Clock.System());
|
||||
}
|
||||
|
||||
public void testExpiredDeleteFamily(Clock clock) throws Exception {
|
||||
long now = System.currentTimeMillis();
|
||||
TimestampType timestampType = clock.getTimestampType();
|
||||
KeyValue [] kvs = new KeyValue[] {
|
||||
new KeyValue(Bytes.toBytes("R1"), Bytes.toBytes("cf"), null, now-1000,
|
||||
KeyValue.Type.DeleteFamily),
|
||||
KeyValueTestUtil.create("R1", "cf", "a", now-10, KeyValue.Type.Put,
|
||||
"dont-care"),
|
||||
new KeyValue(Bytes.toBytes("R1"), Bytes.toBytes("cf"), null, timestampType
|
||||
.fromEpochTimeMillisToTimestamp(now-1000), KeyValue.Type.DeleteFamily),
|
||||
KeyValueTestUtil.create("R1", "cf", "a", timestampType.fromEpochTimeMillisToTimestamp
|
||||
(now-10), KeyValue.Type.Put, "dont-care"),
|
||||
};
|
||||
List<KeyValueScanner> scanners = scanFixture(kvs);
|
||||
Scan scan = new Scan();
|
||||
|
@ -906,9 +935,10 @@ public class TestStoreScanner {
|
|||
ScanInfo scanInfo = new ScanInfo(CONF, CF, 0, 1, 500, KeepDeletedCells.FALSE,
|
||||
HConstants.DEFAULT_BLOCKSIZE, 0, CellComparator.COMPARATOR);
|
||||
ScanType scanType = ScanType.USER_SCAN;
|
||||
Store store = mock(HStore.class);
|
||||
when(store.getClock()).thenReturn(clock);
|
||||
try (StoreScanner scanner =
|
||||
new StoreScanner(scan, scanInfo, scanType, null, scanners)) {
|
||||
|
||||
new StoreScanner(store, scan, scanInfo, scanType, null, scanners)) {
|
||||
List<Cell> results = new ArrayList<>();
|
||||
Assert.assertEquals(true, scanner.next(results));
|
||||
Assert.assertEquals(1, results.size());
|
||||
|
@ -921,8 +951,15 @@ public class TestStoreScanner {
|
|||
|
||||
@Test
|
||||
public void testDeleteMarkerLongevity() throws Exception {
|
||||
testDeleteMarkerLongevity(new Clock.HLC());
|
||||
testDeleteMarkerLongevity(new Clock.SystemMonotonic());
|
||||
testDeleteMarkerLongevity(new Clock.System());
|
||||
}
|
||||
|
||||
public void testDeleteMarkerLongevity(Clock clock) throws Exception {
|
||||
try {
|
||||
final long now = System.currentTimeMillis();
|
||||
TimestampType timestampType = clock.getTimestampType();
|
||||
EnvironmentEdgeManagerTestHelper.injectEdge(new EnvironmentEdge() {
|
||||
public long currentTime() {
|
||||
return now;
|
||||
|
@ -930,37 +967,40 @@ public class TestStoreScanner {
|
|||
});
|
||||
KeyValue[] kvs = new KeyValue[]{
|
||||
/*0*/ new KeyValue(Bytes.toBytes("R1"), Bytes.toBytes("cf"), null,
|
||||
now - 100, KeyValue.Type.DeleteFamily), // live
|
||||
timestampType.fromEpochTimeMillisToTimestamp(now - 100), KeyValue.Type.DeleteFamily),
|
||||
// live
|
||||
/*1*/ new KeyValue(Bytes.toBytes("R1"), Bytes.toBytes("cf"), null,
|
||||
now - 1000, KeyValue.Type.DeleteFamily), // expired
|
||||
/*2*/ KeyValueTestUtil.create("R1", "cf", "a", now - 50,
|
||||
KeyValue.Type.Put, "v3"), // live
|
||||
/*3*/ KeyValueTestUtil.create("R1", "cf", "a", now - 55,
|
||||
KeyValue.Type.Delete, "dontcare"), // live
|
||||
/*4*/ KeyValueTestUtil.create("R1", "cf", "a", now - 55,
|
||||
KeyValue.Type.Put, "deleted-version v2"), // deleted
|
||||
/*5*/ KeyValueTestUtil.create("R1", "cf", "a", now - 60,
|
||||
KeyValue.Type.Put, "v1"), // live
|
||||
/*6*/ KeyValueTestUtil.create("R1", "cf", "a", now - 65,
|
||||
KeyValue.Type.Put, "v0"), // max-version reached
|
||||
timestampType.fromEpochTimeMillisToTimestamp(now - 1000), KeyValue.Type.DeleteFamily),
|
||||
// expired
|
||||
/*2*/ KeyValueTestUtil.create("R1", "cf", "a", timestampType
|
||||
.fromEpochTimeMillisToTimestamp(now - 50), KeyValue.Type.Put, "v3"), // live
|
||||
/*3*/ KeyValueTestUtil.create("R1", "cf", "a", timestampType
|
||||
.fromEpochTimeMillisToTimestamp(now - 55), KeyValue.Type.Delete, "dontcare"), // live
|
||||
/*4*/ KeyValueTestUtil.create("R1", "cf", "a", timestampType
|
||||
.fromEpochTimeMillisToTimestamp(now - 55), KeyValue.Type.Put, "deleted-version v2"), // deleted
|
||||
/*5*/ KeyValueTestUtil.create("R1", "cf", "a", timestampType
|
||||
.fromEpochTimeMillisToTimestamp(now - 60), KeyValue.Type.Put, "v1"), // live
|
||||
/*6*/ KeyValueTestUtil.create("R1", "cf", "a", timestampType
|
||||
.fromEpochTimeMillisToTimestamp(now - 65), KeyValue.Type.Put, "v0"), // max-version reached
|
||||
/*7*/ KeyValueTestUtil.create("R1", "cf", "a",
|
||||
now - 100, KeyValue.Type.DeleteColumn, "dont-care"), // max-version
|
||||
/*8*/ KeyValueTestUtil.create("R1", "cf", "b", now - 600,
|
||||
KeyValue.Type.DeleteColumn, "dont-care"), //expired
|
||||
/*9*/ KeyValueTestUtil.create("R1", "cf", "b", now - 70,
|
||||
KeyValue.Type.Put, "v2"), //live
|
||||
/*10*/ KeyValueTestUtil.create("R1", "cf", "b", now - 750,
|
||||
KeyValue.Type.Put, "v1"), //expired
|
||||
/*11*/ KeyValueTestUtil.create("R1", "cf", "c", now - 500,
|
||||
KeyValue.Type.Delete, "dontcare"), //expired
|
||||
/*12*/ KeyValueTestUtil.create("R1", "cf", "c", now - 600,
|
||||
KeyValue.Type.Put, "v1"), //expired
|
||||
/*13*/ KeyValueTestUtil.create("R1", "cf", "c", now - 1000,
|
||||
KeyValue.Type.Delete, "dontcare"), //expired
|
||||
/*14*/ KeyValueTestUtil.create("R1", "cf", "d", now - 60,
|
||||
KeyValue.Type.Put, "expired put"), //live
|
||||
/*15*/ KeyValueTestUtil.create("R1", "cf", "d", now - 100,
|
||||
KeyValue.Type.Delete, "not-expired delete"), //live
|
||||
timestampType.fromEpochTimeMillisToTimestamp(now - 100), KeyValue.Type.DeleteColumn,
|
||||
"dont-care"), // max-version
|
||||
/*8*/ KeyValueTestUtil.create("R1", "cf", "b", timestampType
|
||||
.fromEpochTimeMillisToTimestamp(now - 600), KeyValue.Type.DeleteColumn, "dont-care"), //expired
|
||||
/*9*/ KeyValueTestUtil.create("R1", "cf", "b", timestampType
|
||||
.fromEpochTimeMillisToTimestamp(now - 70), KeyValue.Type.Put, "v2"), //live
|
||||
/*10*/ KeyValueTestUtil.create("R1", "cf", "b", timestampType
|
||||
.fromEpochTimeMillisToTimestamp(now - 750), KeyValue.Type.Put, "v1"), //expired
|
||||
/*11*/ KeyValueTestUtil.create("R1", "cf", "c", timestampType
|
||||
.fromEpochTimeMillisToTimestamp(now - 500), KeyValue.Type.Delete, "dontcare"), //expired
|
||||
/*12*/ KeyValueTestUtil.create("R1", "cf", "c", timestampType
|
||||
.fromEpochTimeMillisToTimestamp(now - 600), KeyValue.Type.Put, "v1"), //expired
|
||||
/*13*/ KeyValueTestUtil.create("R1", "cf", "c", timestampType
|
||||
.fromEpochTimeMillisToTimestamp(now - 1000), KeyValue.Type.Delete, "dontcare"), //expired
|
||||
/*14*/ KeyValueTestUtil.create("R1", "cf", "d", timestampType
|
||||
.fromEpochTimeMillisToTimestamp(now - 60), KeyValue.Type.Put, "expired put"), //live
|
||||
/*15*/ KeyValueTestUtil.create("R1", "cf", "d", timestampType
|
||||
.fromEpochTimeMillisToTimestamp(now - 100), KeyValue.Type.Delete, "not-expired delete"), //live
|
||||
};
|
||||
List<KeyValueScanner> scanners = scanFixture(kvs);
|
||||
Scan scan = new Scan();
|
||||
|
@ -972,10 +1012,10 @@ public class TestStoreScanner {
|
|||
HConstants.DEFAULT_BLOCKSIZE /* block size */,
|
||||
200, /* timeToPurgeDeletes */
|
||||
CellComparator.COMPARATOR);
|
||||
try (StoreScanner scanner =
|
||||
new StoreScanner(scan, scanInfo,
|
||||
ScanType.COMPACT_DROP_DELETES, null, scanners,
|
||||
HConstants.OLDEST_TIMESTAMP)) {
|
||||
Store store = mock(HStore.class);
|
||||
when(store.getClock()).thenReturn(clock);
|
||||
try (StoreScanner scanner = new StoreScanner(store, scan, scanInfo, ScanType
|
||||
.COMPACT_DROP_DELETES, null, scanners, HConstants.OLDEST_TIMESTAMP)) {
|
||||
List<Cell> results = new ArrayList<>();
|
||||
results = new ArrayList<>();
|
||||
Assert.assertEquals(true, scanner.next(results));
|
||||
|
|
|
@ -19,6 +19,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;
|
||||
|
@ -32,6 +33,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.ChoreService;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
|
@ -203,6 +206,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);
|
||||
|
|
|
@ -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.*;
|
||||
import org.apache.hadoop.hbase.Clock;
|
||||
import org.apache.hadoop.hbase.ClockType;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
|
@ -657,13 +659,23 @@ public abstract class AbstractTestWALReplay {
|
|||
* @throws IOException
|
||||
*/
|
||||
@Test
|
||||
public void testReplayEditsAfterAbortingFlush() throws IOException {
|
||||
public void testReplayEditsAfterAbortingFlush() throws Exception {
|
||||
testReplayEditsAfterAbortingFlush(new Clock.System());
|
||||
setUp();
|
||||
testReplayEditsAfterAbortingFlush(new Clock.SystemMonotonic());
|
||||
tearDown();
|
||||
setUp();
|
||||
testReplayEditsAfterAbortingFlush(new Clock.HLC());
|
||||
}
|
||||
|
||||
public void testReplayEditsAfterAbortingFlush(Clock clock) throws IOException {
|
||||
final TableName tableName =
|
||||
TableName.valueOf("testReplayEditsAfterAbortingFlush");
|
||||
final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
|
||||
final Path basedir = FSUtils.getTableDir(this.hbaseRootDir, tableName);
|
||||
deleteDir(basedir);
|
||||
final HTableDescriptor htd = createBasic3FamilyHTD(tableName);
|
||||
htd.setClockType(clock.clockType);
|
||||
HRegion region3 = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd);
|
||||
HBaseTestingUtility.closeRegionAndWAL(region3);
|
||||
// Write countPerFamily edits into the three families. Do a flush on one
|
||||
|
@ -672,6 +684,7 @@ public abstract class AbstractTestWALReplay {
|
|||
WAL wal = createWAL(this.conf, hbaseRootDir, logName);
|
||||
RegionServerServices rsServices = Mockito.mock(RegionServerServices.class);
|
||||
Mockito.doReturn(false).when(rsServices).isAborted();
|
||||
when(rsServices.getRegionServerClock(clock.clockType)).thenReturn(clock);
|
||||
when(rsServices.getServerName()).thenReturn(ServerName.valueOf("foo", 10, 10));
|
||||
Configuration customConf = new Configuration(this.conf);
|
||||
customConf.set(DefaultStoreEngine.DEFAULT_STORE_FLUSHER_CLASS_KEY,
|
||||
|
|
|
@ -18,22 +18,28 @@
|
|||
package org.apache.hadoop.hbase.security.access;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.AuthUtil;
|
||||
import org.apache.hadoop.hbase.ClockType;
|
||||
import org.apache.hadoop.hbase.Coprocessor;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.TimestampType;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
|
@ -49,9 +55,12 @@ import org.apache.hadoop.hbase.security.access.Permission.Action;
|
|||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SecurityTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.DefaultEnvironmentEdge;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
|
||||
import org.apache.hadoop.hbase.util.TestTableName;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.hadoop.util.Time;
|
||||
import org.apache.log4j.Level;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.junit.After;
|
||||
|
@ -61,9 +70,25 @@ import org.junit.BeforeClass;
|
|||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
import org.junit.runners.Parameterized.Parameters;
|
||||
|
||||
@Category({SecurityTests.class, MediumTests.class})
|
||||
@RunWith(Parameterized.class)
|
||||
public class TestCellACLWithMultipleVersions extends SecureTestUtil {
|
||||
|
||||
@Parameters()
|
||||
public static Iterable<Object> data() {
|
||||
return Arrays.asList(new Object[] {ClockType
|
||||
.SYSTEM, ClockType.SYSTEM_MONOTONIC, ClockType.HLC});
|
||||
}
|
||||
|
||||
public TestCellACLWithMultipleVersions(ClockType clockType) {
|
||||
this.clockType = clockType;
|
||||
this.timestampType = clockType.timestampType();
|
||||
}
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(TestCellACLWithMultipleVersions.class);
|
||||
|
||||
static {
|
||||
|
@ -72,6 +97,9 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
|
|||
Logger.getLogger(TableAuthManager.class).setLevel(Level.TRACE);
|
||||
}
|
||||
|
||||
private final ClockType clockType;
|
||||
private final TimestampType timestampType;
|
||||
|
||||
@Rule
|
||||
public TestTableName TEST_TABLE = new TestTableName();
|
||||
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
|
@ -93,6 +121,8 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
|
|||
private static User USER_OTHER2;
|
||||
|
||||
private static String[] usersAndGroups;
|
||||
private static ManualEnvironmentEdge mee = new ManualEnvironmentEdge();
|
||||
private static long now;
|
||||
|
||||
@BeforeClass
|
||||
public static void setupBeforeClass() throws Exception {
|
||||
|
@ -136,7 +166,9 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
|
|||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
EnvironmentEdgeManager.injectEdge(new DefaultEnvironmentEdge());
|
||||
HTableDescriptor htd = new HTableDescriptor(TEST_TABLE.getTableName());
|
||||
htd.setClockType(clockType);
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY1);
|
||||
hcd.setMaxVersions(4);
|
||||
htd.setOwner(USER_OWNER);
|
||||
|
@ -385,6 +417,12 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
|
|||
public void testDeleteWithFutureTimestamp() throws Exception {
|
||||
// Store two values, one in the future
|
||||
|
||||
// Setting of future timestamps is not allowed with System Monotonic and HLC.
|
||||
// So need not run this test against these two clocks.
|
||||
if (clockType == ClockType.HLC || clockType == ClockType.SYSTEM_MONOTONIC) {
|
||||
assertTrue(true);
|
||||
return;
|
||||
}
|
||||
verifyAllowed(new AccessTestAction() {
|
||||
@Override
|
||||
public Object run() throws Exception {
|
||||
|
@ -484,8 +522,11 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
|
|||
try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
|
||||
// This version (TS = 123) with rw ACL for USER_OTHER and USER_OTHER2
|
||||
Put p = new Put(TEST_ROW);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q1, 123L, ZERO);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q2, 123L, ZERO);
|
||||
EnvironmentEdgeManager.injectEdge(mee);
|
||||
now += 10000;
|
||||
mee.setValue(now);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //123
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO); //123
|
||||
p.setACL(prepareCellPermissions(
|
||||
new String[] { USER_OTHER.getShortName(), AuthUtil.toGroupEntry(GROUP),
|
||||
USER_OTHER2.getShortName() }, Permission.Action.READ, Permission.Action.WRITE));
|
||||
|
@ -493,8 +534,9 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
|
|||
|
||||
// This version (TS = 125) with rw ACL for USER_OTHER
|
||||
p = new Put(TEST_ROW);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q1, 125L, ONE);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q2, 125L, ONE);
|
||||
mee.setValue(now+2);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q1, HConstants.LATEST_TIMESTAMP, ONE); //125
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q2, HConstants.LATEST_TIMESTAMP, ONE); //125
|
||||
p.setACL(prepareCellPermissions(
|
||||
new String[] { USER_OTHER.getShortName(), AuthUtil.toGroupEntry(GROUP) },
|
||||
Action.READ, Action.WRITE));
|
||||
|
@ -502,8 +544,9 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
|
|||
|
||||
// This version (TS = 127) with rw ACL for USER_OTHER
|
||||
p = new Put(TEST_ROW);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q1, 127L, TWO);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q2, 127L, TWO);
|
||||
mee.setValue(now+4);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q1, HConstants.LATEST_TIMESTAMP, TWO); //127
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q2, HConstants.LATEST_TIMESTAMP, TWO); //127
|
||||
p.setACL(prepareCellPermissions(
|
||||
new String[] { USER_OTHER.getShortName(), AuthUtil.toGroupEntry(GROUP) },
|
||||
Action.READ, Action.WRITE));
|
||||
|
@ -521,7 +564,8 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
|
|||
public Object run() throws Exception {
|
||||
try (Connection connection = ConnectionFactory.createConnection(conf)) {
|
||||
try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
|
||||
Delete d = new Delete(TEST_ROW, 124L);
|
||||
Delete d = new Delete(TEST_ROW, timestampType.toTimestamp(TimeUnit.MILLISECONDS,
|
||||
now+1, timestampType.getMaxLogicalTime())); //124
|
||||
d.addColumns(TEST_FAMILY1, TEST_Q1);
|
||||
t.delete(d);
|
||||
}
|
||||
|
@ -537,7 +581,8 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
|
|||
try (Connection connection = ConnectionFactory.createConnection(conf)) {
|
||||
try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
|
||||
Delete d = new Delete(TEST_ROW);
|
||||
d.addColumns(TEST_FAMILY1, TEST_Q2, 124L);
|
||||
d.addColumns(TEST_FAMILY1, TEST_Q2, timestampType.toTimestamp(TimeUnit.MILLISECONDS,
|
||||
now+1, timestampType.getMaxLogicalTime())); // 124
|
||||
t.delete(d);
|
||||
}
|
||||
}
|
||||
|
@ -569,37 +614,45 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
|
|||
prepareCellPermissions(
|
||||
new String[] { user2.getShortName(), AuthUtil.toGroupEntry(GROUP),
|
||||
USER_OWNER.getShortName() }, Action.READ, Action.WRITE);
|
||||
now = EnvironmentEdgeManager.currentTime();
|
||||
EnvironmentEdgeManager.injectEdge(mee);
|
||||
now += 10000;
|
||||
mee.setValue(now);
|
||||
Put p = new Put(TEST_ROW1);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q1, (long) 123, ZERO);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //123
|
||||
p.setACL(permsU1andOwner);
|
||||
t.put(p);
|
||||
p = new Put(TEST_ROW1);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q2, (long) 123, ZERO);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO); //123
|
||||
p.setACL(permsU2andGUandOwner);
|
||||
t.put(p);
|
||||
p = new Put(TEST_ROW1);
|
||||
p.addColumn(TEST_FAMILY2, TEST_Q1, (long) 123, ZERO);
|
||||
p.addColumn(TEST_FAMILY2, TEST_Q2, (long) 123, ZERO);
|
||||
p.addColumn(TEST_FAMILY2, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //123
|
||||
p.addColumn(TEST_FAMILY2, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO); //123
|
||||
p.setACL(permsU2andGUandOwner);
|
||||
t.put(p);
|
||||
|
||||
mee.setValue(now+2);
|
||||
p = new Put(TEST_ROW1);
|
||||
p.addColumn(TEST_FAMILY2, TEST_Q1, (long) 125, ZERO);
|
||||
p.addColumn(TEST_FAMILY2, TEST_Q2, (long) 125, ZERO);
|
||||
p.addColumn(TEST_FAMILY2, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //125
|
||||
p.addColumn(TEST_FAMILY2, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO); //125
|
||||
p.setACL(permsU1andOwner);
|
||||
t.put(p);
|
||||
|
||||
mee.setValue(now+4);
|
||||
p = new Put(TEST_ROW1);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q1, (long) 127, ZERO);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //127
|
||||
p.setACL(permsU2andGUandOwner);
|
||||
t.put(p);
|
||||
p = new Put(TEST_ROW1);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q2, (long) 127, ZERO);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO); //127
|
||||
p.setACL(permsU1andOwner);
|
||||
t.put(p);
|
||||
|
||||
mee.setValue(now+6);
|
||||
p = new Put(TEST_ROW1);
|
||||
p.addColumn(TEST_FAMILY2, TEST_Q1, (long) 129, ZERO);
|
||||
p.addColumn(TEST_FAMILY2, TEST_Q2, (long) 129, ZERO);
|
||||
p.addColumn(TEST_FAMILY2, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //129
|
||||
p.addColumn(TEST_FAMILY2, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO); //129
|
||||
p.setACL(permsU1andOwner);
|
||||
t.put(p);
|
||||
}
|
||||
|
@ -616,9 +669,11 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
|
|||
try (Connection connection = ConnectionFactory.createConnection(conf)) {
|
||||
try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
|
||||
Delete d = new Delete(TEST_ROW1);
|
||||
d.addColumn(TEST_FAMILY1, TEST_Q1, 123);
|
||||
d.addColumn(TEST_FAMILY1, TEST_Q1, timestampType.toTimestamp(TimeUnit.MILLISECONDS,
|
||||
now, timestampType.getMaxLogicalTime())); //123
|
||||
d.addColumn(TEST_FAMILY1, TEST_Q2);
|
||||
d.addFamilyVersion(TEST_FAMILY2, 125);
|
||||
d.addFamilyVersion(TEST_FAMILY2, timestampType.toTimestamp(TimeUnit.MILLISECONDS,
|
||||
now+2, timestampType.getMaxLogicalTime())); //125
|
||||
t.delete(d);
|
||||
}
|
||||
}
|
||||
|
@ -637,10 +692,12 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
|
|||
public Void run() throws Exception {
|
||||
try (Connection connection = ConnectionFactory.createConnection(conf)) {
|
||||
try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
|
||||
Delete d = new Delete(row, 127);
|
||||
Delete d = new Delete(row, timestampType.toTimestamp(TimeUnit.MILLISECONDS,
|
||||
now+4, timestampType.getMaxLogicalTime())); //127
|
||||
d.addColumns(TEST_FAMILY1, q1);
|
||||
d.addColumns(TEST_FAMILY1, q2);
|
||||
d.addFamily(TEST_FAMILY2, 129);
|
||||
d.addFamily(TEST_FAMILY2, timestampType.toTimestamp(TimeUnit.MILLISECONDS,
|
||||
now+6, timestampType.getMaxLogicalTime())); //129
|
||||
t.delete(d);
|
||||
fail(user.getShortName() + " can not do the delete");
|
||||
} catch (Exception e) {
|
||||
|
@ -675,21 +732,27 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
|
|||
prepareCellPermissions(
|
||||
new String[] { user2.getShortName(), AuthUtil.toGroupEntry(GROUP),
|
||||
USER_OWNER.getShortName() }, Action.READ, Action.WRITE);
|
||||
now = EnvironmentEdgeManager.currentTime();
|
||||
EnvironmentEdgeManager.injectEdge(mee);
|
||||
now += 10000;
|
||||
mee.setValue(now);
|
||||
Put p = new Put(TEST_ROW1);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q1, (long) 123, ZERO);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //123
|
||||
p.setACL(permsU1andOwner);
|
||||
t.put(p);
|
||||
p = new Put(TEST_ROW1);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q2, (long) 123, ZERO);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO); //123
|
||||
p.setACL(permsU2andGUandOwner);
|
||||
t.put(p);
|
||||
|
||||
System.out.println(now+4);
|
||||
mee.setValue(now+4);
|
||||
p = new Put(TEST_ROW1);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q1, (long) 127, ZERO);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //127
|
||||
p.setACL(permsU2andGUandOwner);
|
||||
t.put(p);
|
||||
p = new Put(TEST_ROW1);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q2, (long) 127, ZERO);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO); //127
|
||||
p.setACL(permsU1andOwner);
|
||||
t.put(p);
|
||||
}
|
||||
|
@ -705,7 +768,8 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
|
|||
try (Connection connection = ConnectionFactory.createConnection(conf)) {
|
||||
try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
|
||||
Increment inc = new Increment(TEST_ROW1);
|
||||
inc.setTimeRange(0, 123);
|
||||
inc.setTimeRange(0, timestampType.toTimestamp(TimeUnit.MILLISECONDS, now,
|
||||
timestampType.getMaxLogicalTime()));
|
||||
inc.addColumn(TEST_FAMILY1, TEST_Q1, 2L);
|
||||
t.increment(inc);
|
||||
t.incrementColumnValue(TEST_ROW1, TEST_FAMILY1, TEST_Q2, 1L);
|
||||
|
@ -727,7 +791,9 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
|
|||
try (Connection connection = ConnectionFactory.createConnection(conf)) {
|
||||
try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
|
||||
Increment inc = new Increment(row);
|
||||
inc.setTimeRange(0, 127);
|
||||
System.out.println(now+4);
|
||||
inc.setTimeRange(0, timestampType.toTimestamp(TimeUnit.MILLISECONDS, now+4,
|
||||
timestampType.getMaxLogicalTime()));
|
||||
inc.addColumn(TEST_FAMILY1, q1, 2L);
|
||||
t.increment(inc);
|
||||
fail(user.getShortName() + " cannot do the increment.");
|
||||
|
@ -742,6 +808,14 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
|
|||
|
||||
@Test
|
||||
public void testCellPermissionsForPutWithMultipleVersions() throws Exception {
|
||||
|
||||
// This test relies is dependent on non monotonic timestamp updates which doesn't happen with
|
||||
// HLC and System Monotonic Clocks.
|
||||
if (clockType == ClockType.HLC || clockType == ClockType.SYSTEM_MONOTONIC) {
|
||||
assertTrue(true);
|
||||
return;
|
||||
}
|
||||
|
||||
final byte[] TEST_ROW1 = Bytes.toBytes("r1");
|
||||
final byte[] TEST_Q1 = Bytes.toBytes("q1");
|
||||
final byte[] TEST_Q2 = Bytes.toBytes("q2");
|
||||
|
@ -857,38 +931,45 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
|
|||
Action.WRITE);
|
||||
Map<String, Permission> permsU1andU2andGUandOwner =
|
||||
prepareCellPermissions(new String[] { user1.getShortName(), user2.getShortName(),
|
||||
AuthUtil.toGroupEntry(GROUP), USER_OWNER.getShortName() }, Action.READ,
|
||||
AuthUtil.toGroupEntry(GROUP), USER_OWNER.getShortName() }, Action.READ,
|
||||
Action.WRITE);
|
||||
Map<String, Permission> permsU1_U2andGU =
|
||||
prepareCellPermissions(new String[] { user1.getShortName(), user2.getShortName(),
|
||||
AuthUtil.toGroupEntry(GROUP) }, Action.READ, Action.WRITE);
|
||||
|
||||
now = EnvironmentEdgeManager.currentTime();
|
||||
EnvironmentEdgeManager.injectEdge(mee);
|
||||
now += 5000;
|
||||
|
||||
mee.setValue(now);
|
||||
Put p = new Put(TEST_ROW1);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q1, (long) 120, ZERO);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q2, (long) 120, ZERO);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q3, (long) 120, ZERO);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //120
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q3, HConstants.LATEST_TIMESTAMP, ZERO);
|
||||
p.setACL(permsU1andU2andGUandOwner);
|
||||
t.put(p);
|
||||
|
||||
mee.setValue(now+3);
|
||||
p = new Put(TEST_ROW1);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q1, (long) 123, ZERO);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q2, (long) 123, ZERO);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q3, (long) 123, ZERO);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //123
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q3, HConstants.LATEST_TIMESTAMP, ZERO);
|
||||
p.setACL(permsU1andOwner);
|
||||
t.put(p);
|
||||
|
||||
mee.setValue(now+7);
|
||||
p = new Put(TEST_ROW1);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q1, (long) 127, ZERO);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //127
|
||||
p.setACL(permsU1_U2andGU);
|
||||
t.put(p);
|
||||
|
||||
p = new Put(TEST_ROW1);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q2, (long) 127, ZERO);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO); //127
|
||||
p.setACL(user2.getShortName(), new Permission(Permission.Action.READ));
|
||||
t.put(p);
|
||||
|
||||
p = new Put(TEST_ROW1);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q3, 127, ZERO);
|
||||
p.addColumn(TEST_FAMILY1, TEST_Q3, HConstants.LATEST_TIMESTAMP, ZERO); //127
|
||||
p.setACL(AuthUtil.toGroupEntry(GROUP), new Permission(Permission.Action.READ));
|
||||
t.put(p);
|
||||
}
|
||||
|
@ -905,7 +986,8 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
|
|||
try (Connection connection = ConnectionFactory.createConnection(conf)) {
|
||||
try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
|
||||
Delete d = new Delete(TEST_ROW1);
|
||||
d.addColumns(TEST_FAMILY1, TEST_Q1, 120);
|
||||
d.addColumns(TEST_FAMILY1, TEST_Q1, timestampType.toTimestamp(TimeUnit.MILLISECONDS,
|
||||
now, timestampType.getMaxLogicalTime())); //120
|
||||
t.checkAndDelete(TEST_ROW1, TEST_FAMILY1, TEST_Q1, ZERO, d);
|
||||
}
|
||||
}
|
||||
|
@ -941,7 +1023,8 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
|
|||
try (Connection connection = ConnectionFactory.createConnection(conf)) {
|
||||
try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
|
||||
Delete d = new Delete(row);
|
||||
d.addColumn(TEST_FAMILY1, q1, 120);
|
||||
d.addColumn(TEST_FAMILY1, q1, timestampType.toTimestamp(TimeUnit.MILLISECONDS,
|
||||
now, timestampType.getMaxLogicalTime()));
|
||||
t.checkAndDelete(row, TEST_FAMILY1, q1, value, d);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -31,6 +31,7 @@ import java.util.NavigableSet;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.ClockType;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
|
@ -147,13 +148,20 @@ public class TestCoprocessorScanPolicy {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testTTL() throws Exception {
|
||||
public void testTTl() throws Exception {
|
||||
testTTL(ClockType.HLC);
|
||||
testTTL(ClockType.SYSTEM_MONOTONIC);
|
||||
testTTL(ClockType.SYSTEM);
|
||||
}
|
||||
|
||||
public void testTTL(ClockType clockType) throws Exception {
|
||||
TableName tableName =
|
||||
TableName.valueOf("testTTL");
|
||||
if (TEST_UTIL.getAdmin().tableExists(tableName)) {
|
||||
TEST_UTIL.deleteTable(tableName);
|
||||
}
|
||||
HTableDescriptor desc = new HTableDescriptor(tableName);
|
||||
desc.setClockType(clockType);
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(F)
|
||||
.setMaxVersions(10)
|
||||
.setTimeToLive(1);
|
||||
|
@ -162,10 +170,10 @@ public class TestCoprocessorScanPolicy {
|
|||
Table t = TEST_UTIL.getConnection().getTable(tableName);
|
||||
long now = EnvironmentEdgeManager.currentTime();
|
||||
ManualEnvironmentEdge me = new ManualEnvironmentEdge();
|
||||
me.setValue(now);
|
||||
me.setValue(now-2000);
|
||||
EnvironmentEdgeManagerTestHelper.injectEdge(me);
|
||||
// 2s in the past
|
||||
long ts = now - 2000;
|
||||
long ts = Long.MAX_VALUE;
|
||||
// Set the TTL override to 3s
|
||||
Put p = new Put(R);
|
||||
p.setAttribute("ttl", new byte[]{});
|
||||
|
@ -175,12 +183,15 @@ public class TestCoprocessorScanPolicy {
|
|||
p = new Put(R);
|
||||
p.addColumn(F, Q, ts, Q);
|
||||
t.put(p);
|
||||
|
||||
me.setValue(now-1999);
|
||||
p = new Put(R);
|
||||
p.addColumn(F, Q, ts + 1, Q);
|
||||
p.addColumn(F, Q, ts , Q);
|
||||
t.put(p);
|
||||
|
||||
// these two should be expired but for the override
|
||||
// (their ts was 2s in the past)
|
||||
me.setValue(now);
|
||||
Get g = new Get(R);
|
||||
g.setMaxVersions(10);
|
||||
Result r = t.get(g);
|
||||
|
|
|
@ -46,7 +46,9 @@ public class TestTableName extends TestWatcher {
|
|||
*/
|
||||
@Override
|
||||
protected void starting(Description description) {
|
||||
tableName = TableName.valueOf(description.getMethodName());
|
||||
tableName = TableName.valueOf(description.getMethodName()
|
||||
.replace("[", ".")
|
||||
.replace("]", ""));
|
||||
}
|
||||
|
||||
public TableName getTableName() {
|
||||
|
|
Loading…
Reference in New Issue