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