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.master;
20
21 import org.apache.commons.logging.Log;
22 import org.apache.commons.logging.LogFactory;
23 import org.apache.hadoop.classification.InterfaceAudience;
24 import org.apache.hadoop.conf.Configuration;
25 import org.apache.hadoop.fs.FileStatus;
26 import org.apache.hadoop.fs.Path;
27 import org.apache.hadoop.hbase.Abortable;
28 import org.apache.hadoop.hbase.HConstants;
29 import org.apache.hadoop.hbase.client.HConnectionManager;
30 import org.apache.hadoop.hbase.master.cleaner.BaseLogCleanerDelegate;
31 import org.apache.hadoop.hbase.replication.ReplicationFactory;
32 import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
33 import org.apache.hadoop.hbase.replication.ReplicationQueuesClientZKImpl;
34 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
35 import org.apache.zookeeper.KeeperException;
36
37 import java.io.IOException;
38 import java.util.HashSet;
39 import java.util.List;
40 import java.util.Set;
41
42
43
44
45
46 @InterfaceAudience.Private
47 public class ReplicationLogCleaner extends BaseLogCleanerDelegate implements Abortable {
48 private static final Log LOG = LogFactory.getLog(ReplicationLogCleaner.class);
49 private ZooKeeperWatcher zkw;
50 private ReplicationQueuesClient replicationQueues;
51 private final Set<String> hlogs = new HashSet<String>();
52 private boolean stopped = false;
53 private boolean aborted;
54
55
56 @Override
57 public boolean isLogDeletable(FileStatus fStat) {
58
59
60
61 if (this.getConf() == null) {
62 return true;
63 }
64 String log = fStat.getPath().getName();
65
66
67 if (this.hlogs.contains(log)) {
68 return false;
69 }
70
71
72
73
74 return !refreshHLogsAndSearch(log);
75 }
76
77
78
79
80
81
82
83
84 private boolean refreshHLogsAndSearch(String searchedLog) {
85 this.hlogs.clear();
86 final boolean lookForLog = searchedLog != null;
87 List<String> rss = replicationQueues.getListOfReplicators();
88 if (rss == null) {
89 LOG.debug("Didn't find any region server that replicates, deleting: " +
90 searchedLog);
91 return false;
92 }
93 for (String rs: rss) {
94 List<String> listOfPeers = replicationQueues.getAllQueues(rs);
95
96 if (listOfPeers == null) {
97 continue;
98 }
99 for (String id : listOfPeers) {
100 List<String> peersHlogs = replicationQueues.getLogsInQueue(rs, id);
101 if (peersHlogs != null) {
102 this.hlogs.addAll(peersHlogs);
103 }
104
105 if(lookForLog && this.hlogs.contains(searchedLog)) {
106 LOG.debug("Found log in ZK, keeping: " + searchedLog);
107 return true;
108 }
109 }
110 }
111 LOG.debug("Didn't find this log in ZK, deleting: " + searchedLog);
112 return false;
113 }
114
115 @Override
116 public void setConf(Configuration config) {
117
118 if (!config.getBoolean(HConstants.REPLICATION_ENABLE_KEY, false)) {
119 return;
120 }
121
122
123 Configuration conf = new Configuration(config);
124 super.setConf(conf);
125 try {
126 this.zkw = new ZooKeeperWatcher(conf, "replicationLogCleaner", null);
127 this.replicationQueues = ReplicationFactory.getReplicationQueuesClient(zkw, conf, this);
128 this.replicationQueues.init();
129 } catch (KeeperException e) {
130 LOG.error("Error while configuring " + this.getClass().getName(), e);
131 } catch (IOException e) {
132 LOG.error("Error while configuring " + this.getClass().getName(), e);
133 }
134 refreshHLogsAndSearch(null);
135 }
136
137
138 @Override
139 public void stop(String why) {
140 if (this.stopped) return;
141 this.stopped = true;
142 if (this.zkw != null) {
143 LOG.info("Stopping " + this.zkw);
144 this.zkw.close();
145 }
146
147 HConnectionManager.deleteConnection(this.getConf());
148 }
149
150 @Override
151 public boolean isStopped() {
152 return this.stopped;
153 }
154
155 @Override
156 public void abort(String why, Throwable e) {
157 LOG.warn("Aborting ReplicationLogCleaner because " + why, e);
158 this.aborted = true;
159 stop(why);
160 }
161
162 @Override
163 public boolean isAborted() {
164 return this.aborted;
165 }
166 }