1   /*
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
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.ReplicationQueuesClient;
31  import org.apache.hadoop.hbase.replication.ReplicationQueuesClientZKImpl;
32  import org.apache.hadoop.hbase.replication.ReplicationStateImpl;
33  import org.apache.hadoop.hbase.replication.ReplicationStateInterface;
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   * Implementation of a log cleaner that checks if a log is still scheduled for
44   * replication before deleting it when its TTL is over.
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 ReplicationStateInterface replicationState;
52    private final Set<String> hlogs = new HashSet<String>();
53    private boolean stopped = false;
54    private boolean aborted;
55  
56  
57    @Override
58    public boolean isLogDeletable(Path filePath) {
59  
60      try {
61        if (!replicationState.getState()) {
62          return false;
63        }
64      } catch (KeeperException e) {
65        abort("Cannot get the state of replication", e);
66        return false;
67      }
68  
69      // all members of this class are null if replication is disabled, and we
70      // return true since false would render the LogsCleaner useless
71      if (this.getConf() == null) {
72        return true;
73      }
74      String log = filePath.getName();
75      // If we saw the hlog previously, let's consider it's still used
76      // At some point in the future we will refresh the list and it will be gone
77      if (this.hlogs.contains(log)) {
78        return false;
79      }
80  
81      // Let's see it's still there
82      // This solution makes every miss very expensive to process since we
83      // almost completely refresh the cache each time
84      return !refreshHLogsAndSearch(log);
85    }
86  
87    /**
88     * Search through all the hlogs we have in ZK to refresh the cache
89     * If a log is specified and found, then we early out and return true
90     * @param searchedLog log we are searching for, pass null to cache everything
91     *                    that's in zookeeper.
92     * @return false until a specified log is found.
93     */
94    private boolean refreshHLogsAndSearch(String searchedLog) {
95      this.hlogs.clear();
96      final boolean lookForLog = searchedLog != null;
97      List<String> rss = replicationQueues.getListOfReplicators();
98      if (rss == null) {
99        LOG.debug("Didn't find any region server that replicates, deleting: " +
100           searchedLog);
101       return false;
102     }
103     for (String rs: rss) {
104       List<String> listOfPeers = replicationQueues.getAllQueues(rs);
105       // if rs just died, this will be null
106       if (listOfPeers == null) {
107         continue;
108       }
109       for (String id : listOfPeers) {
110         List<String> peersHlogs = replicationQueues.getLogsInQueue(rs, id);
111         if (peersHlogs != null) {
112           this.hlogs.addAll(peersHlogs);
113         }
114         // early exit if we found the log
115         if(lookForLog && this.hlogs.contains(searchedLog)) {
116           LOG.debug("Found log in ZK, keeping: " + searchedLog);
117           return true;
118         }
119       }
120     }
121     LOG.debug("Didn't find this log in ZK, deleting: " + searchedLog);
122     return false;
123   }
124 
125   @Override
126   public void setConf(Configuration config) {
127     // If replication is disabled, keep all members null
128     if (!config.getBoolean(HConstants.REPLICATION_ENABLE_KEY, false)) {
129       return;
130     }
131     // Make my own Configuration.  Then I'll have my own connection to zk that
132     // I can close myself when comes time.
133     Configuration conf = new Configuration(config);
134     super.setConf(conf);
135     try {
136       this.zkw = new ZooKeeperWatcher(conf, "replicationLogCleaner", null);
137       this.replicationQueues = new ReplicationQueuesClientZKImpl(zkw, conf, this);
138       this.replicationState = new ReplicationStateImpl(zkw, conf, this);
139     } catch (KeeperException e) {
140       LOG.error("Error while configuring " + this.getClass().getName(), e);
141     } catch (IOException e) {
142       LOG.error("Error while configuring " + this.getClass().getName(), e);
143     }
144     refreshHLogsAndSearch(null);
145   }
146 
147 
148   @Override
149   public void stop(String why) {
150     if (this.stopped) return;
151     this.stopped = true;
152     if (this.zkw != null) {
153       LOG.info("Stopping " + this.zkw);
154       this.zkw.close();
155     }
156     if (this.replicationState != null) {
157       LOG.info("Stopping " + this.replicationState);
158       try {
159         this.replicationState.close();
160       } catch (IOException e) {
161         LOG.error("Error while stopping " + this.replicationState, e);
162       }
163     }
164     // Not sure why we're deleting a connection that we never acquired or used
165     HConnectionManager.deleteConnection(this.getConf());
166   }
167 
168   @Override
169   public boolean isStopped() {
170     return this.stopped;
171   }
172 
173   @Override
174   public void abort(String why, Throwable e) {
175     LOG.warn("Aborting ReplicationLogCleaner because " + why, e);
176     this.aborted = true;
177     stop(why);
178   }
179 
180   @Override
181   public boolean isAborted() {
182     return this.aborted;
183   }
184 }