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/07 01:54:19 UTC

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

saintstack opened a new pull request #995: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/995
 
 
       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.

----------------------------------------------------------------
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 #995: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack commented on issue #995: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/995#issuecomment-572321045
 
 
   Fixed checkstyle and findbugs complaint.

----------------------------------------------------------------
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] bharathv commented on a change in pull request #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
 ##########
 @@ -948,22 +950,13 @@ protected Void waitOperationResult(long deadlineTs) throws IOException, TimeoutE
   @Override
   public boolean isTableEnabled(final TableName tableName) throws IOException {
     checkTableExists(tableName);
-    return executeCallable(new RpcRetryingCallable<Boolean>() {
-      @Override
-      protected Boolean rpcCall(int callTimeout) throws Exception {
-        TableState tableState = MetaTableAccessor.getTableState(getConnection(), tableName);
-        if (tableState == null) {
-          throw new TableNotFoundException(tableName);
-        }
-        return tableState.inStates(TableState.State.ENABLED);
-      }
-    });
+    return this.connection.getTableState(tableName).isEnabled();
 
 Review comment:
   Ack.

----------------------------------------------------------------
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] bharathv commented on a change in pull request #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegistry.java
 ##########
 @@ -29,12 +30,26 @@
  */
 @InterfaceAudience.Private
 interface AsyncRegistry extends Closeable {
+  /**
+   * A completed CompletableFuture to host default hbase:meta table state (ENABLED).
+   */
+  TableState ENABLED_META_TABLE_STATE =
+    new TableState(TableName.META_TABLE_NAME, TableState.State.ENABLED);
+  CompletableFuture<TableState> COMPLETED_GET_META_TABLE_STATE =
+    CompletableFuture.completedFuture(ENABLED_META_TABLE_STATE);
 
   /**
    * Get the location of meta region.
    */
   CompletableFuture<RegionLocations> getMetaRegionLocation();
 
+  /**
+   * The hbase:meta table state.
+   */
+  default CompletableFuture<TableState> getMetaTableState() {
 
 Review comment:
   More of a question: whats the need for a default implementation? This is almost always overriden by an implementation?

----------------------------------------------------------------
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] bharathv commented on a change in pull request #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java
 ##########
 @@ -229,6 +244,44 @@ private void getMetaRegionLocation(CompletableFuture<RegionLocations> future,
         });
   }
 
+  @Override
+  public CompletableFuture<TableState> getMetaTableState() {
 
 Review comment:
    think you missed the tests for this? we can add to TestZKAsyncRegistry.

----------------------------------------------------------------
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 #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
 ##########
 @@ -661,42 +662,54 @@ public void run(PRESP resp) {
       new DisableTableProcedureBiConsumer(tableName));
   }
 
+  /**
+   * Utility for completing passed {@link CompletableFuture} <code>future</code>
+   * using passed parameters.
+   */
+  private static CompletableFuture<Boolean> completeBoolean(CompletableFuture<Boolean> future,
 
 Review comment:
   Added small test in next push. It was interesting since TestZKRegistry has replicas on meta itself which made things more interesting.

----------------------------------------------------------------
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] bharathv commented on a change in pull request #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseMetaEdit.java
 ##########
 @@ -0,0 +1,111 @@
+/*
+ * 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 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;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutionException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * 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, ExecutionException, InterruptedException {
 
 Review comment:
   nit: ExecutionException, InterruptedException are not thrown anywhere?

----------------------------------------------------------------
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 #995: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #995: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/995#issuecomment-572370285
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 46s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  1s |  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 9 new or modified test files.  |
   ||| _ branch-2 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 25s |  branch-2 passed  |
   | +1 :green_heart: |  compile  |   2m 30s |  branch-2 passed  |
   | +1 :green_heart: |  checkstyle  |   3m  6s |  branch-2 passed  |
   | +1 :green_heart: |  shadedjars  |   4m 14s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 54s |  branch-2 passed  |
   | +0 :ok: |  spotbugs  |   0m 33s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   6m  1s |  branch-2 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 18s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 47s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 30s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 30s |  the patch passed  |
   | -1 :x: |  checkstyle  |   0m 45s |  hbase-client: The patch generated 2 new + 347 unchanged - 6 fixed = 349 total (was 353)  |
   | -1 :x: |  checkstyle  |   1m 29s |  hbase-server: The patch generated 6 new + 416 unchanged - 30 fixed = 422 total (was 446)  |
   | +1 :green_heart: |  rubocop  |   0m  3s |  There were no new rubocop issues.  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedjars  |   4m 13s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  15m 28s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   1m 52s |  the patch passed  |
   | -1 :x: |  findbugs  |   3m 35s |  hbase-server generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   3m  7s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   3m 39s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 50s |  hbase-zookeeper in the patch passed.  |
   | -1 :x: |  unit  | 156m  9s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |   8m  3s |  hbase-shell in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   2m 40s |  The patch does not generate ASF License warnings.  |
   |  |   | 244m 49s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hbase-server |
   |  |  Dead store to rootDir in org.apache.hadoop.hbase.snapshot.SnapshotManifest.consolidate()  At SnapshotManifest.java:org.apache.hadoop.hbase.snapshot.SnapshotManifest.consolidate()  At SnapshotManifest.java:[line 475] |
   | Failed junit tests | hadoop.hbase.TestClusterPortAssignment |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/5/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/995 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile rubocop |
   | uname | Linux 3ed10968b7b7 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/HBase-PreCommit-GitHub-PR_PR-995/out/precommit/personality/provided.sh |
   | git revision | branch-2 / e242d84b97 |
   | Default Java | 1.8.0_181 |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/5/artifact/out/diff-checkstyle-hbase-client.txt |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/5/artifact/out/diff-checkstyle-hbase-server.txt |
   | findbugs | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/5/artifact/out/new-findbugs-hbase-server.html |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/5/artifact/out/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/5/testReport/ |
   | Max. process+thread count | 4267 (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-995/5/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] bharathv commented on a change in pull request #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
 ##########
 @@ -948,22 +950,13 @@ protected Void waitOperationResult(long deadlineTs) throws IOException, TimeoutE
   @Override
   public boolean isTableEnabled(final TableName tableName) throws IOException {
     checkTableExists(tableName);
-    return executeCallable(new RpcRetryingCallable<Boolean>() {
-      @Override
-      protected Boolean rpcCall(int callTimeout) throws Exception {
-        TableState tableState = MetaTableAccessor.getTableState(getConnection(), tableName);
-        if (tableState == null) {
-          throw new TableNotFoundException(tableName);
-        }
-        return tableState.inStates(TableState.State.ENABLED);
-      }
-    });
+    return this.connection.getTableState(tableName).isEnabled();
 
 Review comment:
   nit: is this change in behavior intentional? Retrying -> non-retrying

----------------------------------------------------------------
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 #995: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #995: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/995#issuecomment-571973956
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 37s |  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 7 new or modified test files.  |
   ||| _ branch-2 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 14s |  branch-2 passed  |
   | +1 :green_heart: |  compile  |   2m 30s |  branch-2 passed  |
   | +1 :green_heart: |  checkstyle  |   3m  8s |  branch-2 passed  |
   | +1 :green_heart: |  shadedjars  |   4m 13s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 51s |  branch-2 passed  |
   | +0 :ok: |  spotbugs  |   0m 35s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   6m  2s |  branch-2 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 50s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 28s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 28s |  the patch passed  |
   | -1 :x: |  checkstyle  |   0m 45s |  hbase-client: The patch generated 6 new + 348 unchanged - 5 fixed = 354 total (was 353)  |
   | -1 :x: |  checkstyle  |   1m 28s |  hbase-server: The patch generated 2 new + 412 unchanged - 28 fixed = 414 total (was 440)  |
   | +1 :green_heart: |  rubocop  |   0m  3s |  There were no new rubocop issues.  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedjars  |   4m 14s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  15m 35s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   1m 53s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   6m 30s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   3m  7s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   3m 41s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 50s |  hbase-zookeeper in the patch passed.  |
   | -1 :x: |  unit  | 150m  8s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |   8m  1s |  hbase-shell in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   2m 41s |  The patch does not generate ASF License warnings.  |
   |  |   | 238m 35s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hbase.master.assignment.TestCloseRegionWhileRSCrash |
   |   | hadoop.hbase.client.TestSnapshotTemporaryDirectory |
   |   | hadoop.hbase.client.TestAsyncTableAdminApi3 |
   |   | hadoop.hbase.client.TestSnapshotDFSTemporaryDirectory |
   |   | hadoop.hbase.client.TestSnapshotTemporaryDirectoryWithRegionReplicas |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/3/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/995 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile rubocop |
   | uname | Linux 24394696d0cd 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/HBase-PreCommit-GitHub-PR_PR-995/out/precommit/personality/provided.sh |
   | git revision | branch-2 / 4f7de9e27e |
   | Default Java | 1.8.0_181 |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/3/artifact/out/diff-checkstyle-hbase-client.txt |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/3/artifact/out/diff-checkstyle-hbase-server.txt |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/3/artifact/out/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/3/testReport/ |
   | Max. process+thread count | 4388 (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-995/3/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 #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
 ##########
 @@ -661,42 +662,54 @@ public void run(PRESP resp) {
       new DisableTableProcedureBiConsumer(tableName));
   }
 
+  /**
+   * Utility for completing passed {@link CompletableFuture} <code>future</code>
+   * using passed parameters.
+   */
+  private static CompletableFuture<Boolean> completeBoolean(CompletableFuture<Boolean> future,
 
 Review comment:
   Let me fix in next 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] saintstack commented on a change in pull request #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java
 ##########
 @@ -128,4 +130,42 @@ public void testNoMetaAvailable() throws InterruptedException {
       }
     }
   }
+
+  /**
+   * Test meta tablestate implementation.
+   * Test is a bit involved because meta has replicas... Replica assign lags so check
+   * between steps all assigned.
+   */
+  @Test
+  public void testMetaTableState() throws IOException, ExecutionException, InterruptedException {
+    assertTrue(TEST_UTIL.getMiniHBaseCluster().getMaster().isActiveMaster());
+    while (!TEST_UTIL.getMiniHBaseCluster().getMaster().isInitialized()) {
 
 Review comment:
   Leftover from debug. Purged.

----------------------------------------------------------------
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 #995: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack commented on issue #995: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/995#issuecomment-571844515
 
 
   Address helpful feedback. Fix test failures.

----------------------------------------------------------------
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 #995: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #995: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/995#issuecomment-572310549
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 42s |  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.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 9 new or modified test files.  |
   ||| _ branch-2 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 55s |  branch-2 passed  |
   | +1 :green_heart: |  compile  |   2m 42s |  branch-2 passed  |
   | +1 :green_heart: |  checkstyle  |   3m 14s |  branch-2 passed  |
   | +1 :green_heart: |  shadedjars  |   4m 22s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 52s |  branch-2 passed  |
   | +0 :ok: |  spotbugs  |   0m 36s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   6m 21s |  branch-2 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m 23s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 33s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 33s |  the patch passed  |
   | -1 :x: |  checkstyle  |   0m 46s |  hbase-client: The patch generated 2 new + 347 unchanged - 6 fixed = 349 total (was 353)  |
   | -1 :x: |  checkstyle  |   1m 30s |  hbase-server: The patch generated 5 new + 416 unchanged - 30 fixed = 421 total (was 446)  |
   | +1 :green_heart: |  rubocop  |   0m  3s |  There were no new rubocop issues.  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedjars  |   4m 21s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  15m 54s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   1m 54s |  the patch passed  |
   | -1 :x: |  findbugs  |   3m 48s |  hbase-server generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   3m  0s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   3m 39s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 51s |  hbase-zookeeper in the patch passed.  |
   | +1 :green_heart: |  unit  | 147m 35s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |   7m 14s |  hbase-shell in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   2m 44s |  The patch does not generate ASF License warnings.  |
   |  |   | 238m  6s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hbase-server |
   |  |  Dead store to rootDir in org.apache.hadoop.hbase.snapshot.SnapshotManifest.consolidate()  At SnapshotManifest.java:org.apache.hadoop.hbase.snapshot.SnapshotManifest.consolidate()  At SnapshotManifest.java:[line 475] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/4/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/995 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile rubocop |
   | uname | Linux 6e9de063fc55 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/HBase-PreCommit-GitHub-PR_PR-995/out/precommit/personality/provided.sh |
   | git revision | branch-2 / e242d84b97 |
   | Default Java | 1.8.0_181 |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/4/artifact/out/diff-checkstyle-hbase-client.txt |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/4/artifact/out/diff-checkstyle-hbase-server.txt |
   | findbugs | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/4/artifact/out/new-findbugs-hbase-server.html |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/4/testReport/ |
   | Max. process+thread count | 5585 (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-995/4/console |
   | versions | git=2.11.0 maven=2018-06-17T18:33:14Z) findbugs=3.1.11 rubocop=0.78.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 #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
 ##########
 @@ -330,6 +342,26 @@ public void newDead(ServerName sn) {
       close();
       throw e;
     }
+    // Create tablestate cache. Add a loader that know how to find table state.
+    int duration = this.conf.getInt(TABLESTATE_CACHE_DURATION_MS, 1000);
+    this.tableStateCache = CacheBuilder.newBuilder().
+      expireAfterWrite(duration, TimeUnit.MILLISECONDS).
+      recordStats().
 
 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 commented on a change in pull request #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
 ##########
 @@ -154,6 +156,16 @@
   public static final String RETRIES_BY_SERVER_KEY = "hbase.client.retries.by.server";
   private static final Logger LOG = LoggerFactory.getLogger(ConnectionImplementation.class);
 
+  /**
+   * TableState cache.
+   */
+  private final LoadingCache<TableName, TableState> tableStateCache;
 
 Review comment:
   Added more comment on the cache data member explaining why reproduced here:
   
      * Table States change super rarely. In synchronous client, state can be queried a lot
      * particularly when Regions are moving. Its ok if we are not super responsive noticing
      * Table State change. So, cache the last look up for a period. Use
      * {@link #TABLESTATE_CACHE_DURATION_MS} to change default of one second.
   
   On your nit, yeah, I think it fine reading table state from cache even if it stale. It will update after 1000ms. Region location lookups are more likely because of server crash or region balance... If table is being offlined, there'll be a little lag because of the cache but in scheme of things -- especially if a table disable -- I think it ok.
   
   There is no cache for async client. It goes against async API. TODO.

----------------------------------------------------------------
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 #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegistry.java
 ##########
 @@ -29,12 +30,26 @@
  */
 @InterfaceAudience.Private
 interface AsyncRegistry extends Closeable {
+  /**
+   * A completed CompletableFuture to host default hbase:meta table state (ENABLED).
+   */
+  TableState ENABLED_META_TABLE_STATE =
+    new TableState(TableName.META_TABLE_NAME, TableState.State.ENABLED);
+  CompletableFuture<TableState> COMPLETED_GET_META_TABLE_STATE =
+    CompletableFuture.completedFuture(ENABLED_META_TABLE_STATE);
 
   /**
    * Get the location of meta region.
    */
   CompletableFuture<RegionLocations> getMetaRegionLocation();
 
+  /**
+   * The hbase:meta table state.
+   */
+  default CompletableFuture<TableState> getMetaTableState() {
 
 Review comment:
   No. The opposite I'd say. Up to this, meta table state was hard-wired ENABLED -- i.e. the default here. There are two Registry implementations currently. The ZK-based one and the one you are working on. The ZKRegistry got an impl as part of this patch. Will work on one for when your Registry lands. Meantime, default impl.

----------------------------------------------------------------
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] bharathv commented on a change in pull request #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java
 ##########
 @@ -128,4 +130,42 @@ public void testNoMetaAvailable() throws InterruptedException {
       }
     }
   }
+
+  /**
+   * Test meta tablestate implementation.
+   * Test is a bit involved because meta has replicas... Replica assign lags so check
+   * between steps all assigned.
+   */
+  @Test
+  public void testMetaTableState() throws IOException, ExecutionException, InterruptedException {
+    assertTrue(TEST_UTIL.getMiniHBaseCluster().getMaster().isActiveMaster());
+    while (!TEST_UTIL.getMiniHBaseCluster().getMaster().isInitialized()) {
 
 Review comment:
   q: Isn't this guaranteed by startMiniCluster()? It doesn't return until there is an active initialized master and it can scan meta. If so, can be omitted?

----------------------------------------------------------------
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 #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java
 ##########
 @@ -229,6 +244,44 @@ private void getMetaRegionLocation(CompletableFuture<RegionLocations> future,
         });
   }
 
+  @Override
+  public CompletableFuture<TableState> getMetaTableState() {
 
 Review comment:
   Added small test in next push. It was interesting since TestZKRegistry has replicas on meta itself which made things more interesting.

----------------------------------------------------------------
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] bharathv commented on a change in pull request #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
 ##########
 @@ -330,6 +342,26 @@ public void newDead(ServerName sn) {
       close();
       throw e;
     }
+    // Create tablestate cache. Add a loader that know how to find table state.
+    int duration = this.conf.getInt(TABLESTATE_CACHE_DURATION_MS, 1000);
+    this.tableStateCache = CacheBuilder.newBuilder().
+      expireAfterWrite(duration, TimeUnit.MILLISECONDS).
+      recordStats().
 
 Review comment:
   nit: The patch doesn't use the recorded states anywhere. Unnecessary overhead, can be removed?

----------------------------------------------------------------
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] bharathv commented on a change in pull request #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java
 ##########
 @@ -128,4 +130,42 @@ public void testNoMetaAvailable() throws InterruptedException {
       }
     }
   }
+
+  /**
+   * Test meta tablestate implementation.
+   * Test is a bit involved because meta has replicas... Replica assign lags so check
+   * between steps all assigned.
+   */
+  @Test
+  public void testMetaTableState() throws IOException, ExecutionException, InterruptedException {
+    assertTrue(TEST_UTIL.getMiniHBaseCluster().getMaster().isActiveMaster());
+    while (!TEST_UTIL.getMiniHBaseCluster().getMaster().isInitialized()) {
+      Threads.sleep(10);
+    }
+    while (TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager().
 
 Review comment:
   nit: s/ waitUntilNoRegionsInTransition(timeout)

----------------------------------------------------------------
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] bharathv commented on a change in pull request #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
 ##########
 @@ -154,6 +156,16 @@
   public static final String RETRIES_BY_SERVER_KEY = "hbase.client.retries.by.server";
   private static final Logger LOG = LoggerFactory.getLogger(ConnectionImplementation.class);
 
+  /**
+   * TableState cache.
+   */
+  private final LoadingCache<TableName, TableState> tableStateCache;
 
 Review comment:
   Q: whats the motivation to use a cache here? To reduce the number of rpcs for table state look ups? 
   
   nit: One behavior I noticed is that the general patten of client is to force reload if something doesn't look as expected (retries..). For example relocateRegion() doesn't care if any meta hrls are cached, it force reloads them. Does that pattern also apply for table state cache? For example, in the following snippet, isTableDisabled can now be served from cache. Not big of a deal probably because the cache ttl is very low and table states probably  don't flip between disabled/enabled that often  but wanted to point it out.
   
   ```
   @Override
     public RegionLocations relocateRegion(final TableName tableName,
         final byte [] row, int replicaId) throws IOException{
       if (isTableDisabled(tableName)) {
         throw new TableNotEnabledException(tableName.getNameAsString() + " is disabled.");
       }
       return locateRegion(tableName, row, false, true, replicaId);
     } 
   ```

----------------------------------------------------------------
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 #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseMetaEdit.java
 ##########
 @@ -0,0 +1,111 @@
+/*
+ * 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 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;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutionException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * 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, ExecutionException, InterruptedException {
 
 Review comment:
   Thanks.

----------------------------------------------------------------
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 #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java
 ##########
 @@ -229,6 +244,44 @@ private void getMetaRegionLocation(CompletableFuture<RegionLocations> future,
         });
   }
 
+  @Override
+  public CompletableFuture<TableState> getMetaTableState() {
 
 Review comment:
   Yes. Let me add. Thanks.

----------------------------------------------------------------
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] bharathv commented on a change in pull request #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
 ##########
 @@ -661,42 +662,54 @@ public void run(PRESP resp) {
       new DisableTableProcedureBiConsumer(tableName));
   }
 
+  /**
+   * Utility for completing passed {@link CompletableFuture} <code>future</code>
+   * using passed parameters.
+   */
+  private static CompletableFuture<Boolean> completeBoolean(CompletableFuture<Boolean> future,
 
 Review comment:
   nit: I think checkTableState() or something would be a more appropriate name since this is not a generic function that works for any boolean? 

----------------------------------------------------------------
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 #995: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack commented on issue #995: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/995#issuecomment-572246784
 
 
   @bharathv Thanks for review help.

----------------------------------------------------------------
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] bharathv commented on a change in pull request #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
 ##########
 @@ -1112,7 +1119,7 @@ public static RegionInfo getRegionInfo(final Result r, byte [] qualifier) {
   public static TableState getTableState(Connection conn, TableName tableName)
       throws IOException {
     if (tableName.equals(TableName.META_TABLE_NAME)) {
-      return new TableState(tableName, TableState.State.ENABLED);
+      throw new IllegalAccessError("Not for hbase:meta table state");
 
 Review comment:
   nit: can be simplified to Preconditions.checkArgument()?

----------------------------------------------------------------
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] bharathv commented on a change in pull request #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java
 ##########
 @@ -128,4 +130,42 @@ public void testNoMetaAvailable() throws InterruptedException {
       }
     }
   }
+
+  /**
+   * Test meta tablestate implementation.
+   * Test is a bit involved because meta has replicas... Replica assign lags so check
+   * between steps all assigned.
+   */
+  @Test
+  public void testMetaTableState() throws IOException, ExecutionException, InterruptedException {
+    assertTrue(TEST_UTIL.getMiniHBaseCluster().getMaster().isActiveMaster());
+    while (!TEST_UTIL.getMiniHBaseCluster().getMaster().isInitialized()) {
+      Threads.sleep(10);
+    }
+    while (TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager().
+        hasRegionsInTransition()) {
+      Threads.sleep(10);
+    }
+    LOG.info("MASTER INITIALIZED");
+    try (ZKAsyncRegistry registry = new ZKAsyncRegistry(TEST_UTIL.getConfiguration())) {
+      assertEquals(TableState.State.ENABLED, registry.getMetaTableState().get().getState());
+      LOG.info("META ENABLED");
+      try (Admin admin = TEST_UTIL.getConnection().getAdmin()) {
+        admin.disableTable(TableName.META_TABLE_NAME);
+        assertEquals(TableState.State.DISABLED, registry.getMetaTableState().get().getState());
+        while (TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager().
+          hasRegionsInTransition()) {
+          Threads.sleep(10);
+        }
+        LOG.info("META DISABLED");
+        admin.enableTable(TableName.META_TABLE_NAME);
+        assertEquals(TableState.State.ENABLED, registry.getMetaTableState().get().getState());
+        while (TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager().
 
 Review comment:
   same.

----------------------------------------------------------------
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 #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java
 ##########
 @@ -229,6 +242,33 @@ private void getMetaRegionLocation(CompletableFuture<RegionLocations> future,
         });
   }
 
+  @Override
+  public CompletableFuture<TableState> getMetaTableState() {
+    CompletableFuture<TableState> result =
+      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);
+      });
+    // If completed exceptionally return default ENABLED state.
+    return result.isCompletedExceptionally()? COMPLETED_GET_META_TABLE_STATE: result;
 
 Review comment:
   You are right. Undid this.  Letting any exception out instead of trying to paper it over with a default.
   
   Will add test.

----------------------------------------------------------------
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 #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
 ##########
 @@ -948,22 +950,13 @@ protected Void waitOperationResult(long deadlineTs) throws IOException, TimeoutE
   @Override
   public boolean isTableEnabled(final TableName tableName) throws IOException {
     checkTableExists(tableName);
-    return executeCallable(new RpcRetryingCallable<Boolean>() {
-      @Override
-      protected Boolean rpcCall(int callTimeout) throws Exception {
-        TableState tableState = MetaTableAccessor.getTableState(getConnection(), tableName);
-        if (tableState == null) {
-          throw new TableNotFoundException(tableName);
-        }
-        return tableState.inStates(TableState.State.ENABLED);
-      }
-    });
+    return this.connection.getTableState(tableName).isEnabled();
 
 Review comment:
   Yes. Removed it so symmetric with isDisabled and retrying cached value seemed odd.
   
   Maybe I should put this back though just to minimize differences.

----------------------------------------------------------------
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 #995: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #995: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/995#issuecomment-571893043
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 35s |  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.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 5 new or modified test files.  |
   ||| _ branch-2 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 13s |  branch-2 passed  |
   | +1 :green_heart: |  compile  |   2m 29s |  branch-2 passed  |
   | +1 :green_heart: |  checkstyle  |   3m  6s |  branch-2 passed  |
   | +1 :green_heart: |  shadedjars  |   4m 17s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 51s |  branch-2 passed  |
   | +0 :ok: |  spotbugs  |   0m 35s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   5m 57s |  branch-2 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 51s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 30s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 30s |  the patch passed  |
   | -1 :x: |  checkstyle  |   0m 44s |  hbase-client: The patch generated 10 new + 348 unchanged - 5 fixed = 358 total (was 353)  |
   | -1 :x: |  checkstyle  |   1m 30s |  hbase-server: The patch generated 4 new + 400 unchanged - 28 fixed = 404 total (was 428)  |
   | +1 :green_heart: |  rubocop  |   0m  3s |  There were no new rubocop issues.  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedjars  |   4m 13s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  15m 17s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   1m 51s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   6m 25s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   3m  6s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   3m 42s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 49s |  hbase-zookeeper in the patch passed.  |
   | -1 :x: |  unit  | 148m 54s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |   7m 58s |  hbase-shell in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   2m 40s |  The patch does not generate ASF License warnings.  |
   |  |   | 236m 45s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hbase.client.TestAdmin2 |
   |   | hadoop.hbase.client.TestAsyncTableAdminApi2 |
   |   | hadoop.hbase.client.TestSnapshotTemporaryDirectory |
   |   | hadoop.hbase.client.TestAsyncTableAdminApi3 |
   |   | hadoop.hbase.client.TestSnapshotDFSTemporaryDirectory |
   |   | hadoop.hbase.client.TestSnapshotTemporaryDirectoryWithRegionReplicas |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/2/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/995 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile rubocop |
   | uname | Linux d56b82ad10c9 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/HBase-PreCommit-GitHub-PR_PR-995/out/precommit/personality/provided.sh |
   | git revision | branch-2 / 4f7de9e27e |
   | Default Java | 1.8.0_181 |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/2/artifact/out/diff-checkstyle-hbase-client.txt |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/2/artifact/out/diff-checkstyle-hbase-server.txt |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/2/artifact/out/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/2/testReport/ |
   | Max. process+thread count | 4494 (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-995/2/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] Apache-HBase commented on issue #995: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #995: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/995#issuecomment-572434235
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 40s |  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 9 new or modified test files.  |
   ||| _ branch-2 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 19s |  branch-2 passed  |
   | +1 :green_heart: |  compile  |   2m 34s |  branch-2 passed  |
   | +1 :green_heart: |  checkstyle  |   3m  9s |  branch-2 passed  |
   | +1 :green_heart: |  shadedjars  |   4m 21s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 55s |  branch-2 passed  |
   | +0 :ok: |  spotbugs  |   0m 36s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   6m 15s |  branch-2 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m  3s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 30s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 30s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 26s |  The patch passed checkstyle in hbase-common  |
   | +1 :green_heart: |  checkstyle  |   0m 47s |  hbase-client: The patch generated 0 new + 348 unchanged - 5 fixed = 348 total (was 353)  |
   | +1 :green_heart: |  checkstyle  |   0m 14s |  The patch passed checkstyle in hbase-zookeeper  |
   | -1 :x: |  checkstyle  |   1m 28s |  hbase-server: The patch generated 1 new + 416 unchanged - 30 fixed = 417 total (was 446)  |
   | +1 :green_heart: |  checkstyle  |   0m 12s |  The patch passed checkstyle in hbase-shell  |
   | +1 :green_heart: |  rubocop  |   0m  3s |  There were no new rubocop issues.  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedjars  |   4m 39s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  15m 45s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   1m 54s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   6m 41s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   3m 10s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   3m 45s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 50s |  hbase-zookeeper in the patch passed.  |
   | +1 :green_heart: |  unit  | 148m 53s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |   8m  2s |  hbase-shell in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   2m 42s |  The patch does not generate ASF License warnings.  |
   |  |   | 239m 40s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/6/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/995 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile rubocop |
   | uname | Linux 1aae5c1c1f6e 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/HBase-PreCommit-GitHub-PR_PR-995/out/precommit/personality/provided.sh |
   | git revision | branch-2 / e242d84b97 |
   | Default Java | 1.8.0_181 |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/6/artifact/out/diff-checkstyle-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/6/testReport/ |
   | Max. process+thread count | 4425 (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-995/6/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] bharathv commented on a change in pull request #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterKeepAliveConnection.java
 ##########
 @@ -35,7 +38,8 @@
  * final user code. Hence it's package protected.
  */
 @InterfaceAudience.Private
-interface MasterKeepAliveConnection extends MasterProtos.MasterService.BlockingInterface {
+interface MasterKeepAliveConnection extends
+    MasterProtos.MasterService.BlockingInterface, Closeable {
   // Do this instead of implement Closeable because closeable returning IOE is PITA.
 
 Review comment:
   nit: update the comment? Thats contradicting.

----------------------------------------------------------------
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 #995: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack commented on issue #995: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/995#issuecomment-572677533
 
 
   Merged manually to branch-2 + after fixing single checkstyle complaint above about unused import. Thanks for reviews @bharathv .

----------------------------------------------------------------
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 #995: HBASE-23055 Alter hbase:meta

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

----------------------------------------------------------------
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 #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java
 ##########
 @@ -128,4 +130,42 @@ public void testNoMetaAvailable() throws InterruptedException {
       }
     }
   }
+
+  /**
+   * Test meta tablestate implementation.
+   * Test is a bit involved because meta has replicas... Replica assign lags so check
+   * between steps all assigned.
+   */
+  @Test
+  public void testMetaTableState() throws IOException, ExecutionException, InterruptedException {
+    assertTrue(TEST_UTIL.getMiniHBaseCluster().getMaster().isActiveMaster());
+    while (!TEST_UTIL.getMiniHBaseCluster().getMaster().isInitialized()) {
+      Threads.sleep(10);
+    }
+    while (TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager().
 
 Review comment:
   Your suggestion is better than what I had.

----------------------------------------------------------------
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 #995: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack commented on issue #995: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/995#issuecomment-572238258
 
 
   Address most recent set of @bharathv  comments (thanks for review!)

----------------------------------------------------------------
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 #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterKeepAliveConnection.java
 ##########
 @@ -35,7 +38,8 @@
  * final user code. Hence it's package protected.
  */
 @InterfaceAudience.Private
-interface MasterKeepAliveConnection extends MasterProtos.MasterService.BlockingInterface {
+interface MasterKeepAliveConnection extends
+    MasterProtos.MasterService.BlockingInterface, Closeable {
   // Do this instead of implement Closeable because closeable returning IOE is PITA.
 
 Review comment:
   You are right.

----------------------------------------------------------------
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 #995: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack commented on issue #995: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/995#issuecomment-571901659
 
 
   Thanks for the review @bharathv . Helps. I just pushed some more checkstyle and test fixes. Will address your helpful comments in the morning. Thanks.

----------------------------------------------------------------
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 #995: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
saintstack commented on issue #995: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/995#issuecomment-572225043
 
 
   Fix tests and checkstyle. Address @bharathv 's last set of comments.

----------------------------------------------------------------
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 #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
 ##########
 @@ -1112,7 +1119,7 @@ public static RegionInfo getRegionInfo(final Result r, byte [] qualifier) {
   public static TableState getTableState(Connection conn, TableName tableName)
       throws IOException {
     if (tableName.equals(TableName.META_TABLE_NAME)) {
-      return new TableState(tableName, TableState.State.ENABLED);
+      throw new IllegalAccessError("Not for hbase:meta table state");
 
 Review comment:
   Done

----------------------------------------------------------------
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 #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
 ##########
 @@ -661,42 +662,54 @@ public void run(PRESP resp) {
       new DisableTableProcedureBiConsumer(tableName));
   }
 
+  /**
+   * Utility for completing passed {@link CompletableFuture} <code>future</code>
+   * using passed parameters.
+   */
+  private static CompletableFuture<Boolean> completeBoolean(CompletableFuture<Boolean> future,
 
 Review comment:
   Added small test in next push. It was interesting since TestZKRegistry has replicas on meta itself which made things more interesting.

----------------------------------------------------------------
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 #995: HBASE-23055 Alter hbase:meta

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #995: HBASE-23055 Alter hbase:meta
URL: https://github.com/apache/hbase/pull/995#issuecomment-571449703
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   2m  6s |  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 3 new or modified test files.  |
   ||| _ branch-2 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 19s |  branch-2 passed  |
   | +1 :green_heart: |  compile  |   2m 27s |  branch-2 passed  |
   | +1 :green_heart: |  checkstyle  |   3m  8s |  branch-2 passed  |
   | +1 :green_heart: |  shadedjars  |   4m 21s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 55s |  branch-2 passed  |
   | +0 :ok: |  spotbugs  |   0m 36s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   6m  0s |  branch-2 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 45s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 28s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 28s |  the patch passed  |
   | -1 :x: |  checkstyle  |   0m 45s |  hbase-client: The patch generated 4 new + 348 unchanged - 5 fixed = 352 total (was 353)  |
   | -1 :x: |  checkstyle  |   1m 27s |  hbase-server: The patch generated 5 new + 374 unchanged - 28 fixed = 379 total (was 402)  |
   | +1 :green_heart: |  rubocop  |   0m  3s |  There were no new rubocop issues.  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedjars  |   4m 15s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  15m 33s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   1m 52s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   6m 28s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   3m  7s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   3m 41s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 49s |  hbase-zookeeper in the patch passed.  |
   | -1 :x: |  unit  | 150m 23s |  hbase-server in the patch failed.  |
   | -1 :x: |  unit  |   8m  2s |  hbase-shell in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   2m 43s |  The patch does not generate ASF License warnings.  |
   |  |   | 240m 29s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hbase.util.TestFSTableDescriptors |
   |   | hadoop.hbase.client.TestAdmin2 |
   |   | hadoop.hbase.client.TestAsyncTableAdminApi2 |
   |   | hadoop.hbase.regionserver.TestClusterId |
   |   | hadoop.hbase.TestMetaTableAccessorNoCluster |
   |   | hadoop.hbase.client.TestSnapshotMetadata |
   |   | hadoop.hbase.regionserver.TestPriorityRpc |
   |   | hadoop.hbase.coprocessor.TestMasterObserver |
   |   | hadoop.hbase.TestClusterBootOrder |
   |   | hadoop.hbase.client.TestAsyncNonMetaRegionLocator |
   |   | hadoop.hbase.client.TestSnapshotTemporaryDirectory |
   |   | hadoop.hbase.client.TestAsyncTableAdminApi3 |
   |   | hadoop.hbase.client.TestSnapshotDFSTemporaryDirectory |
   |   | hadoop.hbase.client.TestSnapshotTemporaryDirectoryWithRegionReplicas |
   |   | hadoop.hbase.client.TestAdminShell2 |
   |   | hadoop.hbase.client.TestAdminShell |
   |   | hadoop.hbase.client.TestShell |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/995 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile rubocop |
   | uname | Linux 8c6432b9d7de 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/HBase-PreCommit-GitHub-PR_PR-995/out/precommit/personality/provided.sh |
   | git revision | branch-2 / 1415a82d41 |
   | Default Java | 1.8.0_181 |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/1/artifact/out/diff-checkstyle-hbase-client.txt |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/1/artifact/out/diff-checkstyle-hbase-server.txt |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/1/artifact/out/patch-unit-hbase-server.txt |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/1/artifact/out/patch-unit-hbase-shell.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-995/1/testReport/ |
   | Max. process+thread count | 4430 (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-995/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] bharathv commented on a change in pull request #995: HBASE-23055 Alter hbase:meta

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

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java
 ##########
 @@ -229,6 +242,33 @@ private void getMetaRegionLocation(CompletableFuture<RegionLocations> future,
         });
   }
 
+  @Override
+  public CompletableFuture<TableState> getMetaTableState() {
+    CompletableFuture<TableState> result =
+      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);
+      });
+    // If completed exceptionally return default ENABLED state.
+    return result.isCompletedExceptionally()? COMPLETED_GET_META_TABLE_STATE: result;
 
 Review comment:
   Putting it out here means its blocking and not async right? (or did I read it incorrectly?)
   
   I think this method could use some test coverage for a bunch of table states.

----------------------------------------------------------------
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