001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.replication.master;
019
020import java.io.IOException;
021import java.util.Collections;
022import java.util.Set;
023import org.apache.hadoop.conf.Configuration;
024import org.apache.hadoop.fs.FileStatus;
025import org.apache.hadoop.hbase.HBaseInterfaceAudience;
026import org.apache.hadoop.hbase.master.cleaner.BaseLogCleanerDelegate;
027import org.apache.hadoop.hbase.replication.ReplicationException;
028import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
029import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
030import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
031import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
032import org.apache.yetus.audience.InterfaceAudience;
033import org.slf4j.Logger;
034import org.slf4j.LoggerFactory;
035
036import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
037import org.apache.hbase.thirdparty.com.google.common.base.Predicate;
038import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
039
040/**
041 * Implementation of a log cleaner that checks if a log is still scheduled for
042 * replication before deleting it when its TTL is over.
043 */
044@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
045public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
046  private static final Logger LOG = LoggerFactory.getLogger(ReplicationLogCleaner.class);
047  private ZKWatcher zkw;
048  private ReplicationQueueStorage queueStorage;
049  private boolean stopped = false;
050  private Set<String> wals;
051  private long readZKTimestamp = 0;
052
053  @Override
054  public void preClean() {
055    readZKTimestamp = EnvironmentEdgeManager.currentTime();
056    try {
057      // The concurrently created new WALs may not be included in the return list,
058      // but they won't be deleted because they're not in the checking set.
059      wals = queueStorage.getAllWALs();
060    } catch (ReplicationException e) {
061      LOG.warn("Failed to read zookeeper, skipping checking deletable files");
062      wals = null;
063    }
064  }
065
066  @Override
067  public Iterable<FileStatus> getDeletableFiles(Iterable<FileStatus> files) {
068    // all members of this class are null if replication is disabled,
069    // so we cannot filter the files
070    if (this.getConf() == null) {
071      return files;
072    }
073
074    if (wals == null) {
075      return Collections.emptyList();
076    }
077    return Iterables.filter(files, new Predicate<FileStatus>() {
078      @Override
079      public boolean apply(FileStatus file) {
080        // just for overriding the findbugs NP warnings, as the parameter is marked as Nullable in
081        // the guava Predicate.
082        if (file == null) {
083          return false;
084        }
085        String wal = file.getPath().getName();
086        boolean logInReplicationQueue = wals.contains(wal);
087          if (logInReplicationQueue) {
088            LOG.debug("Found up in ZooKeeper, NOT deleting={}", wal);
089        }
090        return !logInReplicationQueue && (file.getModificationTime() < readZKTimestamp);
091      }
092    });
093  }
094
095  @Override
096  public void setConf(Configuration config) {
097    // Make my own Configuration.  Then I'll have my own connection to zk that
098    // I can close myself when comes time.
099    Configuration conf = new Configuration(config);
100    try {
101      setConf(conf, new ZKWatcher(conf, "replicationLogCleaner", null));
102    } catch (IOException e) {
103      LOG.error("Error while configuring " + this.getClass().getName(), e);
104    }
105  }
106
107  @VisibleForTesting
108  public void setConf(Configuration conf, ZKWatcher zk) {
109    super.setConf(conf);
110    try {
111      this.zkw = zk;
112      this.queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zk, conf);
113    } catch (Exception e) {
114      LOG.error("Error while configuring " + this.getClass().getName(), e);
115    }
116  }
117  
118  @Override
119  public void stop(String why) {
120    if (this.stopped) return;
121    this.stopped = true;
122    if (this.zkw != null) {
123      LOG.info("Stopping " + this.zkw);
124      this.zkw.close();
125    }
126  }
127
128  @Override
129  public boolean isStopped() {
130    return this.stopped;
131  }
132}