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 * --<region dir> 073 * | 074 * --data 075 * | | 076 * | --/<ns>/<table>/<encoded-region-name> <---- The region data 077 * | | 078 * | --replay <---- The edits to replay 079 * | 080 * --WALs 081 * | 082 * --<master-server-name> <---- The WAL dir for active master 083 * | 084 * --<master-server-name>-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}