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.Map;
023import java.util.Set;
024import org.apache.hadoop.conf.Configuration;
025import org.apache.hadoop.fs.FileStatus;
026import org.apache.hadoop.hbase.HBaseInterfaceAudience;
027import org.apache.hadoop.hbase.master.HMaster;
028import org.apache.hadoop.hbase.master.cleaner.BaseLogCleanerDelegate;
029import org.apache.hadoop.hbase.replication.ReplicationException;
030import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
031import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
032import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
033import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
034import org.apache.yetus.audience.InterfaceAudience;
035import org.slf4j.Logger;
036import org.slf4j.LoggerFactory;
037
038import org.apache.hbase.thirdparty.com.google.common.base.Predicate;
039import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
040import org.apache.hbase.thirdparty.org.apache.commons.collections4.MapUtils;
041
042/**
043 * Implementation of a log cleaner that checks if a log is still scheduled for replication before
044 * deleting it when its TTL is over.
045 */
046@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
047public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
048  private static final Logger LOG = LoggerFactory.getLogger(ReplicationLogCleaner.class);
049  private ZKWatcher zkw = null;
050  private boolean shareZK = false;
051  private ReplicationQueueStorage queueStorage;
052  private boolean stopped = false;
053  private Set<String> wals;
054  private long readZKTimestamp = 0;
055
056  @Override
057  public void preClean() {
058    readZKTimestamp = EnvironmentEdgeManager.currentTime();
059    try {
060      // The concurrently created new WALs may not be included in the return list,
061      // but they won't be deleted because they're not in the checking set.
062      wals = queueStorage.getAllWALs();
063    } catch (ReplicationException e) {
064      LOG.warn("Failed to read zookeeper, skipping checking deletable files");
065      wals = null;
066    }
067  }
068
069  @Override
070  public Iterable<FileStatus> getDeletableFiles(Iterable<FileStatus> files) {
071    // all members of this class are null if replication is disabled,
072    // so we cannot filter the files
073    if (this.getConf() == null) {
074      return files;
075    }
076
077    if (wals == null) {
078      return Collections.emptyList();
079    }
080    return Iterables.filter(files, new Predicate<FileStatus>() {
081      @Override
082      public boolean apply(FileStatus file) {
083        // just for overriding the findbugs NP warnings, as the parameter is marked as Nullable in
084        // the guava Predicate.
085        if (file == null) {
086          return false;
087        }
088        String wal = file.getPath().getName();
089        boolean logInReplicationQueue = wals.contains(wal);
090        if (logInReplicationQueue) {
091          LOG.debug("Found up in ZooKeeper, NOT deleting={}", wal);
092        }
093        return !logInReplicationQueue && (file.getModificationTime() < readZKTimestamp);
094      }
095    });
096  }
097
098  @Override
099  public void init(Map<String, Object> params) {
100    super.init(params);
101    try {
102      if (MapUtils.isNotEmpty(params)) {
103        Object master = params.get(HMaster.MASTER);
104        if (master != null && master instanceof HMaster) {
105          zkw = ((HMaster) master).getZooKeeper();
106          shareZK = true;
107        }
108      }
109      if (zkw == null) {
110        zkw = new ZKWatcher(getConf(), "replicationLogCleaner", null);
111      }
112      this.queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zkw, getConf());
113    } catch (IOException e) {
114      LOG.error("Error while configuring " + this.getClass().getName(), e);
115    }
116  }
117
118  @InterfaceAudience.Private
119  public void setConf(Configuration conf, ZKWatcher zk) {
120    super.setConf(conf);
121    try {
122      this.zkw = zk;
123      this.queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zk, conf);
124    } catch (Exception e) {
125      LOG.error("Error while configuring " + this.getClass().getName(), e);
126    }
127  }
128
129  @Override
130  public void stop(String why) {
131    if (this.stopped) return;
132    this.stopped = true;
133    if (!shareZK && this.zkw != null) {
134      LOG.info("Stopping " + this.zkw);
135      this.zkw.close();
136    }
137  }
138
139  @Override
140  public boolean isStopped() {
141    return this.stopped;
142  }
143}