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.master.HMaster;
27 import org.apache.hadoop.hbase.master.LogCleanerDelegate;
28 import org.apache.hadoop.hbase.master.TimeToLiveLogCleaner;
29 import org.apache.hadoop.hbase.replication.ReplicationZookeeperWrapper;
30 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper;
31 import org.apache.zookeeper.WatchedEvent;
32 import org.apache.zookeeper.Watcher;
33
34 import java.io.IOException;
35 import java.util.HashSet;
36 import java.util.List;
37 import java.util.Set;
38 import java.util.concurrent.atomic.AtomicBoolean;
39
40
41
42
43
44 public class ReplicationLogCleaner implements LogCleanerDelegate, Watcher {
45
46 private static final Log LOG =
47 LogFactory.getLog(ReplicationLogCleaner.class);
48 private TimeToLiveLogCleaner ttlCleaner;
49 private Configuration conf;
50 private ReplicationZookeeperWrapper zkHelper;
51 private Set<String> hlogs = new HashSet<String>();
52
53
54
55
56 public ReplicationLogCleaner() {}
57
58 @Override
59 public boolean isLogDeletable(Path filePath) {
60
61
62 if (!ttlCleaner.isLogDeletable(filePath)) {
63 LOG.debug("Won't delete log since not past due " + filePath);
64 return false;
65 }
66 String log = filePath.getName();
67
68
69 if (this.hlogs.contains(log)) {
70 return false;
71 }
72
73
74
75
76 return !refreshHLogsAndSearch(log);
77 }
78
79
80
81
82
83
84
85
86 private boolean refreshHLogsAndSearch(String searchedLog) {
87 this.hlogs.clear();
88 final boolean lookForLog = searchedLog != null;
89 List<String> rss = zkHelper.getListOfReplicators(this);
90 if (rss == null) {
91 LOG.debug("Didn't find any region server that replicates, deleting: " +
92 searchedLog);
93 return false;
94 }
95 for (String rs: rss) {
96 List<String> listOfPeers = zkHelper.getListPeersForRS(rs, this);
97
98 if (listOfPeers == null) {
99 continue;
100 }
101 for (String id : listOfPeers) {
102 List<String> peersHlogs = zkHelper.getListHLogsForPeerForRS(rs, id, this);
103 if (peersHlogs != null) {
104 this.hlogs.addAll(peersHlogs);
105 }
106
107 if(lookForLog && this.hlogs.contains(searchedLog)) {
108 LOG.debug("Found log in ZK, keeping: " + searchedLog);
109 return true;
110 }
111 }
112 }
113 LOG.debug("Didn't find this log in ZK, deleting: " + searchedLog);
114 return false;
115 }
116
117 @Override
118 public void setConf(Configuration conf) {
119 this.conf = conf;
120 this.ttlCleaner = new TimeToLiveLogCleaner();
121 this.ttlCleaner.setConf(conf);
122 try {
123 this.zkHelper = new ReplicationZookeeperWrapper(
124 ZooKeeperWrapper.createInstance(this.conf,
125 HMaster.class.getName()),
126 this.conf, new AtomicBoolean(true), null);
127 } catch (IOException e) {
128 LOG.error(e);
129 }
130 refreshHLogsAndSearch(null);
131 }
132
133 @Override
134 public Configuration getConf() {
135 return conf;
136 }
137
138 @Override
139 public void process(WatchedEvent watchedEvent) {}
140 }