导语
1.Put、Delete、Increment如何基于行锁以及mvcc实现事务以及事务的可见性?
2.Get、Scan如何实现高效的查询?
3.Flush、Split、Compact如何实现Region Store的管理?
4.LoadBalancer、RegionSplitPolicy、CompactionPolicy各策略算法实现? //TODO:
5.CombinedBlockCache(LRUBlockCache + BucketCache)实现? //TODO:
以下基于hbase1.2,目前已经出了2.0版本,2.0具体细节没跟过,但是基本思路不变,优化了offheap、AssignmentManager(不再依赖zk做state数据交换,感觉这个思路有点像kafka的GroupCoordinator的思路,启动另一个管理节点,然后通过接口将结果分发到各工作节点,而尽量只用zk做小范围的一致性协调,减小数据写入,毕竟zk的写入吞吐量有限)等优化
一.Index
1.1 Put & Delete
org.apache.hadoop.hbase.regionserver.HRegion.put
->startRegionOperation(Operation.PUT); //read lock,guard closes
->org.apache.hadoop.hbase.regionserver.HRegion.doBatchMutate
->org.apache.hadoop.hbase.regionserver.HRegion.batchMutate
->doPreMutationHook(batchOp); //hook
->HRegion.doMiniBatchMutation
->RowLock rowLock = getRowLock(mutation.getRow(), true);//获取行锁的读锁,HRegion.lockedRows保存了各个row的读写锁,put、delete由于并非CAS操作,不需要考虑之前的状态,所以获取读锁就可以了,而checkAndMutate、Increment这类CAS操作就需要获取写锁了
->updateCellTimestamps(familyMaps[i].values(), byteNow);// Update any LATEST_TIMESTAMP timestamps
->lock(this.updatesLock.readLock(), numReadyToWrite);//获取HRegion的update读锁,flush是会对此锁加写锁,暂停memorystore的更新
->Durability tmpDur = getEffectiveDurability(m.getDurability());//获取持久化策略
->HRegion.addFamilyMapToWALEdit(familyMaps[i], walEdit);
->walKey = new HLogKey(this.getRegionInfo().getEncodedNameAsBytes(),this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, now,mutation.getClusterIds(), currentNonceGroup, currentNonce, mvcc)
->txid = this.wal.append(this.htableDescriptor, this.getRegionInfo(), walKey, walEdit, true);//将此批次数据加入追加到wal中,等待disruptor处理,并分配region级别seqID
->FSWALEntry.stampRegionSequenceId
->getKey().getMvcc().begin().getWriteNumber(); //开启mvcc事务,自增region级别的writePoint并将WriteEntry加入MultiVersionConcurrencyControl.writeQueue,等待本次事务完成时调用MultiVersionConcurrencyControl.complete时将queue已完成的WriteEntry移除,同时后移readPoint
->CellUtil.setSequenceId(c, regionSequenceId);//cell写入mvcc ID
->WALKey.seqNumAssignedLatch.countDown() //释放阻塞,即此时已分配mvcc作为SequenceId
->writeEntry = walKey.getWriteEntry(); //等待上部wal分配其SequenceId
->mvccNum = writeEntry.getWriteNumber() //同上
->HRegion.applyFamilyMapToMemstore //加入Memstore,此时还未syn日志
->CellUtil.setSequenceId(cell, mvccNum)
->HStore.add(cell)//加入MemStore
->this.updatesLock.readLock().unlock();//释放updatesLock锁,允许flush
->releaseRowLocks(acquiredRowLocks)//释放行锁
->HRegion.syncOrDefer(txid, durability)// Sync wal.
->this.wal.sync(txid);
->mvcc.completeAndWait(writeEntry);//更新mvcc,并等待readPoint到达此次写入的点
->isFlushSize(newSize) requestFlush();//插件是否符合flush条件
->closeRegionOperation(Operation.PUT);//realse lock
put、delete思路基本一样
1.2 Increment
org.apache.hadoop.hbase.regionserver.HRegion.increment
->HRegion.doIncrement
->rowLock = getRowLock(increment.getRow()); //加入写锁
->this.mvcc.await(); //等待已有mvcc完成以保证我们可以拿到截止现在的所有的稳定状态(此处直接region mvcc等待,按道理想快的话应该加列锁?)
->HRegion.applyIncrementsToColumnFamily
->List currentValues = getIncrementCurrentValue(increment, columnFamilyName, sortedIncrements, isolation);//获取当前值
->incrementAmount += getLongValue(currentValue);//加值
->CellUtil.setSequenceId(newValue, mvccNum);//更新mvcc
->walKey = new HLogKey(this.getRegionInfo().getEncodedNameAsBytes(),this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, nonceGroup, nonce,getMVCC());//写入WAL
->this.wal.append(this.htableDescriptor, this.getRegionInfo(), walKey, walEdits, true);
->walKey.getWriteEntry();//等待分配sequence id
->CellUtil.setSequenceId(cell, walKey.getWriteEntry().getWriteNumber());//cell更新seqID
->store.add(cell)//更新到store中
->syncOrDefer(txid, durability);
->rowLock.release();//释放写锁
|
1.3 Get & Scan
org.apache.hadoop.hbase.regionserver.HRegion.get
->Scan scan = new Scan(get); //scan与get都采用scan,get的startRow与stopRow相同
->HRegion.getScanner(scan, null)
->startRegionOperation(Operation.SCAN);//同上,guard closes
->new RegionScannerImpl(scan, additionalScanners, this)
->KeyValueScanner scanner = HStore.getScanner(scan, entry.getValue(), this.readPt);//基于Family以及readpoint构建scanner
->HStore.lock.readLock().lock();//store级别的读锁,此锁只有close以及compact在切换阶段才会write
->HStore.createScanner(scan, targetCols, readPt, scanner)
->StoreScanner.StoreScanner(this,getScanInfo(), scan, targetCols, readPt)//构造函数
->matcher = new ScanQueryMatcher(scan, scanInfo, columns,ScanType.USER_SCAN, Long.MAX_VALUE, HConstants.LATEST_TIMESTAMP,oldestUnexpiredTS, now, store.getCoprocessorHost());//构建ScanQueryMatcher,后面用来对scan进行引导
->TimeRange timeRange = scan.getColumnFamilyTimeRange().get(scanInfo.getFamily());//构建TimeRange
->this.deletes = new ScanDeleteTracker(); //构建DeleteTracker
->this.filter = scan.getFilter();
->this.columns = new ExplicitColumnTracker(columns, scanInfo.getMinVersions(), maxVersions,oldestUnexpiredTS)//这里用指定Column的查询方式的ColumnTracker
->StoreScanner.getScannersNoCompaction()
->HStore.getScanners
-> storeFilesToScan = this.storeEngine.getStoreFileManager().getFilesForScanOrGet(isGet, startRow, stopRow)
-> memStoreScanners = this.memstore.getScanners(readPt);
->StoreFileScanner.getScannersForStoreFiles(storeFilesToScan,cacheBlocks, usePread, isCompaction, false, matcher, readPt, isPrimaryReplicaStore())
->StoreFile.createReader(canUseDrop)
->StoreFile.open() //加载hfile的Trailer以及Opening-time data section,并保存StoreFile对象中
->org.apache.hadoop.hbase.regionserver.StoreFile.Reader.getStoreFileScanner
->new StoreFileScanner(this, getScanner(cacheBlocks, pread, isCompaction), !isCompaction, reader.hasMVCCInfo(),readPt, scannerOrder)
->scanner.setScanQueryMatcher(matcher)
->scanners.addAll(sfScanners);scanners.addAll(memStoreScanners);//mem以及hfile的scanner都加入list
->StoreScanner.selectScannersFrom(allScanners)//提前过滤掉本次scan不会涉及的scanner,调用各scanner的shouldUseScanner,这里以StoreFileScanner为例;基本思路就是基于Bloom filter, time range, and TTL三方面过滤
->StoreFileScanner.shouldUseScanner
->org.apache.hadoop.hbase.regionserver.StoreFile.Reader.passesTimerangeFilter //Timerange
->org.apache.hadoop.hbase.regionserver.StoreFile.Reader.passesKeyRangeFilter //rowkey range
->org.apache.hadoop.hbase.regionserver.StoreFile.Reader.passesBloomFilter
->org.apache.hadoop.hbase.regionserver.StoreFile.Reader.passesGeneralBloomFilter //BloomFilter
-> bloom = HFile.Reader.getMetaBlock(HFile.BLOOM_FILTER_DATA_KEY
->org.apache.hadoop.hbase.util.CompoundBloomFilter.contains(key, 0, key.length, bloom)
->bloomBlock = reader.readBlock(index.getRootBlockOffset(block),index.getRootBlockDataSize(block), true, true, false, true,BlockType.BLOOM_CHUNK, null)
->ByteBloomFilter.contains(key, keyOffset, keyLength,bloomBuf, bloomBlock.headerSize(),bloomBlock.getUncompressedSizeWithoutHeader(), hash, hashCount)//基于BloomFilter过滤
->org.apache.hadoop.hbase.regionserver.StoreScanner.seekScanners //seek所有的scanner到相应的位置,这里涉及lazyseek会不执行真正的seek,仅仅设置虚拟seek位置的,但是不影响堆排序
->org.apache.hadoop.hbase.regionserver.StoreFileScanner.requestSeek //设置StoreFileScanner.cur 指示当前cell
->reader.passesGeneralBloomFilter()//首先检查Cell是否包含在hfile中,如果不包含,就不需要seek此hfile,直接设置为本Column的最大的位置
->setCurrentCell(KeyValueUtil.createLastOnRowCol(kv));
->if (seekTimestamp > maxTimestampInFile) setCurrentCell(KeyValueUtil.createFirstOnRowColTS(kv, maxTimestampInFile)); //同理,如果scan Cell 规定了Timestamp,而本file最晚更新时间都低于Timestamp,说明没有符合条件的Column,则直接设置到高位置
->setCurrentCell(KeyValueUtil.createFirstOnRowColTS(kv, maxTimestampInFile));
->org.apache.hadoop.hbase.regionserver.StoreFileScanner.enforceSeek
->StoreFileScanner.seek
->StoreFileScanner.seekAtOrAfter //seek到当前位置或是向后一个位置
->setCurrentCell(hfs.getKeyValue()); //设置StoreFileScanner.cur
->StoreScanner.resetKVHeap //Combine all seeked scanners with a heap
->StoreScanner.heap = new KeyValueHeap(scanners, comparator)
->KeyValueHeap.add(scanner);
->KeyValueHeap.current = pollRealKV();
->this.current = KeyValueHeap.pollRealKV() //弹出堆顶Scanner作为当前head的current,如果堆顶Scanner没有real seek,则enforceSeek
->//如果堆顶已经realSeekDone,则直接返回
->!kvScanner.realSeekDone() //如果没有real seek
->kvScanner.enforceSeek();//首先强制seek,此时current等于或是落到下一个位置上
->Cell curKV = kvScanner.peek();//获取当前元素
->KeyValueScanner nextEarliestScanner = heap.peek()//获取第二大的scanner
->Cell nextKV = nextEarliestScanner.peek();
->comparator.compare(curKV, nextKV) < 0 {return kvScanner;};//如果当前节点的cell的下一个节点(其实是堆顶节点的下一个节点,即走了2步,因为先enforceseek了,cur一定落到了真实节点上)小于下一Scanner的下一个真实节点,则不再放回heap,直接继续用,因为可以保证小顶堆的堆顶scanner依然是下次要使用的scanner,否则加入heap继续参与竞争
->comparator.compare(curKV, nextKV) >= 0 {heap.add(kvScanner) return kvScanner}; //不能确保当前堆顶Scanner依然是下次最小的,则加入head,继续参与竞争
->HStore.lock.readLock().unlock();
->RegionScannerImpl.initializeKVHeap(scanners, joinedScanners, region); //初始化region级别的heap,元素为各个store
->this.storeHeap = new KeyValueHeap(scanners, region.comparator);//同上,选出堆顶scanner
->closeRegionOperation(Operation.SCAN);
->org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl.next(outResults, defaultScannerContext) //由于GET为单行,此处不基于返回值循环获取更多row
->RegionScannerImpl.nextRaw(outResults, scannerContext);
->RegionScannerImpl.nextInternal
->while (true)
->Cell current = this.storeHeap.peek();//堆顶元素
->RegionScannerImpl.populateResult(results, this.storeHeap, scannerContext, currentRow, offset, length);//抓取本row的所有元素知道达到指定条件
->do {//循环抓取各family的cell
->heap.next(results, scannerContext); //store级别heap
->InternalScanner currentAsInternal = (InternalScanner) this.current; //栈顶store
->boolean moreCells = currentAsInternal.next(result, scannerContext); //搜索当前store的下个元素
->StoreScanner.next(result, scannerContext)
->Cell cell = this.heap.peek() //获取此storescanner下一元素
->ScanQueryMatcher.setRow(row, offset, length);
->LOOP: do { //进入循环取当前family(heap)中的cell
->ScanQueryMatcher.MatchCode qcode = matcher.match(cell); //ScanQueryMatcher其实是针对同一row的同一family的
->if (filter.filterAllRemaining() MatchCode.DONE_SCAN) //过滤器停止扫描的机会
->this.rowComparator.compareRows(row, this.rowOffset, this.rowLength,cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()) <=-1 {return MatchCode.SEEK_NEXT_ROW;} //如果peek的cell大于row了,说明本row完事了,SEEK_NEXT_ROW
->if (this.columns.done()) {MatchCode.SEEK_NEXT_ROW} //检查columns是否满足要求了
->ExplicitColumnTracker.done //以精确设置Column为例,this.index >= columns.length则表示抓取的列够了
->columns.isDone(timestamp) //基于timestamp检查是否达到阈值
->ExplicitColumnTracker.doneWithColumn(bytes, offset,qualLength);//本column抓取完成
->if (getColumnHint() == null) {return MatchCode.SEEK_NEXT_ROW;} else {return MatchCode.SEEK_NEXT_COL;}//还有没抓取完的列,则SEEK_NEXT_COL,否则SEEK_NEXT_ROW
->if (HStore.isCellTTLExpired(cell, this.oldestUnexpiredTS, this.now)) {return MatchCode.SKIP;} //检查当前cell TTL是否过期,过期直接skip
->if (CellUtil.isDelete(cell)) //如果delete类型,则加入DeleteTracker中用于后续cell的过滤
->this.deletes.add(cell);
->if (!this.deletes.isEmpty()) //delete不为空,则此cell检查delete逻辑
->DeleteResult deleteResult = deletes.isDeleted(cell);
->FAMILY_DELETED || COLUMN_DELETED columns.getNextRowOrNextColumn(cell.getQualifierArray(),qualifierOffset, qualifierLength) //famliy或是COLUMN_DELETED则完成本colum抓取,调到下一colum或是row
->VERSION_DELETED || FAMILY_VERSION_DELETED return MatchCode.SKIP;//删除特定版本,则只需要跳过当前CELL就可以了,不影响其他的cell
->TimeRange.compare(timestamp); //检查TimeRange,不符合条件则过滤或是跳过到下一列或是下一行
->filterResponse = filter.filterKeyValue(cell) //filter过滤本cell
->columns.checkVersions(cell.getQualifierArray(), qualifierOffset,qualifierLength, timestamp, typeByte,mvccVersion > maxReadPointToTrackVersions) //检查抓取的最大版本数量
->int count = this.column.increment();
->if (count >= maxVersions || (count >= minVersions && isExpired(timestamp))) //检查当前column最大version是否满足要求
->if (done()) {return ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_ROW;} //如果所有列完成,则转到下一行
->this.column = this.columns[this.index]; return ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_COL; //当前列完成,更新ColumnTracker到下一列,并转到下一列抓取
->switch(qcode):
->INCLUDE
->cell = Filter.transformCell(cell); //给filter一次转化cell的机会
->Case
->SEEK_NEXT_ROW->seekToNextRow(cell);
->reseek(KeyValueUtil.createLastOnRow(kv));//seek到本row的最尾
->SEEK_NEXT_COL->seekAsDirection(matcher.getKeyForNextColumn(cell));
->seekAsDirection(matcher.getKeyForNextColumn(cell))//seek到需要搜索的下一column的头
->this.heap.next();//普通的迭代到当前column的下一cell的情况系,使下一scanner赋值为current
->KeyValueHeap.next
->Cell kvReturn = this.current.next();当前堆顶Scanner迭代到下一元素
->Cell kvNext = this.current.peek(); //取出元素
->if (kvNext == null) {this.current.close(); this.current = null; this.current = pollRealKV();} //如果堆顶scanner没有剩余元素,则关闭此scanner,并重置current scanner
->else //否则
->KeyValueScanner topScanner = this.heap.peek();取出当前堆顶scanner
->if (topScanner != null && this.comparator.compare(kvNext, topScanner.peek()) >= 0) {this.heap.add(this.current);this.current = pollRealKV();}{//如果current比堆顶scanner还要小,则可以不参与堆重新排序,继续next,因为下次next必然不涉及heap中的元素,否则,继续加入heap参与竞争堆顶
->while((cell = this.heap.peek()) != null)
->Cell pee = this.current.peek();
->this.heap.add(this.current);
->this.current = pollRealKV();//重置堆顶,设置current
->nextKv = heap.peek();
->moreCellsInRow = moreCellsInRow(nextKv, currentRow, offset, length); //基于nextkv以及currentRow是否依然是同一row,判断是否还有更多的cell(其实是family)读取
->} while (moreCellsInRow);
->Cell nextKv = this.storeHeap.peek();//取堆顶下个rowkey判断是否到达stoprow
->if (hasFilterRow) {filter.filterRowCellsWithRet(results) } //对row查询结果走过滤器方法
->this.filter.filterRowCells(kvs); //调用filter的filterRowCells方法过滤cells
->this.filter.filterRow() //决定是否过滤整个row
->if (stopRow) {scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();} else {scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();} //判断是否需要继续下一row
->scanner.close();
get、scan思路也基本一样,hbase为基于row的查询
1.4 Flush
org.apache.hadoop.hbase.regionserver.HRegion.requestFlush
->MemStoreFlusher.flushRegion
->HRegion.flush->HRegion.flushcache
->Collection specificStoresToFlush = forceFlushAllStores ? stores.values() : flushPolicy.selectStoresToFlush();//基于flushPolicy选择需要flush的store
->HRegion.internalFlushcache
->HRegion.internalPrepareFlushCache
->this.updatesLock.writeLock().lock();//HRegion更新写锁,防止多个flush同时操作
->MultiVersionConcurrencyControl.WriteEntry writeEntry = mvcc.begin(); //开启mvcc事务
->FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.START_FLUSH,getRegionInfo(), flushOpSeqId, committedFiles)
->trxId = WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(),desc, false, mvcc)//将flush描述写入wal
->org.apache.hadoop.hbase.regionserver.HStore.StoreFlusherImpl.prepare
->this.snapshot = memstore.snapshot();
->org.apache.hadoop.hbase.regionserver.DefaultMemStore.snapshot()//生成memstore的snapshot
->this.snapshot = this.cellSet;//将memstore中的数据赋值给snapshot
->this.cellSet = new CellSkipListSet(this.comparator);//cellSet初始化一个新的
->MemStoreSnapshot memStoreSnapshot = new MemStoreSnapshot(this.snapshotId, snapshot.size(), this.snapshotSize,this.snapshotTimeRangeTracker, new CollectionBackedScanner(snapshot, this.comparator),this.tagsPresent) //初始化memStore
->this.updatesLock.writeLock().unlock();//释放HRegion更新写锁
->wal.sync(); //同步wal
->mvcc.completeAndWait(writeEntry);//等待writeEntry之前的事务完成,防止hfile刷写一半事务
->HRegion.internalFlushCacheAndCommit
->HStore.flushCache //snapshot刷写到一个临时目录
->org.apache.hadoop.hbase.regionserver.DefaultStoreFlusher.flushSnapshot
->InternalScanner scanner = createScanner(snapshot.getScanner(), smallestReadPoint);//基于snapshot创建scanner
->writer = store.createWriterInTmp(cellsCount, store.getFamily().getCompression(),false,true,snapshot.isTagsPresent(),false);
->StoreFlusher.performFlush//执行flush
->finalizeWriter(writer, cacheFlushId, status);//刷入Metadata
->StoreFlusherImpl.commit
->HStore.commitFile
->HRegionFileSystem.commitStoreFile //将文件从临时目录变更到正式目录
->HStore.createStoreFileAndReader //初始化storefile跟reader
->StoreFile storeFile = new StoreFile(this.getFileSystem(), info, this.conf, this.cacheConf,this.family.getBloomFilterType());
->StoreFile.Reader r = storeFile.createReader();
->HStore.this.updateStorefiles(storeFiles, snapshot.getId())
->this.lock.writeLock().lock();//hstore写锁,阻塞读取与写入
->this.storeEngine.getStoreFileManager().insertNewFiles(sfs);//将刚刷写的hfile挂到store上
->newFiles.addAll(sfs);storefiles = ImmutableList.copyOf(storeFiles);
->this.memstore.clearSnapshot(snapshotId);//清理memstore中的Snapshot,释放内存
->this.lock.writeLock().unlock();//释放写锁
->needsCompaction//检查是否需要compact
->if (shouldSplit) {this.server.compactSplitThread.requestSplit(region);} //检查flush
->if (shouldCompact) {server.compactSplitThread.requestSystemCompaction(region, Thread.currentThread().getName());} //检查compact
1.4 split
org.apache.hadoop.hbase.regionserver.RSRpcServices.splitRegion
->HRegion.checkSplit
->splitPolicy.getSplitPoint();//基于RegionSplitPolicy计算分割点
->org.apache.hadoop.hbase.regionserver.CompactSplitThread.requestSplit
->org.apache.hadoop.hbase.regionserver.SplitRequest.doSplitting
->tableLock = server.getTableLockManager().readLock(parent.getTableDesc().getTableName(), "SPLIT_REGION:" + parent.getRegionInfo().getRegionNameAsString()) tableLock.acquire(); //获取基于zk实现的锁
->SplitTransactionImpl.prepare
->long rid = getDaughterRegionIdTimestamp(hri);
->this.hri_a = new HRegionInfo(hri.getTable(), startKey, this.splitrow, false, rid);
->this.hri_b = new HRegionInfo(hri.getTable(), this.splitrow, endKey, false, rid);
->transition(SplitTransactionPhase.PREPARED);
->SplitTransactionImpl.execute //执行split
->SplitTransactionImpl.createDaughters //主要是flush mem,下线parent,生成Daughter的reference并创建HRegion对象
->SplitTransactionImpl.stepsBeforePONR
->server.getCoordinatedStateManager()).getSplitTransactionCoordination().startSplitTransaction(parent, server.getServerName(),hri_a, hri_b);//创建PENDING_SPLIT的临时节点
->server.getCoordinatedStateManager()).getSplitTransactionCoordination().waitForSplitTransaction(services, parent, hri_a,hri_b, std)//等待hmaster响应pending_split
->this.parent.getRegionFileSystem().createSplitsDir(); //创建.splits目录
->HRegion.close //下线parent hregion,客户端请求会抛出NotServingRegionException,并自动重试
->HRegion.doClose
->lock.writeLock().lock() //close写锁避免数据更新或是其他close操作
->this.closing.set(true);
->internalFlushcache(status); //先将memsotre刷写为hfile
->HStore.close //关闭Hstore,关闭其StoreFile的reader
->writeRegionCloseMarker(wal) //Writing region close event to WAL
->this.closed.set(true);
->lock.writeLock().unlock();
->services.removeFromOnlineRegions(this.parent, null); //讲region从regionserver.onlineRegions中移除
->SplitTransactionImpl.splitStoreFiles //创建hfile的reference文件(此处可以理解为类似软引用,并不copy数据,所以性能很快)
->SplitTransactionImpl.splitStoreFile
->HRegionFileSystem.splitStoreFile
->Region a = this.parent.createDaughterRegionFromSplits(this.hri_a);
->HRegionFileSystem.commitDaughterRegion //讲文件移动到正式目录下
->HRegion r = HRegion.newHRegion(this.fs.getTableDir(), this.getWAL(), fs.getFileSystem(),this.getBaseConf(), hri, this.getTableDesc(), rsServices);//创建region对象
->Region b = this.parent.createDaughterRegionFromSplits(this.hri_b) //同上
->transition(SplitTransactionPhase.PONR);
->SplitTransactionImpl.offlineParentInMetaAndputMetaEntries //更新meta表,更新parent meta并添加Daughter region信息
->MetaTableAccessor.addDaughtersToPut(putParent, splitA, splitB)
->Put putA = MetaTableAccessor.makePutFromRegionInfo(splitA);Put putB = MetaTableAccessor.makePutFromRegionInfo(splitB);
->MetaTableAccessor.mutateMetaTable(hConnection, mutations);
->SplitTransactionImpl.stepsAfterPONR
->SplitTransactionImpl.openDaughters //开启Daughter
->SplitTransactionImpl.openDaughterRegion
->HRegion.openHRegion
->HRegionServer.addToOnlineRegions //将region加入HRegionServer.onlineRegions
->server.getCoordinatedStateManager()).getSplitTransactionCoordination().completeSplitTransaction(services, regions.getFirst(),regions.getSecond(), std, parent) //更新zk状态完成split,此处的region虽然分裂了,但是依然还是同一个regionserver上,hmaster会有balancer将其迁移到其他regionserver上
->transition(SplitTransactionPhase.COMPLETED);
https://zh.hortonworks.com/blog/apache-hbase-region-splitting-and-merging/
merge为split逆操作
1.5 Compact
org.apache.hadoop.hbase.regionserver.CompactSplitThread.CompactionRunner.doCompaction
->CompactSplitThread.selectCompaction //基于策略选择需要合并的file
->HRegion.compact //执行compact
->HStore.compact
->DefaultCompactionContext.compact
->DefaultCompactor.compact
->FileDetails fd = getFileDetails(request.getFiles(), request.isAllFiles());//提取一些需要合并的file文件的meta,因为这些文件保存成单一hfile时要用
->scanner = createScanner(store, scanners, scanType, smallestReadPoint, fd.earliestPutTs);//基于这些file创建scanner
->writer = createTmpWriter(fd, store.throttleCompaction(request.getSize()));//创建新文件的writer
->org.apache.hadoop.hbase.regionserver.compactions.Compactor.performCompaction
->do {
-> hasMore = scanner.next(cells, scannerContext); //遍历数据写入文件中
->CellUtil.setSequenceId(c, 0);
->writer.append(c);
->while (hasMore);
-> writer.appendMetadata(fd.maxSeqId, request.isAllFiles());//写入meta
->HStore.moveCompatedFilesIntoPlace //移动到正式目录
->HStore.writeCompactionWalRecord //写入wal
->HStore.replaceStoreFiles //把compact过的file替换为新的,注意此处加的是写锁,即交换瞬间阻塞hstore的读写
->this.lock.writeLock().lock();
->this.storeEngine.getStoreFileManager().addCompactionResults(compactedFiles, result);
->this.lock.writeLock().unlock();
->HStore.completeCompaction //关闭compactedFile reader,archive历史hfile
->compactedFile.closeReader(evictOnClose);
->this.fs.removeStoreFiles(this.getColumnFamilyName(), compactedFiles);
->this.storeSize += r.length();
->requestSplit(region)//检查是否需要split
二.reference
- http://hbasefly.com/category/hbase/
- https://blog.csdn.net/map_lixiupeng/article/details/40861791/
- HBase权威指南
三.下篇
APACHE ZOOKEEPER 3.5.3 CODE REVIEW