You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by nd...@apache.org on 2020/09/18 20:30:34 UTC

[hbase] branch branch-2.3 updated: HBASE-24896 'Stuck' in static initialization creating RegionInfo instance

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

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


The following commit(s) were added to refs/heads/branch-2.3 by this push:
     new b28c3bd  HBASE-24896 'Stuck' in static initialization creating RegionInfo instance
b28c3bd is described below

commit b28c3bd5cfac8b1abb39cfa513554f19a9365b1f
Author: stack <st...@apache.org>
AuthorDate: Wed Aug 19 22:23:25 2020 -0700

    HBASE-24896 'Stuck' in static initialization creating RegionInfo instance
    
    Untangle RegionInfo, RegionInfoBuilder, and MutableRegionInfo static
    initializations some. Move MutableRegionInfo from inner-class of
    RegionInfoBuilder to be (package private) standalone. Undo static
    initializing references from RI to RIB.
    
    Co-authored-by: Nick Dimiduk <nd...@apache.org>
    Signed-off-by: Bharath Vissapragada <bh...@apache.org>
    Signed-off-by: Duo Zhang <zh...@apache.org>
    Signed-off-by: Viraj Jasani <vj...@apache.org>
---
 .../org/apache/hadoop/hbase/client/RegionInfo.java |  9 ++-
 .../TestRegionInfoStaticInitialization.java        | 71 ++++++++++++++++++++++
 2 files changed, 77 insertions(+), 3 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
index 7a3a9af..493b389 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
@@ -71,7 +71,9 @@ public interface RegionInfo extends Comparable<RegionInfo> {
    */
   @Deprecated
   @InterfaceAudience.Private
-  RegionInfo UNDEFINED = RegionInfoBuilder.newBuilder(TableName.valueOf("__UNDEFINED__")).build();
+  // Not using RegionInfoBuilder intentionally to avoid a static loading deadlock: HBASE-24627
+  RegionInfo UNDEFINED = new MutableRegionInfo(0, TableName.valueOf("__UNDEFINED__"),
+    RegionInfo.DEFAULT_REPLICA_ID);
 
   /**
    * Separator used to demarcate the encodedName in a region name
@@ -588,8 +590,9 @@ public interface RegionInfo extends Comparable<RegionInfo> {
    * @return the MOB {@link RegionInfo}.
    */
   static RegionInfo createMobRegionInfo(TableName tableName) {
-    return RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes(".mob")).
-      setRegionId(0).build();
+    // Skipping reference to RegionInfoBuilder in this class.
+    return new MutableRegionInfo(tableName, Bytes.toBytes(".mob"),
+      HConstants.EMPTY_END_ROW, false, 0, DEFAULT_REPLICA_ID, false);
   }
 
   /**
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfoStaticInitialization.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfoStaticInitialization.java
new file mode 100644
index 0000000..e56bfcb
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfoStaticInitialization.java
@@ -0,0 +1,71 @@
+/*
+ * 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.regionserver;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import java.util.stream.Stream;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test for the tangled mess that is static initialization of our our {@link HRegionInfo} and
+ * {@link RegionInfoBuilder}, as reported on HBASE-24896. The condition being tested can only be
+ * reproduced the first time a JVM loads the classes under test. Thus, this test is marked as a
+ * {@link LargeTests} because, under their current configuration, tests in that category are run
+ * in their own JVM instances.
+ */
+@SuppressWarnings("deprecation")
+@Category({ RegionServerTests.class, LargeTests.class})
+public class TestRegionInfoStaticInitialization {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestRegionInfoStaticInitialization.class);
+
+  @Test
+  public void testParallelStaticInitialization() throws Exception {
+    // The JVM loads symbols lazily. These suppliers reference two symbols that, before this patch,
+    // are mutually dependent and expose a deadlock in the loading of symbols from RegionInfo and
+    // RegionInfoBuilder.
+    final Supplier<RegionInfo> retrieveUNDEFINED = () -> HRegionInfo.UNDEFINED;
+    final Supplier<RegionInfo> retrieveMetaRegionInfo =
+      () -> RegionInfoBuilder.FIRST_META_REGIONINFO;
+
+    // The test runs multiple threads that reference these mutually dependent symbols. In order to
+    // express this bug, these threads need to access these symbols at roughly the same time, so
+    // that the classloader is asked to materialize these symbols concurrently. These Suppliers are
+    // run on threads that have already been allocated, managed by the system's ForkJoin pool.
+    final CompletableFuture<?>[] futures = Stream.of(
+      retrieveUNDEFINED, retrieveMetaRegionInfo, retrieveUNDEFINED, retrieveMetaRegionInfo)
+      .map(CompletableFuture::supplyAsync)
+      .toArray(CompletableFuture<?>[]::new);
+
+    // Loading classes should be relatively fast. 5 seconds is an arbitrary choice of timeout. It
+    // was chosen under the assumption that loading these symbols should complete much faster than
+    // this window.
+    CompletableFuture.allOf(futures).get(5, TimeUnit.SECONDS);
+  }
+}