You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by en...@apache.org on 2013/02/06 01:00:43 UTC

svn commit: r1442803 - in /hbase/branches/0.94/src: main/java/org/apache/hadoop/hbase/regionserver/ test/java/org/apache/hadoop/hbase/regionserver/

Author: enis
Date: Wed Feb  6 00:00:42 2013
New Revision: 1442803

URL: http://svn.apache.org/viewvc?rev=1442803&view=rev
Log:
HBASE-7748. Add DelimitedKeyPrefixRegionSplitPolicy

Added:
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/DelimitedKeyPrefixRegionSplitPolicy.java
Modified:
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/KeyPrefixRegionSplitPolicy.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java

Added: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/DelimitedKeyPrefixRegionSplitPolicy.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/DelimitedKeyPrefixRegionSplitPolicy.java?rev=1442803&view=auto
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/DelimitedKeyPrefixRegionSplitPolicy.java (added)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/DelimitedKeyPrefixRegionSplitPolicy.java Wed Feb  6 00:00:42 2013
@@ -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;
+    }
+  }
+}

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/KeyPrefixRegionSplitPolicy.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/KeyPrefixRegionSplitPolicy.java?rev=1442803&r1=1442802&r2=1442803&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/KeyPrefixRegionSplitPolicy.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/KeyPrefixRegionSplitPolicy.java Wed Feb  6 00:00:42 2013
@@ -27,12 +27,14 @@ import org.apache.commons.logging.LogFac
  * 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.
  */
 public class KeyPrefixRegionSplitPolicy extends IncreasingToUpperBoundRegionSplitPolicy {
   private static final Log LOG = LogFactory
       .getLog(KeyPrefixRegionSplitPolicy.class);
-  public static 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;
 
@@ -46,10 +48,14 @@ public class KeyPrefixRegionSplitPolicy 
       String prefixLengthString = region.getTableDesc().getValue(
           PREFIX_LENGTH_KEY);
       if (prefixLengthString == null) {
-        LOG.error(PREFIX_LENGTH_KEY + " not specified for table "
-            + region.getTableDesc().getNameAsString()
-            + ". Using default RegionSplitPolicy");
-        return;
+        //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);
@@ -75,4 +81,4 @@ public class KeyPrefixRegionSplitPolicy 
       return splitPoint;
     }
   }
-}
\ No newline at end of file
+}

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java?rev=1442803&r1=1442802&r2=1442803&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java Wed Feb  6 00:00:42 2013
@@ -250,7 +250,43 @@ 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();
+
+    Store mockStore = Mockito.mock(Store.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()));
+  }
+
   @org.junit.Rule
   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
-}
\ No newline at end of file
+}