HBASE-26412 Handle sink failure in RegionReplicationSink (#3815)
[hbase.git] / hbase-server / src / main / java / org / apache / hadoop / hbase / regionserver / DelimitedKeyPrefixRegionSplitPolicy.java
blob241c062c6497445d5201f28fc97c4a1ff3b4c16c
1 /**
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.
19 package org.apache.hadoop.hbase.regionserver;
21 import java.util.Arrays;
23 import org.apache.yetus.audience.InterfaceAudience;
24 import org.slf4j.Logger;
25 import org.slf4j.LoggerFactory;
26 import org.apache.hadoop.hbase.util.Bytes;
28 /**
29 * A custom RegionSplitPolicy implementing a SplitPolicy that groups
30 * rows by a prefix of the row-key with a delimiter. Only the first delimiter
31 * for the row key will define the prefix of the row key that is used for grouping.
33 * This ensures that a region is not split "inside" a prefix of a row key.
34 * I.e. rows can be co-located in a region by their prefix.
36 * As an example, if you have row keys delimited with <code>_</code>, like
37 * <code>userid_eventtype_eventid</code>, and use prefix delimiter _, this split policy
38 * ensures that all rows starting with the same userid, belongs to the same region.
39 * @see KeyPrefixRegionSplitPolicy
41 * @deprecated since 3.0.0 and will be removed in 4.0.0. Use {@link RegionSplitRestriction},
42 * instead.
44 @Deprecated
45 @InterfaceAudience.Private
46 public class DelimitedKeyPrefixRegionSplitPolicy extends IncreasingToUpperBoundRegionSplitPolicy {
48 private static final Logger LOG = LoggerFactory
49 .getLogger(DelimitedKeyPrefixRegionSplitPolicy.class);
50 public static final String DELIMITER_KEY = "DelimitedKeyPrefixRegionSplitPolicy.delimiter";
52 private byte[] delimiter = null;
54 @Override
55 public String toString() {
56 return "DelimitedKeyPrefixRegionSplitPolicy{" + "delimiter=" + Bytes.toStringBinary(delimiter) +
57 ", " + super.toString() + '}';
60 @Override
61 protected void configureForRegion(HRegion region) {
62 super.configureForRegion(region);
63 // read the prefix length from the table descriptor
64 String delimiterString = region.getTableDescriptor().getValue(DELIMITER_KEY);
65 if (delimiterString == null || delimiterString.length() == 0) {
66 LOG.error(DELIMITER_KEY + " not specified for table " +
67 region.getTableDescriptor().getTableName() + ". Using default RegionSplitPolicy");
68 return;
70 delimiter = Bytes.toBytes(delimiterString);
73 @Override
74 protected byte[] getSplitPoint() {
75 byte[] splitPoint = super.getSplitPoint();
76 if (splitPoint != null && delimiter != null) {
78 //find the first occurrence of delimiter in split point
79 int index =
80 org.apache.hbase.thirdparty.com.google.common.primitives.Bytes.indexOf(splitPoint, delimiter);
81 if (index < 0) {
82 LOG.warn("Delimiter " + Bytes.toString(delimiter) + " not found for split key "
83 + Bytes.toString(splitPoint));
84 return splitPoint;
87 // group split keys by a prefix
88 return Arrays.copyOf(splitPoint, Math.min(index, splitPoint.length));
89 } else {
90 return splitPoint;