1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.regionserver;
21
22 import org.apache.commons.logging.Log;
23 import org.apache.commons.logging.LogFactory;
24 import org.apache.hadoop.fs.Path;
25 import org.apache.hadoop.hbase.HConstants;
26 import org.apache.hadoop.hbase.HRegionInfo;
27 import org.apache.hadoop.hbase.RemoteExceptionHandler;
28 import org.apache.hadoop.hbase.Server;
29 import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
30 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
31 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
32 import org.apache.hadoop.hbase.regionserver.wal.WALObserver;
33 import org.apache.hadoop.hbase.util.Bytes;
34
35 import java.io.IOException;
36 import java.util.concurrent.atomic.AtomicBoolean;
37 import java.util.concurrent.locks.ReentrantLock;
38
39
40
41
42
43
44
45
46 class LogRoller extends Thread implements WALObserver {
47 static final Log LOG = LogFactory.getLog(LogRoller.class);
48 private final ReentrantLock rollLock = new ReentrantLock();
49 private final AtomicBoolean rollLog = new AtomicBoolean(false);
50 private final Server server;
51 private final RegionServerServices services;
52 private volatile long lastrolltime = System.currentTimeMillis();
53
54 private final long rollperiod;
55 private final int threadWakeFrequency;
56
57
58 public LogRoller(final Server server, final RegionServerServices services) {
59 super();
60 this.server = server;
61 this.services = services;
62 this.rollperiod = this.server.getConfiguration().
63 getLong("hbase.regionserver.logroll.period", 3600000);
64 this.threadWakeFrequency = this.server.getConfiguration().
65 getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000);
66 }
67
68 @Override
69 public void run() {
70 while (!server.isStopped()) {
71 long now = System.currentTimeMillis();
72 boolean periodic = false;
73 if (!rollLog.get()) {
74 periodic = (now - this.lastrolltime) > this.rollperiod;
75 if (!periodic) {
76 synchronized (rollLog) {
77 try {
78 rollLog.wait(this.threadWakeFrequency);
79 } catch (InterruptedException e) {
80
81 }
82 }
83 continue;
84 }
85
86 if (LOG.isDebugEnabled()) {
87 LOG.debug("Hlog roll period " + this.rollperiod + "ms elapsed");
88 }
89 }
90 rollLock.lock();
91 try {
92 this.lastrolltime = now;
93
94 byte [][] regionsToFlush = this.services.getWAL().rollWriter();
95 if (regionsToFlush != null) {
96 for (byte [] r: regionsToFlush) scheduleFlush(r);
97 }
98 } catch (FailedLogCloseException e) {
99 server.abort("Failed log close in log roller", e);
100 } catch (java.net.ConnectException e) {
101 server.abort("Failed log close in log roller", e);
102 } catch (IOException ex) {
103
104 server.abort("IOE in log roller",
105 RemoteExceptionHandler.checkIOException(ex));
106 } catch (Exception ex) {
107 LOG.error("Log rolling failed", ex);
108 server.abort("Log rolling failed", ex);
109 } finally {
110 rollLog.set(false);
111 rollLock.unlock();
112 }
113 }
114 LOG.info("LogRoller exiting.");
115 }
116
117
118
119
120 private void scheduleFlush(final byte [] region) {
121 boolean scheduled = false;
122 HRegion r = this.services.getFromOnlineRegions(Bytes.toString(region));
123 FlushRequester requester = null;
124 if (r != null) {
125 requester = this.services.getFlushRequester();
126 if (requester != null) {
127 requester.requestFlush(r);
128 scheduled = true;
129 }
130 }
131 if (!scheduled) {
132 LOG.warn("Failed to schedule flush of " +
133 Bytes.toString(region) + "r=" + r + ", requester=" + requester);
134 }
135 }
136
137 public void logRollRequested() {
138 synchronized (rollLog) {
139 rollLog.set(true);
140 rollLog.notifyAll();
141 }
142 }
143
144
145
146
147
148 public void interruptIfNecessary() {
149 try {
150 rollLock.lock();
151 this.interrupt();
152 } finally {
153 rollLock.unlock();
154 }
155 }
156
157 @Override
158 public void logRolled(Path newFile) {
159
160 }
161
162 @Override
163 public void visitLogEntryBeforeWrite(HRegionInfo info, HLogKey logKey,
164 WALEdit logEdit) {
165
166 }
167
168 @Override
169 public void logCloseRequested() {
170
171 }
172 }