View Javadoc

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 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   * Implementation of a log cleaner that checks if a log is still scheduled for
51   * replication before deleting it when its TTL is over.
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     // all members of this class are null if replication is disabled,
65     // so we cannot filter the files
66      if (this.getConf() == null) {
67        return files;
68      }
69  
70      final Set<String> wals;
71      try {
72        // The concurrently created new WALs may not be included in the return list,
73        // but they won't be deleted because they're not in the checking set.
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     * Load all wals in all replication queues from ZK. This method guarantees to return a
97     * snapshot which contains all WALs in the zookeeper at the start of this call even there
98     * is concurrent queue failover. However, some newly created WALs during the call may
99     * not be included.
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       // We should also check cversions of all rs nodes to Prevent missing of WAL which are claiming
110       // by other regionServer. For details, please see HBASE-26482
111       Map<String, Integer> rsToCversionBefore = replicationQueues.getReplicatorsZNodeCversion();
112       Set<String> wals = Sets.newHashSet();
113       for (String rs : rss) {
114         List<String> listOfPeers = replicationQueues.getAllQueues(rs);
115         // if rs just died, this will be null
116         if (listOfPeers == null) {
117           continue;
118         }
119         for (String id : listOfPeers) {
120           List<String> peersWals = replicationQueues.getLogsInQueue(rs, id);
121           if (peersWals != null) {
122             wals.addAll(peersWals);
123           }
124         }
125       }
126       int v1 = replicationQueues.getQueuesZNodeCversion();
127       Map<String, Integer> rsToCversionAfter = replicationQueues.getReplicatorsZNodeCversion();
128       if (v0 == v1 && rsToCversionBefore.equals(rsToCversionAfter)) {
129         return wals;
130       }
131       LOG.info(String.format("Replication queue node cversion changed from %d to %d, retry = %d",
132           v0, v1, retry));
133     }
134   }
135 
136   @Override
137   public void setConf(Configuration config) {
138     // If replication is disabled, keep all members null
139     if (!config.getBoolean(HConstants.REPLICATION_ENABLE_KEY,
140         HConstants.REPLICATION_ENABLE_DEFAULT)) {
141       LOG.warn("Not configured - allowing all wals to be deleted");
142       return;
143     }
144     super.setConf(config);
145   }
146 
147   @Override
148   public void init(Map<String, Object> params) {
149     if (getConf() == null) {
150       // Replication is disabled so do nothing.
151       return;
152     }
153 
154     if (MapUtils.isNotEmpty(params)) {
155       Object master = params.get(HMaster.MASTER);
156       if (master != null && master instanceof HMaster) {
157         this.master = (HMaster)master;
158         zkw = ((HMaster) master).getZooKeeper();
159         shareZK = true;
160       }
161     }
162     init(getConf(), this.zkw, null);
163   }
164 
165   @InterfaceAudience.Private
166   public void init(Configuration conf, ZooKeeperWatcher zk,
167       ReplicationQueuesClient replicationQueuesClient) {
168     super.setConf(conf);
169     try {
170       if (zk != null) {
171         this.zkw = zk;
172       } else {
173         this.zkw = new ZooKeeperWatcher(getConf(), "replicationLogCleaner", null);
174       }
175       Preconditions.checkNotNull(this.zkw, "Zookeeper watcher cannot be null");
176       if (replicationQueuesClient != null) {
177         this.replicationQueues = replicationQueuesClient;
178       } else {
179         this.replicationQueues =
180           ReplicationFactory.getReplicationQueuesClient(zkw, getConf(), master);
181         this.replicationQueues.init();
182       }
183       Preconditions.checkNotNull(this.replicationQueues,
184         "ReplicationQueues cannot be null");
185     } catch (IOException | ReplicationException e) {
186       LOG.error("Error while configuring " + this.getClass().getName(), e);
187     }
188   }
189 
190   @Override
191   public void stop(String why) {
192     if (this.stopped) return;
193     this.stopped = true;
194     if (!shareZK && this.zkw != null) {
195       LOG.info("Stopping " + this.zkw);
196       this.zkw.close();
197     }
198   }
199 
200   @Override
201   public boolean isStopped() {
202     return this.stopped;
203   }
204 }