You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2020/01/09 19:14:17 UTC

[GitHub] [hbase] saintstack opened a new pull request #1011: HBASE-23055 Alter hbase:meta

saintstack opened a new pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011
 
 
       Make it so hbase:meta can be altered. TableState for hbase:meta
       was hardcoded ENABLED. Make it dynamic. State is now kept in
       current active Master. It is transient so falls back to default
       if Master crashes. Add to registry a getMetaTableState
       which reads mirrored state from zookeeper (NOT from Master and
       defaults ENABLED if no implementation or error fetching state).
       hbase:meta schema will be bootstrapped from the filesystem.
       Changes to filesystem schema are atomic so we should be ok if
       Master fails mid-edit (TBD). Undoes a bunch of guards that
       prevented our being able to edit hbase:meta.
   
       TODO: Tests, more clarity around hbase:meta table state, and undoing
       references to hard-coded hbase:meta regioninfo.
   
       M hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
        Throw illegal access exception if you try to use MetaTableAccessor
        getting state of the hbase:meta table.
   
       M hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
        Add fetching of hbase:meta table state from registry. Adds cache of
        tablestates w/ a ttl of 1 second (adjustable).
   
       M hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
       M hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
         Add querying registry for hbase:meta table state.
   
       M hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java
         Add querying of mirrored table state for hbase:meta table.
   
       M hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java
        Shutdown access.
   
       M hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptors.java
        Just cleanup.
   
       M hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
        Add state holder for hbase:meta. Removed unused methods.
   
       M hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
        Shut down access.
   
       M hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
        Allow hbase:meta to be disabled.
   
       M hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java
        Allow hbase:meta to be enabled.
   
   Signed-off-by: Bharath Vissapragada <bh...@apache.org>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#discussion_r366641320
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseMetaEdit.java
 ##########
 @@ -0,0 +1,110 @@
+/*
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+/**
+ * Test being able to edit hbase:meta.
+ */
+@Category({MiscTests.class, LargeTests.class})
+public class TestHBaseMetaEdit {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestHBaseMetaEdit.class);
+  @Rule
+  public TestName name = new TestName();
+  private final static HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  @Before
+  public void before() throws Exception {
+    UTIL.startMiniCluster();
+  }
+
+  @After
+  public void after() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  /**
+   * Set versions, set HBASE-16213 indexed block encoding, and add a column family.
+   * Verify they are all in place by looking at TableDescriptor AND by checking
+   * what the RegionServer sees after opening Region.
+   */
+  @Test
+  public void testEditMeta() throws IOException {
+    Admin admin = UTIL.getAdmin();
+    admin.tableExists(TableName.META_TABLE_NAME);
+    admin.disableTable(TableName.META_TABLE_NAME);
+    assertTrue(admin.isTableDisabled(TableName.META_TABLE_NAME));
+    TableDescriptor descriptor = admin.getDescriptor(TableName.META_TABLE_NAME);
+    ColumnFamilyDescriptor cfd = descriptor.getColumnFamily(HConstants.CATALOG_FAMILY);
+    byte [] extraColumnFamilyName = Bytes.toBytes("xtra");
+    ColumnFamilyDescriptor newCfd =
+        ColumnFamilyDescriptorBuilder.newBuilder(extraColumnFamilyName).build();
+    int oldVersions = cfd.getMaxVersions();
+    // Add '1' to current versions count.
+    cfd = ColumnFamilyDescriptorBuilder.newBuilder(cfd).setMaxVersions(oldVersions + 1).
+        setConfiguration(ColumnFamilyDescriptorBuilder.DATA_BLOCK_ENCODING,
+            DataBlockEncoding.ROW_INDEX_V1.toString()).build();
+    admin.modifyColumnFamily(TableName.META_TABLE_NAME, cfd);
+    admin.addColumnFamily(TableName.META_TABLE_NAME, newCfd);
 
 Review comment:
   Added blocking of delete to ModifyTableProcedure if hbase:meta and a core column family.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#discussion_r365998779
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
 ##########
 @@ -666,42 +667,55 @@ public void run(PRESP resp) {
       new DisableTableProcedureBiConsumer(tableName));
   }
 
+  /**
+   * Utility for completing passed TableState {@link CompletableFuture} <code>future</code>
+   * using passed parameters.
+   */
+  private static CompletableFuture<Boolean> completeCheckTableState(
 
 Review comment:
   No. Not a holdover. The boolean is true if table is in target state. Let me add javadoc.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#discussion_r365324010
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
 ##########
 @@ -666,42 +667,55 @@ public void run(PRESP resp) {
       new DisableTableProcedureBiConsumer(tableName));
   }
 
+  /**
+   * Utility for completing passed TableState {@link CompletableFuture} <code>future</code>
+   * using passed parameters.
+   */
+  private static CompletableFuture<Boolean> completeCheckTableState(
 
 Review comment:
   Looks like this a holdover. I can make it void.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache9 commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#discussion_r365230905
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseMetaEdit.java
 ##########
 @@ -0,0 +1,110 @@
+/*
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+/**
+ * Test being able to edit hbase:meta.
+ */
+@Category({MiscTests.class, LargeTests.class})
+public class TestHBaseMetaEdit {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestHBaseMetaEdit.class);
+  @Rule
+  public TestName name = new TestName();
+  private final static HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  @Before
+  public void before() throws Exception {
+    UTIL.startMiniCluster();
+  }
+
+  @After
+  public void after() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  /**
+   * Set versions, set HBASE-16213 indexed block encoding, and add a column family.
+   * Verify they are all in place by looking at TableDescriptor AND by checking
+   * what the RegionServer sees after opening Region.
+   */
+  @Test
+  public void testEditMeta() throws IOException {
+    Admin admin = UTIL.getAdmin();
+    admin.tableExists(TableName.META_TABLE_NAME);
+    admin.disableTable(TableName.META_TABLE_NAME);
+    assertTrue(admin.isTableDisabled(TableName.META_TABLE_NAME));
+    TableDescriptor descriptor = admin.getDescriptor(TableName.META_TABLE_NAME);
+    ColumnFamilyDescriptor cfd = descriptor.getColumnFamily(HConstants.CATALOG_FAMILY);
+    byte [] extraColumnFamilyName = Bytes.toBytes("xtra");
+    ColumnFamilyDescriptor newCfd =
+        ColumnFamilyDescriptorBuilder.newBuilder(extraColumnFamilyName).build();
+    int oldVersions = cfd.getMaxVersions();
+    // Add '1' to current versions count.
+    cfd = ColumnFamilyDescriptorBuilder.newBuilder(cfd).setMaxVersions(oldVersions + 1).
+        setConfiguration(ColumnFamilyDescriptorBuilder.DATA_BLOCK_ENCODING,
+            DataBlockEncoding.ROW_INDEX_V1.toString()).build();
+    admin.modifyColumnFamily(TableName.META_TABLE_NAME, cfd);
+    admin.addColumnFamily(TableName.META_TABLE_NAME, newCfd);
 
 Review comment:
   Now we allow users to add/remove families of meta table from client side? A bit dangerous. I think we should only allow a sub set of alter operations for meta table and system tables from client side? At least, we should not let them add or remove a column family, it will introduce very critical problems.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache9 commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#discussion_r365224489
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java
 ##########
 @@ -229,6 +244,43 @@ private void getMetaRegionLocation(CompletableFuture<RegionLocations> future,
         });
   }
 
+  @Override
+  public CompletableFuture<TableState> getMetaTableState() {
+    return getAndConvert(this.znodeMirroredMetaTableState, ZKAsyncRegistry::getTableState).
+      thenApply(state -> {
+        return state == null || state.equals(ENABLED_META_TABLE_STATE.getState())?
+          ENABLED_META_TABLE_STATE: new TableState(TableName.META_TABLE_NAME, state);
+      }).exceptionally(e -> {
 
 Review comment:
   Currently in HBase, usually we will create a new CompletableFuture and use FutureUtils.addListener to complete it. The code in exceptionally are a bit tricky, where we throw a CompletionException...

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on issue #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack commented on issue #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#issuecomment-572711092
 
 
   For master branch

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#discussion_r365337598
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptors.java
 ##########
 @@ -25,51 +25,39 @@
 
 /**
  * Get, remove and modify table descriptors.
- * Used by servers to host descriptors.
  */
 @InterfaceAudience.Private
 public interface TableDescriptors {
   /**
-   * @param tableName
    * @return TableDescriptor for tablename
-   * @throws IOException
    */
-  TableDescriptor get(final TableName tableName)
-      throws IOException;
+  TableDescriptor get(final TableName tableName) throws IOException;
 
   /**
    * Get Map of all NamespaceDescriptors for a given namespace.
    * @return Map of all descriptors.
-   * @throws IOException
    */
-  Map<String, TableDescriptor> getByNamespace(String name)
-  throws IOException;
+  Map<String, TableDescriptor> getByNamespace(String name) throws IOException;
 
   /**
    * Get Map of all TableDescriptors. Populates the descriptor cache as a
    * side effect.
    * Notice: the key of map is the table name which contains namespace. It was generated by
    * {@link TableName#getNameWithNamespaceInclAsString()}.
    * @return Map of all descriptors.
-   * @throws IOException
    */
   Map<String, TableDescriptor> getAll() throws IOException;
 
   /**
    * Add or update descriptor
    * @param htd Descriptor to set into TableDescriptors
-   * @throws IOException
    */
-  void add(final TableDescriptor htd)
-  throws IOException;
+  void add(final TableDescriptor htd) throws IOException;
 
 Review comment:
   Let me fix.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack opened a new pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack opened a new pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011
 
 
       Make it so hbase:meta can be altered. TableState for hbase:meta
       was hardcoded ENABLED. Make it dynamic. State is now kept in
       current active Master. It is transient so falls back to default
       if Master crashes. Add to registry a getMetaTableState
       which reads mirrored state from zookeeper (NOT from Master and
       defaults ENABLED if no implementation or error fetching state).
       hbase:meta schema will be bootstrapped from the filesystem.
       Changes to filesystem schema are atomic so we should be ok if
       Master fails mid-edit (TBD). Undoes a bunch of guards that
       prevented our being able to edit hbase:meta.
   
       TODO: Tests, more clarity around hbase:meta table state, and undoing
       references to hard-coded hbase:meta regioninfo.
   
       M hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
        Throw illegal access exception if you try to use MetaTableAccessor
        getting state of the hbase:meta table.
   
       M hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
        Add fetching of hbase:meta table state from registry. Adds cache of
        tablestates w/ a ttl of 1 second (adjustable).
   
       M hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
       M hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
         Add querying registry for hbase:meta table state.
   
       M hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java
         Add querying of mirrored table state for hbase:meta table.
   
       M hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java
        Shutdown access.
   
       M hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptors.java
        Just cleanup.
   
       M hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
        Add state holder for hbase:meta. Removed unused methods.
   
       M hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
        Shut down access.
   
       M hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
        Allow hbase:meta to be disabled.
   
       M hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java
        Allow hbase:meta to be enabled.
   
   Signed-off-by: Bharath Vissapragada <bh...@apache.org>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache9 commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#discussion_r365221342
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java
 ##########
 @@ -58,8 +62,19 @@
 
   private final ZNodePaths znodePaths;
 
+  /**
+   * A znode maintained by MirroringTableStateManager.
+   * MirroringTableStateManager is deprecated to be removed in hbase3. It can also be disabled.
+   * Make sure it is enabled if you want to alter hbase:meta table in hbase2. In hbase3,
+   * TBD how metatable state will be hosted; likely on active hbase master.
 
 Review comment:
   So I do not think we should name this as mirrored, it is the original data. The state in master's memory, is a cache, actually.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on issue #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack commented on issue #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#issuecomment-572821124
 
 
   Merged manually after fixing checkstyle. Tried the failed test locally a few times and passes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack closed pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack closed pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache-HBase commented on issue #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#issuecomment-573270424
 
 
   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 32s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | -0 :warning: |  test4tests  |   0m  0s |  The patch doesn't appear to include any new or modified tests. Please justify why no new tests are needed for this patch. Also please list what manual steps were performed to verify this patch.  |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   5m 36s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 59s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 38s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m  7s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 38s |  master passed  |
   | +0 :ok: |  spotbugs  |   4m 35s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   4m 34s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   5m 20s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 57s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 57s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   1m 19s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedjars  |   4m 40s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  15m 52s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   4m 31s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 153m 32s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 37s |  The patch does not generate ASF License warnings.  |
   |  |   | 212m 59s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1011/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1011 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux 22d8c4b69881 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/Base-PreCommit-GitHub-PR_PR-1011/out/precommit/personality/provided.sh |
   | git revision | master / 8ca614857d |
   | Default Java | 1.8.0_181 |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1011/1/testReport/ |
   | Max. process+thread count | 5355 (vs. ulimit of 10000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1011/1/console |
   | versions | git=2.11.0 maven=2018-06-17T18:33:14Z) findbugs=3.1.11 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack closed pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack closed pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache9 commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#discussion_r365233018
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptors.java
 ##########
 @@ -25,51 +25,39 @@
 
 /**
  * Get, remove and modify table descriptors.
- * Used by servers to host descriptors.
  */
 @InterfaceAudience.Private
 public interface TableDescriptors {
   /**
-   * @param tableName
    * @return TableDescriptor for tablename
-   * @throws IOException
    */
-  TableDescriptor get(final TableName tableName)
-      throws IOException;
+  TableDescriptor get(final TableName tableName) throws IOException;
 
   /**
    * Get Map of all NamespaceDescriptors for a given namespace.
    * @return Map of all descriptors.
-   * @throws IOException
    */
-  Map<String, TableDescriptor> getByNamespace(String name)
-  throws IOException;
+  Map<String, TableDescriptor> getByNamespace(String name) throws IOException;
 
   /**
    * Get Map of all TableDescriptors. Populates the descriptor cache as a
    * side effect.
    * Notice: the key of map is the table name which contains namespace. It was generated by
    * {@link TableName#getNameWithNamespaceInclAsString()}.
    * @return Map of all descriptors.
-   * @throws IOException
    */
   Map<String, TableDescriptor> getAll() throws IOException;
 
   /**
    * Add or update descriptor
    * @param htd Descriptor to set into TableDescriptors
-   * @throws IOException
    */
-  void add(final TableDescriptor htd)
-  throws IOException;
+  void add(final TableDescriptor htd) throws IOException;
 
 Review comment:
   This method is a bit odd. We do not call it in CreateTableProcedure, but only in ModifyTableProcedure. Why not just call it update?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#discussion_r366641964
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java
 ##########
 @@ -58,8 +62,19 @@
 
   private final ZNodePaths znodePaths;
 
+  /**
+   * A znode maintained by MirroringTableStateManager.
+   * MirroringTableStateManager is deprecated to be removed in hbase3. It can also be disabled.
+   * Make sure it is enabled if you want to alter hbase:meta table in hbase2. In hbase3,
+   * TBD how metatable state will be hosted; likely on active hbase master.
 
 Review comment:
   Not relevant in new patch.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache9 commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#discussion_r365220125
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
 ##########
 @@ -666,42 +667,55 @@ public void run(PRESP resp) {
       new DisableTableProcedureBiConsumer(tableName));
   }
 
+  /**
+   * Utility for completing passed TableState {@link CompletableFuture} <code>future</code>
+   * using passed parameters.
+   */
+  private static CompletableFuture<Boolean> completeCheckTableState(
 
 Review comment:
   What is the return value used for?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on issue #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack commented on issue #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#issuecomment-574444740
 
 
   Closing messed up push.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache9 commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#discussion_r365227172
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RejectReplicationRequestStateChecker.java
 ##########
 @@ -1,4 +1,6 @@
-/**
+/*
+ * Copyright The Apache Software Foundation
 
 Review comment:
   What's this?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#discussion_r366586004
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
 ##########
 @@ -135,16 +136,32 @@ public FSTableDescriptors(final Configuration conf, final FileSystem fs,
    *                     TODO: This is a workaround. Should remove this ugly code...
    */
   public FSTableDescriptors(final Configuration conf, final FileSystem fs,
-                            final Path rootdir, final boolean fsreadonly, final boolean usecache,
-                            Function<TableDescriptorBuilder, TableDescriptorBuilder> metaObserver) throws IOException {
+       final Path rootdir, final boolean fsreadonly, final boolean usecache,
+       Function<TableDescriptorBuilder, TableDescriptorBuilder> metaObserver) throws IOException {
     this.fs = fs;
     this.rootdir = rootdir;
     this.fsreadonly = fsreadonly;
     this.usecache = usecache;
-    this.metaTableDescriptor = metaObserver == null ? createMetaTableDescriptor(conf)
-          : metaObserver.apply(createMetaTableDescriptorBuilder(conf)).build();
+    TableDescriptor td = null;
+    try {
+      td = getTableDescriptorFromFs(fs, rootdir, TableName.META_TABLE_NAME);
+    } catch (TableInfoMissingException e) {
+      td = metaObserver == null? createMetaTableDescriptor(conf):
+        metaObserver.apply(createMetaTableDescriptorBuilder(conf)).build();
+      if (!fsreadonly) {
+        LOG.info("Creating new hbase:meta table default descriptor/schema {}", td);
+        updateTableDescriptor(td);
+      }
+    }
+    this.metaTableDescriptor = td;
   }
 
+  /**
+   *
+   * Should be private
+   * @deprecated Since 2.3.0. Should be for internal use only. Used by testing.
 
 Review comment:
   Leaving it public for now (after removing the deprecated).  Need to undo one reference in test before can take it private.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#discussion_r366642125
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
 ##########
 @@ -666,42 +667,55 @@ public void run(PRESP resp) {
       new DisableTableProcedureBiConsumer(tableName));
   }
 
+  /**
+   * Utility for completing passed TableState {@link CompletableFuture} <code>future</code>
+   * using passed parameters.
+   */
+  private static CompletableFuture<Boolean> completeCheckTableState(
 
 Review comment:
   The returned boolean is actually used. Boolean is true if state matches passed in expected state. Leaving.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache-HBase commented on issue #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#issuecomment-572815795
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 33s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 8 new or modified test files.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 38s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 46s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 45s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   3m  7s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   4m 55s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m  3s |  master passed  |
   | +0 :ok: |  spotbugs  |   4m 57s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   7m 39s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m 34s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 49s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 49s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 27s |  The patch passed checkstyle in hbase-common  |
   | +1 :green_heart: |  checkstyle  |   0m 38s |  hbase-client: The patch generated 0 new + 254 unchanged - 4 fixed = 254 total (was 258)  |
   | +1 :green_heart: |  checkstyle  |   0m 15s |  The patch passed checkstyle in hbase-zookeeper  |
   | -1 :x: |  checkstyle  |   1m 35s |  hbase-server: The patch generated 2 new + 382 unchanged - 30 fixed = 384 total (was 412)  |
   | +1 :green_heart: |  checkstyle  |   0m 13s |  The patch passed checkstyle in hbase-shell  |
   | +1 :green_heart: |  rubocop  |   0m  5s |  There were no new rubocop issues.  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedjars  |   5m 26s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  19m 15s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   2m  5s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   9m 40s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   3m 44s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m  8s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 55s |  hbase-zookeeper in the patch passed.  |
   | -1 :x: |  unit  | 188m 23s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |   7m 33s |  hbase-shell in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   2m 49s |  The patch does not generate ASF License warnings.  |
   |  |   | 290m 19s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hbase.master.procedure.TestMasterFailoverWithProcedures |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1011/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1011 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile rubocop |
   | uname | Linux 075ace95d3ec 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/Base-PreCommit-GitHub-PR_PR-1011/out/precommit/personality/provided.sh |
   | git revision | master / b6a1f0098a |
   | Default Java | 1.8.0_181 |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1011/1/artifact/out/diff-checkstyle-hbase-server.txt |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1011/1/artifact/out/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1011/1/testReport/ |
   | Max. process+thread count | 5107 (vs. ulimit of 10000) |
   | modules | C: hbase-common hbase-client hbase-zookeeper hbase-server hbase-shell U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1011/1/console |
   | versions | git=2.11.0 maven=2018-06-17T18:33:14Z) findbugs=3.1.11 rubocop=0.79.0 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#discussion_r365340349
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RejectReplicationRequestStateChecker.java
 ##########
 @@ -1,4 +1,6 @@
-/**
+/*
+ * Copyright The Apache Software Foundation
 
 Review comment:
   We've had this discussion before. Its not javadoc so seems right to remove the extra '*'. We should do up an agreement on way to go. Did you say why you think we should preserve the '**'? Is it because its in the formatter?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#discussion_r366641900
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java
 ##########
 @@ -229,6 +244,43 @@ private void getMetaRegionLocation(CompletableFuture<RegionLocations> future,
         });
   }
 
+  @Override
+  public CompletableFuture<TableState> getMetaTableState() {
+    return getAndConvert(this.znodeMirroredMetaTableState, ZKAsyncRegistry::getTableState).
+      thenApply(state -> {
+        return state == null || state.equals(ENABLED_META_TABLE_STATE.getState())?
+          ENABLED_META_TABLE_STATE: new TableState(TableName.META_TABLE_NAME, state);
+      }).exceptionally(e -> {
 
 Review comment:
   Not relevant in new patch

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#discussion_r365337458
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseMetaEdit.java
 ##########
 @@ -0,0 +1,110 @@
+/*
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+/**
+ * Test being able to edit hbase:meta.
+ */
+@Category({MiscTests.class, LargeTests.class})
+public class TestHBaseMetaEdit {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestHBaseMetaEdit.class);
+  @Rule
+  public TestName name = new TestName();
+  private final static HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  @Before
+  public void before() throws Exception {
+    UTIL.startMiniCluster();
+  }
+
+  @After
+  public void after() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  /**
+   * Set versions, set HBASE-16213 indexed block encoding, and add a column family.
+   * Verify they are all in place by looking at TableDescriptor AND by checking
+   * what the RegionServer sees after opening Region.
+   */
+  @Test
+  public void testEditMeta() throws IOException {
+    Admin admin = UTIL.getAdmin();
+    admin.tableExists(TableName.META_TABLE_NAME);
+    admin.disableTable(TableName.META_TABLE_NAME);
+    assertTrue(admin.isTableDisabled(TableName.META_TABLE_NAME));
+    TableDescriptor descriptor = admin.getDescriptor(TableName.META_TABLE_NAME);
+    ColumnFamilyDescriptor cfd = descriptor.getColumnFamily(HConstants.CATALOG_FAMILY);
+    byte [] extraColumnFamilyName = Bytes.toBytes("xtra");
+    ColumnFamilyDescriptor newCfd =
+        ColumnFamilyDescriptorBuilder.newBuilder(extraColumnFamilyName).build();
+    int oldVersions = cfd.getMaxVersions();
+    // Add '1' to current versions count.
+    cfd = ColumnFamilyDescriptorBuilder.newBuilder(cfd).setMaxVersions(oldVersions + 1).
+        setConfiguration(ColumnFamilyDescriptorBuilder.DATA_BLOCK_ENCODING,
+            DataBlockEncoding.ROW_INDEX_V1.toString()).build();
+    admin.modifyColumnFamily(TableName.META_TABLE_NAME, cfd);
+    admin.addColumnFamily(TableName.META_TABLE_NAME, newCfd);
 
 Review comment:
   This is a good point.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#discussion_r365338167
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
 ##########
 @@ -135,16 +136,32 @@ public FSTableDescriptors(final Configuration conf, final FileSystem fs,
    *                     TODO: This is a workaround. Should remove this ugly code...
    */
   public FSTableDescriptors(final Configuration conf, final FileSystem fs,
-                            final Path rootdir, final boolean fsreadonly, final boolean usecache,
-                            Function<TableDescriptorBuilder, TableDescriptorBuilder> metaObserver) throws IOException {
+       final Path rootdir, final boolean fsreadonly, final boolean usecache,
+       Function<TableDescriptorBuilder, TableDescriptorBuilder> metaObserver) throws IOException {
     this.fs = fs;
     this.rootdir = rootdir;
     this.fsreadonly = fsreadonly;
     this.usecache = usecache;
-    this.metaTableDescriptor = metaObserver == null ? createMetaTableDescriptor(conf)
-          : metaObserver.apply(createMetaTableDescriptorBuilder(conf)).build();
+    TableDescriptor td = null;
+    try {
+      td = getTableDescriptorFromFs(fs, rootdir, TableName.META_TABLE_NAME);
+    } catch (TableInfoMissingException e) {
+      td = metaObserver == null? createMetaTableDescriptor(conf):
+        metaObserver.apply(createMetaTableDescriptorBuilder(conf)).build();
+      if (!fsreadonly) {
+        LOG.info("Creating new hbase:meta table default descriptor/schema {}", td);
+        updateTableDescriptor(td);
+      }
+    }
+    this.metaTableDescriptor = td;
   }
 
+  /**
+   *
+   * Should be private
+   * @deprecated Since 2.3.0. Should be for internal use only. Used by testing.
 
 Review comment:
   You are right. I wanted to take it private/protected but as you note, its Private so I can just do it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache9 commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#discussion_r365233838
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
 ##########
 @@ -135,16 +136,32 @@ public FSTableDescriptors(final Configuration conf, final FileSystem fs,
    *                     TODO: This is a workaround. Should remove this ugly code...
    */
   public FSTableDescriptors(final Configuration conf, final FileSystem fs,
-                            final Path rootdir, final boolean fsreadonly, final boolean usecache,
-                            Function<TableDescriptorBuilder, TableDescriptorBuilder> metaObserver) throws IOException {
+       final Path rootdir, final boolean fsreadonly, final boolean usecache,
+       Function<TableDescriptorBuilder, TableDescriptorBuilder> metaObserver) throws IOException {
     this.fs = fs;
     this.rootdir = rootdir;
     this.fsreadonly = fsreadonly;
     this.usecache = usecache;
-    this.metaTableDescriptor = metaObserver == null ? createMetaTableDescriptor(conf)
-          : metaObserver.apply(createMetaTableDescriptorBuilder(conf)).build();
+    TableDescriptor td = null;
+    try {
+      td = getTableDescriptorFromFs(fs, rootdir, TableName.META_TABLE_NAME);
+    } catch (TableInfoMissingException e) {
+      td = metaObserver == null? createMetaTableDescriptor(conf):
+        metaObserver.apply(createMetaTableDescriptorBuilder(conf)).build();
+      if (!fsreadonly) {
+        LOG.info("Creating new hbase:meta table default descriptor/schema {}", td);
+        updateTableDescriptor(td);
+      }
+    }
+    this.metaTableDescriptor = td;
   }
 
+  /**
+   *
+   * Should be private
+   * @deprecated Since 2.3.0. Should be for internal use only. Used by testing.
 
 Review comment:
   I do not think this should be deprecated? The class is IA.Private so it is OK to put internal methods here.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#discussion_r365330389
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java
 ##########
 @@ -58,8 +62,19 @@
 
   private final ZNodePaths znodePaths;
 
+  /**
+   * A znode maintained by MirroringTableStateManager.
+   * MirroringTableStateManager is deprecated to be removed in hbase3. It can also be disabled.
+   * Make sure it is enabled if you want to alter hbase:meta table in hbase2. In hbase3,
+   * TBD how metatable state will be hosted; likely on active hbase master.
 
 Review comment:
   The 'mirroring' feature predates this patch. This is an old facility this patch just makes use of. Changing this old features name is outside realm of this work.
   
   You get the bit that edit of meta is a rare, short-lived event and you get the bit that this implementation fails the state back to ENABLED if ever an issue, intentionally, to minimize our running into exotic situations. I don't want a permanently DISABLED state for hbase:meta, at least not at this stage in the game. We can do that later

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#discussion_r366641819
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
 ##########
 @@ -54,8 +53,20 @@
 // TODO: Make this a guava Service
 @InterfaceAudience.Private
 public class TableStateManager {
-
   private static final Logger LOG = LoggerFactory.getLogger(TableStateManager.class);
+
+  /**
+   * All table state is kept in hbase:meta except that of hbase:meta itself.
 
 Review comment:
   This discussion has been by-passed by your suggestion that we just support alter of meta, and not disable.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache9 commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#discussion_r365221090
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java
 ##########
 @@ -58,8 +62,19 @@
 
   private final ZNodePaths znodePaths;
 
+  /**
+   * A znode maintained by MirroringTableStateManager.
+   * MirroringTableStateManager is deprecated to be removed in hbase3. It can also be disabled.
+   * Make sure it is enabled if you want to alter hbase:meta table in hbase2. In hbase3,
+   * TBD how metatable state will be hosted; likely on active hbase master.
 
 Review comment:
   I think the state will either be on zk or on hdfs? HMaster itself is state less...
   
   In general, in the current architecture, I think it the state should be placed on zk, of course you could cache it in master and let client go to master to ask for the state.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#discussion_r365331618
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java
 ##########
 @@ -229,6 +244,43 @@ private void getMetaRegionLocation(CompletableFuture<RegionLocations> future,
         });
   }
 
+  @Override
+  public CompletableFuture<TableState> getMetaTableState() {
+    return getAndConvert(this.znodeMirroredMetaTableState, ZKAsyncRegistry::getTableState).
+      thenApply(state -> {
+        return state == null || state.equals(ENABLED_META_TABLE_STATE.getState())?
+          ENABLED_META_TABLE_STATE: new TableState(TableName.META_TABLE_NAME, state);
+      }).exceptionally(e -> {
 
 Review comment:
   Yeah, saw that. Here, if no znode -- probably because the mirroring table state manager is turned off -- then I want the return to be ENABLED (Completed CF). Should I change this?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache9 commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#discussion_r365488937
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RejectReplicationRequestStateChecker.java
 ##########
 @@ -1,4 +1,6 @@
-/**
+/*
+ * Copyright The Apache Software Foundation
 
 Review comment:
   I mean the 'Copyright The Apache Software Foundation'.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#discussion_r365337029
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
 ##########
 @@ -54,8 +53,20 @@
 // TODO: Make this a guava Service
 @InterfaceAudience.Private
 public class TableStateManager {
-
   private static final Logger LOG = LoggerFactory.getLogger(TableStateManager.class);
+
+  /**
+   * All table state is kept in hbase:meta except that of hbase:meta itself.
 
 Review comment:
   bq. We should persist it somewhere, and just cache it in master's memory.
   
   I explain why state is intentionally transient for now so table falls-back to ENABLED if problem like Master crash during alter; its one less thing for the operator to decipher and hbck2 navigate starting up a cluster; in other words we fallback to the 'known' state rather than persist DISABLED meta, a new condition that is sure to have holes in it.
   
   bq. And the solution here just assume that we only have one meta region? 
   
   All of the code base presumes this, not just this patch (this patch is part work to undo this presumption letting go of hardcoded meta schema).
   
   bq. I think this patch also aims to implement splittable meta in the future? No?
   
   Splittable meta is another project, not this one.
   
   bq. And we do not need to disable a table when altering any more? 
   
   This patch makes hbase:meta dynamic. Maybe the above will work. Let me try.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on issue #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack commented on issue #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#issuecomment-573125703
 
 
   @Apache9 Thanks for the review. Let me open a follow-on to address helpful feedback (Boolean => Void, Preventing CF deletion, etc.).
   
   On the 'bad design', your objections seem a little out-of-place ('... presumes single meta region...') or easy to counter and your worries that this patch unsettles the codebase seem to pale compared to what has already gone into this minor branch. How to proceed? A discussion on dev list? Or would a write up on how this works help?  A one-pager? Let me know.
   
   Let me reopen this while we are commenting.....

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache9 commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#discussion_r365489207
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
 ##########
 @@ -54,8 +53,20 @@
 // TODO: Make this a guava Service
 @InterfaceAudience.Private
 public class TableStateManager {
-
   private static final Logger LOG = LoggerFactory.getLogger(TableStateManager.class);
+
+  /**
+   * All table state is kept in hbase:meta except that of hbase:meta itself.
 
 Review comment:
   I know splittable meta is not implemented by this one but I think we should prepare for it? Not adding new difficulty, by introducing more 'there is only one meta region' assumptions...
   
   And I do not think we need to disable a table before altering it. The issue here is named 'Altering meta', not 'disabling meta', so let's try remove the disable/enable stuff and just implement the alter?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache9 commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #1011: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/1011#discussion_r365226730
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
 ##########
 @@ -54,8 +53,20 @@
 // TODO: Make this a guava Service
 @InterfaceAudience.Private
 public class TableStateManager {
-
   private static final Logger LOG = LoggerFactory.getLogger(TableStateManager.class);
+
+  /**
+   * All table state is kept in hbase:meta except that of hbase:meta itself.
 
 Review comment:
   In general I do not think this is a good design. We should persist it somewhere, and just cache it in master's memory. And we do not need to disable a table when altering any more? And the solution here just assume that we only have one meta region? So the altering operation can be atomic? What if we have multiple meta regions and we crash in the middle? I think this patch also aims to implement splittable meta in the future? No?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services