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