[
https://issues.apache.org/jira/browse/HBASE-1730?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13083892#comment-13083892
]
Ted Yu commented on HBASE-1730:
-------------------------------
Nice work. In the future, please use https://reviews.apache.org for code review.
Is the following needed in HbaseObjectWritable.java where Serializable is
handled already:
{code}
+ addToMap(Pair.class, code++);
{code}
Please remove log for debugging or change it to LOG.debug():
{code}
+ LOG.info("======== in assignment manager");
{code}
{code}
+ RegionState regionState = regionsInTransition.get(hri.getEncodedName());
+ if (!regionsToReopen.isEmpty()) {
+ regionsToReopen.remove(regionState.getRegion().getEncodedName());
+ }
new ClosedRegionHandler(this.master, this, hri).process();
{code}
Should regionsToReopen.remove() call be placed inside
ClosedRegionHandler.process() (toward the end)?
And why use regionState.getRegion().getEncodedName() where hri.getEncodedName()
gives the same value ?
{code}
+ int waitTime = this.master.getConfiguration().getInt(
+ "hbase.bulk.reopen.waitafter.reopen", 0);
{code}
Do we need to retrieve the wait time every time in this method ? A better name
for the knob is hbase.bulk.waitbetween.reopen
> Near-instantaneous online schema and table state updates
> --------------------------------------------------------
>
> Key: HBASE-1730
> URL: https://issues.apache.org/jira/browse/HBASE-1730
> Project: HBase
> Issue Type: Improvement
> Reporter: Andrew Purtell
> Assignee: stack
> Priority: Critical
> Fix For: 0.92.0
>
> Attachments: 1730-v2.patch, 1730-v3.patch, 1730.patch,
> HBASE-1730.patch
>
>
> We should not need to take a table offline to update HCD or HTD.
> One option for that is putting HTDs and HCDs up into ZK, with mirror on disk
> catalog tables to be used only for cold init scenarios, as discussed on IRC.
> In this scheme, regionservers hosting regions of a table would watch
> permanent nodes in ZK associated with that table for schema updates and take
> appropriate actions out of the watcher. In effect, schema updates become
> another item in the ToDo list.
> {{/hbase/tables/<table-name>/schema}}
> Must be associated with a write locking scheme also handled with ZK
> primitives to avoid situations where one concurrent update clobbers another.
--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira