You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by el...@apache.org on 2014/09/22 22:48:02 UTC

[02/16] git commit: ACCUMULO-715 Migrated ZKInstance to curator

ACCUMULO-715 Migrated ZKInstance to curator



git-svn-id: https://svn.apache.org/repos/asf/accumulo/branches/ACCUMULO-CURATOR@1492844 13f79535-47bb-0310-9956-ffa450edef68


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

Branch: refs/heads/ACCUMULO-CURATOR
Commit: e927de8c938479b033a5474f26dd36f89caeb5b0
Parents: cfb357f
Author: John Vines <vi...@apache.org>
Authored: Thu Jun 13 20:16:00 2013 +0000
Committer: John Vines <vi...@apache.org>
Committed: Thu Jun 13 20:16:00 2013 +0000

----------------------------------------------------------------------
 .../org/apache/accumulo/core/Constants.java     |  10 +-
 .../accumulo/core/client/ZooKeeperInstance.java | 153 ++++++++++++-------
 2 files changed, 107 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/e927de8c/core/src/main/java/org/apache/accumulo/core/Constants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/Constants.java b/core/src/main/java/org/apache/accumulo/core/Constants.java
index 3324a44..0c48db3 100644
--- a/core/src/main/java/org/apache/accumulo/core/Constants.java
+++ b/core/src/main/java/org/apache/accumulo/core/Constants.java
@@ -99,7 +99,8 @@ public class Constants {
   public static final String METADATA_DELETE_FLAG_PREFIX = "~del";
   public static final String METADATA_DELETE_FLAG_FOR_METADATA_PREFIX = "!!" + METADATA_DELETE_FLAG_PREFIX;
   public static final Range METADATA_DELETES_KEYSPACE = new Range(new Key(new Text(METADATA_DELETE_FLAG_PREFIX)), true, new Key(new Text("~dem")), false);
-  public static final Range METADATA_DELETES_FOR_METADATA_KEYSPACE = new Range(new Key(new Text(METADATA_DELETE_FLAG_FOR_METADATA_PREFIX)), true, new Key(new Text("!!~dem")), false);
+  public static final Range METADATA_DELETES_FOR_METADATA_KEYSPACE = new Range(new Key(new Text(METADATA_DELETE_FLAG_FOR_METADATA_PREFIX)), true, new Key(
+      new Text("!!~dem")), false);
   public static final String METADATA_BLIP_FLAG_PREFIX = "~blip"; // BLIP = bulk load in progress
   public static final Range METADATA_BLIP_KEYSPACE = new Range(new Key(new Text(METADATA_BLIP_FLAG_PREFIX)), true, new Key(new Text("~bliq")), false);
   
@@ -197,7 +198,6 @@ public class Constants {
   public static String getRootTabletDir(final AccumuloConfiguration conf) {
     return getMetadataTableDir(conf) + ZROOT_TABLET;
   }
-
   
   /**
    * @param conf
@@ -206,6 +206,8 @@ public class Constants {
   public static String getWalDirectory(final AccumuloConfiguration conf) {
     return getBaseDir(conf) + "/wal";
   }
-
-    public static final String AUDITLOG = "Audit";
+  
+  public static final String AUDITLOG = "Audit";
+  public static final String ZROOT_CURATOR_SERVICE = "root_tablet";
+  public static final String MASTER_CURATOR_SERVICE = "master";
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e927de8c/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
----------------------------------------------------------------------
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 8f62f8f..5004635 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
@@ -37,8 +37,14 @@ import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.OpTimer;
 import org.apache.accumulo.core.util.TextUtil;
-import org.apache.accumulo.core.zookeeper.ZooUtil;
-import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.curator.CuratorUtil;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.x.discovery.ServiceDiscovery;
+import org.apache.curator.x.discovery.ServiceDiscoveryBuilder;
+import org.apache.curator.x.discovery.ServiceProvider;
+import org.apache.curator.x.discovery.strategies.RandomStrategy;
+import org.apache.curator.x.discovery.strategies.StickyStrategy;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -67,9 +73,11 @@ public class ZooKeeperInstance implements Instance {
   private String instanceId = null;
   private String instanceName = null;
   
-  private final ZooCache zooCache;
-  
-  private final String zooKeepers;
+  private final CuratorFramework curator;
+  // http://curator.incubator.apache.org/curator-x-discovery/index.html
+  private ServiceDiscovery<String> discovery;
+  private ServiceProvider<String> rootService;
+  private ServiceProvider<String> masterService;
   
   private final int zooKeepersSessionTimeOut;
   
@@ -98,10 +106,15 @@ public class ZooKeeperInstance implements Instance {
   public ZooKeeperInstance(String instanceName, String zooKeepers, int sessionTimeout) {
     ArgumentChecker.notNull(instanceName, zooKeepers);
     this.instanceName = instanceName;
-    this.zooKeepers = zooKeepers;
     this.zooKeepersSessionTimeOut = sessionTimeout;
-    zooCache = ZooCache.getInstance(zooKeepers, sessionTimeout);
-    getInstanceID();
+    
+    // Need to create curator for getInstanceId
+    curator = constructCurator(zooKeepers, sessionTimeout).usingNamespace(Constants.ZROOT + Constants.ZINSTANCES);
+    this.instanceId = getInstanceID();
+    
+    // And now that we have the ID, we can set the namespace
+    curator.usingNamespace(Constants.ZROOT + '/' + getInstanceID());
+    setupDiscoveries(curator);
   }
   
   /**
@@ -129,29 +142,59 @@ public class ZooKeeperInstance implements Instance {
   public ZooKeeperInstance(UUID instanceId, String zooKeepers, int sessionTimeout) {
     ArgumentChecker.notNull(instanceId, zooKeepers);
     this.instanceId = instanceId.toString();
-    this.zooKeepers = zooKeepers;
     this.zooKeepersSessionTimeOut = sessionTimeout;
-    zooCache = ZooCache.getInstance(zooKeepers, sessionTimeout);
+    curator = constructCurator(zooKeepers, sessionTimeout).usingNamespace(Constants.ZROOT + '/' + getInstanceID());
+    
+    setupDiscoveries(curator);
+  }
+  
+  private CuratorFramework constructCurator(String zookeeperConnectString, int sessionTimeoutMs) {
+    return CuratorFrameworkFactory.builder().canBeReadOnly(true).sessionTimeoutMs(sessionTimeoutMs).retryPolicy(CuratorUtil.retry)
+        .connectString(zookeeperConnectString).build();
+  }
+  
+  private void setupDiscoveries(CuratorFramework curator2) {
+    try {
+      discovery = ServiceDiscoveryBuilder.builder(String.class).client(curator).basePath(Constants.ZROOT_TABLET_LOCATION).build();
+      discovery.start();
+      rootService = discovery.serviceProviderBuilder().serviceName(Constants.ZROOT_CURATOR_SERVICE)
+          .providerStrategy(new StickyStrategy<String>(new RandomStrategy<String>())).build();
+      masterService = discovery.serviceProviderBuilder().serviceName(Constants.MASTER_CURATOR_SERVICE)
+          .providerStrategy(new StickyStrategy<String>(new RandomStrategy<String>())).build();
+      rootService.start();
+      masterService.start();
+    } catch (Exception e) {
+      // We should have encountered any known Zookeeper issues by now.
+      throw new RuntimeException(e);
+    }
   }
   
   @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) {
+      // want the instance id to be stable for the life of this instance object, so only get it once
+      // And this will ONLY be invoked once iff the constructors using instanceName are used
+      // And the namespace will already be set to the instance path
+      byte[] iidb;
+      try {
+        iidb = curator.getData().forPath(instanceName);
+      } catch (Exception e) {
         throw new RuntimeException("Instance name " + instanceName
-            + " does not exist in zookeeper.  Run \"accumulo org.apache.accumulo.server.util.ListInstances\" to see a list.");
+            + " does not exist in zookeeper.  Run \"accumulo org.apache.accumulo.server.util.ListInstances\" to see a list.", e);
       }
+      
       instanceId = new String(iidb);
     }
     
-    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");
+    try {
+      if (curator.usingNamespace(Constants.ZROOT).checkExists().forPath(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");
+      }
+    } catch (Exception e) {
+      // Should only happen if things are in a very bad state, I think
+      throw new RuntimeException(e);
     }
     
     return instanceId;
@@ -159,10 +202,16 @@ public class ZooKeeperInstance implements Instance {
   
   @Override
   public List<String> getMasterLocations() {
-    String masterLocPath = ZooUtil.getRoot(this) + Constants.ZMASTER_LOCK;
-    
-    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Looking up master location in zoocache.");
-    byte[] loc = ZooUtil.getLockData(zooCache, masterLocPath);
+    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Looking up master location using curator service discovery.");
+    String loc;
+    try {
+      loc = masterService.getInstance().getPayload();
+    } catch (Exception e) {
+      opTimer.stop("Failed to find master location in curator discovery service");
+      // Zookeeper errors are handles, big ones hit already. This is probably very bad?
+      log.error(e,e);
+      return Collections.emptyList();
+    }
     opTimer.stop("Found master at " + (loc == null ? null : new String(loc)) + " in %DURATION%");
     
     if (loc == null) {
@@ -174,30 +223,31 @@ public class ZooKeeperInstance implements Instance {
   
   @Override
   public String getRootTabletLocation() {
-    String zRootLocPath = ZooUtil.getRoot(this) + Constants.ZROOT_TABLET_LOCATION;
-    
-    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Looking up root tablet location in zookeeper.");
-    byte[] loc = zooCache.get(zRootLocPath);
-    opTimer.stop("Found root tablet at " + (loc == null ? null : new String(loc)) + " in %DURATION%");
-    
-    if (loc == null) {
+    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Looking up root tablet location using curator service discovery.");
+    String loc;
+    try {
+      loc = rootService.getInstance().getPayload();
+    } catch (Exception e) {
+      opTimer.stop("Failed to find root tablet in curator discovery service");
+      // Zookeeper errors are handles, big ones hit already. This is probably very bad?
+      log.error(e,e);
       return null;
     }
-    
-    return new String(loc).split("\\|")[0];
+    opTimer.stop("Found root tablet at " + (loc == null ? null : new String(loc)) + " in %DURATION%");
+    return loc;
   }
   
   @Override
   public String getInstanceName() {
     if (instanceName == null)
-      instanceName = lookupInstanceName(zooCache, UUID.fromString(getInstanceID()));
+      instanceName = lookupInstanceName(curator, UUID.fromString(getInstanceID()));
     
     return instanceName;
   }
   
   @Override
   public String getZooKeepers() {
-    return zooKeepers;
+    return curator.getZookeeperClient().getCurrentConnectionString();
   }
   
   @Override
@@ -248,30 +298,29 @@ public class ZooKeeperInstance implements Instance {
   }
   
   /**
-   * @deprecated Use {@link #lookupInstanceName(org.apache.accumulo.fate.zookeeper.ZooCache, UUID)} instead
-   */
-  @Deprecated
-  public static String lookupInstanceName(org.apache.accumulo.core.zookeeper.ZooCache zooCache, UUID instanceId) {
-    return lookupInstanceName((ZooCache) zooCache, instanceId);
-  }
-  
-  /**
    * Given a zooCache and instanceId, look up the instance name.
    * 
-   * @param zooCache
+   * @param curator
    * @param instanceId
    * @return the instance name
    */
-  public static String lookupInstanceName(ZooCache zooCache, UUID instanceId) {
-    ArgumentChecker.notNull(zooCache, instanceId);
-    for (String name : zooCache.getChildren(Constants.ZROOT + Constants.ZINSTANCES)) {
-      String instanceNamePath = Constants.ZROOT + Constants.ZINSTANCES + "/" + name;
-      UUID iid = UUID.fromString(new String(zooCache.get(instanceNamePath)));
-      if (iid.equals(instanceId)) {
-        return name;
+  public static String lookupInstanceName(CuratorFramework curator, UUID instanceId) {
+    ArgumentChecker.notNull(curator, instanceId);
+    curator = curator.usingNamespace(Constants.ZROOT);
+    try {
+      for (String name : curator.getChildren().forPath(Constants.ZINSTANCES)) {
+        String instanceNamePath = Constants.ZINSTANCES + "/" + name;
+        UUID iid = UUID.fromString(new String(curator.getData().forPath(instanceNamePath)));
+        if (iid.equals(instanceId)) {
+          return name;
+        }
       }
+      return null;
+    } catch (Exception e) {
+      // Should only happen if things are in a very bad state, I think
+      log.error(e,e);
+      return null;
     }
-    return null;
   }
   
   /**