You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2019/06/19 19:44:58 UTC

[accumulo] branch master updated: Fix #1212 re-apply #1211 preserving prior behavior (#1216)

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

ctubbsii pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/master by this push:
     new be44359  Fix #1212 re-apply #1211 preserving prior behavior (#1216)
be44359 is described below

commit be44359c8ca9447b7c56d45e2ab94bbdea7e6ee7
Author: Christopher Tubbs <ct...@apache.org>
AuthorDate: Wed Jun 19 15:44:54 2019 -0400

    Fix #1212 re-apply #1211 preserving prior behavior (#1216)
    
    Revert "Revert "De-dupe some code (#1211)""
    
    This reverts commit ca5b62d81b21a35bb260f1c5d3257811eeb9ae6e.
    (This reverts the revert, and then adds fixes)
    
    Preserve behavior prior to #1211 that verifies (using ZooCache) that the
    instanceId actually points to a valid instanceId path in ZooKeeper.
    This verification is done regardless of whether instanceId is resolved
    using the instanceName, or if it is set explicitly in client
    configuration (in the case of ZooKeeperInstance). It also verifies on
    each call to getInstance() (just in case the instance is deleted from
    ZooCache unexpectedly, things won't continue operating as though
    everything is fine).
---
 .../accumulo/core/client/ZooKeeperInstance.java    | 47 ++-------------------
 .../accumulo/core/clientImpl/ClientContext.java    | 47 ++-------------------
 .../core/clientImpl/InstanceOperationsImpl.java    |  1 -
 .../apache/accumulo/fate/zookeeper/ZooUtil.java    | 49 ++++++++++++++++++++++
 4 files changed, 55 insertions(+), 89 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
index 9b3bd24..9d6faa0 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
@@ -17,15 +17,12 @@
 package org.apache.accumulo.core.client;
 
 import static com.google.common.base.Preconditions.checkArgument;
-import static java.nio.charset.StandardCharsets.UTF_8;
 
-import java.util.Collections;
 import java.util.List;
 import java.util.Properties;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.clientImpl.ClientConfConverter;
 import org.apache.accumulo.core.clientImpl.ClientContext;
@@ -125,53 +122,15 @@ public class ZooKeeperInstance implements Instance {
   @Override
   public String getInstanceID() {
     if (instanceId == null) {
-      // want the instance id to be stable for the life of this instance object,
-      // so only get it once
-      String instanceNamePath = Constants.ZROOT + Constants.ZINSTANCES + "/" + instanceName;
-      byte[] iidb = zooCache.get(instanceNamePath);
-      if (iidb == null) {
-        throw new RuntimeException(
-            "Instance name " + instanceName + " does not exist in zookeeper. "
-                + "Run \"accumulo org.apache.accumulo.server.util.ListInstances\" to see a list.");
-      }
-      instanceId = new String(iidb, UTF_8);
+      instanceId = ZooUtil.getInstanceID(zooCache, instanceName);
     }
-
-    if (zooCache.get(Constants.ZROOT + "/" + instanceId) == null) {
-      if (instanceName == null)
-        throw new RuntimeException("Instance id " + instanceId + " does not exist in zookeeper");
-      throw new RuntimeException("Instance id " + instanceId + " pointed to by the name "
-          + instanceName + " does not exist in zookeeper");
-    }
-
+    ZooUtil.verifyInstanceId(zooCache, instanceId, instanceName);
     return instanceId;
   }
 
   @Override
   public List<String> getMasterLocations() {
-    String masterLocPath = ZooUtil.getRoot(getInstanceID()) + Constants.ZMASTER_LOCK;
-
-    OpTimer timer = null;
-
-    if (log.isTraceEnabled()) {
-      log.trace("tid={} Looking up master location in zookeeper.", Thread.currentThread().getId());
-      timer = new OpTimer().start();
-    }
-
-    byte[] loc = ZooUtil.getLockData(zooCache, masterLocPath);
-
-    if (timer != null) {
-      timer.stop();
-      log.trace("tid={} Found master at {} in {}", Thread.currentThread().getId(),
-          (loc == null ? "null" : new String(loc, UTF_8)),
-          String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)));
-    }
-
-    if (loc == null) {
-      return Collections.emptyList();
-    }
-
-    return Collections.singletonList(new String(loc, UTF_8));
+    return ZooUtil.getMasterLocations(zooCache, getInstanceID());
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
index fa8b73e..abaf282 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
@@ -17,11 +17,9 @@
 package org.apache.accumulo.core.clientImpl;
 
 import static com.google.common.base.Preconditions.checkArgument;
-import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION;
 
 import java.nio.file.Path;
-import java.util.Collections;
 import java.util.List;
 import java.util.Objects;
 import java.util.Properties;
@@ -29,7 +27,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
 import java.util.function.Supplier;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -366,29 +363,7 @@ public class ClientContext implements AccumuloClient {
    */
   public List<String> getMasterLocations() {
     ensureOpen();
-    String masterLocPath = getZooKeeperRoot() + Constants.ZMASTER_LOCK;
-
-    OpTimer timer = null;
-
-    if (log.isTraceEnabled()) {
-      log.trace("tid={} Looking up master location in zookeeper.", Thread.currentThread().getId());
-      timer = new OpTimer().start();
-    }
-
-    byte[] loc = ZooUtil.getLockData(zooCache, masterLocPath);
-
-    if (timer != null) {
-      timer.stop();
-      log.trace("tid={} Found master at {} in {}", Thread.currentThread().getId(),
-          (loc == null ? "null" : new String(loc, UTF_8)),
-          String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)));
-    }
-
-    if (loc == null) {
-      return Collections.emptyList();
-    }
-
-    return Collections.singletonList(new String(loc, UTF_8));
+    return ZooUtil.getMasterLocations(zooCache, getInstanceID());
   }
 
   /**
@@ -400,25 +375,9 @@ public class ClientContext implements AccumuloClient {
     ensureOpen();
     final String instanceName = info.getInstanceName();
     if (instanceId == null) {
-      // want the instance id to be stable for the life of this instance object,
-      // so only get it once
-      String instanceNamePath = Constants.ZROOT + Constants.ZINSTANCES + "/" + instanceName;
-      byte[] iidb = zooCache.get(instanceNamePath);
-      if (iidb == null) {
-        throw new RuntimeException(
-            "Instance name " + instanceName + " does not exist in zookeeper. "
-                + "Run \"accumulo org.apache.accumulo.server.util.ListInstances\" to see a list.");
-      }
-      instanceId = new String(iidb, UTF_8);
-    }
-
-    if (zooCache.get(Constants.ZROOT + "/" + instanceId) == null) {
-      if (instanceName == null)
-        throw new RuntimeException("Instance id " + instanceId + " does not exist in zookeeper");
-      throw new RuntimeException("Instance id " + instanceId + " pointed to by the name "
-          + instanceName + " does not exist in zookeeper");
+      instanceId = ZooUtil.getInstanceID(zooCache, instanceName);
     }
-
+    ZooUtil.verifyInstanceId(zooCache, instanceId, instanceName);
     return instanceId;
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java
index a452ce4..79411cb 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java
@@ -235,7 +235,6 @@ public class InstanceOperationsImpl implements InstanceOperations {
 
   @Override
   public String getInstanceID() {
-
     return context.getInstanceID();
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/fate/zookeeper/ZooUtil.java b/core/src/main/java/org/apache/accumulo/fate/zookeeper/ZooUtil.java
index 6573614..52b3dd9 100644
--- a/core/src/main/java/org/apache/accumulo/fate/zookeeper/ZooUtil.java
+++ b/core/src/main/java/org/apache/accumulo/fate/zookeeper/ZooUtil.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.fate.zookeeper;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.Objects.requireNonNull;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 
@@ -31,6 +32,7 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.util.OpTimer;
 import org.apache.accumulo.core.volume.VolumeConfiguration;
 import org.apache.accumulo.fate.util.Retry;
 import org.apache.accumulo.fate.util.Retry.RetryFactory;
@@ -614,4 +616,51 @@ public class ZooUtil {
     }
   }
 
+  public static String getInstanceID(ZooCache zooCache, String instanceName) {
+    requireNonNull(zooCache, "zooCache cannot be null");
+    requireNonNull(instanceName, "instanceName cannot be null");
+    String instanceNamePath = Constants.ZROOT + Constants.ZINSTANCES + "/" + instanceName;
+    byte[] data = zooCache.get(instanceNamePath);
+    if (data == null) {
+      throw new RuntimeException("Instance name " + instanceName + " does not exist in zookeeper. "
+          + "Run \"accumulo org.apache.accumulo.server.util.ListInstances\" to see a list.");
+    }
+    return new String(data, UTF_8);
+  }
+
+  public static void verifyInstanceId(ZooCache zooCache, String instanceId, String instanceName) {
+    requireNonNull(zooCache, "zooCache cannot be null");
+    requireNonNull(instanceId, "instanceId cannot be null");
+    if (zooCache.get(Constants.ZROOT + "/" + instanceId) == null) {
+      throw new RuntimeException("Instance id " + instanceId
+          + (instanceName == null ? "" : " pointed to by the name " + instanceName)
+          + " does not exist in zookeeper");
+    }
+  }
+
+  public static List<String> getMasterLocations(ZooCache zooCache, String instanceId) {
+    String masterLocPath = ZooUtil.getRoot(instanceId) + Constants.ZMASTER_LOCK;
+
+    OpTimer timer = null;
+
+    if (log.isTraceEnabled()) {
+      log.trace("tid={} Looking up master location in zookeeper.", Thread.currentThread().getId());
+      timer = new OpTimer().start();
+    }
+
+    byte[] loc = ZooUtil.getLockData(zooCache, masterLocPath);
+
+    if (timer != null) {
+      timer.stop();
+      log.trace("tid={} Found master at {} in {}", Thread.currentThread().getId(),
+          (loc == null ? "null" : new String(loc, UTF_8)),
+          String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)));
+    }
+
+    if (loc == null) {
+      return Collections.emptyList();
+    }
+
+    return Collections.singletonList(new String(loc, UTF_8));
+  }
 }