You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2014/09/03 00:42:12 UTC

git commit: HBASE-11878 TestVisibilityLabelsWithDistributedLogReplay#testAddVisibilityLabelsOnRSRestart sometimes fails due to VisibilityController not yet initialized

Repository: hbase
Updated Branches:
  refs/heads/0.98 fd10bde5a -> 8ac090c6d


HBASE-11878 TestVisibilityLabelsWithDistributedLogReplay#testAddVisibilityLabelsOnRSRestart sometimes fails due to VisibilityController not yet initialized


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/8ac090c6
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/8ac090c6
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/8ac090c6

Branch: refs/heads/0.98
Commit: 8ac090c6d4ff45cc188e15a04ba1f29ebd4ee725
Parents: fd10bde
Author: Ted Yu <te...@apache.org>
Authored: Tue Sep 2 22:42:01 2014 +0000
Committer: Ted Yu <te...@apache.org>
Committed: Tue Sep 2 22:42:01 2014 +0000

----------------------------------------------------------------------
 .../VisibilityControllerNotReadyException.java  | 35 +++++++++++++++
 .../visibility/VisibilityController.java        | 19 +++++---
 ...ibilityLabelsWithDefaultVisLabelService.java | 46 +++++++++++++++-----
 3 files changed, 82 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8ac090c6/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityControllerNotReadyException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityControllerNotReadyException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityControllerNotReadyException.java
new file mode 100644
index 0000000..d0f6a61
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityControllerNotReadyException.java
@@ -0,0 +1,35 @@
+/**
+ * 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.security.visibility;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/*
+ * This exception indicates that VisibilityController hasn't finished initialization.
+ */
+@InterfaceAudience.Public
+public class VisibilityControllerNotReadyException extends IOException {
+
+  private static final long serialVersionUID = 1725986525207989173L;
+
+  public VisibilityControllerNotReadyException(String msg) {
+    super(msg);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8ac090c6/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
index 04b1790..7f0c9c7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
@@ -249,6 +249,7 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
   public void postLogReplay(ObserverContext<RegionCoprocessorEnvironment> e) {
     if (this.labelsRegion) {
       initVisibilityLabelService(e.getEnvironment());
+      LOG.debug("post labels region log replay");
     }
   }
 
@@ -411,7 +412,9 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
   @Override
   public RegionScanner preScannerOpen(ObserverContext<RegionCoprocessorEnvironment> e, Scan scan,
       RegionScanner s) throws IOException {
-    if (!initialized) throw new IOException("VisibilityController not yet initialized!!");
+    if (!initialized) {
+      throw new VisibilityControllerNotReadyException("VisibilityController not yet initialized!");
+    }
     HRegion region = e.getEnvironment().getRegion();
     Authorizations authorizations = null;
     try {
@@ -507,7 +510,9 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
   @Override
   public void preGetOp(ObserverContext<RegionCoprocessorEnvironment> e, Get get, List<Cell> results)
       throws IOException {
-    if (!initialized) throw new IOException("VisibilityController not yet initialized!!");
+    if (!initialized) {
+      throw new VisibilityControllerNotReadyException("VisibilityController not yet initialized!");
+    }
     HRegion region = e.getEnvironment().getRegion();
     Authorizations authorizations = null;
     try {
@@ -635,8 +640,9 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
     VisibilityLabelsResponse.Builder response = VisibilityLabelsResponse.newBuilder();
     List<VisibilityLabel> visLabels = request.getVisLabelList();
     if (!initialized) {
-      setExceptionResults(visLabels.size(), new CoprocessorException(
-          "VisibilityController not yet initialized"), response);
+      setExceptionResults(visLabels.size(),
+        new VisibilityControllerNotReadyException("VisibilityController not yet initialized!"),
+        response);
     } else {
       try {
         checkCallingUserAuth();
@@ -688,8 +694,9 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
     VisibilityLabelsResponse.Builder response = VisibilityLabelsResponse.newBuilder();
     List<ByteString> auths = request.getAuthList();
     if (!initialized) {
-      setExceptionResults(auths.size(), new CoprocessorException(
-          "VisibilityController not yet initialized"), response);
+      setExceptionResults(auths.size(),
+        new VisibilityControllerNotReadyException("VisibilityController not yet initialized!"),
+        response);
     } else {
       try {
         checkCallingUserAuth();

http://git-wip-us.apache.org/repos/asf/hbase/blob/8ac090c6/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java
index 9abdd85..d15e8dd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java
@@ -25,18 +25,25 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair;
 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
+import org.apache.hadoop.hbase.util.Threads;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -44,6 +51,7 @@ import org.junit.experimental.categories.Category;
 
 @Category(MediumTests.class)
 public class TestVisibilityLabelsWithDefaultVisLabelService extends TestVisibilityLabels {
+  final Log LOG = LogFactory.getLog(getClass());
 
   @BeforeClass
   public static void setupBeforeClass() throws Exception {
@@ -104,19 +112,33 @@ public class TestVisibilityLabelsWithDefaultVisLabelService extends TestVisibili
     // Start one new RS
     RegionServerThread rs = TEST_UTIL.getHBaseCluster().startRegionServer();
     waitForLabelsRegionAvailability(rs.getRegionServer());
-    PrivilegedExceptionAction<VisibilityLabelsResponse> action =
-        new PrivilegedExceptionAction<VisibilityLabelsResponse>() {
-      public VisibilityLabelsResponse run() throws Exception {
-        String[] labels = { SECRET, CONFIDENTIAL, PRIVATE, "ABC", "XYZ" };
-        try {
-          VisibilityClient.addLabels(conf, labels);
-        } catch (Throwable t) {
-          throw new IOException(t);
+    final AtomicBoolean vcInitialized = new AtomicBoolean(true);
+    do {
+      PrivilegedExceptionAction<VisibilityLabelsResponse> action =
+          new PrivilegedExceptionAction<VisibilityLabelsResponse>() {
+        public VisibilityLabelsResponse run() throws Exception {
+          String[] labels = { SECRET, CONFIDENTIAL, PRIVATE, "ABC", "XYZ" };
+          try {
+            VisibilityLabelsResponse resp = VisibilityClient.addLabels(conf, labels);
+            List<RegionActionResult> results = resp.getResultList();
+            if (results.get(0).hasException()) {
+              NameBytesPair pair = results.get(0).getException();
+              Throwable t = ProtobufUtil.toException(pair);
+              LOG.debug("Got exception writing labels", t);
+              if (t instanceof VisibilityControllerNotReadyException) {
+                vcInitialized.set(false);
+                LOG.warn("VisibilityController was not yet initialized");
+                Threads.sleep(10);
+              }
+            } else LOG.debug("new labels added: " + resp);
+          } catch (Throwable t) {
+            throw new IOException(t);
+          }
+          return null;
         }
-        return null;
-      }
-    };
-    SUPERUSER.runAs(action);
+      };
+      SUPERUSER.runAs(action);
+    } while (!vcInitialized.get());
     // Scan the visibility label
     Scan s = new Scan();
     s.setAuthorizations(new Authorizations(VisibilityUtils.SYSTEM_LABEL));