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.master.cleaner;
019
020import java.io.IOException;
021import java.util.concurrent.locks.ReentrantReadWriteLock;
022import org.apache.hadoop.conf.Configuration;
023import org.apache.hadoop.fs.FileStatus;
024import org.apache.hadoop.fs.FileSystem;
025import org.apache.hadoop.fs.Path;
026import org.apache.hadoop.hbase.HBaseInterfaceAudience;
027import org.apache.hadoop.hbase.HConstants;
028import org.apache.hadoop.hbase.io.HFileLink;
029import org.apache.hadoop.hbase.mob.MobUtils;
030import org.apache.hadoop.hbase.util.CommonFSUtils;
031import org.apache.yetus.audience.InterfaceAudience;
032import org.slf4j.Logger;
033import org.slf4j.LoggerFactory;
034
035/**
036 * HFileLink cleaner that determines if a hfile should be deleted. HFiles can be deleted only if
037 * there're no links to them. When a HFileLink is created a back reference file is created in:
038 * /hbase/archive/table/region/cf/.links-hfile/ref-region.ref-table To check if the hfile can be
039 * deleted the back references folder must be empty.
040 */
041@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
042public class HFileLinkCleaner extends BaseHFileCleanerDelegate {
043  private static final Logger LOG = LoggerFactory.getLogger(HFileLinkCleaner.class);
044
045  private FileSystem fs = null;
046  private ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
047
048  @Override
049  public boolean isFileDeletable(FileStatus fStat) {
050    lock.readLock().lock();
051    try {
052      if (this.fs == null) {
053        return false;
054      }
055      Path filePath = fStat.getPath();
056      // HFile Link is always deletable
057      if (HFileLink.isHFileLink(filePath)) {
058        return true;
059      }
060
061      // If the file is inside a link references directory, means that it is a back ref link.
062      // The back ref can be deleted only if the referenced file doesn't exists.
063      Path parentDir = filePath.getParent();
064      if (HFileLink.isBackReferencesDir(parentDir)) {
065        Path hfilePath = null;
066        try {
067          // Also check if the HFile is in the HBASE_TEMP_DIRECTORY; this is where the referenced
068          // file gets created when cloning a snapshot.
069          hfilePath = HFileLink.getHFileFromBackReference(
070            new Path(CommonFSUtils.getRootDir(getConf()), HConstants.HBASE_TEMP_DIRECTORY),
071            filePath);
072          if (fs.exists(hfilePath)) {
073            return false;
074          }
075          // check whether the HFileLink still exists in mob dir.
076          hfilePath = HFileLink.getHFileFromBackReference(MobUtils.getMobHome(getConf()), filePath);
077          if (fs.exists(hfilePath)) {
078            return false;
079          }
080          hfilePath =
081            HFileLink.getHFileFromBackReference(CommonFSUtils.getRootDir(getConf()), filePath);
082          return !fs.exists(hfilePath);
083        } catch (IOException e) {
084          if (LOG.isDebugEnabled()) {
085            LOG.debug("Couldn't verify if the referenced file still exists, keep it just in case: "
086              + hfilePath);
087          }
088          return false;
089        }
090      }
091
092      // HFile is deletable only if has no links
093      Path backRefDir = HFileLink.getBackReferencesDir(parentDir, filePath.getName());
094      try {
095        FileStatus[] fileStatuses = CommonFSUtils.listStatus(fs, backRefDir);
096        // for empty reference directory, retain the logic to be deletable
097        if (fileStatuses == null) {
098          return true;
099        }
100        // reuse the found back reference files, check if the forward reference exists.
101        // with this optimization, the chore could save one round compute time if we're visiting
102        // the archive HFile earlier than the HFile Link
103        for (FileStatus fileStatus : fileStatuses) {
104          if (!isFileDeletable(fileStatus)) {
105            return false;
106          }
107        }
108        // all the found back reference files are clear, we can delete it.
109        return true;
110      } catch (IOException e) {
111        if (LOG.isDebugEnabled()) {
112          LOG.debug("Couldn't get the references, not deleting file, just in case. filePath="
113            + filePath + ", backRefDir=" + backRefDir);
114        }
115        return false;
116      }
117    } finally {
118      lock.readLock().unlock();
119    }
120  }
121
122  @Override
123  public void setConf(Configuration conf) {
124    super.setConf(conf);
125
126    // setup filesystem
127    lock.writeLock().lock();
128    try {
129      this.fs = FileSystem.get(this.getConf());
130    } catch (IOException e) {
131      if (LOG.isDebugEnabled()) {
132        LOG.debug("Couldn't instantiate the file system, not deleting file, just in case. "
133          + FileSystem.FS_DEFAULT_NAME_KEY + "="
134          + getConf().get(FileSystem.FS_DEFAULT_NAME_KEY, FileSystem.DEFAULT_FS));
135      }
136    } finally {
137      lock.writeLock().unlock();
138    }
139  }
140}