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 com.google.common.base.Preconditions;
22 import java.util.Map;
23 import org.apache.commons.collections.MapUtils;
24 import org.apache.commons.logging.Log;
25 import org.apache.commons.logging.LogFactory;
26 import org.apache.hadoop.hbase.classification.InterfaceAudience;
27 import org.apache.hadoop.conf.Configuration;
28 import org.apache.hadoop.fs.FileStatus;
29 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
30 import org.apache.hadoop.hbase.HConstants;
31 import org.apache.hadoop.hbase.master.HMaster;
32 import org.apache.hadoop.hbase.master.MasterServices;
33 import org.apache.hadoop.hbase.master.cleaner.BaseLogCleanerDelegate;
34 import org.apache.hadoop.hbase.replication.ReplicationException;
35 import org.apache.hadoop.hbase.replication.ReplicationFactory;
36 import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
37 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
38 import java.io.IOException;
39 import java.util.Collections;
40 import java.util.List;
41 import java.util.Set;
42
43 import com.google.common.base.Predicate;
44 import com.google.common.collect.ImmutableSet;
45 import com.google.common.collect.Iterables;
46 import com.google.common.collect.Sets;
47 import org.apache.zookeeper.KeeperException;
48
49
50
51
52
53 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
54 public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
55 private static final Log LOG = LogFactory.getLog(ReplicationLogCleaner.class);
56 private ZooKeeperWatcher zkw = null;
57 private ReplicationQueuesClient replicationQueues;
58 private boolean stopped = false;
59 private MasterServices master;
60 private boolean shareZK = false;
61
62 @Override
63 public Iterable<FileStatus> getDeletableFiles(Iterable<FileStatus> files) {
64
65
66 if (this.getConf() == null) {
67 return files;
68 }
69
70 final Set<String> wals;
71 try {
72
73
74 wals = loadWALsFromQueues();
75 } catch (KeeperException e) {
76 LOG.warn("Failed to read zookeeper, skipping checking deletable files", e);
77 return Collections.emptyList();
78 }
79 return Iterables.filter(files, new Predicate<FileStatus>() {
80 @Override
81 public boolean apply(FileStatus file) {
82 String wal = file.getPath().getName();
83 boolean logInReplicationQueue = wals.contains(wal);
84 if (LOG.isDebugEnabled()) {
85 if (logInReplicationQueue) {
86 LOG.debug("Found log in ZK, keeping: " + wal);
87 } else {
88 LOG.debug("Didn't find this log in ZK, deleting: " + wal);
89 }
90 }
91 return !logInReplicationQueue;
92 }});
93 }
94
95
96
97
98
99
100
101 private Set<String> loadWALsFromQueues() throws KeeperException {
102 for (int retry = 0; ; retry++) {
103 int v0 = replicationQueues.getQueuesZNodeCversion();
104 List<String> rss = replicationQueues.getListOfReplicators();
105 if (rss == null || rss.isEmpty()) {
106 LOG.debug("Didn't find any region server that replicates, won't prevent any deletions.");
107 return ImmutableSet.of();
108 }
109 Set<String> wals = Sets.newHashSet();
110 for (String rs : rss) {
111 List<String> listOfPeers = replicationQueues.getAllQueues(rs);
112
113 if (listOfPeers == null) {
114 continue;
115 }
116 for (String id : listOfPeers) {
117 List<String> peersWals = replicationQueues.getLogsInQueue(rs, id);
118 if (peersWals != null) {
119 wals.addAll(peersWals);
120 }
121 }
122 }
123 int v1 = replicationQueues.getQueuesZNodeCversion();
124 if (v0 == v1) {
125 return wals;
126 }
127 LOG.info(String.format("Replication queue node cversion changed from %d to %d, retry = %d",
128 v0, v1, retry));
129 }
130 }
131
132 @Override
133 public void setConf(Configuration config) {
134
135 if (!config.getBoolean(HConstants.REPLICATION_ENABLE_KEY,
136 HConstants.REPLICATION_ENABLE_DEFAULT)) {
137 LOG.warn("Not configured - allowing all wals to be deleted");
138 return;
139 }
140 super.setConf(config);
141 }
142
143 @Override
144 public void init(Map<String, Object> params) {
145 if (getConf() == null) {
146
147 return;
148 }
149
150 if (MapUtils.isNotEmpty(params)) {
151 Object master = params.get(HMaster.MASTER);
152 if (master != null && master instanceof HMaster) {
153 this.master = (HMaster)master;
154 zkw = ((HMaster) master).getZooKeeper();
155 shareZK = true;
156 }
157 }
158 init(getConf(), this.zkw, null);
159 }
160
161 @InterfaceAudience.Private
162 public void init(Configuration conf, ZooKeeperWatcher zk,
163 ReplicationQueuesClient replicationQueuesClient) {
164 super.setConf(conf);
165 try {
166 if (zk != null) {
167 this.zkw = zk;
168 } else {
169 this.zkw = new ZooKeeperWatcher(getConf(), "replicationLogCleaner", null);
170 }
171 Preconditions.checkNotNull(this.zkw, "Zookeeper watcher cannot be null");
172 if (replicationQueuesClient != null) {
173 this.replicationQueues = replicationQueuesClient;
174 } else {
175 this.replicationQueues =
176 ReplicationFactory.getReplicationQueuesClient(zkw, getConf(), master);
177 this.replicationQueues.init();
178 }
179 Preconditions.checkNotNull(this.replicationQueues,
180 "ReplicationQueues cannot be null");
181 } catch (IOException | ReplicationException e) {
182 LOG.error("Error while configuring " + this.getClass().getName(), e);
183 }
184 }
185
186 @Override
187 public void stop(String why) {
188 if (this.stopped) return;
189 this.stopped = true;
190 if (!shareZK && this.zkw != null) {
191 LOG.info("Stopping " + this.zkw);
192 this.zkw.close();
193 }
194 }
195
196 @Override
197 public boolean isStopped() {
198 return this.stopped;
199 }
200 }