View Javadoc

1   /*
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
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   * Implementation of a log cleaner that checks if a log is still scheduled for
42   * replication before deleting it when its TTL is over.
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     * Instantiates the cleaner, does nothing more.
55     */
56    public ReplicationLogCleaner() {}
57  
58    @Override
59    public boolean isLogDeletable(Path filePath) {
60  
61      // Don't bother going further if the hlog isn't even expired
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      // If we saw the hlog previously, let's consider it's still used
68      // At some point in the future we will refresh the list and it will be gone
69      if (this.hlogs.contains(log)) {
70        return false;
71      }
72  
73      // Let's see it's still there
74      // This solution makes every miss very expensive to process since we
75      // almost completly refresh the cache each time
76      return !refreshHLogsAndSearch(log);
77    }
78  
79    /**
80     * Search through all the hlogs we have in ZK to refresh the cache
81     * If a log is specified and found, then we early out and return true
82     * @param searchedLog log we are searching for, pass null to cache everything
83     *                    that's in zookeeper.
84     * @return false until a specified log is found.
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        // if rs just died, this will be null
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         // early exit if we found the log
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 }