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_LOGDIR_NAME;
021
022import com.google.errorprone.annotations.RestrictedApi;
023import java.io.IOException;
024import java.util.List;
025import org.apache.hadoop.conf.Configuration;
026import org.apache.hadoop.fs.FileStatus;
027import org.apache.hadoop.fs.FileSystem;
028import org.apache.hadoop.fs.Path;
029import org.apache.hadoop.hbase.HBaseIOException;
030import org.apache.hadoop.hbase.Server;
031import org.apache.hadoop.hbase.TableName;
032import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
033import org.apache.hadoop.hbase.client.Get;
034import org.apache.hadoop.hbase.client.RegionInfo;
035import org.apache.hadoop.hbase.client.RegionInfoBuilder;
036import org.apache.hadoop.hbase.client.Result;
037import org.apache.hadoop.hbase.client.ResultScanner;
038import org.apache.hadoop.hbase.client.Scan;
039import org.apache.hadoop.hbase.client.TableDescriptor;
040import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
041import org.apache.hadoop.hbase.regionserver.HRegion;
042import org.apache.hadoop.hbase.regionserver.HRegion.FlushResult;
043import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
044import org.apache.hadoop.hbase.regionserver.RegionScanner;
045import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
046import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTracker;
047import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
048import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL;
049import org.apache.hadoop.hbase.util.Bytes;
050import org.apache.hadoop.hbase.util.CommonFSUtils;
051import org.apache.hadoop.hbase.util.FSTableDescriptors;
052import org.apache.hadoop.hbase.util.FSUtils;
053import org.apache.hadoop.hbase.util.HFileArchiveUtil;
054import org.apache.hadoop.hbase.util.RecoverLeaseFSUtils;
055import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
056import org.apache.hadoop.hbase.wal.WAL;
057import org.apache.hadoop.hbase.wal.WALFactory;
058import org.apache.yetus.audience.InterfaceAudience;
059import org.slf4j.Logger;
060import org.slf4j.LoggerFactory;
061
062import org.apache.hbase.thirdparty.com.google.common.math.IntMath;
063
064/**
065 * A region that stores data in a separated directory, which can be used to store master local data.
066 * <p/>
067 * FileSystem layout:
068 *
069 * <pre>
070 * hbase
071 *   |
072 *   --&lt;region dir&gt;
073 *       |
074 *       --data
075 *       |  |
076 *       |  --/&lt;ns&gt/&lt;table&gt/&lt;encoded-region-name&gt; <---- The region data
077 *       |      |
078 *       |      --replay <---- The edits to replay
079 *       |
080 *       --WALs
081 *          |
082 *          --&lt;master-server-name&gt; <---- The WAL dir for active master
083 *          |
084 *          --&lt;master-server-name&gt;-dead <---- The WAL dir for dead master
085 * </pre>
086 *
087 * Notice that, you can use different root file system and WAL file system. Then the above directory
088 * will be on two file systems, the root file system will have the data directory while the WAL
089 * filesystem will have the WALs directory. The archived HFile will be moved to the global HFile
090 * archived directory with the {@link MasterRegionParams#archivedWalSuffix()} suffix. The archived
091 * WAL will be moved to the global WAL archived directory with the
092 * {@link MasterRegionParams#archivedHFileSuffix()} suffix.
093 */
094@InterfaceAudience.Private
095public final class MasterRegion {
096
097  private static final Logger LOG = LoggerFactory.getLogger(MasterRegion.class);
098
099  private static final String REPLAY_EDITS_DIR = "recovered.wals";
100
101  private static final String DEAD_WAL_DIR_SUFFIX = "-dead";
102
103  static final String INITIALIZING_FLAG = ".initializing";
104
105  static final String INITIALIZED_FLAG = ".initialized";
106
107  private static final int REGION_ID = 1;
108
109  private final WALFactory walFactory;
110
111  final HRegion region;
112
113  final MasterRegionFlusherAndCompactor flusherAndCompactor;
114
115  private MasterRegionWALRoller walRoller;
116
117  private MasterRegion(HRegion region, WALFactory walFactory,
118    MasterRegionFlusherAndCompactor flusherAndCompactor, MasterRegionWALRoller walRoller) {
119    this.region = region;
120    this.walFactory = walFactory;
121    this.flusherAndCompactor = flusherAndCompactor;
122    this.walRoller = walRoller;
123  }
124
125  private void closeRegion(boolean abort) {
126    try {
127      region.close(abort);
128    } catch (IOException e) {
129      LOG.warn("Failed to close region", e);
130    }
131  }
132
133  private void shutdownWAL() {
134    try {
135      walFactory.shutdown();
136    } catch (IOException e) {
137      LOG.warn("Failed to shutdown WAL", e);
138    }
139  }
140
141  public void update(UpdateMasterRegion action) throws IOException {
142    action.update(region);
143    flusherAndCompactor.onUpdate();
144  }
145
146  public Result get(Get get) throws IOException {
147    return region.get(get);
148  }
149
150  public ResultScanner getScanner(Scan scan) throws IOException {
151    return new RegionScannerAsResultScanner(region.getScanner(scan));
152  }
153
154  public RegionScanner getRegionScanner(Scan scan) throws IOException {
155    return region.getScanner(scan);
156  }
157
158  public FlushResult flush(boolean force) throws IOException {
159    flusherAndCompactor.resetChangesAfterLastFlush();
160    FlushResult flushResult = region.flush(force);
161    flusherAndCompactor.recordLastFlushTime();
162    return flushResult;
163  }
164
165  @RestrictedApi(explanation = "Should only be called in tests", link = "",
166      allowedOnPath = ".*/src/test/.*")
167  public void requestRollAll() {
168    walRoller.requestRollAll();
169  }
170
171  @RestrictedApi(explanation = "Should only be called in tests", link = "",
172      allowedOnPath = ".*/src/test/.*")
173  public void waitUntilWalRollFinished() throws InterruptedException {
174    walRoller.waitUntilWalRollFinished();
175  }
176
177  public void close(boolean abort) {
178    LOG.info("Closing local region {}, isAbort={}", region.getRegionInfo(), abort);
179    if (flusherAndCompactor != null) {
180      flusherAndCompactor.close();
181    }
182    // if abort, we shutdown wal first to fail the ongoing updates to the region, and then close the
183    // region, otherwise there will be dead lock.
184    if (abort) {
185      shutdownWAL();
186      closeRegion(true);
187    } else {
188      closeRegion(false);
189      shutdownWAL();
190    }
191
192    if (walRoller != null) {
193      walRoller.close();
194    }
195  }
196
197  private static WAL createWAL(WALFactory walFactory, MasterRegionWALRoller walRoller,
198    String serverName, FileSystem walFs, Path walRootDir, RegionInfo regionInfo)
199    throws IOException {
200    String logName = AbstractFSWALProvider.getWALDirectoryName(serverName);
201    Path walDir = new Path(walRootDir, logName);
202    LOG.debug("WALDir={}", walDir);
203    if (walFs.exists(walDir)) {
204      throw new HBaseIOException(
205        "Already created wal directory at " + walDir + " for local region " + regionInfo);
206    }
207    if (!walFs.mkdirs(walDir)) {
208      throw new IOException(
209        "Can not create wal directory " + walDir + " for local region " + regionInfo);
210    }
211    WAL wal = walFactory.getWAL(regionInfo);
212    walRoller.addWAL(wal);
213    return wal;
214  }
215
216  private static HRegion bootstrap(Configuration conf, TableDescriptor td, FileSystem fs,
217    Path rootDir, FileSystem walFs, Path walRootDir, WALFactory walFactory,
218    MasterRegionWALRoller walRoller, String serverName, boolean touchInitializingFlag)
219    throws IOException {
220    TableName tn = td.getTableName();
221    RegionInfo regionInfo = RegionInfoBuilder.newBuilder(tn).setRegionId(REGION_ID).build();
222    Path tableDir = CommonFSUtils.getTableDir(rootDir, tn);
223    // persist table descriptor
224    FSTableDescriptors.createTableDescriptorForTableDirectory(fs, tableDir, td, true);
225    HRegion.createHRegion(conf, regionInfo, fs, tableDir, td).close();
226    Path initializedFlag = new Path(tableDir, INITIALIZED_FLAG);
227    if (!fs.mkdirs(initializedFlag)) {
228      throw new IOException("Can not touch initialized flag: " + initializedFlag);
229    }
230    Path initializingFlag = new Path(tableDir, INITIALIZING_FLAG);
231    if (!fs.delete(initializingFlag, true)) {
232      LOG.warn("failed to clean up initializing flag: " + initializingFlag);
233    }
234    WAL wal = createWAL(walFactory, walRoller, serverName, walFs, walRootDir, regionInfo);
235    return HRegion.openHRegionFromTableDir(conf, fs, tableDir, regionInfo, td, wal, null, null);
236  }
237
238  private static RegionInfo loadRegionInfo(FileSystem fs, Path tableDir) throws IOException {
239    // on branch-2, the RegionInfo.isEncodedRegionName will returns true for .initializing and
240    // .initialized, see HBASE-25368. Since RegionInfo is IA.Public, changing the implementation may
241    // raise compatibility concerns, so here we just skip them by our own.
242    Path regionDir = fs.listStatus(tableDir, p -> !p.getName().startsWith(".")
243      && RegionInfo.isEncodedRegionName(Bytes.toBytes(p.getName())))[0].getPath();
244    return HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
245  }
246
247  private static HRegion open(Configuration conf, TableDescriptor td, RegionInfo regionInfo,
248    FileSystem fs, Path rootDir, FileSystem walFs, Path walRootDir, WALFactory walFactory,
249    MasterRegionWALRoller walRoller, String serverName) throws IOException {
250    Path tableDir = CommonFSUtils.getTableDir(rootDir, td.getTableName());
251    Path walRegionDir = FSUtils.getRegionDirFromRootDir(walRootDir, regionInfo);
252    Path replayEditsDir = new Path(walRegionDir, REPLAY_EDITS_DIR);
253    if (!walFs.exists(replayEditsDir) && !walFs.mkdirs(replayEditsDir)) {
254      throw new IOException("Failed to create replay directory: " + replayEditsDir);
255    }
256
257    // Replay any WALs for the Master Region before opening it.
258    Path walsDir = new Path(walRootDir, HREGION_LOGDIR_NAME);
259    // In open(...), we expect that the WAL directory for the MasterRegion to already exist.
260    // This is in contrast to bootstrap() where we create the MasterRegion data and WAL dir.
261    // However, it's possible that users directly remove the WAL directory. We expect walsDir
262    // to always exist in normal situations, but we should guard against users changing the
263    // filesystem outside of HBase's line of sight.
264    if (walFs.exists(walsDir)) {
265      replayWALs(conf, walFs, walRootDir, walsDir, regionInfo, serverName, replayEditsDir);
266    } else {
267      LOG.error(
268        "UNEXPECTED: WAL directory for MasterRegion is missing." + " {} is unexpectedly missing.",
269        walsDir);
270    }
271
272    // Create a new WAL
273    WAL wal = createWAL(walFactory, walRoller, serverName, walFs, walRootDir, regionInfo);
274    conf.set(HRegion.SPECIAL_RECOVERED_EDITS_DIR,
275      replayEditsDir.makeQualified(walFs.getUri(), walFs.getWorkingDirectory()).toString());
276    // we do not do WAL splitting here so it is possible to have uncleanly closed WAL files, so we
277    // need to ignore EOFException.
278    conf.setBoolean(HRegion.RECOVERED_EDITS_IGNORE_EOF, true);
279    return HRegion.openHRegionFromTableDir(conf, fs, tableDir, regionInfo, td, wal, null, null);
280  }
281
282  private static void replayWALs(Configuration conf, FileSystem walFs, Path walRootDir,
283    Path walsDir, RegionInfo regionInfo, String serverName, Path replayEditsDir)
284    throws IOException {
285    for (FileStatus walDir : walFs.listStatus(walsDir)) {
286      if (!walDir.isDirectory()) {
287        continue;
288      }
289      if (walDir.getPath().getName().startsWith(serverName)) {
290        LOG.warn("This should not happen in real production as we have not created our WAL "
291          + "directory yet, ignore if you are running a local region related UT");
292      }
293      Path deadWALDir;
294      if (!walDir.getPath().getName().endsWith(DEAD_WAL_DIR_SUFFIX)) {
295        deadWALDir =
296          new Path(walDir.getPath().getParent(), walDir.getPath().getName() + DEAD_WAL_DIR_SUFFIX);
297        if (!walFs.rename(walDir.getPath(), deadWALDir)) {
298          throw new IOException("Can not rename " + walDir + " to " + deadWALDir
299            + " when recovering lease of proc store");
300        }
301        LOG.info("Renamed {} to {} as it is dead", walDir.getPath(), deadWALDir);
302      } else {
303        deadWALDir = walDir.getPath();
304        LOG.info("{} is already marked as dead", deadWALDir);
305      }
306      for (FileStatus walFile : walFs.listStatus(deadWALDir)) {
307        Path replayEditsFile = new Path(replayEditsDir, walFile.getPath().getName());
308        RecoverLeaseFSUtils.recoverFileLease(walFs, walFile.getPath(), conf);
309        if (!walFs.rename(walFile.getPath(), replayEditsFile)) {
310          throw new IOException("Can not rename " + walFile.getPath() + " to " + replayEditsFile
311            + " when recovering lease for local region");
312        }
313        LOG.info("Renamed {} to {}", walFile.getPath(), replayEditsFile);
314      }
315      LOG.info("Delete empty local region wal dir {}", deadWALDir);
316      walFs.delete(deadWALDir, true);
317    }
318  }
319
320  private static void tryMigrate(Configuration conf, FileSystem fs, Path tableDir,
321    RegionInfo regionInfo, TableDescriptor oldTd, TableDescriptor newTd) throws IOException {
322    Class<? extends StoreFileTracker> oldSft =
323      StoreFileTrackerFactory.getTrackerClass(oldTd.getValue(StoreFileTrackerFactory.TRACKER_IMPL));
324    Class<? extends StoreFileTracker> newSft =
325      StoreFileTrackerFactory.getTrackerClass(newTd.getValue(StoreFileTrackerFactory.TRACKER_IMPL));
326    if (oldSft.equals(newSft)) {
327      LOG.debug("old store file tracker {} is the same with new store file tracker, skip migration",
328        StoreFileTrackerFactory.getStoreFileTrackerName(oldSft));
329      if (!oldTd.equals(newTd)) {
330        // we may change other things such as adding a new family, so here we still need to persist
331        // the new table descriptor
332        LOG.info("Update table descriptor from {} to {}", oldTd, newTd);
333        FSTableDescriptors.createTableDescriptorForTableDirectory(fs, tableDir, newTd, true);
334      }
335      return;
336    }
337    LOG.info("Migrate store file tracker from {} to {}", oldSft.getSimpleName(),
338      newSft.getSimpleName());
339    HRegionFileSystem hfs =
340      HRegionFileSystem.openRegionFromFileSystem(conf, fs, tableDir, regionInfo, false);
341    for (ColumnFamilyDescriptor oldCfd : oldTd.getColumnFamilies()) {
342      StoreFileTracker oldTracker = StoreFileTrackerFactory.create(conf, oldTd, oldCfd, hfs);
343      StoreFileTracker newTracker = StoreFileTrackerFactory.create(conf, oldTd, oldCfd, hfs);
344      List<StoreFileInfo> files = oldTracker.load();
345      LOG.debug("Store file list for {}: {}", oldCfd.getNameAsString(), files);
346      newTracker.set(oldTracker.load());
347    }
348    // persist the new table descriptor after migration
349    LOG.info("Update table descriptor from {} to {}", oldTd, newTd);
350    FSTableDescriptors.createTableDescriptorForTableDirectory(fs, tableDir, newTd, true);
351  }
352
353  public static MasterRegion create(MasterRegionParams params) throws IOException {
354    TableDescriptor td = params.tableDescriptor();
355    LOG.info("Create or load local region for table " + td);
356    Server server = params.server();
357    Configuration baseConf = server.getConfiguration();
358    FileSystem fs = CommonFSUtils.getRootDirFileSystem(baseConf);
359    FileSystem walFs = CommonFSUtils.getWALFileSystem(baseConf);
360    Path globalRootDir = CommonFSUtils.getRootDir(baseConf);
361    Path globalWALRootDir = CommonFSUtils.getWALRootDir(baseConf);
362    Path rootDir = new Path(globalRootDir, params.regionDirName());
363    Path walRootDir = new Path(globalWALRootDir, params.regionDirName());
364    // we will override some configurations so create a new one.
365    Configuration conf = new Configuration(baseConf);
366    CommonFSUtils.setRootDir(conf, rootDir);
367    CommonFSUtils.setWALRootDir(conf, walRootDir);
368    MasterRegionFlusherAndCompactor.setupConf(conf, params.flushSize(), params.flushPerChanges(),
369      params.flushIntervalMs());
370    conf.setInt(AbstractFSWAL.MAX_LOGS, params.maxWals());
371    if (params.useHsync() != null) {
372      conf.setBoolean(HRegion.WAL_HSYNC_CONF_KEY, params.useHsync());
373    }
374    if (params.useMetaCellComparator() != null) {
375      conf.setBoolean(HRegion.USE_META_CELL_COMPARATOR, params.useMetaCellComparator());
376    }
377    conf.setInt(AbstractFSWAL.RING_BUFFER_SLOT_COUNT,
378      IntMath.ceilingPowerOfTwo(params.ringBufferSlotCount()));
379
380    MasterRegionWALRoller walRoller = MasterRegionWALRoller.create(
381      td.getTableName() + "-WAL-Roller", conf, server, walFs, walRootDir, globalWALRootDir,
382      params.archivedWalSuffix(), params.rollPeriodMs(), params.flushSize());
383    walRoller.start();
384
385    WALFactory walFactory = new WALFactory(conf, server.getServerName().toString());
386    Path tableDir = CommonFSUtils.getTableDir(rootDir, td.getTableName());
387    Path initializingFlag = new Path(tableDir, INITIALIZING_FLAG);
388    Path initializedFlag = new Path(tableDir, INITIALIZED_FLAG);
389    HRegion region;
390    if (!fs.exists(tableDir)) {
391      // bootstrap, no doubt
392      if (!fs.mkdirs(initializedFlag)) {
393        throw new IOException("Can not touch initialized flag");
394      }
395      region = bootstrap(conf, td, fs, rootDir, walFs, walRootDir, walFactory, walRoller,
396        server.getServerName().toString(), true);
397    } else {
398      if (!fs.exists(initializedFlag)) {
399        if (!fs.exists(initializingFlag)) {
400          // should be old style, where we do not have the initializing or initialized file, persist
401          // the table descriptor, touch the initialized flag and then open the region.
402          // the store file tracker must be DEFAULT
403          LOG.info("No {} or {} file, try upgrading", INITIALIZING_FLAG, INITIALIZED_FLAG);
404          TableDescriptor oldTd =
405            TableDescriptorBuilder.newBuilder(td).setValue(StoreFileTrackerFactory.TRACKER_IMPL,
406              StoreFileTrackerFactory.Trackers.DEFAULT.name()).build();
407          FSTableDescriptors.createTableDescriptorForTableDirectory(fs, tableDir, oldTd, true);
408          if (!fs.mkdirs(initializedFlag)) {
409            throw new IOException("Can not touch initialized flag: " + initializedFlag);
410          }
411          RegionInfo regionInfo = loadRegionInfo(fs, tableDir);
412          tryMigrate(conf, fs, tableDir, regionInfo, oldTd, td);
413          region = open(conf, td, regionInfo, fs, rootDir, walFs, walRootDir, walFactory, walRoller,
414            server.getServerName().toString());
415        } else {
416          // delete all contents besides the initializing flag, here we can make sure tableDir
417          // exists(unless someone delete it manually...), so we do not do null check here.
418          for (FileStatus status : fs.listStatus(tableDir)) {
419            if (!status.getPath().getName().equals(INITIALIZING_FLAG)) {
420              fs.delete(status.getPath(), true);
421            }
422          }
423          region = bootstrap(conf, td, fs, rootDir, walFs, walRootDir, walFactory, walRoller,
424            server.getServerName().toString(), false);
425        }
426      } else {
427        if (fs.exists(initializingFlag) && !fs.delete(initializingFlag, true)) {
428          LOG.warn("failed to clean up initializing flag: " + initializingFlag);
429        }
430        // open it, make sure to load the table descriptor from fs
431        TableDescriptor oldTd = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir);
432        RegionInfo regionInfo = loadRegionInfo(fs, tableDir);
433        tryMigrate(conf, fs, tableDir, regionInfo, oldTd, td);
434        region = open(conf, td, regionInfo, fs, rootDir, walFs, walRootDir, walFactory, walRoller,
435          server.getServerName().toString());
436      }
437    }
438
439    Path globalArchiveDir = HFileArchiveUtil.getArchivePath(baseConf);
440    MasterRegionFlusherAndCompactor flusherAndCompactor = new MasterRegionFlusherAndCompactor(conf,
441      server, region, params.flushSize(), params.flushPerChanges(), params.flushIntervalMs(),
442      params.compactMin(), globalArchiveDir, params.archivedHFileSuffix());
443    walRoller.setFlusherAndCompactor(flusherAndCompactor);
444    Path archiveDir = HFileArchiveUtil.getArchivePath(conf);
445    if (!fs.mkdirs(archiveDir)) {
446      LOG.warn("Failed to create archive directory {}. Usually this should not happen but it will"
447        + " be created again when we actually archive the hfiles later, so continue", archiveDir);
448    }
449    return new MasterRegion(region, walFactory, flusherAndCompactor, walRoller);
450  }
451}