[ 
https://issues.apache.org/jira/browse/HBASE-25766?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Toshihiro Suzuki updated HBASE-25766:
-------------------------------------
    Release Note: 
After HBASE-25766, we can specify a split restriction, "KeyPrefix" or 
"DelimitedKeyPrefix", to a table with the 
"hbase.regionserver.region.split_restriction.type" property. The "KeyPrefix" 
split restriction groups rows by a prefix of the row-key. And the 
"DelimitedKeyPrefix" split restriction groups rows by a prefix of the row-key 
with a delimiter.

For example:
```
# Create a table with a "KeyPrefix" split restriction, where the prefix length 
is 2 bytes
hbase> create 'tbl1', 'fam', {CONFIGURATION => 
{'hbase.regionserver.region.split_restriction.type' => 'KeyPrefix', 
'hbase.regionserver.region.split_restriction.prefix_length' => '2'}}

# Create a table with a "DelimitedKeyPrefix" split restriction, where the 
delimiter is a comma (,)
hbase> create 'tbl2', 'fam', {CONFIGURATION => 
{'hbase.regionserver.region.split_restriction.type' => 'DelimitedKeyPrefix', 
'hbase.regionserver.region.split_restriction.delimiter' => ','}}
```

Instead of specifying a split restriction to a table directly, we can also set 
the properties in hbase-site.xml. In this case, the specified split restriction 
is applied for all the tables.

Note that after this change, the split restriction is also applied to a 
user-specified split point so that we don't allow users to break the 
restriction, which is a behavior change.

  was:
After HBASE-25766, we can specify a split restriction, "KeyPrefix" or 
"DelimitedKeyPrefix", to a table with the 
"hbase.regionserver.region.split_restriction.type" property. The "KeyPrefix" 
split restriction groups rows by a prefix of the row-key. And the 
"DelimitedKeyPrefix" split restriction groups rows by a prefix of the row-key 
with a delimiter.

For example:
```
# Create a table with a "KeyPrefix" split restriction, where the prefix length 
is 2 bytes
hbase> create 'tbl1', 'fam', {CONFIGURATION => 
{'hbase.regionserver.region.split_restriction.type' => 'KeyPrefix', 
'hbase.regionserver.region.split_restriction.prefix_length' => '2'}}

# Create a table with a "DelimitedKeyPrefix" split restriction, where the 
delimiter is a comma (,)
hbase> create 'tbl2', 'fam', {CONFIGURATION => 
{'hbase.regionserver.region.split_restriction.type' => 'DelimitedKeyPrefix', 
'hbase.regionserver.region.split_restriction.delimiter' => ','}}
```

Instead of specifying a split restriction to a table directly, we can also set 
the properties in hbase-site.xml. In this case, the specified split restriction 
is applied for all the tables.



> Introduce RegionSplitRestriction that restricts the pattern of the split point
> ------------------------------------------------------------------------------
>
>                 Key: HBASE-25766
>                 URL: https://issues.apache.org/jira/browse/HBASE-25766
>             Project: HBase
>          Issue Type: Improvement
>            Reporter: Toshihiro Suzuki
>            Assignee: Toshihiro Suzuki
>            Priority: Major
>             Fix For: 3.0.0-alpha-1, 2.4.3
>
>
> As discussed in HBASE-25706, we can introduce RegionSplitRestriction that 
> restricts the pattern of the split point.
> See the following comment for the details:
> https://issues.apache.org/jira/browse/HBASE-25706?focusedCommentId=17310190&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17310190
> CC: [~zhangduo] [~stack]



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to