Skip to content

Commit 71583a2

Browse files
committed
HBASE-25913 Introduce EnvironmentEdge.Clock and Clock.currentTimeAdvancing
- Use a per region Clock in HRegion to ensure the time advances - Update unit tests as needed
1 parent ee72ef9 commit 71583a2

File tree

16 files changed

+257
-175
lines changed

16 files changed

+257
-175
lines changed

hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java

Lines changed: 11 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -28,9 +28,8 @@
2828
import org.apache.hadoop.hbase.TableName;
2929
import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
3030
import org.apache.hadoop.hbase.testclassification.LargeTests;
31-
import org.apache.hadoop.hbase.util.EnvironmentEdge;
31+
import org.apache.hadoop.hbase.util.BaseEnvironmentEdge;
3232
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
33-
import org.apache.hadoop.hbase.util.HashedBytes;
3433
import org.apache.hadoop.util.ToolRunner;
3534
import org.junit.Assert;
3635
import org.junit.ClassRule;
@@ -113,21 +112,7 @@ public void testBackupDeleteCommand() throws Exception {
113112
public void testBackupPurgeOldBackupsCommand() throws Exception {
114113
LOG.info("test backup delete (purge old backups) on a single table with data: command-line");
115114
List<TableName> tableList = Lists.newArrayList(table1);
116-
EnvironmentEdgeManager.injectEdge(new EnvironmentEdge() {
117-
// time - 2 days
118-
@Override
119-
public long currentTime() {
120-
return System.currentTimeMillis() - 2 * 24 * 3600 * 1000 ;
121-
}
122-
@Override
123-
public Clock getClock(HashedBytes name) {
124-
return null;
125-
}
126-
@Override
127-
public boolean removeClock(Clock clock) {
128-
return false;
129-
}
130-
});
115+
EnvironmentEdgeManager.injectEdge(new TwoDaysBackEnvironmentEdge());
131116
String backupId = fullTableBackup(tableList);
132117
assertTrue(checkSucceeded(backupId));
133118

@@ -169,4 +154,13 @@ public boolean removeClock(Clock clock) {
169154
LOG.info(baos.toString());
170155
assertTrue(output.indexOf("Deleted 1 backups") >= 0);
171156
}
157+
158+
static class TwoDaysBackEnvironmentEdge extends BaseEnvironmentEdge {
159+
// time - 2 days
160+
@Override
161+
public long currentTime() {
162+
return System.currentTimeMillis() - 2 * 24 * 3600 * 1000 ;
163+
}
164+
}
165+
172166
}

hbase-common/src/main/java/org/apache/hadoop/hbase/util/BaseEnvironmentEdge.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -60,7 +60,8 @@ public long currentTime() {
6060
}
6161
@Override
6262
public long currentTimeAdvancing() {
63-
throw new UnsupportedOperationException("Default clock does not implement currentTimeAdvancing()");
63+
throw new UnsupportedOperationException(
64+
"Default clock does not implement currentTimeAdvancing()");
6465
}
6566
@Override
6667
public void get() {

hbase-common/src/main/java/org/apache/hadoop/hbase/util/EnvironmentEdge.java

Lines changed: 3 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -41,13 +41,13 @@ public interface EnvironmentEdge {
4141
/**
4242
* Get the clock associated with the given identifier.
4343
* @param name clock identifier
44-
* @returns the clock instance for the given identifier
44+
* @return the clock instance for the given identifier
4545
*/
4646
Clock getClock(HashedBytes name);
4747

4848
/**
49-
* Remove the clock with the given identifier.
50-
* @param clockId clock identifier
49+
* Release the reference to and possible remove this clock.
50+
* @param clock the clock
5151
* @return true if the clock was removed, false if it did not exist
5252
*/
5353
boolean removeClock(Clock clock);
@@ -64,7 +64,6 @@ public interface Clock {
6464

6565
/**
6666
* Returns the current time using a named clock.
67-
* @param clockId clock identifier
6867
* @return The current time, according to the given named clock.
6968
*/
7069
long currentTime();
@@ -74,7 +73,6 @@ public interface Clock {
7473
* at least one tick before returning.
7574
* <p>
7675
* This method may block the current thread's execution or cause it to yield.
77-
* @param clockId clock identifier
7876
* @return The current time, according to the given named clock.
7977
* @throws InterruptedException if interrupted while waiting for the clock to advance
8078
*/

hbase-common/src/main/java/org/apache/hadoop/hbase/util/HashedBytes.java

Lines changed: 10 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -36,7 +36,12 @@ public class HashedBytes {
3636

3737
public HashedBytes(byte[] bytes) {
3838
this.bytes = bytes;
39-
hashCode = Bytes.hashCode(bytes);
39+
hashCode = Bytes.hashCode(this.bytes);
40+
}
41+
42+
public HashedBytes(byte[]... bytes) {
43+
this.bytes = Bytes.add(bytes);
44+
hashCode = Bytes.hashCode(this.bytes);
4045
}
4146

4247
public byte[] getBytes() {
@@ -50,10 +55,12 @@ public int hashCode() {
5055

5156
@Override
5257
public boolean equals(Object obj) {
53-
if (this == obj)
58+
if (this == obj) {
5459
return true;
55-
if (obj == null || getClass() != obj.getClass())
60+
}
61+
if (obj == null || getClass() != obj.getClass()) {
5662
return false;
63+
}
5764
HashedBytes other = (HashedBytes) obj;
5865
return (hashCode == other.hashCode) && Arrays.equals(bytes, other.bytes);
5966
}

hbase-common/src/main/java/org/apache/hadoop/hbase/util/IncrementingEnvironmentEdge.java

Lines changed: 44 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,40 @@
2626
@InterfaceAudience.Private
2727
public class IncrementingEnvironmentEdge extends BaseEnvironmentEdge {
2828

29+
class ManualIncrementingClock implements EnvironmentEdge.Clock {
30+
31+
private HashedBytes name;
32+
33+
public ManualIncrementingClock(HashedBytes name) {
34+
this.name = name;
35+
}
36+
37+
@Override
38+
public HashedBytes getName() {
39+
return name;
40+
}
41+
42+
@Override
43+
public long currentTime() {
44+
return System.currentTimeMillis() + timeIncrement;
45+
}
46+
47+
@Override
48+
public long currentTimeAdvancing() {
49+
return System.currentTimeMillis() + timeIncrement;
50+
}
51+
52+
@Override
53+
public void get() {
54+
}
55+
56+
@Override
57+
public boolean remove() {
58+
return true;
59+
}
60+
61+
}
62+
2963
private long timeIncrement;
3064

3165
/**
@@ -63,4 +97,14 @@ public synchronized long incrementTime(long amount) {
6397
return timeIncrement;
6498
}
6599

100+
@Override
101+
public Clock getClock(HashedBytes name) {
102+
return new ManualIncrementingClock(name);
103+
}
104+
105+
@Override
106+
public boolean removeClock(Clock clock) {
107+
return true;
108+
}
109+
66110
}

hbase-server/src/main/java/org/apache/hadoop/hbase/util/ManualEnvironmentEdge.java renamed to hbase-common/src/main/java/org/apache/hadoop/hbase/util/ManualEnvironmentEdge.java

Lines changed: 44 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,40 @@
2727
@InterfaceAudience.Private
2828
public class ManualEnvironmentEdge extends BaseEnvironmentEdge {
2929

30+
class ManualFixedClock implements EnvironmentEdge.Clock {
31+
32+
private HashedBytes name;
33+
34+
public ManualFixedClock(HashedBytes name) {
35+
this.name = name;
36+
}
37+
38+
@Override
39+
public HashedBytes getName() {
40+
return name;
41+
}
42+
43+
@Override
44+
public long currentTime() {
45+
return value;
46+
}
47+
48+
@Override
49+
public long currentTimeAdvancing() {
50+
return value;
51+
}
52+
53+
@Override
54+
public void get() {
55+
}
56+
57+
@Override
58+
public boolean remove() {
59+
return true;
60+
}
61+
62+
}
63+
3064
protected long value;
3165

3266
public ManualEnvironmentEdge() {
@@ -51,4 +85,14 @@ public long currentTime() {
5185
return this.value;
5286
}
5387

88+
@Override
89+
public Clock getClock(HashedBytes name) {
90+
return new ManualFixedClock(name);
91+
}
92+
93+
@Override
94+
public boolean removeClock(Clock clock) {
95+
return true;
96+
}
97+
5498
}

hbase-common/src/main/java/org/apache/hadoop/hbase/util/clock/BoundedIncrementSpinAdvancingClock.java

Lines changed: 5 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -50,12 +50,11 @@ public long currentTimeAdvancing() throws InterruptedException {
5050
// If we have advanced too far, now we have to wait for the system clock to
5151
// catch up.
5252
if (currentAdvance.incrementAndGet() >= MAX_ADVANCE) {
53-
long now;
5453
while (true) {
55-
now = System.currentTimeMillis();
54+
long now = System.currentTimeMillis();
5655
if (now > lastTime.get()) {
57-
final long updateTime = now;
58-
return lastTime.updateAndGet(x -> update(updateTime));
56+
lastTime.set(update(now));
57+
return now;
5958
}
6059
spin();
6160
}
@@ -76,15 +75,15 @@ protected long advance(long last) {
7675
// System clock hasn't moved forward. Increment our notion of current tick to keep
7776
// the time advancing.
7877
currentAdvance.incrementAndGet();
79-
return super.advance(last);
78+
return last + 1;
8079
}
8180

8281
// Broken out to inlinable method for subclassing and instrumentation.
8382
protected long update(long now) {
8483
// System clock has moved ahead of our notion of current tick. Move us forward to match.
8584
// We do that just by returning the current time, which was passed in to us as 'n'.
8685
currentAdvance.set(0);
87-
return super.update(now);
86+
return now;
8887
}
8988

9089
// Broken out to inlinable method for subclassing and instrumentation.

hbase-common/src/main/java/org/apache/hadoop/hbase/util/clock/BoundedIncrementYieldAdvancingClock.java

Lines changed: 6 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,7 @@
3232
@InterfaceAudience.Private
3333
public class BoundedIncrementYieldAdvancingClock extends IncrementAdvancingClock {
3434

35-
static final int MAX_ADVANCE = 1000; // one second in milliseconds
35+
static final int MAX_ADVANCE = 1000;
3636

3737
protected AtomicLong currentAdvance = new AtomicLong(0);
3838

@@ -50,12 +50,11 @@ public long currentTimeAdvancing() throws InterruptedException {
5050
// If we have advanced too far, now we have to wait for the system clock to
5151
// catch up.
5252
if (currentAdvance.incrementAndGet() >= MAX_ADVANCE) {
53-
long now;
5453
while (true) {
55-
now = System.currentTimeMillis();
54+
long now = System.currentTimeMillis();
5655
if (now > lastTime.get()) {
57-
final long updateTime = now;
58-
return lastTime.updateAndGet(x -> update(updateTime));
56+
lastTime.set(update(now));
57+
return now;
5958
}
6059
spin();
6160
}
@@ -76,15 +75,15 @@ protected long advance(long last) {
7675
// System clock hasn't moved forward. Increment our notion of current tick to keep
7776
// the time advancing.
7877
currentAdvance.incrementAndGet();
79-
return super.advance(last);
78+
return last + 1;
8079
}
8180

8281
// Broken out to inlinable method for subclassing and instrumentation.
8382
protected long update(long now) {
8483
// System clock has moved ahead of our notion of current tick. Move us forward to match.
8584
// We do that just by returning the current time, which was passed in to us as 'n'.
8685
currentAdvance.set(0);
87-
return super.update(now);
86+
return now;
8887
}
8988

9089
// Broken out to inlinable method for subclassing and instrumentation.

hbase-common/src/main/java/org/apache/hadoop/hbase/util/clock/IncrementAdvancingClock.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,8 +20,8 @@
2020
import java.util.concurrent.atomic.AtomicInteger;
2121
import java.util.concurrent.atomic.AtomicLong;
2222

23-
import org.apache.hadoop.hbase.util.HashedBytes;
2423
import org.apache.hadoop.hbase.util.EnvironmentEdge.Clock;
24+
import org.apache.hadoop.hbase.util.HashedBytes;
2525
import org.apache.yetus.audience.InterfaceAudience;
2626

2727
/**

hbase-common/src/main/java/org/apache/hadoop/hbase/util/clock/SpinAdvancingClock.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,8 +20,8 @@
2020
import java.util.concurrent.atomic.AtomicInteger;
2121
import java.util.concurrent.atomic.AtomicLong;
2222

23-
import org.apache.hadoop.hbase.util.HashedBytes;
2423
import org.apache.hadoop.hbase.util.EnvironmentEdge.Clock;
24+
import org.apache.hadoop.hbase.util.HashedBytes;
2525
import org.apache.yetus.audience.InterfaceAudience;
2626

2727
/**

0 commit comments

Comments
 (0)