1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.replication.regionserver;
20
21 import java.io.IOException;
22 import java.util.List;
23 import java.util.NavigableMap;
24 import java.util.TreeMap;
25 import java.util.UUID;
26 import java.util.concurrent.Executors;
27 import java.util.concurrent.ScheduledExecutorService;
28 import java.util.concurrent.TimeUnit;
29
30 import com.google.common.util.concurrent.ThreadFactoryBuilder;
31 import org.apache.commons.logging.Log;
32 import org.apache.commons.logging.LogFactory;
33 import org.apache.hadoop.classification.InterfaceAudience;
34 import org.apache.hadoop.conf.Configuration;
35 import org.apache.hadoop.fs.FileSystem;
36 import org.apache.hadoop.fs.Path;
37 import org.apache.hadoop.hbase.CellScanner;
38 import org.apache.hadoop.hbase.HRegionInfo;
39 import org.apache.hadoop.hbase.HTableDescriptor;
40 import org.apache.hadoop.hbase.KeyValue;
41 import org.apache.hadoop.hbase.Server;
42 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
43 import org.apache.hadoop.hbase.regionserver.ReplicationSourceService;
44 import org.apache.hadoop.hbase.regionserver.ReplicationSinkService;
45 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
46 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
47 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
48 import org.apache.hadoop.hbase.replication.ReplicationFactory;
49 import org.apache.hadoop.hbase.replication.ReplicationPeers;
50 import org.apache.hadoop.hbase.replication.ReplicationQueues;
51 import org.apache.hadoop.hbase.replication.ReplicationTracker;
52 import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner;
53 import org.apache.hadoop.hbase.util.Bytes;
54 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
55 import org.apache.zookeeper.KeeperException;
56
57 import static org.apache.hadoop.hbase.HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS;
58 import static org.apache.hadoop.hbase.HConstants.REPLICATION_ENABLE_KEY;
59 import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_LOCAL;
60
61
62
63
64 @InterfaceAudience.Private
65 public class Replication implements WALActionsListener,
66 ReplicationSourceService, ReplicationSinkService {
67 private static final Log LOG =
68 LogFactory.getLog(Replication.class);
69 private boolean replication;
70 private ReplicationSourceManager replicationManager;
71 private ReplicationQueues replicationQueues;
72 private ReplicationPeers replicationPeers;
73 private ReplicationTracker replicationTracker;
74 private Configuration conf;
75 private ReplicationSink replicationSink;
76
77 private Server server;
78
79 private ScheduledExecutorService scheduleThreadPool;
80 private int statsThreadPeriod;
81
82
83
84
85
86
87
88
89
90 public Replication(final Server server, final FileSystem fs,
91 final Path logDir, final Path oldLogDir) throws IOException{
92 initialize(server, fs, logDir, oldLogDir);
93 }
94
95
96
97
98 public Replication() {
99 }
100
101 public void initialize(final Server server, final FileSystem fs,
102 final Path logDir, final Path oldLogDir) throws IOException {
103 this.server = server;
104 this.conf = this.server.getConfiguration();
105 this.replication = isReplication(this.conf);
106 this.scheduleThreadPool = Executors.newScheduledThreadPool(1,
107 new ThreadFactoryBuilder()
108 .setNameFormat(server.getServerName().toShortString() + "Replication Statistics #%d")
109 .setDaemon(true)
110 .build());
111 if (replication) {
112 try {
113 this.replicationQueues =
114 ReplicationFactory.getReplicationQueues(server.getZooKeeper(), this.conf, this.server);
115 this.replicationQueues.init(this.server.getServerName().toString());
116 this.replicationPeers =
117 ReplicationFactory.getReplicationPeers(server.getZooKeeper(), this.conf, this.server);
118 this.replicationPeers.init();
119 this.replicationTracker =
120 ReplicationFactory.getReplicationTracker(server.getZooKeeper(), this.replicationPeers,
121 this.conf, this.server, this.server);
122 } catch (KeeperException ke) {
123 throw new IOException("Failed replication handler create", ke);
124 }
125 UUID clusterId = null;
126 try {
127 clusterId = ZKClusterId.getUUIDForCluster(this.server.getZooKeeper());
128 } catch (KeeperException ke) {
129 throw new IOException("Could not read cluster id", ke);
130 }
131 this.replicationManager =
132 new ReplicationSourceManager(replicationQueues, replicationPeers, replicationTracker,
133 conf, this.server, fs, logDir, oldLogDir, clusterId);
134 this.statsThreadPeriod =
135 this.conf.getInt("replication.stats.thread.period.seconds", 5 * 60);
136 LOG.debug("ReplicationStatisticsThread " + this.statsThreadPeriod);
137 } else {
138 this.replicationManager = null;
139 this.replicationQueues = null;
140 this.replicationPeers = null;
141 this.replicationTracker = null;
142 }
143 }
144
145
146
147
148
149 public static boolean isReplication(final Configuration c) {
150 return c.getBoolean(REPLICATION_ENABLE_KEY, false);
151 }
152
153
154
155
156 public WALActionsListener getWALActionsListener() {
157 return this;
158 }
159
160
161
162 public void stopReplicationService() {
163 join();
164 }
165
166
167
168
169 public void join() {
170 if (this.replication) {
171 this.replicationManager.join();
172 if (this.replicationSink != null) {
173 this.replicationSink.stopReplicationSinkServices();
174 }
175 }
176 scheduleThreadPool.shutdown();
177 }
178
179
180
181
182
183
184
185
186
187 public void replicateLogEntries(List<WALEntry> entries, CellScanner cells) throws IOException {
188 if (this.replication) {
189 this.replicationSink.replicateEntries(entries, cells);
190 }
191 }
192
193
194
195
196
197
198 public void startReplicationService() throws IOException {
199 if (this.replication) {
200 this.replicationManager.init();
201 this.replicationSink = new ReplicationSink(this.conf, this.server);
202 this.scheduleThreadPool.scheduleAtFixedRate(
203 new ReplicationStatisticsThread(this.replicationSink, this.replicationManager),
204 statsThreadPeriod, statsThreadPeriod, TimeUnit.SECONDS);
205 }
206 }
207
208
209
210
211
212 public ReplicationSourceManager getReplicationManager() {
213 return this.replicationManager;
214 }
215
216 @Override
217 public void visitLogEntryBeforeWrite(HRegionInfo info, HLogKey logKey,
218 WALEdit logEdit) {
219
220 }
221
222 @Override
223 public void visitLogEntryBeforeWrite(HTableDescriptor htd, HLogKey logKey,
224 WALEdit logEdit) {
225 scopeWALEdits(htd, logKey, logEdit);
226 }
227
228
229
230
231
232
233
234
235 public static void scopeWALEdits(HTableDescriptor htd, HLogKey logKey,
236 WALEdit logEdit) {
237 NavigableMap<byte[], Integer> scopes =
238 new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
239 byte[] family;
240 for (KeyValue kv : logEdit.getKeyValues()) {
241 family = kv.getFamily();
242 if (kv.matchingFamily(WALEdit.METAFAMILY)) continue;
243
244 int scope = htd.getFamily(family).getScope();
245 if (scope != REPLICATION_SCOPE_LOCAL &&
246 !scopes.containsKey(family)) {
247 scopes.put(family, scope);
248 }
249 }
250 if (!scopes.isEmpty()) {
251 logKey.setScopes(scopes);
252 }
253 }
254
255 @Override
256 public void preLogRoll(Path oldPath, Path newPath) throws IOException {
257 getReplicationManager().preLogRoll(newPath);
258 }
259
260 @Override
261 public void postLogRoll(Path oldPath, Path newPath) throws IOException {
262 getReplicationManager().postLogRoll(newPath);
263 }
264
265 @Override
266 public void preLogArchive(Path oldPath, Path newPath) throws IOException {
267
268 }
269
270 @Override
271 public void postLogArchive(Path oldPath, Path newPath) throws IOException {
272
273 }
274
275
276
277
278
279
280 public static void decorateMasterConfiguration(Configuration conf) {
281 if (!isReplication(conf)) {
282 return;
283 }
284 String plugins = conf.get(HBASE_MASTER_LOGCLEANER_PLUGINS);
285 String cleanerClass = ReplicationLogCleaner.class.getCanonicalName();
286 if (!plugins.contains(cleanerClass)) {
287 conf.set(HBASE_MASTER_LOGCLEANER_PLUGINS, plugins + "," + cleanerClass);
288 }
289 }
290
291 @Override
292 public void logRollRequested() {
293
294 }
295
296 @Override
297 public void logCloseRequested() {
298
299 }
300
301
302
303
304 static class ReplicationStatisticsThread extends Thread {
305
306 private final ReplicationSink replicationSink;
307 private final ReplicationSourceManager replicationManager;
308
309 public ReplicationStatisticsThread(final ReplicationSink replicationSink,
310 final ReplicationSourceManager replicationManager) {
311 super("ReplicationStatisticsThread");
312 this.replicationManager = replicationManager;
313 this.replicationSink = replicationSink;
314 }
315
316 @Override
317 public void run() {
318 printStats(this.replicationManager.getStats());
319 printStats(this.replicationSink.getStats());
320 }
321
322 private void printStats(String stats) {
323 if (!stats.isEmpty()) {
324 LOG.info(stats);
325 }
326 }
327 }
328 }