You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by wc...@apache.org on 2022/01/05 13:10:18 UTC

[hbase] branch branch-2 updated: HBASE-26585 Add SFT configuration to META table descriptor when creating META (#3998)

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

wchevreuil pushed a commit to branch branch-2
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2 by this push:
     new 755b3b4  HBASE-26585 Add SFT configuration to META table descriptor when creating META (#3998)
755b3b4 is described below

commit 755b3b4cb82d0580dc5e3ec51408df4d650ae410
Author: Wellington Ramos Chevreuil <wc...@apache.org>
AuthorDate: Wed Jan 5 12:20:07 2022 +0000

    HBASE-26585 Add SFT configuration to META table descriptor when creating META (#3998)
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
    Signed-off-by: Josh Elser <el...@apache.org>
    
    (cherry picked from commit baeb51ff8adf1bc3b0309f50af39a9f0df1487ae)
---
 .../hadoop/hbase/util/FSTableDescriptors.java      |  4 +-
 .../TestMasterFileSystemWithStoreFileTracking.java | 69 ++++++++++++++++++++++
 2 files changed, 72 insertions(+), 1 deletion(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
index f40736d..4adcc73 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.regionserver.BloomType;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -127,7 +128,8 @@ public class FSTableDescriptors implements TableDescriptors {
       return getTableDescriptorFromFs(fs, rootdir, TableName.META_TABLE_NAME);
     } catch (TableInfoMissingException e) {
       TableDescriptorBuilder builder = createMetaTableDescriptorBuilder(conf);
-      TableDescriptor td = builder.build();
+      TableDescriptor td = StoreFileTrackerFactory.
+        updateWithTrackerConfigs(conf, builder.build());
       LOG.info("Creating new hbase:meta table descriptor {}", td);
       TableName tableName = td.getTableName();
       Path tableDir = CommonFSUtils.getTableDir(rootdir, tableName);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithStoreFileTracking.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithStoreFileTracking.java
new file mode 100644
index 0000000..f2b8a58
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithStoreFileTracking.java
@@ -0,0 +1,69 @@
+/**
+ * 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.master;
+
+import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACKER_IMPL;
+import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.Trackers.FILE;
+import static org.junit.Assert.assertEquals;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+/**
+ * Test the master filesystem in a local cluster with
+ * Store File Tracking explicitly set in global config
+ */
+@Category({MasterTests.class, MediumTests.class})
+public class TestMasterFileSystemWithStoreFileTracking {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMasterFileSystemWithStoreFileTracking.class);
+
+  @Rule
+  public TestName name = new TestName();
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  @BeforeClass
+  public static void setupTest() throws Exception {
+    UTIL.getConfiguration().set(TRACKER_IMPL, FILE.name());
+    UTIL.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void teardownTest() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void tesMetaDescriptorHasSFTConfig() throws Exception {
+    TableDescriptor descriptor = UTIL.getAdmin().getDescriptor(TableName.META_TABLE_NAME);
+    assertEquals(FILE.name(), descriptor.getValue(TRACKER_IMPL));
+  }
+}