HBASE-7748. Add DelimitedKeyPrefixRegionSplitPolicy

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1442408 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Enis Soztutar 2013-02-04 22:30:50 +00:00
parent b9b6c9293f
commit 419aca3b6c
3 changed files with 136 additions and 7 deletions

View File

@ -0,0 +1,88 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import java.util.Arrays;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes;
/**
* A custom RegionSplitPolicy implementing a SplitPolicy that groups
* rows by a prefix of the row-key with a delimiter. Only the first delimiter
* for the row key will define the prefix of the row key that is used for grouping.
*
* This ensures that a region is not split "inside" a prefix of a row key.
* I.e. rows can be co-located in a region by their prefix.
*
* As an example, if you have row keys delimited with <code>_</code>, like
* <code>userid_eventtype_eventid</code>, and use prefix delimiter _, this split policy
* ensures that all rows starting with the same userid, belongs to the same region.
* @see KeyPrefixRegionSplitPolicy
*/
@InterfaceAudience.Private
public class DelimitedKeyPrefixRegionSplitPolicy extends IncreasingToUpperBoundRegionSplitPolicy {
private static final Log LOG = LogFactory
.getLog(DelimitedKeyPrefixRegionSplitPolicy.class);
public static final String DELIMITER_KEY = "DelimitedKeyPrefixRegionSplitPolicy.delimiter";
private byte[] delimiter = null;
@Override
protected void configureForRegion(HRegion region) {
super.configureForRegion(region);
if (region != null) {
// read the prefix length from the table descriptor
String delimiterString = region.getTableDesc().getValue(
DELIMITER_KEY);
if (delimiterString == null || delimiterString.length() == 0) {
LOG.error(DELIMITER_KEY + " not specified for table "
+ region.getTableDesc().getNameAsString()
+ ". Using default RegionSplitPolicy");
return;
}
delimiter = Bytes.toBytes(delimiterString);
}
}
@Override
protected byte[] getSplitPoint() {
byte[] splitPoint = super.getSplitPoint();
if (delimiter != null) {
//find the first occurrence of delimiter in split point
int index = com.google.common.primitives.Bytes.indexOf(splitPoint, delimiter);
if (index < 0) {
LOG.warn("Delimiter " + Bytes.toString(delimiter) + " not found for split key "
+ Bytes.toString(splitPoint));
return splitPoint;
}
// group split keys by a prefix
return Arrays.copyOf(splitPoint, Math.min(index, splitPoint.length));
} else {
return splitPoint;
}
}
}

View File

@ -28,13 +28,15 @@ import org.apache.hadoop.classification.InterfaceAudience;
* rows by a prefix of the row-key
*
* This ensures that a region is not split "inside" a prefix of a row key.
* I.e. rows can be co-located in a regionb by their prefix.
* I.e. rows can be co-located in a region by their prefix.
*/
@InterfaceAudience.Private
public class KeyPrefixRegionSplitPolicy extends IncreasingToUpperBoundRegionSplitPolicy {
private static final Log LOG = LogFactory
.getLog(KeyPrefixRegionSplitPolicy.class);
public static final String PREFIX_LENGTH_KEY = "prefix_split_key_policy.prefix_length";
@Deprecated
public static final String PREFIX_LENGTH_KEY_DEPRECATED = "prefix_split_key_policy.prefix_length";
public static final String PREFIX_LENGTH_KEY = "KeyPrefixRegionSplitPolicy.prefix_length";
private int prefixLength = 0;
@ -47,12 +49,16 @@ public class KeyPrefixRegionSplitPolicy extends IncreasingToUpperBoundRegionSpli
// read the prefix length from the table descriptor
String prefixLengthString = region.getTableDesc().getValue(
PREFIX_LENGTH_KEY);
if (prefixLengthString == null) {
//read the deprecated value
prefixLengthString = region.getTableDesc().getValue(PREFIX_LENGTH_KEY_DEPRECATED);
if (prefixLengthString == null) {
LOG.error(PREFIX_LENGTH_KEY + " not specified for table "
+ region.getTableDesc().getNameAsString()
+ ". Using default RegionSplitPolicy");
return;
}
}
try {
prefixLength = Integer.parseInt(prefixLengthString);
} catch (NumberFormatException nfe) {

View File

@ -28,7 +28,6 @@ import java.util.List;
import java.util.TreeMap;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CompoundConfiguration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
@ -251,4 +250,40 @@ public class TestRegionSplitPolicy {
Bytes.toString(policy.getSplitPoint()));
}
@Test
public void testDelimitedKeyPrefixRegionSplitPolicy() throws IOException {
HTableDescriptor myHtd = new HTableDescriptor();
myHtd.setValue(HTableDescriptor.SPLIT_POLICY,
DelimitedKeyPrefixRegionSplitPolicy.class.getName());
myHtd.setValue(DelimitedKeyPrefixRegionSplitPolicy.DELIMITER_KEY, ",");
HRegion myMockRegion = Mockito.mock(HRegion.class);
Mockito.doReturn(myHtd).when(myMockRegion).getTableDesc();
Mockito.doReturn(stores).when(myMockRegion).getStores();
HStore mockStore = Mockito.mock(HStore.class);
Mockito.doReturn(2000L).when(mockStore).getSize();
Mockito.doReturn(true).when(mockStore).canSplit();
Mockito.doReturn(Bytes.toBytes("ab,cd")).when(mockStore).getSplitPoint();
stores.put(new byte[] { 1 }, mockStore);
DelimitedKeyPrefixRegionSplitPolicy policy = (DelimitedKeyPrefixRegionSplitPolicy) RegionSplitPolicy
.create(myMockRegion, conf);
assertEquals("ab", Bytes.toString(policy.getSplitPoint()));
Mockito.doReturn(true).when(myMockRegion).shouldForceSplit();
Mockito.doReturn(Bytes.toBytes("efg,h")).when(myMockRegion)
.getExplicitSplitPoint();
policy = (DelimitedKeyPrefixRegionSplitPolicy) RegionSplitPolicy
.create(myMockRegion, conf);
assertEquals("efg", Bytes.toString(policy.getSplitPoint()));
Mockito.doReturn(Bytes.toBytes("ijk")).when(myMockRegion)
.getExplicitSplitPoint();
assertEquals("ijk", Bytes.toString(policy.getSplitPoint()));
}
}