2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements. See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership. The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License. You may obtain a copy of the License at
10 * http://www.apache.org/licenses/LICENSE-2.0
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
18 package org
.apache
.hadoop
.hbase
.regionserver
;
20 import java
.io
.IOException
;
21 import java
.util
.Arrays
;
22 import org
.apache
.hadoop
.conf
.Configuration
;
23 import org
.apache
.hadoop
.hbase
.client
.TableDescriptor
;
24 import org
.apache
.hadoop
.hbase
.util
.Bytes
;
25 import org
.apache
.yetus
.audience
.InterfaceAudience
;
26 import org
.slf4j
.Logger
;
27 import org
.slf4j
.LoggerFactory
;
30 * A {@link RegionSplitRestriction} implementation that groups rows by a prefix of the row-key with
31 * a delimiter. Only the first delimiter for the row key will define the prefix of the row key that
32 * is used for grouping.
34 * This ensures that a region is not split "inside" a prefix of a row key.
35 * I.e. rows can be co-located in a region by their prefix.
37 * As an example, if you have row keys delimited with <code>_</code>, like
38 * <code>userid_eventtype_eventid</code>, and use prefix delimiter _, this split policy ensures
39 * that all rows starting with the same userid, belongs to the same region.
41 @InterfaceAudience.Private
42 public class DelimitedKeyPrefixRegionSplitRestriction
extends RegionSplitRestriction
{
43 private static final Logger LOG
=
44 LoggerFactory
.getLogger(DelimitedKeyPrefixRegionSplitRestriction
.class);
46 public static final String DELIMITER_KEY
=
47 "hbase.regionserver.region.split_restriction.delimiter";
49 private byte[] delimiter
= null;
52 public void initialize(TableDescriptor tableDescriptor
, Configuration conf
) throws IOException
{
53 String delimiterString
= tableDescriptor
.getValue(DELIMITER_KEY
);
54 if (delimiterString
== null || delimiterString
.length() == 0) {
55 delimiterString
= conf
.get(DELIMITER_KEY
);
56 if (delimiterString
== null || delimiterString
.length() == 0) {
57 LOG
.error("{} not specified for table {}. "
58 + "Using the default RegionSplitRestriction", DELIMITER_KEY
,
59 tableDescriptor
.getTableName());
63 delimiter
= Bytes
.toBytes(delimiterString
);
67 public byte[] getRestrictedSplitPoint(byte[] splitPoint
) {
68 if (delimiter
!= null) {
69 // find the first occurrence of delimiter in split point
70 int index
= org
.apache
.hbase
.thirdparty
.com
.google
.common
.primitives
.Bytes
.indexOf(
71 splitPoint
, delimiter
);
73 LOG
.warn("Delimiter {} not found for split key {}", Bytes
.toString(delimiter
),
74 Bytes
.toStringBinary(splitPoint
));
78 // group split keys by a prefix
79 return Arrays
.copyOf(splitPoint
, Math
.min(index
, splitPoint
.length
));