基于HBase trunk代码http://svn.apache.org/repos/asf/hbase/trunk,Revision: 1431176
Create table 的过程,Create table时 只有一个region 的assign 过程。 图示
Client HMaster HRegionServer
Client 发送请求到HMaster (HBaseAdmin)
HMaster收到请求
创建CreateTableHandler,CreateTableHandler的构造函数
里面会
(1)调用
MetaReader.tableExists(CatalogTracker catalogTracker,String tableName)来
扫描META表,检查table是否已存在
(2)在ZK中设置table的状态是ZooKeeperProtos.Table.State.ENABLING
/hbase/table/tablename
CreateTableHandler构造好了之后放入线程池(ExecutorService),
由线程池异步去执行
构造好CreateTableResponse,返回client
Client收到CreateTableResponse,
然后就wait,知道table创建好
HMaster开始执行CreateTableHandler,调用CreateTableHandler.process
(1)在HDFS上面创建Table的目录,并写上HTableDescriptor(文件.tableinfo)
(2) 创建HRegion,HRegion.createHRegion
(3) Close HRegion,region.close();
(4) RegionInfo信息往META表里面put
MetaEditor.addRegionsToMeta(this.catalogTracker, regionInfos);
//这里只有 info:regioninfo 这个cq,
没有info:server、info:serverstartcode等其他cq。
这里region的location是null,在AssignmentManager的rebuildUserRegions
里面有对这块(region的location为null时)单独进行处理。
(5) 在RegionStates里面为每个Region创建一个RegionState,状态为State.OFFLINE
(6) assign region,调用 AssignmentManager.assign(List<HRegionInfo> regions)
assign region出去
(1) 调用LoadBalancer进行Round Robin分配,给每个region分配一个Server,生成分配计划
(2)new 一个 GeneralBulkAssigner 按照生成的分配计划把region assign 出去
BulkAssigner ba = new GeneralBulkAssigner(this.server, bulkPlan, this);
ba.bulkAssign();
BulkAssigner.bulkAssign()
BulkAssigner.bulkAssign(boolean sync)
GeneralBulkAssigner.populatePool 具体调用SingleServerBulkAssigner来完成
GeneralBulkAssigner.waitUntilDone
这个的分配计划是Map<ServerName, List<HRegionInfo>>,Server对应一些HRegion,
GeneralBulkAssigner执行时每一个Server对应一个SingleServerBulkAssigner,由线程池去执行
SingleServerBulkAssigner,而自己只需要waitUntilDone
这里GeneralBulkAssigner的waitUntilDone条件是:
所有需要分配的Region在RegionStates中的状态满足
【1】在regionsInTransition没有这个region
【2】在regionAssignments中有这个region
SingleServerBulkAssigner的执行
SingleServerBulkAssigner.run
assignmentManager.assign(regionserver, regions)
AssignmentManager.assign(final ServerName destination,
final List<HRegionInfo> regions)的执行
(1)asyncSetOfflineInZooKeeper
(1.1)regionStates中更新Region的状态,从OFFLINE->OFFLINE
(1.2)ZK中/hbase/unassigned下面异步创建Region对应的节点,
数据RegionTransition是EventType.M_ZK_REGION_OFFLINE
其中有注册回调函数OfflineCallback
(1.3)每个Region创建一个RegionPlan
HMaster收到NodeChildrenChanged事件,path是/hbase/unassigned
触发回调函数OfflineCallback以及OfflineCallback.ExistCallback
这样HMaster就能往下执行了
// Wait until all unassigned nodes have been put up and watchers set.
把RegionPlan put到regionPlans中
更新Region的状态,从OFFLINE->PENDING_OPEN,ServerName不为null,同时会放入regionsInTransition
HMaster向HRegionServer发送rpc请求(open regions)
HRegionServer收到open region请求
创建OpenRegionHandler,丢到线程池(ExecutorService),
由线程池运行
返回HMaster Response,RegionOpeningState.OPENED
HMaster收到HRegionServer端的Response
AssignmentManager.assign(final ServerName destination,
final List<HRegionInfo> regions) 结束
SingleServerBulkAssigner.run结束
但是GeneralBulkAssigner还没有结束,还在运行waitUntilDone
HRegionServer运行OpenRegionHandler
(1)第一步修改zk上面region节点的信息,
M_ZK_REGION_OFFLINE->RS_ZK_REGION_OPENING
transitionZookeeperOfflineToOpening(encodedName,
versionOfOfflineNode)
因为上面HRegionServer修改了ZK节点的信息,所以HMaster会收到 HRegionServer open region,HRegion region = openRegion();
ZK的事件NodeDataChanged,path是 HRegion r = HRegion.newHRegion
/hbase/unassigned/98ab07ecd6bbfa1e3a170092f3fc4de6 r.openHRegion(reporter);
HMaster处理zk事件 HRegion.initialize
更新Region的状态,PENDING_OPEN->OPENING,ServerName不为null HRegion.initializeRegionInternals
创建 .regioninfo 文件
初始化HStore
HRegion open完成
OpenRegionHandler 接着处理HRegion open完后的逻辑
tickleOpening("post_region_open")
retransitionNodeOpening
RS_ZK_REGION_OPENING->RS_ZK_REGION_OPENING
HMaster收到ZK事件,NodeDataChanged,path是 执行OpenRegionHandler里面的updateMeta
/hbase/unassigned/98ab07ecd6bbfa1e3a170092f3fc4de6 启动线程PostOpenDeployTasksThread,并start运行
HMaster处理zk事件 PostOpenDeployTasksThread运行HRegionServer.postOpenDeployTasks
更新Region的状态,OPENING->OPENING,ServerName不为null HRegionServer: 更新META表,修改Region的Location信息
(Location原先是没有的)。
OpenRegionHandler 接着处理,修改zk中region的状态为
RS_ZK_REGION_OPENED
HMaster收到ZK事件,NodeDataChanged,path是 添加Region到OnlineRegions列表
/hbase/unassigned/98ab07ecd6bbfa1e3a170092f3fc4de6 OpenRegionHandler处理完成
事件是RS_ZK_REGION_OPENED
更新Region的状态,OPENING->OPEN,ServerName不为null
同时更新regionsInTransition中Region的状态为OPEN
HMaster创建OpenedRegionHandler,并且丢到线程池去运行
HMaster运行OpenedRegionHandler
OpenedRegionHandler主要是删除Region的ZK节点
/hbase/unassigned/98ab07ecd6bbfa1e3a170092f3fc4de6
HMaster收到ZK事件,NodeDeleted,path是
/hbase/unassigned/98ab07ecd6bbfa1e3a170092f3fc4de6
HMaster收到ZK事件,NodeChildrenChanged,path是
/hbase/unassigned
HMaster处理nodeDeleted事件
运行AssignmentManager.nodeDeleted
从regionsInTransition中获得Region的state状态
AssignmentManager.regionOnline
(1) RegionStates.regionOnline,
更新Region的状态,从OPEN->OPEN
从regionsInTransition中删除region的state信息
往regionAssignments中put数据
(2)AssignmentManager.clearRegionPlan
从regionPlans中删除region的RegionPlan
(3)AssignmentManager.addToServersInUpdatingTimer
添加ServerName到serversInUpdatingTimer中
GeneralBulkAssigner的waitUntilDone可以结束了,返回到
AssignmentManager.assign(List<HRegionInfo> regions)
AssignmentManager.assign结束,返回到
CreateTableHandler.handleCreateTable
CreateTableHandler.handleCreateTable
在ZK中设置table的状态是ZooKeeperProtos.Table.State.ENABLED
/hbase/table/tablename
CreateTableHandler结束
(1)Client扫描META表发现有这个Table的regions,
并且这些regions都已经在某个regionserver
上面了,那么设置doneWithMetaScan=true
MetaScanner.metaScan(conf, visitor, desc.getName());
(2)发现这个Table已经是enabled状态了,那么就return
isTableEnabled(desc.getName())