1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.errorhandling;
19
20 import java.util.Timer;
21 import java.util.TimerTask;
22
23 import org.apache.commons.logging.Log;
24 import org.apache.commons.logging.LogFactory;
25 import org.apache.hadoop.hbase.classification.InterfaceAudience;
26 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
27
28
29
30
31
32
33
34
35 @InterfaceAudience.Private
36 public class TimeoutExceptionInjector {
37
38 private static final Log LOG = LogFactory.getLog(TimeoutExceptionInjector.class);
39
40 private final long maxTime;
41 private volatile boolean complete;
42 private final Timer timer;
43 private final TimerTask timerTask;
44 private long start = -1;
45
46
47
48
49
50
51
52 public TimeoutExceptionInjector(final ForeignExceptionListener listener, final long maxTime) {
53 this.maxTime = maxTime;
54 timer = new Timer();
55 timerTask = new TimerTask() {
56 @Override
57 public void run() {
58
59 synchronized (this) {
60
61 if (TimeoutExceptionInjector.this.complete) return;
62
63 TimeoutExceptionInjector.this.complete = true;
64 }
65 long end = EnvironmentEdgeManager.currentTime();
66 TimeoutException tee = new TimeoutException(
67 "Timeout caused Foreign Exception", start, end, maxTime);
68 String source = "timer-" + timer;
69 listener.receive(new ForeignException(source, tee));
70 }
71 };
72 }
73
74 public long getMaxTime() {
75 return maxTime;
76 }
77
78
79
80
81 public void complete() {
82 synchronized (this.timerTask) {
83 if (this.complete) {
84 LOG.warn("Timer already marked completed, ignoring!");
85 return;
86 }
87 if (LOG.isDebugEnabled()) {
88 LOG.debug("Marking timer as complete - no error notifications will be received for " +
89 "this timer.");
90 }
91 this.complete = true;
92 }
93 this.timer.cancel();
94 }
95
96
97
98
99
100
101
102
103 public synchronized void start() throws IllegalStateException {
104 if (this.start >= 0) {
105 LOG.warn("Timer already started, can't be started again. Ignoring second request.");
106 return;
107 }
108 LOG.debug("Scheduling process timer to run in: " + maxTime + " ms");
109 timer.schedule(timerTask, maxTime);
110 this.start = EnvironmentEdgeManager.currentTime();
111 }
112
113
114
115
116
117
118 public void trigger() {
119 synchronized (timerTask) {
120 if (this.complete) {
121 LOG.warn("Timer already completed, not triggering.");
122 return;
123 }
124 LOG.debug("Triggering timer immediately!");
125 this.timer.cancel();
126 this.timerTask.run();
127 }
128 }
129 }