You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by me...@apache.org on 2021/09/22 10:23:01 UTC

[hbase] branch master updated: HBASE-26261 Store configuration loss when use update_config (#3664)

This is an automated email from the ASF dual-hosted git repository.

meiyi pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/master by this push:
     new 96fa015  HBASE-26261 Store configuration loss when use update_config (#3664)
96fa015 is described below

commit 96fa015043e5ba6419bb8725899c2ac9dc9d89c5
Author: meiyi <my...@gmail.com>
AuthorDate: Wed Sep 22 18:22:30 2021 +0800

    HBASE-26261 Store configuration loss when use update_config (#3664)
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
---
 .../org/apache/hadoop/hbase/regionserver/HStore.java   | 18 ++++--------------
 .../apache/hadoop/hbase/regionserver/StoreUtils.java   | 10 ++++++++++
 .../TestRegionServerOnlineConfigChange.java            | 14 +++++++++++++-
 3 files changed, 27 insertions(+), 15 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index 4466504..6118244 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -67,6 +67,7 @@ import org.apache.hadoop.hbase.backup.FailedArchiveException;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.conf.ConfigurationManager;
 import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
 import org.apache.hadoop.hbase.coprocessor.ReadOnlyConfiguration;
@@ -241,14 +242,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
   protected HStore(final HRegion region, final ColumnFamilyDescriptor family,
       final Configuration confParam, boolean warmup) throws IOException {
 
-    // 'conf' renamed to 'confParam' b/c we use this.conf in the constructor
-    // CompoundConfiguration will look for keys in reverse order of addition, so we'd
-    // add global config first, then table and cf overrides, then cf metadata.
-    this.conf = new CompoundConfiguration()
-      .add(confParam)
-      .addBytesMap(region.getTableDescriptor().getValues())
-      .addStringMap(family.getConfiguration())
-      .addBytesMap(family.getValues());
+    this.conf = StoreUtils.createStoreConfiguration(confParam, region.getTableDescriptor(), family);
 
     this.region = region;
     this.storeContext = initializeStoreContext(family);
@@ -2523,14 +2517,10 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
     return this.offPeakHours;
   }
 
-  /**
-   * {@inheritDoc}
-   */
   @Override
   public void onConfigurationChange(Configuration conf) {
-    this.conf = new CompoundConfiguration()
-            .add(conf)
-            .addBytesMap(getColumnFamilyDescriptor().getValues());
+    this.conf = StoreUtils.createStoreConfiguration(conf, region.getTableDescriptor(),
+      getColumnFamilyDescriptor());
     this.storeEngine.compactionPolicy.setConf(conf);
     this.offPeakHours = OffPeakHours.getInstance(conf);
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java
index ac5955f..454b244 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java
@@ -28,7 +28,10 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.CompoundConfiguration;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.util.ChecksumType;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -161,4 +164,11 @@ public class StoreUtils {
         HFile.DEFAULT_BYTES_PER_CHECKSUM);
   }
 
+  public static Configuration createStoreConfiguration(Configuration conf, TableDescriptor td,
+      ColumnFamilyDescriptor cfd) {
+    // CompoundConfiguration will look for keys in reverse order of addition, so we'd
+    // add global config first, then table and cf overrides, then cf metadata.
+    return new CompoundConfiguration().add(conf).addBytesMap(td.getValues())
+        .addStringMap(cfd.getConfiguration()).addBytesMap(cfd.getValues());
+  }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerOnlineConfigChange.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerOnlineConfigChange.java
index de3335b..943b8a9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerOnlineConfigChange.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerOnlineConfigChange.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -76,13 +77,16 @@ public class TestRegionServerOnlineConfigChange {
   private final static String columnFamily1Str = "columnFamily1";
   private final static TableName TABLE1 = TableName.valueOf(table1Str);
   private final static byte[] COLUMN_FAMILY1 = Bytes.toBytes(columnFamily1Str);
+  private final static long MAX_FILE_SIZE = 20 * 1024 * 1024L;
 
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     conf = hbaseTestingUtility.getConfiguration();
     hbaseTestingUtility.startMiniCluster(2);
-    t1 = hbaseTestingUtility.createTable(TABLE1, COLUMN_FAMILY1);
+    t1 = hbaseTestingUtility.createTable(
+      TableDescriptorBuilder.newBuilder(TABLE1).setMaxFileSize(MAX_FILE_SIZE).build(),
+      new byte[][] { COLUMN_FAMILY1 }, conf);
   }
 
   @AfterClass
@@ -259,4 +263,12 @@ public class TestRegionServerOnlineConfigChange {
       });
     }
   }
+
+  @Test
+  public void testStoreConfigurationOnlineChange() {
+    rs1.getConfigurationManager().notifyAllObservers(conf);
+    long actualMaxFileSize = r1.getStore(COLUMN_FAMILY1).getReadOnlyConfiguration()
+        .getLong(TableDescriptorBuilder.MAX_FILESIZE, -1);
+    assertEquals(MAX_FILE_SIZE, actualMaxFileSize);
+  }
 }