You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zg...@apache.org on 2018/08/13 08:49:12 UTC

hbase git commit: HBASE-20985 add two attributes when we do normalization

Repository: hbase
Updated Branches:
  refs/heads/master 911a6ef41 -> 21e02813f


HBASE-20985 add two attributes when we do normalization

Signed-off-by: Guanghao Zhang <zg...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/21e02813
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/21e02813
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/21e02813

Branch: refs/heads/master
Commit: 21e02813fedc6846c9ef381e1ff9b49534f20ec9
Parents: 911a6ef
Author: jingyuntian <ti...@gmail.com>
Authored: Wed Aug 8 19:33:03 2018 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Mon Aug 13 16:43:54 2018 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/HTableDescriptor.java   |  24 ++++
 .../hadoop/hbase/client/TableDescriptor.java    |  16 +++
 .../hbase/client/TableDescriptorBuilder.java    |  60 ++++++++++
 .../normalizer/SimpleRegionNormalizer.java      |  28 ++++-
 .../normalizer/TestSimpleRegionNormalizer.java  | 109 +++++++++++++++++++
 hbase-shell/src/main/ruby/hbase/admin.rb        |   2 +
 .../src/main/ruby/shell/commands/alter.rb       |   3 +-
 hbase-shell/src/test/ruby/hbase/admin_test.rb   |   7 ++
 8 files changed, 247 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/21e02813/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
index cab1acf..e239966 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
@@ -65,6 +65,10 @@ public class HTableDescriptor implements TableDescriptor, Comparable<HTableDescr
   public static final String REGION_REPLICATION = TableDescriptorBuilder.REGION_REPLICATION;
   public static final String REGION_MEMSTORE_REPLICATION = TableDescriptorBuilder.REGION_MEMSTORE_REPLICATION;
   public static final String NORMALIZATION_ENABLED = TableDescriptorBuilder.NORMALIZATION_ENABLED;
+  public static final String NORMALIZER_TARGET_REGION_COUNT =
+      TableDescriptorBuilder.NORMALIZER_TARGET_REGION_COUNT;
+  public static final String NORMALIZER_TARGET_REGION_SIZE =
+      TableDescriptorBuilder.NORMALIZER_TARGET_REGION_SIZE;
   public static final String PRIORITY = TableDescriptorBuilder.PRIORITY;
   public static final boolean DEFAULT_READONLY = TableDescriptorBuilder.DEFAULT_READONLY;
   public static final boolean DEFAULT_COMPACTION_ENABLED = TableDescriptorBuilder.DEFAULT_COMPACTION_ENABLED;
@@ -288,6 +292,26 @@ public class HTableDescriptor implements TableDescriptor, Comparable<HTableDescr
     return this;
   }
 
+  @Override
+  public int getNormalizerTargetRegionCount() {
+    return getDelegateeForModification().getNormalizerTargetRegionCount();
+  }
+
+  public HTableDescriptor setNormalizerTargetRegionCount(final int regionCount) {
+    getDelegateeForModification().setNormalizerTargetRegionCount(regionCount);
+    return this;
+  }
+
+  @Override
+  public long getNormalizerTargetRegionSize() {
+    return getDelegateeForModification().getNormalizerTargetRegionSize();
+  }
+
+  public HTableDescriptor setNormalizerTargetRegionSize(final long regionSize) {
+    getDelegateeForModification().setNormalizerTargetRegionSize(regionSize);
+    return this;
+  }
+
   /**
    * Sets the {@link Durability} setting for the table. This defaults to Durability.USE_DEFAULT.
    * @param durability enum value

http://git-wip-us.apache.org/repos/asf/hbase/blob/21e02813/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java
index 4f713d1..82e08c3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java
@@ -261,6 +261,22 @@ public interface TableDescriptor {
   boolean isNormalizationEnabled();
 
   /**
+   * Check if there is the target region count. If so, the normalize plan will
+   * be calculated based on the target region count.
+   *
+   * @return target region count after normalize done
+   */
+  int getNormalizerTargetRegionCount();
+
+  /**
+   * Check if there is the target region size. If so, the normalize plan will
+   * be calculated based on the target region size.
+   *
+   * @return target region size after normalize done
+   */
+  long getNormalizerTargetRegionSize();
+
+  /**
    * Check if the readOnly flag of the table is set. If the readOnly flag is set
    * then the contents of the table can only be read from but not modified.
    *

http://git-wip-us.apache.org/repos/asf/hbase/blob/21e02813/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
index 78f25ec..d1c3f78 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
@@ -150,6 +150,17 @@ public class TableDescriptorBuilder {
   private static final Bytes NORMALIZATION_ENABLED_KEY
           = new Bytes(Bytes.toBytes(NORMALIZATION_ENABLED));
 
+  @InterfaceAudience.Private
+  public static final String NORMALIZER_TARGET_REGION_COUNT =
+      "NORMALIZER_TARGET_REGION_COUNT";
+  private static final Bytes NORMALIZER_TARGET_REGION_COUNT_KEY =
+      new Bytes(Bytes.toBytes(NORMALIZER_TARGET_REGION_COUNT));
+
+  @InterfaceAudience.Private
+  public static final String NORMALIZER_TARGET_REGION_SIZE = "NORMALIZER_TARGET_REGION_SIZE";
+  private static final Bytes NORMALIZER_TARGET_REGION_SIZE_KEY =
+      new Bytes(Bytes.toBytes(NORMALIZER_TARGET_REGION_SIZE));
+
   /**
    * Default durability for HTD is USE_DEFAULT, which defaults to HBase-global
    * default value
@@ -380,6 +391,16 @@ public class TableDescriptorBuilder {
     return this;
   }
 
+  public TableDescriptorBuilder setNormalizerTargetRegionCount(final int regionCount) {
+    desc.setNormalizerTargetRegionCount(regionCount);
+    return this;
+  }
+
+  public TableDescriptorBuilder setNormalizerTargetRegionSize(final long regionSize) {
+    desc.setNormalizerTargetRegionSize(regionSize);
+    return this;
+  }
+
   public TableDescriptorBuilder setNormalizationEnabled(final boolean isEnable) {
     desc.setNormalizationEnabled(isEnable);
     return this;
@@ -710,6 +731,27 @@ public class TableDescriptorBuilder {
     }
 
     /**
+     * Check if there is the target region count. If so, the normalize plan will be calculated based
+     * on the target region count.
+     * @return target region count after normalize done
+     */
+    @Override
+    public int getNormalizerTargetRegionCount() {
+      return getOrDefault(NORMALIZER_TARGET_REGION_COUNT_KEY, Integer::valueOf,
+        Integer.valueOf(-1));
+    }
+
+    /**
+     * Check if there is the target region size. If so, the normalize plan will be calculated based
+     * on the target region size.
+     * @return target region size after normalize done
+     */
+    @Override
+    public long getNormalizerTargetRegionSize() {
+      return getOrDefault(NORMALIZER_TARGET_REGION_SIZE_KEY, Long::valueOf, Long.valueOf(-1));
+    }
+
+    /**
      * Setting the table normalization enable flag.
      *
      * @param isEnable True if enable normalization.
@@ -720,6 +762,24 @@ public class TableDescriptorBuilder {
     }
 
     /**
+     * Setting the target region count of table normalization .
+     * @param regionCount the target region count.
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor setNormalizerTargetRegionCount(final int regionCount) {
+      return setValue(NORMALIZER_TARGET_REGION_COUNT_KEY, Integer.toString(regionCount));
+    }
+
+    /**
+     * Setting the target region size of table normalization.
+     * @param regionSize the target region size.
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor setNormalizerTargetRegionSize(final long regionSize) {
+      return setValue(NORMALIZER_TARGET_REGION_SIZE_KEY, Long.toString(regionSize));
+    }
+
+    /**
      * Sets the {@link Durability} setting for the table. This defaults to
      * Durability.USE_DEFAULT.
      *

http://git-wip-us.apache.org/repos/asf/hbase/blob/21e02813/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
index f02ccf4..a30a13b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.master.normalizer;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
@@ -30,6 +31,7 @@ import org.apache.hadoop.hbase.Size;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.master.MasterRpcServices;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
@@ -156,8 +158,32 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
         totalSizeMb += regionSize;
       }
     }
+    int targetRegionCount = -1;
+    long targetRegionSize = -1;
+    try {
+      TableDescriptor tableDescriptor = masterServices.getTableDescriptors().get(table);
+      if(tableDescriptor != null) {
+        targetRegionCount =
+            tableDescriptor.getNormalizerTargetRegionCount();
+        targetRegionSize =
+            tableDescriptor.getNormalizerTargetRegionSize();
+        LOG.debug("Table {}:  target region count is {}, target region size is {}", table,
+            targetRegionCount, targetRegionSize);
+      }
+    } catch (IOException e) {
+      LOG.warn(
+        "cannot get the target number and target size of table {}, they will be default value -1.",
+        table);
+    }
 
-    double avgRegionSize = acutalRegionCnt == 0 ? 0 : totalSizeMb / (double) acutalRegionCnt;
+    double avgRegionSize;
+    if (targetRegionSize > 0) {
+      avgRegionSize = targetRegionSize;
+    } else if (targetRegionCount > 0) {
+      avgRegionSize = totalSizeMb / (double) targetRegionCount;
+    } else {
+      avgRegionSize = acutalRegionCnt == 0 ? 0 : totalSizeMb / (double) acutalRegionCnt;
+    }
 
     LOG.debug("Table " + table + ", total aggregated regions size: " + totalSizeMb);
     LOG.debug("Table " + table + ", average region size: " + avgRegionSize);

http://git-wip-us.apache.org/repos/asf/hbase/blob/21e02813/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
index 34489ad..743ec2b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Rule;
@@ -367,6 +368,114 @@ public class TestSimpleRegionNormalizer {
     assertEquals(hri4, ((SplitNormalizationPlan) plan).getRegionInfo());
   }
 
+  @Test
+  public void testSplitWithTargetRegionCount() throws Exception {
+    final TableName tableName = TableName.valueOf(name.getMethodName());
+    List<RegionInfo> RegionInfo = new ArrayList<>();
+    Map<byte[], Integer> regionSizes = new HashMap<>();
+
+    RegionInfo hri1 = RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("aaa"))
+        .setEndKey(Bytes.toBytes("bbb")).build();
+    RegionInfo.add(hri1);
+    regionSizes.put(hri1.getRegionName(), 20);
+
+    RegionInfo hri2 = RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("bbb"))
+        .setEndKey(Bytes.toBytes("ccc")).build();
+    RegionInfo.add(hri2);
+    regionSizes.put(hri2.getRegionName(), 40);
+
+    RegionInfo hri3 = RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("ccc"))
+        .setEndKey(Bytes.toBytes("ddd")).build();
+    RegionInfo.add(hri3);
+    regionSizes.put(hri3.getRegionName(), 60);
+
+    RegionInfo hri4 = RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("ddd"))
+        .setEndKey(Bytes.toBytes("eee")).build();
+    RegionInfo.add(hri4);
+    regionSizes.put(hri4.getRegionName(), 80);
+
+    RegionInfo hri5 = RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("eee"))
+        .setEndKey(Bytes.toBytes("fff")).build();
+    RegionInfo.add(hri5);
+    regionSizes.put(hri5.getRegionName(), 100);
+
+    RegionInfo hri6 = RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("fff"))
+        .setEndKey(Bytes.toBytes("ggg")).build();
+    RegionInfo.add(hri6);
+    regionSizes.put(hri6.getRegionName(), 120);
+
+    setupMocksForNormalizer(regionSizes, RegionInfo);
+
+    // test when target region size is 20
+    when(masterServices.getTableDescriptors().get(any()).getNormalizerTargetRegionSize())
+        .thenReturn(20L);
+    List<NormalizationPlan> plans = normalizer.computePlanForTable(tableName);
+    Assert.assertEquals(4, plans.size());
+
+    for (NormalizationPlan plan : plans) {
+      assertTrue(plan instanceof SplitNormalizationPlan);
+    }
+
+    // test when target region size is 200
+    when(masterServices.getTableDescriptors().get(any()).getNormalizerTargetRegionSize())
+        .thenReturn(200L);
+    plans = normalizer.computePlanForTable(tableName);
+    Assert.assertEquals(2, plans.size());
+    NormalizationPlan plan = plans.get(0);
+    assertTrue(plan instanceof MergeNormalizationPlan);
+    assertEquals(hri1, ((MergeNormalizationPlan) plan).getFirstRegion());
+    assertEquals(hri2, ((MergeNormalizationPlan) plan).getSecondRegion());
+  }
+
+  @Test
+  public void testSplitWithTargetRegionSize() throws Exception {
+    final TableName tableName = TableName.valueOf(name.getMethodName());
+    List<RegionInfo> RegionInfo = new ArrayList<>();
+    Map<byte[], Integer> regionSizes = new HashMap<>();
+
+    RegionInfo hri1 = RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("aaa"))
+        .setEndKey(Bytes.toBytes("bbb")).build();
+    RegionInfo.add(hri1);
+    regionSizes.put(hri1.getRegionName(), 20);
+
+    RegionInfo hri2 = RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("bbb"))
+        .setEndKey(Bytes.toBytes("ccc")).build();
+    RegionInfo.add(hri2);
+    regionSizes.put(hri2.getRegionName(), 40);
+
+    RegionInfo hri3 = RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("ccc"))
+        .setEndKey(Bytes.toBytes("ddd")).build();
+    RegionInfo.add(hri3);
+    regionSizes.put(hri3.getRegionName(), 60);
+
+    RegionInfo hri4 = RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("ddd"))
+        .setEndKey(Bytes.toBytes("eee")).build();
+    RegionInfo.add(hri4);
+    regionSizes.put(hri4.getRegionName(), 80);
+
+    setupMocksForNormalizer(regionSizes, RegionInfo);
+
+    // test when target region count is 8
+    when(masterServices.getTableDescriptors().get(any()).getNormalizerTargetRegionCount())
+        .thenReturn(8);
+    List<NormalizationPlan> plans = normalizer.computePlanForTable(tableName);
+    Assert.assertEquals(2, plans.size());
+
+    for (NormalizationPlan plan : plans) {
+      assertTrue(plan instanceof SplitNormalizationPlan);
+    }
+
+    // test when target region count is 3
+    when(masterServices.getTableDescriptors().get(any()).getNormalizerTargetRegionCount())
+        .thenReturn(3);
+    plans = normalizer.computePlanForTable(tableName);
+    Assert.assertEquals(1, plans.size());
+    NormalizationPlan plan = plans.get(0);
+    assertTrue(plan instanceof MergeNormalizationPlan);
+    assertEquals(hri1, ((MergeNormalizationPlan) plan).getFirstRegion());
+    assertEquals(hri2, ((MergeNormalizationPlan) plan).getSecondRegion());
+  }
+
   @SuppressWarnings("MockitoCast")
   protected void setupMocksForNormalizer(Map<byte[], Integer> regionSizes,
                                          List<RegionInfo> RegionInfo) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/21e02813/hbase-shell/src/main/ruby/hbase/admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index 75d2de3..2be059d 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -1250,6 +1250,8 @@ module Hbase
       htd.setReadOnly(JBoolean.valueOf(arg.delete(org.apache.hadoop.hbase.HTableDescriptor::READONLY))) if arg.include?(org.apache.hadoop.hbase.HTableDescriptor::READONLY)
       htd.setCompactionEnabled(JBoolean.valueOf(arg.delete(org.apache.hadoop.hbase.HTableDescriptor::COMPACTION_ENABLED))) if arg.include?(org.apache.hadoop.hbase.HTableDescriptor::COMPACTION_ENABLED)
       htd.setNormalizationEnabled(JBoolean.valueOf(arg.delete(org.apache.hadoop.hbase.HTableDescriptor::NORMALIZATION_ENABLED))) if arg.include?(org.apache.hadoop.hbase.HTableDescriptor::NORMALIZATION_ENABLED)
+      htd.setNormalizerTargetRegionCount(JInteger.valueOf(arg.delete(org.apache.hadoop.hbase.HTableDescriptor::NORMALIZER_TARGET_REGION_COUNT))) if arg.include?(org.apache.hadoop.hbase.HTableDescriptor::NORMALIZER_TARGET_REGION_COUNT)
+      htd.setNormalizerTargetRegionSize(JLong.valueOf(arg.delete(org.apache.hadoop.hbase.HTableDescriptor::NORMALIZER_TARGET_REGION_SIZE))) if arg.include?(org.apache.hadoop.hbase.HTableDescriptor::NORMALIZER_TARGET_REGION_SIZE)
       htd.setMemStoreFlushSize(JLong.valueOf(arg.delete(org.apache.hadoop.hbase.HTableDescriptor::MEMSTORE_FLUSHSIZE))) if arg.include?(org.apache.hadoop.hbase.HTableDescriptor::MEMSTORE_FLUSHSIZE)
       htd.setDurability(org.apache.hadoop.hbase.client.Durability.valueOf(arg.delete(org.apache.hadoop.hbase.HTableDescriptor::DURABILITY))) if arg.include?(org.apache.hadoop.hbase.HTableDescriptor::DURABILITY)
       htd.setPriority(JInteger.valueOf(arg.delete(org.apache.hadoop.hbase.HTableDescriptor::PRIORITY))) if arg.include?(org.apache.hadoop.hbase.HTableDescriptor::PRIORITY)

http://git-wip-us.apache.org/repos/asf/hbase/blob/21e02813/hbase-shell/src/main/ruby/shell/commands/alter.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/alter.rb b/hbase-shell/src/main/ruby/shell/commands/alter.rb
index 4aef28c..5ba7551 100644
--- a/hbase-shell/src/main/ruby/shell/commands/alter.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/alter.rb
@@ -47,7 +47,8 @@ To delete the 'f1' column family in table 'ns1:t1', use one of:
   hbase> alter 'ns1:t1', 'delete' => 'f1'
 
 You can also change table-scope attributes like MAX_FILESIZE, READONLY,
-MEMSTORE_FLUSHSIZE, DURABILITY, etc. These can be put at the end;
+MEMSTORE_FLUSHSIZE, NORMALIZATION_ENABLED, NORMALIZER_TARGET_REGION_COUNT,
+NORMALIZER_TARGET_REGION_SIZE(MB), DURABILITY, etc. These can be put at the end;
 for example, to change the max size of a region to 128MB, do:
 
   hbase> alter 't1', MAX_FILESIZE => '134217728'

http://git-wip-us.apache.org/repos/asf/hbase/blob/21e02813/hbase-shell/src/test/ruby/hbase/admin_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/admin_test.rb b/hbase-shell/src/test/ruby/hbase/admin_test.rb
index 69dde45..1a12921 100644
--- a/hbase-shell/src/test/ruby/hbase/admin_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/admin_test.rb
@@ -516,6 +516,13 @@ module Hbase
       assert_match(/12345678/, admin.describe(@test_name))
     end
 
+    define_test 'alter should be able to set the TargetRegionSize and TargetRegionCount' do
+      command(:alter, @test_name, 'NORMALIZER_TARGET_REGION_COUNT' => 156)
+      assert_match(/156/, admin.describe(@test_name))
+      command(:alter, @test_name, 'NORMALIZER_TARGET_REGION_SIZE' => 234)
+      assert_match(/234/, admin.describe(@test_name))
+    end
+
     define_test 'alter should support shortcut DELETE alter specs' do
       assert_equal(['x:', 'y:'], table(@test_name).get_all_columns.sort)
       command(:alter, @test_name, 'delete' => 'y')