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.region;
019
020import static org.apache.hadoop.hbase.HConstants.HREGION_OLDLOGDIR_NAME;
021
022import java.io.IOException;
023import java.util.List;
024import org.apache.hadoop.conf.Configuration;
025import org.apache.hadoop.fs.FileSystem;
026import org.apache.hadoop.fs.Path;
027import org.apache.hadoop.hbase.Abortable;
028import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL;
029import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
030import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
031import org.apache.hadoop.hbase.wal.AbstractWALRoller;
032import org.apache.hadoop.hbase.wal.WALFactory;
033import org.apache.yetus.audience.InterfaceAudience;
034import org.slf4j.Logger;
035import org.slf4j.LoggerFactory;
036
037/**
038 * As long as there is no RegionServerServices for a master local region, we need implement log
039 * roller logic by our own.
040 * <p/>
041 * We can reuse most of the code for normal wal roller, the only difference is that there is only
042 * one region, so in {@link #scheduleFlush(String, List)} method we can just schedule flush for the
043 * master local region.
044 */
045@InterfaceAudience.Private
046public final class MasterRegionWALRoller extends AbstractWALRoller<Abortable> {
047
048  private static final Logger LOG = LoggerFactory.getLogger(MasterRegionWALRoller.class);
049
050  private volatile MasterRegionFlusherAndCompactor flusherAndCompactor;
051
052  private final FileSystem fs;
053
054  private final Path walArchiveDir;
055
056  private final Path globalWALArchiveDir;
057
058  private final String archivedWALSuffix;
059
060  private MasterRegionWALRoller(String name, Configuration conf, Abortable abortable, FileSystem fs,
061    Path walRootDir, Path globalWALRootDir, String archivedWALSuffix) {
062    super(name, conf, abortable);
063    this.fs = fs;
064    this.walArchiveDir = new Path(walRootDir, HREGION_OLDLOGDIR_NAME);
065    this.globalWALArchiveDir = new Path(globalWALRootDir, HREGION_OLDLOGDIR_NAME);
066    this.archivedWALSuffix = archivedWALSuffix;
067  }
068
069  @Override
070  protected void afterWALArchive(Path oldPath, Path newPath) {
071    // move the archived WAL files to the global archive path
072    // here we do not use the newPath directly, so that even if we fail to move some of the
073    // newPaths, we are still safe because every time we will get all the files under the archive
074    // directory.
075    try {
076      MasterRegionUtils.moveFilesUnderDir(fs, walArchiveDir, globalWALArchiveDir,
077        archivedWALSuffix);
078    } catch (IOException e) {
079      LOG.warn("Failed to move archived wals from {} to global dir {}", walArchiveDir,
080        globalWALArchiveDir, e);
081    }
082  }
083
084  @Override
085  protected void scheduleFlush(String encodedRegionName, List<byte[]> families) {
086    MasterRegionFlusherAndCompactor flusher = this.flusherAndCompactor;
087    if (flusher != null) {
088      flusher.requestFlush();
089    }
090  }
091
092  void setFlusherAndCompactor(MasterRegionFlusherAndCompactor flusherAndCompactor) {
093    this.flusherAndCompactor = flusherAndCompactor;
094  }
095
096  static MasterRegionWALRoller create(String name, Configuration conf, Abortable abortable,
097    FileSystem fs, Path walRootDir, Path globalWALRootDir, String archivedWALSuffix,
098    long rollPeriodMs, long flushSize) {
099    // we can not run with wal disabled, so force set it to true.
100    conf.setBoolean(WALFactory.WAL_ENABLED, true);
101    // we do not need this feature, so force disable it.
102    conf.setBoolean(AbstractFSWALProvider.SEPARATE_OLDLOGDIR, false);
103    conf.setLong(WAL_ROLL_PERIOD_KEY, rollPeriodMs);
104    // make the roll size the same with the flush size, as we only have one region here
105    conf.setLong(WALUtil.WAL_BLOCK_SIZE, flushSize * 2);
106    conf.setFloat(AbstractFSWAL.WAL_ROLL_MULTIPLIER, 0.5f);
107    return new MasterRegionWALRoller(name, conf, abortable, fs, walRootDir, globalWALRootDir,
108      archivedWALSuffix);
109  }
110
111}