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

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


The following commit(s) were added to refs/heads/elasticity by this push:
     new 4d9d745e2b removes unused code related to root tablet location (#4669)
4d9d745e2b is described below

commit 4d9d745e2b0e1192976a097ccb227e762c300cd1
Author: Keith Turner <ktur...@apache.org>
AuthorDate: Fri Jun 14 08:52:33 2024 -0400

    removes unused code related to root tablet location (#4669)
---
 .../accumulo/core/clientImpl/ClientContext.java    |   37 -
 .../accumulo/core/metadata/MetadataServicer.java   |   68 -
 .../core/metadata/ServicerForMetadataTable.java    |   33 -
 .../core/metadata/ServicerForRootTable.java        |   48 -
 .../core/metadata/ServicerForUserTables.java       |   34 -
 .../core/metadata/TableMetadataServicer.java       |  149 --
 .../core/clientImpl/thrift/ClientService.java      | 2528 +-------------------
 core/src/main/thrift/client.thrift                 |    5 -
 .../core/clientImpl/ClientTabletCacheImplTest.java |   10 +-
 .../core/metadata/MetadataServicerTest.java        |  104 -
 .../server/client/ClientServiceHandler.java        |   15 -
 .../server/util/VerifyTabletAssignments.java       |   42 +-
 .../monitor/rest/tables/TablesResource.java        |    7 +-
 .../test/performance/scan/CollectTabletStats.java  |   28 +-
 14 files changed, 109 insertions(+), 2999 deletions(-)

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 9cf34500f1..af3e45c0f8 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
@@ -24,7 +24,6 @@ import static 
com.google.common.base.Suppliers.memoizeWithExpiration;
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.SECONDS;
-import static 
org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION;
 
 import java.lang.Thread.UncaughtExceptionHandler;
 import java.lang.reflect.InvocationTargetException;
@@ -85,12 +84,8 @@ import org.apache.accumulo.core.lock.ServiceLock;
 import org.apache.accumulo.core.lock.ServiceLockData;
 import org.apache.accumulo.core.lock.ServiceLockData.ThriftService;
 import org.apache.accumulo.core.manager.state.tables.TableState;
-import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.metadata.schema.Ample;
-import org.apache.accumulo.core.metadata.schema.Ample.ReadConsistency;
 import org.apache.accumulo.core.metadata.schema.AmpleImpl;
-import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
-import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
 import org.apache.accumulo.core.rpc.SaslConnectionParams;
 import org.apache.accumulo.core.rpc.SslConnectionParams;
 import org.apache.accumulo.core.security.Authorizations;
@@ -475,38 +470,6 @@ public class ClientContext implements AccumuloClient {
     return rpcCreds;
   }
 
-  /**
-   * Returns the location of the tablet server that is serving the root tablet.
-   *
-   * @return location in "hostname:port" form
-   */
-  public String getRootTabletLocation() {
-    ensureOpen();
-
-    OpTimer timer = null;
-
-    if (log.isTraceEnabled()) {
-      log.trace("tid={} Looking up root tablet location in zookeeper.",
-          Thread.currentThread().getId());
-      timer = new OpTimer().start();
-    }
-
-    Location loc =
-        getAmple().readTablet(RootTable.EXTENT, ReadConsistency.EVENTUAL, 
LOCATION).getLocation();
-
-    if (timer != null) {
-      timer.stop();
-      log.trace("tid={} Found root tablet at {} in {}", 
Thread.currentThread().getId(), loc,
-          String.format("%.3f secs", timer.scale(SECONDS)));
-    }
-
-    if (loc == null || loc.getType() != LocationType.CURRENT) {
-      return null;
-    }
-
-    return loc.getHostPort();
-  }
-
   /**
    * Returns the location(s) of the accumulo manager and any redundant servers.
    *
diff --git 
a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataServicer.java 
b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataServicer.java
deleted file mode 100644
index fc4e23fa57..0000000000
--- a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataServicer.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * 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
- *
- *   https://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.accumulo.core.metadata;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import java.util.SortedMap;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-
-/**
- * Provides a consolidated API for handling table metadata
- */
-public abstract class MetadataServicer {
-
-  public static MetadataServicer forTableName(ClientContext context, String 
tableName)
-      throws AccumuloException, AccumuloSecurityException {
-    checkArgument(tableName != null, "tableName is null");
-    return forTableId(context, 
TableId.of(context.tableOperations().tableIdMap().get(tableName)));
-  }
-
-  public static MetadataServicer forTableId(ClientContext context, TableId 
tableId) {
-    checkArgument(tableId != null, "tableId is null");
-    if (AccumuloTable.ROOT.tableId().equals(tableId)) {
-      return new ServicerForRootTable(context);
-    } else if (AccumuloTable.METADATA.tableId().equals(tableId)) {
-      return new ServicerForMetadataTable(context);
-    } else {
-      return new ServicerForUserTables(context, tableId);
-    }
-  }
-
-  /**
-   *
-   * @return the table id of the table currently being serviced
-   */
-  public abstract TableId getServicedTableId();
-
-  /**
-   * Populate the provided data structure with the known tablets for the table 
being serviced
-   *
-   * @param tablets A mapping of all known tablets to their location (if 
available, null otherwise)
-   */
-  public abstract void getTabletLocations(SortedMap<KeyExtent,String> tablets)
-      throws AccumuloException, AccumuloSecurityException, 
TableNotFoundException;
-
-}
diff --git 
a/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForMetadataTable.java
 
b/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForMetadataTable.java
deleted file mode 100644
index a4b415074b..0000000000
--- 
a/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForMetadataTable.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * 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
- *
- *   https://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.accumulo.core.metadata;
-
-import org.apache.accumulo.core.clientImpl.ClientContext;
-
-/**
- * A metadata servicer for the metadata table (which holds metadata for user 
tables).<br>
- * The metadata table's metadata is serviced in the root table.
- */
-class ServicerForMetadataTable extends TableMetadataServicer {
-
-  public ServicerForMetadataTable(ClientContext context) {
-    super(context, AccumuloTable.ROOT.tableName(), 
AccumuloTable.METADATA.tableId());
-  }
-
-}
diff --git 
a/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForRootTable.java
 
b/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForRootTable.java
deleted file mode 100644
index 94e86447ed..0000000000
--- 
a/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForRootTable.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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
- *
- *   https://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.accumulo.core.metadata;
-
-import java.util.SortedMap;
-
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-
-/**
- * A metadata servicer for the root table.<br>
- * The root table's metadata is serviced in zookeeper.
- */
-class ServicerForRootTable extends MetadataServicer {
-
-  private final ClientContext context;
-
-  public ServicerForRootTable(ClientContext context) {
-    this.context = context;
-  }
-
-  @Override
-  public TableId getServicedTableId() {
-    return AccumuloTable.ROOT.tableId();
-  }
-
-  @Override
-  public void getTabletLocations(SortedMap<KeyExtent,String> tablets) {
-    tablets.put(RootTable.EXTENT, context.getRootTabletLocation());
-  }
-}
diff --git 
a/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForUserTables.java
 
b/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForUserTables.java
deleted file mode 100644
index 369bcc440f..0000000000
--- 
a/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForUserTables.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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
- *
- *   https://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.accumulo.core.metadata;
-
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.data.TableId;
-
-/**
- * A metadata servicer for user tables.<br>
- * Metadata for user tables are serviced in the metadata table.
- */
-class ServicerForUserTables extends TableMetadataServicer {
-
-  public ServicerForUserTables(ClientContext context, TableId tableId) {
-    super(context, AccumuloTable.METADATA.tableName(), tableId);
-  }
-
-}
diff --git 
a/core/src/main/java/org/apache/accumulo/core/metadata/TableMetadataServicer.java
 
b/core/src/main/java/org/apache/accumulo/core/metadata/TableMetadataServicer.java
deleted file mode 100644
index 032b7f8736..0000000000
--- 
a/core/src/main/java/org/apache/accumulo/core/metadata/TableMetadataServicer.java
+++ /dev/null
@@ -1,149 +0,0 @@
-/*
- * 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
- *
- *   https://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.accumulo.core.metadata;
-
-import java.util.Iterator;
-import java.util.Map.Entry;
-import java.util.SortedMap;
-import java.util.SortedSet;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
-import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily;
-import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.hadoop.io.Text;
-
-/**
- * A {@link MetadataServicer} that is backed by a table
- */
-abstract class TableMetadataServicer extends MetadataServicer {
-
-  private final ClientContext context;
-  private TableId tableIdBeingServiced;
-  private String serviceTableName;
-
-  public TableMetadataServicer(ClientContext context, String serviceTableName,
-      TableId tableIdBeingServiced) {
-    this.context = context;
-    this.serviceTableName = serviceTableName;
-    this.tableIdBeingServiced = tableIdBeingServiced;
-  }
-
-  @Override
-  public TableId getServicedTableId() {
-    return tableIdBeingServiced;
-  }
-
-  public String getServicingTableName() {
-    return serviceTableName;
-  }
-
-  @Override
-  public void getTabletLocations(SortedMap<KeyExtent,String> tablets)
-      throws AccumuloException, TableNotFoundException {
-
-    Scanner scanner = context.createScanner(getServicingTableName(), 
Authorizations.EMPTY);
-
-    TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
-    scanner.fetchColumnFamily(CurrentLocationColumnFamily.NAME);
-
-    // position at first entry in metadata table for given table
-    scanner.setRange(TabletsSection.getRange(getServicedTableId()));
-
-    Text colf = new Text();
-    Text colq = new Text();
-
-    String location = null;
-    Text row = null;
-    // acquire this table's tablets from the metadata table which services it
-    for (Entry<Key,Value> entry : scanner) {
-      if (row != null) {
-        if (!row.equals(entry.getKey().getRow())) {
-          location = null;
-          row = entry.getKey().getRow();
-        }
-      } else {
-        row = entry.getKey().getRow();
-      }
-
-      colf = entry.getKey().getColumnFamily(colf);
-      colq = entry.getKey().getColumnQualifier(colq);
-
-      if (TabletColumnFamily.PREV_ROW_COLUMN.equals(colf, colq)) {
-        KeyExtent currentKeyExtent = KeyExtent.fromMetaPrevRow(entry);
-        tablets.put(currentKeyExtent, location);
-        location = null;
-      } else if (colf.equals(CurrentLocationColumnFamily.NAME)) {
-        location = entry.getValue().toString();
-      }
-
-    }
-
-    validateEntries(tablets);
-  }
-
-  private void validateEntries(SortedMap<KeyExtent,String> tablets) throws 
AccumuloException {
-    SortedSet<KeyExtent> tabletsKeys = (SortedSet<KeyExtent>) tablets.keySet();
-    // sanity check of metadata table entries
-    // make sure tablets has no holes, and that it starts and ends w/ null
-    if (tabletsKeys.isEmpty()) {
-      throw new AccumuloException(
-          "No entries found in metadata table for table " + 
getServicedTableId());
-    }
-
-    if (tabletsKeys.first().prevEndRow() != null) {
-      throw new AccumuloException("Problem with metadata table, first entry 
for table "
-          + getServicedTableId() + "- " + tabletsKeys.first() + " - has non 
null prev end row");
-    }
-
-    if (tabletsKeys.last().endRow() != null) {
-      throw new AccumuloException("Problem with metadata table, last entry for 
table "
-          + getServicedTableId() + "- " + tabletsKeys.first() + " - has non 
null end row");
-    }
-
-    Iterator<KeyExtent> tabIter = tabletsKeys.iterator();
-    Text lastEndRow = tabIter.next().endRow();
-    while (tabIter.hasNext()) {
-      KeyExtent tabke = tabIter.next();
-
-      if (tabke.prevEndRow() == null) {
-        throw new AccumuloException(
-            "Problem with metadata table, it has null prev end row in middle 
of table " + tabke);
-      }
-
-      if (!tabke.prevEndRow().equals(lastEndRow)) {
-        throw new AccumuloException("Problem with metadata table, it has a 
hole "
-            + tabke.prevEndRow() + " != " + lastEndRow);
-      }
-
-      lastEndRow = tabke.endRow();
-    }
-
-    // end METADATA table sanity check
-  }
-
-}
diff --git 
a/core/src/main/thrift-gen-java/org/apache/accumulo/core/clientImpl/thrift/ClientService.java
 
b/core/src/main/thrift-gen-java/org/apache/accumulo/core/clientImpl/thrift/ClientService.java
index 799bb4a6f7..03fe532462 100644
--- 
a/core/src/main/thrift-gen-java/org/apache/accumulo/core/clientImpl/thrift/ClientService.java
+++ 
b/core/src/main/thrift-gen-java/org/apache/accumulo/core/clientImpl/thrift/ClientService.java
@@ -29,12 +29,6 @@ public class ClientService {
 
   public interface Iface {
 
-    public java.lang.String getRootTabletLocation() throws 
org.apache.thrift.TException;
-
-    public java.lang.String getInstanceId() throws 
org.apache.thrift.TException;
-
-    public java.lang.String getZooKeepers() throws 
org.apache.thrift.TException;
-
     public void ping(org.apache.accumulo.core.securityImpl.thrift.TCredentials 
credentials) throws ThriftSecurityException, org.apache.thrift.TException;
 
     public java.util.List<TDiskUsage> 
getDiskUsage(java.util.Set<java.lang.String> tables, 
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws 
ThriftSecurityException, ThriftTableOperationException, 
org.apache.thrift.TException;
@@ -101,12 +95,6 @@ public class ClientService {
 
   public interface AsyncIface {
 
-    public void 
getRootTabletLocation(org.apache.thrift.async.AsyncMethodCallback<java.lang.String>
 resultHandler) throws org.apache.thrift.TException;
-
-    public void 
getInstanceId(org.apache.thrift.async.AsyncMethodCallback<java.lang.String> 
resultHandler) throws org.apache.thrift.TException;
-
-    public void 
getZooKeepers(org.apache.thrift.async.AsyncMethodCallback<java.lang.String> 
resultHandler) throws org.apache.thrift.TException;
-
     public void ping(org.apache.accumulo.core.securityImpl.thrift.TCredentials 
credentials, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) 
throws org.apache.thrift.TException;
 
     public void getDiskUsage(java.util.Set<java.lang.String> tables, 
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, 
org.apache.thrift.async.AsyncMethodCallback<java.util.List<TDiskUsage>> 
resultHandler) throws org.apache.thrift.TException;
@@ -193,75 +181,6 @@ public class ClientService {
       super(iprot, oprot);
     }
 
-    @Override
-    public java.lang.String getRootTabletLocation() throws 
org.apache.thrift.TException
-    {
-      send_getRootTabletLocation();
-      return recv_getRootTabletLocation();
-    }
-
-    public void send_getRootTabletLocation() throws 
org.apache.thrift.TException
-    {
-      getRootTabletLocation_args args = new getRootTabletLocation_args();
-      sendBase("getRootTabletLocation", args);
-    }
-
-    public java.lang.String recv_getRootTabletLocation() throws 
org.apache.thrift.TException
-    {
-      getRootTabletLocation_result result = new getRootTabletLocation_result();
-      receiveBase(result, "getRootTabletLocation");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      throw new 
org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT,
 "getRootTabletLocation failed: unknown result");
-    }
-
-    @Override
-    public java.lang.String getInstanceId() throws org.apache.thrift.TException
-    {
-      send_getInstanceId();
-      return recv_getInstanceId();
-    }
-
-    public void send_getInstanceId() throws org.apache.thrift.TException
-    {
-      getInstanceId_args args = new getInstanceId_args();
-      sendBase("getInstanceId", args);
-    }
-
-    public java.lang.String recv_getInstanceId() throws 
org.apache.thrift.TException
-    {
-      getInstanceId_result result = new getInstanceId_result();
-      receiveBase(result, "getInstanceId");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      throw new 
org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT,
 "getInstanceId failed: unknown result");
-    }
-
-    @Override
-    public java.lang.String getZooKeepers() throws org.apache.thrift.TException
-    {
-      send_getZooKeepers();
-      return recv_getZooKeepers();
-    }
-
-    public void send_getZooKeepers() throws org.apache.thrift.TException
-    {
-      getZooKeepers_args args = new getZooKeepers_args();
-      sendBase("getZooKeepers", args);
-    }
-
-    public java.lang.String recv_getZooKeepers() throws 
org.apache.thrift.TException
-    {
-      getZooKeepers_result result = new getZooKeepers_result();
-      receiveBase(result, "getZooKeepers");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      throw new 
org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT,
 "getZooKeepers failed: unknown result");
-    }
-
     @Override
     public void ping(org.apache.accumulo.core.securityImpl.thrift.TCredentials 
credentials) throws ThriftSecurityException, org.apache.thrift.TException
     {
@@ -1205,102 +1124,6 @@ public class ClientService {
       super(protocolFactory, clientManager, transport);
     }
 
-    @Override
-    public void 
getRootTabletLocation(org.apache.thrift.async.AsyncMethodCallback<java.lang.String>
 resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      getRootTabletLocation_call method_call = new 
getRootTabletLocation_call(resultHandler, this, ___protocolFactory, 
___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class getRootTabletLocation_call extends 
org.apache.thrift.async.TAsyncMethodCall<java.lang.String> {
-      public 
getRootTabletLocation_call(org.apache.thrift.async.AsyncMethodCallback<java.lang.String>
 resultHandler, org.apache.thrift.async.TAsyncClient client, 
org.apache.thrift.protocol.TProtocolFactory protocolFactory, 
org.apache.thrift.transport.TNonblockingTransport transport) throws 
org.apache.thrift.TException {
-        super(client, protocolFactory, transport, resultHandler, false);
-      }
-
-      @Override
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws 
org.apache.thrift.TException {
-        prot.writeMessageBegin(new 
org.apache.thrift.protocol.TMessage("getRootTabletLocation", 
org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getRootTabletLocation_args args = new getRootTabletLocation_args();
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      @Override
-      public java.lang.String getResult() throws org.apache.thrift.TException {
-        if (getState() != 
org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
-          throw new java.lang.IllegalStateException("Method call not 
finished!");
-        }
-        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = 
new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
-        org.apache.thrift.protocol.TProtocol prot = 
client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_getRootTabletLocation();
-      }
-    }
-
-    @Override
-    public void 
getInstanceId(org.apache.thrift.async.AsyncMethodCallback<java.lang.String> 
resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      getInstanceId_call method_call = new getInstanceId_call(resultHandler, 
this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class getInstanceId_call extends 
org.apache.thrift.async.TAsyncMethodCall<java.lang.String> {
-      public 
getInstanceId_call(org.apache.thrift.async.AsyncMethodCallback<java.lang.String>
 resultHandler, org.apache.thrift.async.TAsyncClient client, 
org.apache.thrift.protocol.TProtocolFactory protocolFactory, 
org.apache.thrift.transport.TNonblockingTransport transport) throws 
org.apache.thrift.TException {
-        super(client, protocolFactory, transport, resultHandler, false);
-      }
-
-      @Override
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws 
org.apache.thrift.TException {
-        prot.writeMessageBegin(new 
org.apache.thrift.protocol.TMessage("getInstanceId", 
org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getInstanceId_args args = new getInstanceId_args();
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      @Override
-      public java.lang.String getResult() throws org.apache.thrift.TException {
-        if (getState() != 
org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
-          throw new java.lang.IllegalStateException("Method call not 
finished!");
-        }
-        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = 
new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
-        org.apache.thrift.protocol.TProtocol prot = 
client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_getInstanceId();
-      }
-    }
-
-    @Override
-    public void 
getZooKeepers(org.apache.thrift.async.AsyncMethodCallback<java.lang.String> 
resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      getZooKeepers_call method_call = new getZooKeepers_call(resultHandler, 
this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class getZooKeepers_call extends 
org.apache.thrift.async.TAsyncMethodCall<java.lang.String> {
-      public 
getZooKeepers_call(org.apache.thrift.async.AsyncMethodCallback<java.lang.String>
 resultHandler, org.apache.thrift.async.TAsyncClient client, 
org.apache.thrift.protocol.TProtocolFactory protocolFactory, 
org.apache.thrift.transport.TNonblockingTransport transport) throws 
org.apache.thrift.TException {
-        super(client, protocolFactory, transport, resultHandler, false);
-      }
-
-      @Override
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws 
org.apache.thrift.TException {
-        prot.writeMessageBegin(new 
org.apache.thrift.protocol.TMessage("getZooKeepers", 
org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getZooKeepers_args args = new getZooKeepers_args();
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      @Override
-      public java.lang.String getResult() throws org.apache.thrift.TException {
-        if (getState() != 
org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
-          throw new java.lang.IllegalStateException("Method call not 
finished!");
-        }
-        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = 
new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
-        org.apache.thrift.protocol.TProtocol prot = 
client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_getZooKeepers();
-      }
-    }
-
     @Override
     public void ping(org.apache.accumulo.core.securityImpl.thrift.TCredentials 
credentials, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) 
throws org.apache.thrift.TException {
       checkReady();
@@ -2644,9 +2467,6 @@ public class ClientService {
     }
 
     private static <I extends Iface> java.util.Map<java.lang.String,  
org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>> 
getProcessMap(java.util.Map<java.lang.String, 
org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> 
processMap) {
-      processMap.put("getRootTabletLocation", new getRootTabletLocation());
-      processMap.put("getInstanceId", new getInstanceId());
-      processMap.put("getZooKeepers", new getZooKeepers());
       processMap.put("ping", new ping());
       processMap.put("getDiskUsage", new getDiskUsage());
       processMap.put("listLocalUsers", new listLocalUsers());
@@ -2681,90 +2501,6 @@ public class ClientService {
       return processMap;
     }
 
-    public static class getRootTabletLocation<I extends Iface> extends 
org.apache.thrift.ProcessFunction<I, getRootTabletLocation_args> {
-      public getRootTabletLocation() {
-        super("getRootTabletLocation");
-      }
-
-      @Override
-      public getRootTabletLocation_args getEmptyArgsInstance() {
-        return new getRootTabletLocation_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public getRootTabletLocation_result getResult(I iface, 
getRootTabletLocation_args args) throws org.apache.thrift.TException {
-        getRootTabletLocation_result result = new 
getRootTabletLocation_result();
-        result.success = iface.getRootTabletLocation();
-        return result;
-      }
-    }
-
-    public static class getInstanceId<I extends Iface> extends 
org.apache.thrift.ProcessFunction<I, getInstanceId_args> {
-      public getInstanceId() {
-        super("getInstanceId");
-      }
-
-      @Override
-      public getInstanceId_args getEmptyArgsInstance() {
-        return new getInstanceId_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public getInstanceId_result getResult(I iface, getInstanceId_args args) 
throws org.apache.thrift.TException {
-        getInstanceId_result result = new getInstanceId_result();
-        result.success = iface.getInstanceId();
-        return result;
-      }
-    }
-
-    public static class getZooKeepers<I extends Iface> extends 
org.apache.thrift.ProcessFunction<I, getZooKeepers_args> {
-      public getZooKeepers() {
-        super("getZooKeepers");
-      }
-
-      @Override
-      public getZooKeepers_args getEmptyArgsInstance() {
-        return new getZooKeepers_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public getZooKeepers_result getResult(I iface, getZooKeepers_args args) 
throws org.apache.thrift.TException {
-        getZooKeepers_result result = new getZooKeepers_result();
-        result.success = iface.getZooKeepers();
-        return result;
-      }
-    }
-
     public static class ping<I extends Iface> extends 
org.apache.thrift.ProcessFunction<I, ping_args> {
       public ping() {
         super("ping");
@@ -3804,9 +3540,6 @@ public class ClientService {
     }
 
     private static <I extends AsyncIface> java.util.Map<java.lang.String,  
org.apache.thrift.AsyncProcessFunction<I, ? extends  
org.apache.thrift.TBase,?>> getProcessMap(java.util.Map<java.lang.String,  
org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase, 
?>> processMap) {
-      processMap.put("getRootTabletLocation", new getRootTabletLocation());
-      processMap.put("getInstanceId", new getInstanceId());
-      processMap.put("getZooKeepers", new getZooKeepers());
       processMap.put("ping", new ping());
       processMap.put("getDiskUsage", new getDiskUsage());
       processMap.put("listLocalUsers", new listLocalUsers());
@@ -3841,24 +3574,23 @@ public class ClientService {
       return processMap;
     }
 
-    public static class getRootTabletLocation<I extends AsyncIface> extends 
org.apache.thrift.AsyncProcessFunction<I, getRootTabletLocation_args, 
java.lang.String> {
-      public getRootTabletLocation() {
-        super("getRootTabletLocation");
+    public static class ping<I extends AsyncIface> extends 
org.apache.thrift.AsyncProcessFunction<I, ping_args, Void> {
+      public ping() {
+        super("ping");
       }
 
       @Override
-      public getRootTabletLocation_args getEmptyArgsInstance() {
-        return new getRootTabletLocation_args();
+      public ping_args getEmptyArgsInstance() {
+        return new ping_args();
       }
 
       @Override
-      public org.apache.thrift.async.AsyncMethodCallback<java.lang.String> 
getResultHandler(final 
org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final 
int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 
getResultHandler(final 
org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final 
int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new 
org.apache.thrift.async.AsyncMethodCallback<java.lang.String>() { 
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
           @Override
-          public void onComplete(java.lang.String o) {
-            getRootTabletLocation_result result = new 
getRootTabletLocation_result();
-            result.success = o;
+          public void onComplete(Void o) {
+            ping_result result = new ping_result();
             try {
               fcall.sendResponse(fb, result, 
org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -3873,8 +3605,12 @@ public class ClientService {
           public void onError(java.lang.Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
-            getRootTabletLocation_result result = new 
getRootTabletLocation_result();
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
+            ping_result result = new ping_result();
+            if (e instanceof ThriftSecurityException) {
+              result.sec = (ThriftSecurityException) e;
+              result.setSecIsSet(true);
+              msg = result;
+            } else if (e instanceof 
org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -3903,28 +3639,28 @@ public class ClientService {
       }
 
       @Override
-      public void start(I iface, getRootTabletLocation_args args, 
org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) 
throws org.apache.thrift.TException {
-        iface.getRootTabletLocation(resultHandler);
+      public void start(I iface, ping_args args, 
org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws 
org.apache.thrift.TException {
+        iface.ping(args.credentials,resultHandler);
       }
     }
 
-    public static class getInstanceId<I extends AsyncIface> extends 
org.apache.thrift.AsyncProcessFunction<I, getInstanceId_args, java.lang.String> 
{
-      public getInstanceId() {
-        super("getInstanceId");
+    public static class getDiskUsage<I extends AsyncIface> extends 
org.apache.thrift.AsyncProcessFunction<I, getDiskUsage_args, 
java.util.List<TDiskUsage>> {
+      public getDiskUsage() {
+        super("getDiskUsage");
       }
 
       @Override
-      public getInstanceId_args getEmptyArgsInstance() {
-        return new getInstanceId_args();
+      public getDiskUsage_args getEmptyArgsInstance() {
+        return new getDiskUsage_args();
       }
 
       @Override
-      public org.apache.thrift.async.AsyncMethodCallback<java.lang.String> 
getResultHandler(final 
org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final 
int seqid) {
+      public 
org.apache.thrift.async.AsyncMethodCallback<java.util.List<TDiskUsage>> 
getResultHandler(final 
org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final 
int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new 
org.apache.thrift.async.AsyncMethodCallback<java.lang.String>() { 
+        return new 
org.apache.thrift.async.AsyncMethodCallback<java.util.List<TDiskUsage>>() { 
           @Override
-          public void onComplete(java.lang.String o) {
-            getInstanceId_result result = new getInstanceId_result();
+          public void onComplete(java.util.List<TDiskUsage> o) {
+            getDiskUsage_result result = new getDiskUsage_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, 
org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -3940,8 +3676,16 @@ public class ClientService {
           public void onError(java.lang.Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
-            getInstanceId_result result = new getInstanceId_result();
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
+            getDiskUsage_result result = new getDiskUsage_result();
+            if (e instanceof ThriftSecurityException) {
+              result.sec = (ThriftSecurityException) e;
+              result.setSecIsSet(true);
+              msg = result;
+            } else if (e instanceof ThriftTableOperationException) {
+              result.toe = (ThriftTableOperationException) e;
+              result.setToeIsSet(true);
+              msg = result;
+            } else if (e instanceof 
org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -3970,28 +3714,28 @@ public class ClientService {
       }
 
       @Override
-      public void start(I iface, getInstanceId_args args, 
org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) 
throws org.apache.thrift.TException {
-        iface.getInstanceId(resultHandler);
+      public void start(I iface, getDiskUsage_args args, 
org.apache.thrift.async.AsyncMethodCallback<java.util.List<TDiskUsage>> 
resultHandler) throws org.apache.thrift.TException {
+        iface.getDiskUsage(args.tables, args.credentials,resultHandler);
       }
     }
 
-    public static class getZooKeepers<I extends AsyncIface> extends 
org.apache.thrift.AsyncProcessFunction<I, getZooKeepers_args, java.lang.String> 
{
-      public getZooKeepers() {
-        super("getZooKeepers");
+    public static class listLocalUsers<I extends AsyncIface> extends 
org.apache.thrift.AsyncProcessFunction<I, listLocalUsers_args, 
java.util.Set<java.lang.String>> {
+      public listLocalUsers() {
+        super("listLocalUsers");
       }
 
       @Override
-      public getZooKeepers_args getEmptyArgsInstance() {
-        return new getZooKeepers_args();
+      public listLocalUsers_args getEmptyArgsInstance() {
+        return new listLocalUsers_args();
       }
 
       @Override
-      public org.apache.thrift.async.AsyncMethodCallback<java.lang.String> 
getResultHandler(final 
org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final 
int seqid) {
+      public 
org.apache.thrift.async.AsyncMethodCallback<java.util.Set<java.lang.String>> 
getResultHandler(final 
org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final 
int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new 
org.apache.thrift.async.AsyncMethodCallback<java.lang.String>() { 
+        return new 
org.apache.thrift.async.AsyncMethodCallback<java.util.Set<java.lang.String>>() 
{ 
           @Override
-          public void onComplete(java.lang.String o) {
-            getZooKeepers_result result = new getZooKeepers_result();
+          public void onComplete(java.util.Set<java.lang.String> o) {
+            listLocalUsers_result result = new listLocalUsers_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, 
org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -4007,8 +3751,12 @@ public class ClientService {
           public void onError(java.lang.Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
-            getZooKeepers_result result = new getZooKeepers_result();
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
+            listLocalUsers_result result = new listLocalUsers_result();
+            if (e instanceof ThriftSecurityException) {
+              result.sec = (ThriftSecurityException) e;
+              result.setSecIsSet(true);
+              msg = result;
+            } else if (e instanceof 
org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -4037,19 +3785,19 @@ public class ClientService {
       }
 
       @Override
-      public void start(I iface, getZooKeepers_args args, 
org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) 
throws org.apache.thrift.TException {
-        iface.getZooKeepers(resultHandler);
+      public void start(I iface, listLocalUsers_args args, 
org.apache.thrift.async.AsyncMethodCallback<java.util.Set<java.lang.String>> 
resultHandler) throws org.apache.thrift.TException {
+        iface.listLocalUsers(args.tinfo, args.credentials,resultHandler);
       }
     }
 
-    public static class ping<I extends AsyncIface> extends 
org.apache.thrift.AsyncProcessFunction<I, ping_args, Void> {
-      public ping() {
-        super("ping");
+    public static class createLocalUser<I extends AsyncIface> extends 
org.apache.thrift.AsyncProcessFunction<I, createLocalUser_args, Void> {
+      public createLocalUser() {
+        super("createLocalUser");
       }
 
       @Override
-      public ping_args getEmptyArgsInstance() {
-        return new ping_args();
+      public createLocalUser_args getEmptyArgsInstance() {
+        return new createLocalUser_args();
       }
 
       @Override
@@ -4058,7 +3806,7 @@ public class ClientService {
         return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
           @Override
           public void onComplete(Void o) {
-            ping_result result = new ping_result();
+            createLocalUser_result result = new createLocalUser_result();
             try {
               fcall.sendResponse(fb, result, 
org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -4073,7 +3821,7 @@ public class ClientService {
           public void onError(java.lang.Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
-            ping_result result = new ping_result();
+            createLocalUser_result result = new createLocalUser_result();
             if (e instanceof ThriftSecurityException) {
               result.sec = (ThriftSecurityException) e;
               result.setSecIsSet(true);
@@ -4107,224 +3855,8 @@ public class ClientService {
       }
 
       @Override
-      public void start(I iface, ping_args args, 
org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws 
org.apache.thrift.TException {
-        iface.ping(args.credentials,resultHandler);
-      }
-    }
-
-    public static class getDiskUsage<I extends AsyncIface> extends 
org.apache.thrift.AsyncProcessFunction<I, getDiskUsage_args, 
java.util.List<TDiskUsage>> {
-      public getDiskUsage() {
-        super("getDiskUsage");
-      }
-
-      @Override
-      public getDiskUsage_args getEmptyArgsInstance() {
-        return new getDiskUsage_args();
-      }
-
-      @Override
-      public 
org.apache.thrift.async.AsyncMethodCallback<java.util.List<TDiskUsage>> 
getResultHandler(final 
org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final 
int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new 
org.apache.thrift.async.AsyncMethodCallback<java.util.List<TDiskUsage>>() { 
-          @Override
-          public void onComplete(java.util.List<TDiskUsage> o) {
-            getDiskUsage_result result = new getDiskUsage_result();
-            result.success = o;
-            try {
-              fcall.sendResponse(fb, result, 
org.apache.thrift.protocol.TMessageType.REPLY,seqid);
-            } catch (org.apache.thrift.transport.TTransportException e) {
-              _LOGGER.error("TTransportException writing to internal frame 
buffer", e);
-              fb.close();
-            } catch (java.lang.Exception e) {
-              _LOGGER.error("Exception writing to internal frame buffer", e);
-              onError(e);
-            }
-          }
-          @Override
-          public void onError(java.lang.Exception e) {
-            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
-            org.apache.thrift.TSerializable msg;
-            getDiskUsage_result result = new getDiskUsage_result();
-            if (e instanceof ThriftSecurityException) {
-              result.sec = (ThriftSecurityException) e;
-              result.setSecIsSet(true);
-              msg = result;
-            } else if (e instanceof ThriftTableOperationException) {
-              result.toe = (ThriftTableOperationException) e;
-              result.setToeIsSet(true);
-              msg = result;
-            } else if (e instanceof 
org.apache.thrift.transport.TTransportException) {
-              _LOGGER.error("TTransportException inside handler", e);
-              fb.close();
-              return;
-            } else if (e instanceof org.apache.thrift.TApplicationException) {
-              _LOGGER.error("TApplicationException inside handler", e);
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = (org.apache.thrift.TApplicationException)e;
-            } else {
-              _LOGGER.error("Exception inside handler", e);
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = new 
org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR,
 e.getMessage());
-            }
-            try {
-              fcall.sendResponse(fb,msg,msgType,seqid);
-            } catch (java.lang.Exception ex) {
-              _LOGGER.error("Exception writing to internal frame buffer", ex);
-              fb.close();
-            }
-          }
-        };
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      public void start(I iface, getDiskUsage_args args, 
org.apache.thrift.async.AsyncMethodCallback<java.util.List<TDiskUsage>> 
resultHandler) throws org.apache.thrift.TException {
-        iface.getDiskUsage(args.tables, args.credentials,resultHandler);
-      }
-    }
-
-    public static class listLocalUsers<I extends AsyncIface> extends 
org.apache.thrift.AsyncProcessFunction<I, listLocalUsers_args, 
java.util.Set<java.lang.String>> {
-      public listLocalUsers() {
-        super("listLocalUsers");
-      }
-
-      @Override
-      public listLocalUsers_args getEmptyArgsInstance() {
-        return new listLocalUsers_args();
-      }
-
-      @Override
-      public 
org.apache.thrift.async.AsyncMethodCallback<java.util.Set<java.lang.String>> 
getResultHandler(final 
org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final 
int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new 
org.apache.thrift.async.AsyncMethodCallback<java.util.Set<java.lang.String>>() 
{ 
-          @Override
-          public void onComplete(java.util.Set<java.lang.String> o) {
-            listLocalUsers_result result = new listLocalUsers_result();
-            result.success = o;
-            try {
-              fcall.sendResponse(fb, result, 
org.apache.thrift.protocol.TMessageType.REPLY,seqid);
-            } catch (org.apache.thrift.transport.TTransportException e) {
-              _LOGGER.error("TTransportException writing to internal frame 
buffer", e);
-              fb.close();
-            } catch (java.lang.Exception e) {
-              _LOGGER.error("Exception writing to internal frame buffer", e);
-              onError(e);
-            }
-          }
-          @Override
-          public void onError(java.lang.Exception e) {
-            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
-            org.apache.thrift.TSerializable msg;
-            listLocalUsers_result result = new listLocalUsers_result();
-            if (e instanceof ThriftSecurityException) {
-              result.sec = (ThriftSecurityException) e;
-              result.setSecIsSet(true);
-              msg = result;
-            } else if (e instanceof 
org.apache.thrift.transport.TTransportException) {
-              _LOGGER.error("TTransportException inside handler", e);
-              fb.close();
-              return;
-            } else if (e instanceof org.apache.thrift.TApplicationException) {
-              _LOGGER.error("TApplicationException inside handler", e);
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = (org.apache.thrift.TApplicationException)e;
-            } else {
-              _LOGGER.error("Exception inside handler", e);
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = new 
org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR,
 e.getMessage());
-            }
-            try {
-              fcall.sendResponse(fb,msg,msgType,seqid);
-            } catch (java.lang.Exception ex) {
-              _LOGGER.error("Exception writing to internal frame buffer", ex);
-              fb.close();
-            }
-          }
-        };
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      public void start(I iface, listLocalUsers_args args, 
org.apache.thrift.async.AsyncMethodCallback<java.util.Set<java.lang.String>> 
resultHandler) throws org.apache.thrift.TException {
-        iface.listLocalUsers(args.tinfo, args.credentials,resultHandler);
-      }
-    }
-
-    public static class createLocalUser<I extends AsyncIface> extends 
org.apache.thrift.AsyncProcessFunction<I, createLocalUser_args, Void> {
-      public createLocalUser() {
-        super("createLocalUser");
-      }
-
-      @Override
-      public createLocalUser_args getEmptyArgsInstance() {
-        return new createLocalUser_args();
-      }
-
-      @Override
-      public org.apache.thrift.async.AsyncMethodCallback<Void> 
getResultHandler(final 
org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final 
int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
-          @Override
-          public void onComplete(Void o) {
-            createLocalUser_result result = new createLocalUser_result();
-            try {
-              fcall.sendResponse(fb, result, 
org.apache.thrift.protocol.TMessageType.REPLY,seqid);
-            } catch (org.apache.thrift.transport.TTransportException e) {
-              _LOGGER.error("TTransportException writing to internal frame 
buffer", e);
-              fb.close();
-            } catch (java.lang.Exception e) {
-              _LOGGER.error("Exception writing to internal frame buffer", e);
-              onError(e);
-            }
-          }
-          @Override
-          public void onError(java.lang.Exception e) {
-            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
-            org.apache.thrift.TSerializable msg;
-            createLocalUser_result result = new createLocalUser_result();
-            if (e instanceof ThriftSecurityException) {
-              result.sec = (ThriftSecurityException) e;
-              result.setSecIsSet(true);
-              msg = result;
-            } else if (e instanceof 
org.apache.thrift.transport.TTransportException) {
-              _LOGGER.error("TTransportException inside handler", e);
-              fb.close();
-              return;
-            } else if (e instanceof org.apache.thrift.TApplicationException) {
-              _LOGGER.error("TApplicationException inside handler", e);
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = (org.apache.thrift.TApplicationException)e;
-            } else {
-              _LOGGER.error("Exception inside handler", e);
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = new 
org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR,
 e.getMessage());
-            }
-            try {
-              fcall.sendResponse(fb,msg,msgType,seqid);
-            } catch (java.lang.Exception ex) {
-              _LOGGER.error("Exception writing to internal frame buffer", ex);
-              fb.close();
-            }
-          }
-        };
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      public void start(I iface, createLocalUser_args args, 
org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws 
org.apache.thrift.TException {
-        iface.createLocalUser(args.tinfo, args.credentials, args.principal, 
args.password,resultHandler);
+      public void start(I iface, createLocalUser_args args, 
org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws 
org.apache.thrift.TException {
+        iface.createLocalUser(args.tinfo, args.credentials, args.principal, 
args.password,resultHandler);
       }
     }
 
@@ -6298,1938 +5830,6 @@ public class ClientService {
 
   }
 
-  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-  public static class getRootTabletLocation_args implements 
org.apache.thrift.TBase<getRootTabletLocation_args, 
getRootTabletLocation_args._Fields>, java.io.Serializable, Cloneable, 
Comparable<getRootTabletLocation_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new 
org.apache.thrift.protocol.TStruct("getRootTabletLocation_args");
-
-
-    private static final org.apache.thrift.scheme.SchemeFactory 
STANDARD_SCHEME_FACTORY = new getRootTabletLocation_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory 
TUPLE_SCHEME_FACTORY = new getRootTabletLocation_argsTupleSchemeFactory();
-
-
-    /** The set of fields this struct contains, along with convenience methods 
for finding and manipulating them. */
-    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-;
-
-      private static final java.util.Map<java.lang.String, _Fields> byName = 
new java.util.HashMap<java.lang.String, _Fields>();
-
-      static {
-        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
-          byName.put(field.getFieldName(), field);
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, or null if its not 
found.
-       */
-      @org.apache.thrift.annotation.Nullable
-      public static _Fields findByThriftId(int fieldId) {
-        switch(fieldId) {
-          default:
-            return null;
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, throwing an exception
-       * if it is not found.
-       */
-      public static _Fields findByThriftIdOrThrow(int fieldId) {
-        _Fields fields = findByThriftId(fieldId);
-        if (fields == null) throw new 
java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
-        return fields;
-      }
-
-      /**
-       * Find the _Fields constant that matches name, or null if its not found.
-       */
-      @org.apache.thrift.annotation.Nullable
-      public static _Fields findByName(java.lang.String name) {
-        return byName.get(name);
-      }
-
-      private final short _thriftId;
-      private final java.lang.String _fieldName;
-
-      _Fields(short thriftId, java.lang.String fieldName) {
-        _thriftId = thriftId;
-        _fieldName = fieldName;
-      }
-
-      @Override
-      public short getThriftFieldId() {
-        return _thriftId;
-      }
-
-      @Override
-      public java.lang.String getFieldName() {
-        return _fieldName;
-      }
-    }
-    public static final java.util.Map<_Fields, 
org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-    static {
-      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap 
= new java.util.EnumMap<_Fields, 
org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getRootTabletLocation_args.class,
 metaDataMap);
-    }
-
-    public getRootTabletLocation_args() {
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public getRootTabletLocation_args(getRootTabletLocation_args other) {
-    }
-
-    @Override
-    public getRootTabletLocation_args deepCopy() {
-      return new getRootTabletLocation_args(this);
-    }
-
-    @Override
-    public void clear() {
-    }
-
-    @Override
-    public void setFieldValue(_Fields field, 
@org.apache.thrift.annotation.Nullable java.lang.Object value) {
-      switch (field) {
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public java.lang.Object getFieldValue(_Fields field) {
-      switch (field) {
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been 
assigned a value) and false otherwise */
-    @Override
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new java.lang.IllegalArgumentException();
-      }
-
-      switch (field) {
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof getRootTabletLocation_args)
-        return this.equals((getRootTabletLocation_args)that);
-      return false;
-    }
-
-    public boolean equals(getRootTabletLocation_args that) {
-      if (that == null)
-        return false;
-      if (this == that)
-        return true;
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      int hashCode = 1;
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(getRootTabletLocation_args other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      return 0;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public _Fields fieldForId(int fieldId) {
-      return _Fields.findByThriftId(fieldId);
-    }
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol iprot) throws 
org.apache.thrift.TException {
-      scheme(iprot).read(iprot, this);
-    }
-
-    @Override
-    public void write(org.apache.thrift.protocol.TProtocol oprot) throws 
org.apache.thrift.TException {
-      scheme(oprot).write(oprot, this);
-    }
-
-    @Override
-    public java.lang.String toString() {
-      java.lang.StringBuilder sb = new 
java.lang.StringBuilder("getRootTabletLocation_args(");
-      boolean first = true;
-
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws org.apache.thrift.TException {
-      // check for required fields
-      // check for sub-struct validity
-    }
-
-    private void writeObject(java.io.ObjectOutputStream out) throws 
java.io.IOException {
-      try {
-        write(new org.apache.thrift.protocol.TCompactProtocol(new 
org.apache.thrift.transport.TIOStreamTransport(out)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private void readObject(java.io.ObjectInputStream in) throws 
java.io.IOException, java.lang.ClassNotFoundException {
-      try {
-        read(new org.apache.thrift.protocol.TCompactProtocol(new 
org.apache.thrift.transport.TIOStreamTransport(in)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private static class getRootTabletLocation_argsStandardSchemeFactory 
implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getRootTabletLocation_argsStandardScheme getScheme() {
-        return new getRootTabletLocation_argsStandardScheme();
-      }
-    }
-
-    private static class getRootTabletLocation_argsStandardScheme extends 
org.apache.thrift.scheme.StandardScheme<getRootTabletLocation_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, 
getRootTabletLocation_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            default:
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, 
schemeField.type);
-          }
-          iprot.readFieldEnd();
-        }
-        iprot.readStructEnd();
-
-        // check for required fields of primitive type, which can't be checked 
in the validate method
-        struct.validate();
-      }
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol oprot, 
getRootTabletLocation_args struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class getRootTabletLocation_argsTupleSchemeFactory 
implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getRootTabletLocation_argsTupleScheme getScheme() {
-        return new getRootTabletLocation_argsTupleScheme();
-      }
-    }
-
-    private static class getRootTabletLocation_argsTupleScheme extends 
org.apache.thrift.scheme.TupleScheme<getRootTabletLocation_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, 
getRootTabletLocation_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol oprot = 
(org.apache.thrift.protocol.TTupleProtocol) prot;
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, 
getRootTabletLocation_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol iprot = 
(org.apache.thrift.protocol.TTupleProtocol) prot;
-      }
-    }
-
-    private static <S extends org.apache.thrift.scheme.IScheme> S 
scheme(org.apache.thrift.protocol.TProtocol proto) {
-      return 
(org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? 
STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
-    }
-  }
-
-  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-  public static class getRootTabletLocation_result implements 
org.apache.thrift.TBase<getRootTabletLocation_result, 
getRootTabletLocation_result._Fields>, java.io.Serializable, Cloneable, 
Comparable<getRootTabletLocation_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new 
org.apache.thrift.protocol.TStruct("getRootTabletLocation_result");
-
-    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC 
= new org.apache.thrift.protocol.TField("success", 
org.apache.thrift.protocol.TType.STRING, (short)0);
-
-    private static final org.apache.thrift.scheme.SchemeFactory 
STANDARD_SCHEME_FACTORY = new 
getRootTabletLocation_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory 
TUPLE_SCHEME_FACTORY = new getRootTabletLocation_resultTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable java.lang.String success; // 
required
-
-    /** The set of fields this struct contains, along with convenience methods 
for finding and manipulating them. */
-    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      SUCCESS((short)0, "success");
-
-      private static final java.util.Map<java.lang.String, _Fields> byName = 
new java.util.HashMap<java.lang.String, _Fields>();
-
-      static {
-        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
-          byName.put(field.getFieldName(), field);
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, or null if its not 
found.
-       */
-      @org.apache.thrift.annotation.Nullable
-      public static _Fields findByThriftId(int fieldId) {
-        switch(fieldId) {
-          case 0: // SUCCESS
-            return SUCCESS;
-          default:
-            return null;
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, throwing an exception
-       * if it is not found.
-       */
-      public static _Fields findByThriftIdOrThrow(int fieldId) {
-        _Fields fields = findByThriftId(fieldId);
-        if (fields == null) throw new 
java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
-        return fields;
-      }
-
-      /**
-       * Find the _Fields constant that matches name, or null if its not found.
-       */
-      @org.apache.thrift.annotation.Nullable
-      public static _Fields findByName(java.lang.String name) {
-        return byName.get(name);
-      }
-
-      private final short _thriftId;
-      private final java.lang.String _fieldName;
-
-      _Fields(short thriftId, java.lang.String fieldName) {
-        _thriftId = thriftId;
-        _fieldName = fieldName;
-      }
-
-      @Override
-      public short getThriftFieldId() {
-        return _thriftId;
-      }
-
-      @Override
-      public java.lang.String getFieldName() {
-        return _fieldName;
-      }
-    }
-
-    // isset id assignments
-    public static final java.util.Map<_Fields, 
org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-    static {
-      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap 
= new java.util.EnumMap<_Fields, 
org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new 
org.apache.thrift.meta_data.FieldMetaData("success", 
org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new 
org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getRootTabletLocation_result.class,
 metaDataMap);
-    }
-
-    public getRootTabletLocation_result() {
-    }
-
-    public getRootTabletLocation_result(
-      java.lang.String success)
-    {
-      this();
-      this.success = success;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public getRootTabletLocation_result(getRootTabletLocation_result other) {
-      if (other.isSetSuccess()) {
-        this.success = other.success;
-      }
-    }
-
-    @Override
-    public getRootTabletLocation_result deepCopy() {
-      return new getRootTabletLocation_result(this);
-    }
-
-    @Override
-    public void clear() {
-      this.success = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.lang.String getSuccess() {
-      return this.success;
-    }
-
-    public getRootTabletLocation_result 
setSuccess(@org.apache.thrift.annotation.Nullable java.lang.String success) {
-      this.success = success;
-      return this;
-    }
-
-    public void unsetSuccess() {
-      this.success = null;
-    }
-
-    /** Returns true if field success is set (has been assigned a value) and 
false otherwise */
-    public boolean isSetSuccess() {
-      return this.success != null;
-    }
-
-    public void setSuccessIsSet(boolean value) {
-      if (!value) {
-        this.success = null;
-      }
-    }
-
-    @Override
-    public void setFieldValue(_Fields field, 
@org.apache.thrift.annotation.Nullable java.lang.Object value) {
-      switch (field) {
-      case SUCCESS:
-        if (value == null) {
-          unsetSuccess();
-        } else {
-          setSuccess((java.lang.String)value);
-        }
-        break;
-
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public java.lang.Object getFieldValue(_Fields field) {
-      switch (field) {
-      case SUCCESS:
-        return getSuccess();
-
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been 
assigned a value) and false otherwise */
-    @Override
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new java.lang.IllegalArgumentException();
-      }
-
-      switch (field) {
-      case SUCCESS:
-        return isSetSuccess();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof getRootTabletLocation_result)
-        return this.equals((getRootTabletLocation_result)that);
-      return false;
-    }
-
-    public boolean equals(getRootTabletLocation_result that) {
-      if (that == null)
-        return false;
-      if (this == that)
-        return true;
-
-      boolean this_present_success = true && this.isSetSuccess();
-      boolean that_present_success = true && that.isSetSuccess();
-      if (this_present_success || that_present_success) {
-        if (!(this_present_success && that_present_success))
-          return false;
-        if (!this.success.equals(that.success))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      int hashCode = 1;
-
-      hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
-      if (isSetSuccess())
-        hashCode = hashCode * 8191 + success.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(getRootTabletLocation_result other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      lastComparison = java.lang.Boolean.compare(isSetSuccess(), 
other.isSetSuccess());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetSuccess()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, 
other.success);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      return 0;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public _Fields fieldForId(int fieldId) {
-      return _Fields.findByThriftId(fieldId);
-    }
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol iprot) throws 
org.apache.thrift.TException {
-      scheme(iprot).read(iprot, this);
-    }
-
-    public void write(org.apache.thrift.protocol.TProtocol oprot) throws 
org.apache.thrift.TException {
-      scheme(oprot).write(oprot, this);
-      }
-
-    @Override
-    public java.lang.String toString() {
-      java.lang.StringBuilder sb = new 
java.lang.StringBuilder("getRootTabletLocation_result(");
-      boolean first = true;
-
-      sb.append("success:");
-      if (this.success == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.success);
-      }
-      first = false;
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws org.apache.thrift.TException {
-      // check for required fields
-      // check for sub-struct validity
-    }
-
-    private void writeObject(java.io.ObjectOutputStream out) throws 
java.io.IOException {
-      try {
-        write(new org.apache.thrift.protocol.TCompactProtocol(new 
org.apache.thrift.transport.TIOStreamTransport(out)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private void readObject(java.io.ObjectInputStream in) throws 
java.io.IOException, java.lang.ClassNotFoundException {
-      try {
-        read(new org.apache.thrift.protocol.TCompactProtocol(new 
org.apache.thrift.transport.TIOStreamTransport(in)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private static class getRootTabletLocation_resultStandardSchemeFactory 
implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getRootTabletLocation_resultStandardScheme getScheme() {
-        return new getRootTabletLocation_resultStandardScheme();
-      }
-    }
-
-    private static class getRootTabletLocation_resultStandardScheme extends 
org.apache.thrift.scheme.StandardScheme<getRootTabletLocation_result> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, 
getRootTabletLocation_result struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 0: // SUCCESS
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) 
{
-                struct.success = iprot.readString();
-                struct.setSuccessIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, 
schemeField.type);
-              }
-              break;
-            default:
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, 
schemeField.type);
-          }
-          iprot.readFieldEnd();
-        }
-        iprot.readStructEnd();
-
-        // check for required fields of primitive type, which can't be checked 
in the validate method
-        struct.validate();
-      }
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol oprot, 
getRootTabletLocation_result struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.success != null) {
-          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
-          oprot.writeString(struct.success);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class getRootTabletLocation_resultTupleSchemeFactory 
implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getRootTabletLocation_resultTupleScheme getScheme() {
-        return new getRootTabletLocation_resultTupleScheme();
-      }
-    }
-
-    private static class getRootTabletLocation_resultTupleScheme extends 
org.apache.thrift.scheme.TupleScheme<getRootTabletLocation_result> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, 
getRootTabletLocation_result struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol oprot = 
(org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet optionals = new java.util.BitSet();
-        if (struct.isSetSuccess()) {
-          optionals.set(0);
-        }
-        oprot.writeBitSet(optionals, 1);
-        if (struct.isSetSuccess()) {
-          oprot.writeString(struct.success);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, 
getRootTabletLocation_result struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol iprot = 
(org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet incoming = iprot.readBitSet(1);
-        if (incoming.get(0)) {
-          struct.success = iprot.readString();
-          struct.setSuccessIsSet(true);
-        }
-      }
-    }
-
-    private static <S extends org.apache.thrift.scheme.IScheme> S 
scheme(org.apache.thrift.protocol.TProtocol proto) {
-      return 
(org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? 
STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
-    }
-  }
-
-  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-  public static class getInstanceId_args implements 
org.apache.thrift.TBase<getInstanceId_args, getInstanceId_args._Fields>, 
java.io.Serializable, Cloneable, Comparable<getInstanceId_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new 
org.apache.thrift.protocol.TStruct("getInstanceId_args");
-
-
-    private static final org.apache.thrift.scheme.SchemeFactory 
STANDARD_SCHEME_FACTORY = new getInstanceId_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory 
TUPLE_SCHEME_FACTORY = new getInstanceId_argsTupleSchemeFactory();
-
-
-    /** The set of fields this struct contains, along with convenience methods 
for finding and manipulating them. */
-    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-;
-
-      private static final java.util.Map<java.lang.String, _Fields> byName = 
new java.util.HashMap<java.lang.String, _Fields>();
-
-      static {
-        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
-          byName.put(field.getFieldName(), field);
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, or null if its not 
found.
-       */
-      @org.apache.thrift.annotation.Nullable
-      public static _Fields findByThriftId(int fieldId) {
-        switch(fieldId) {
-          default:
-            return null;
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, throwing an exception
-       * if it is not found.
-       */
-      public static _Fields findByThriftIdOrThrow(int fieldId) {
-        _Fields fields = findByThriftId(fieldId);
-        if (fields == null) throw new 
java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
-        return fields;
-      }
-
-      /**
-       * Find the _Fields constant that matches name, or null if its not found.
-       */
-      @org.apache.thrift.annotation.Nullable
-      public static _Fields findByName(java.lang.String name) {
-        return byName.get(name);
-      }
-
-      private final short _thriftId;
-      private final java.lang.String _fieldName;
-
-      _Fields(short thriftId, java.lang.String fieldName) {
-        _thriftId = thriftId;
-        _fieldName = fieldName;
-      }
-
-      @Override
-      public short getThriftFieldId() {
-        return _thriftId;
-      }
-
-      @Override
-      public java.lang.String getFieldName() {
-        return _fieldName;
-      }
-    }
-    public static final java.util.Map<_Fields, 
org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-    static {
-      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap 
= new java.util.EnumMap<_Fields, 
org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getInstanceId_args.class,
 metaDataMap);
-    }
-
-    public getInstanceId_args() {
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public getInstanceId_args(getInstanceId_args other) {
-    }
-
-    @Override
-    public getInstanceId_args deepCopy() {
-      return new getInstanceId_args(this);
-    }
-
-    @Override
-    public void clear() {
-    }
-
-    @Override
-    public void setFieldValue(_Fields field, 
@org.apache.thrift.annotation.Nullable java.lang.Object value) {
-      switch (field) {
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public java.lang.Object getFieldValue(_Fields field) {
-      switch (field) {
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been 
assigned a value) and false otherwise */
-    @Override
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new java.lang.IllegalArgumentException();
-      }
-
-      switch (field) {
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof getInstanceId_args)
-        return this.equals((getInstanceId_args)that);
-      return false;
-    }
-
-    public boolean equals(getInstanceId_args that) {
-      if (that == null)
-        return false;
-      if (this == that)
-        return true;
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      int hashCode = 1;
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(getInstanceId_args other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      return 0;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public _Fields fieldForId(int fieldId) {
-      return _Fields.findByThriftId(fieldId);
-    }
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol iprot) throws 
org.apache.thrift.TException {
-      scheme(iprot).read(iprot, this);
-    }
-
-    @Override
-    public void write(org.apache.thrift.protocol.TProtocol oprot) throws 
org.apache.thrift.TException {
-      scheme(oprot).write(oprot, this);
-    }
-
-    @Override
-    public java.lang.String toString() {
-      java.lang.StringBuilder sb = new 
java.lang.StringBuilder("getInstanceId_args(");
-      boolean first = true;
-
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws org.apache.thrift.TException {
-      // check for required fields
-      // check for sub-struct validity
-    }
-
-    private void writeObject(java.io.ObjectOutputStream out) throws 
java.io.IOException {
-      try {
-        write(new org.apache.thrift.protocol.TCompactProtocol(new 
org.apache.thrift.transport.TIOStreamTransport(out)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private void readObject(java.io.ObjectInputStream in) throws 
java.io.IOException, java.lang.ClassNotFoundException {
-      try {
-        read(new org.apache.thrift.protocol.TCompactProtocol(new 
org.apache.thrift.transport.TIOStreamTransport(in)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private static class getInstanceId_argsStandardSchemeFactory implements 
org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getInstanceId_argsStandardScheme getScheme() {
-        return new getInstanceId_argsStandardScheme();
-      }
-    }
-
-    private static class getInstanceId_argsStandardScheme extends 
org.apache.thrift.scheme.StandardScheme<getInstanceId_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, 
getInstanceId_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            default:
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, 
schemeField.type);
-          }
-          iprot.readFieldEnd();
-        }
-        iprot.readStructEnd();
-
-        // check for required fields of primitive type, which can't be checked 
in the validate method
-        struct.validate();
-      }
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol oprot, 
getInstanceId_args struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class getInstanceId_argsTupleSchemeFactory implements 
org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getInstanceId_argsTupleScheme getScheme() {
-        return new getInstanceId_argsTupleScheme();
-      }
-    }
-
-    private static class getInstanceId_argsTupleScheme extends 
org.apache.thrift.scheme.TupleScheme<getInstanceId_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, 
getInstanceId_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol oprot = 
(org.apache.thrift.protocol.TTupleProtocol) prot;
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, 
getInstanceId_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol iprot = 
(org.apache.thrift.protocol.TTupleProtocol) prot;
-      }
-    }
-
-    private static <S extends org.apache.thrift.scheme.IScheme> S 
scheme(org.apache.thrift.protocol.TProtocol proto) {
-      return 
(org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? 
STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
-    }
-  }
-
-  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-  public static class getInstanceId_result implements 
org.apache.thrift.TBase<getInstanceId_result, getInstanceId_result._Fields>, 
java.io.Serializable, Cloneable, Comparable<getInstanceId_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new 
org.apache.thrift.protocol.TStruct("getInstanceId_result");
-
-    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC 
= new org.apache.thrift.protocol.TField("success", 
org.apache.thrift.protocol.TType.STRING, (short)0);
-
-    private static final org.apache.thrift.scheme.SchemeFactory 
STANDARD_SCHEME_FACTORY = new getInstanceId_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory 
TUPLE_SCHEME_FACTORY = new getInstanceId_resultTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable java.lang.String success; // 
required
-
-    /** The set of fields this struct contains, along with convenience methods 
for finding and manipulating them. */
-    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      SUCCESS((short)0, "success");
-
-      private static final java.util.Map<java.lang.String, _Fields> byName = 
new java.util.HashMap<java.lang.String, _Fields>();
-
-      static {
-        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
-          byName.put(field.getFieldName(), field);
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, or null if its not 
found.
-       */
-      @org.apache.thrift.annotation.Nullable
-      public static _Fields findByThriftId(int fieldId) {
-        switch(fieldId) {
-          case 0: // SUCCESS
-            return SUCCESS;
-          default:
-            return null;
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, throwing an exception
-       * if it is not found.
-       */
-      public static _Fields findByThriftIdOrThrow(int fieldId) {
-        _Fields fields = findByThriftId(fieldId);
-        if (fields == null) throw new 
java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
-        return fields;
-      }
-
-      /**
-       * Find the _Fields constant that matches name, or null if its not found.
-       */
-      @org.apache.thrift.annotation.Nullable
-      public static _Fields findByName(java.lang.String name) {
-        return byName.get(name);
-      }
-
-      private final short _thriftId;
-      private final java.lang.String _fieldName;
-
-      _Fields(short thriftId, java.lang.String fieldName) {
-        _thriftId = thriftId;
-        _fieldName = fieldName;
-      }
-
-      @Override
-      public short getThriftFieldId() {
-        return _thriftId;
-      }
-
-      @Override
-      public java.lang.String getFieldName() {
-        return _fieldName;
-      }
-    }
-
-    // isset id assignments
-    public static final java.util.Map<_Fields, 
org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-    static {
-      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap 
= new java.util.EnumMap<_Fields, 
org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new 
org.apache.thrift.meta_data.FieldMetaData("success", 
org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new 
org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getInstanceId_result.class,
 metaDataMap);
-    }
-
-    public getInstanceId_result() {
-    }
-
-    public getInstanceId_result(
-      java.lang.String success)
-    {
-      this();
-      this.success = success;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public getInstanceId_result(getInstanceId_result other) {
-      if (other.isSetSuccess()) {
-        this.success = other.success;
-      }
-    }
-
-    @Override
-    public getInstanceId_result deepCopy() {
-      return new getInstanceId_result(this);
-    }
-
-    @Override
-    public void clear() {
-      this.success = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.lang.String getSuccess() {
-      return this.success;
-    }
-
-    public getInstanceId_result 
setSuccess(@org.apache.thrift.annotation.Nullable java.lang.String success) {
-      this.success = success;
-      return this;
-    }
-
-    public void unsetSuccess() {
-      this.success = null;
-    }
-
-    /** Returns true if field success is set (has been assigned a value) and 
false otherwise */
-    public boolean isSetSuccess() {
-      return this.success != null;
-    }
-
-    public void setSuccessIsSet(boolean value) {
-      if (!value) {
-        this.success = null;
-      }
-    }
-
-    @Override
-    public void setFieldValue(_Fields field, 
@org.apache.thrift.annotation.Nullable java.lang.Object value) {
-      switch (field) {
-      case SUCCESS:
-        if (value == null) {
-          unsetSuccess();
-        } else {
-          setSuccess((java.lang.String)value);
-        }
-        break;
-
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public java.lang.Object getFieldValue(_Fields field) {
-      switch (field) {
-      case SUCCESS:
-        return getSuccess();
-
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been 
assigned a value) and false otherwise */
-    @Override
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new java.lang.IllegalArgumentException();
-      }
-
-      switch (field) {
-      case SUCCESS:
-        return isSetSuccess();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof getInstanceId_result)
-        return this.equals((getInstanceId_result)that);
-      return false;
-    }
-
-    public boolean equals(getInstanceId_result that) {
-      if (that == null)
-        return false;
-      if (this == that)
-        return true;
-
-      boolean this_present_success = true && this.isSetSuccess();
-      boolean that_present_success = true && that.isSetSuccess();
-      if (this_present_success || that_present_success) {
-        if (!(this_present_success && that_present_success))
-          return false;
-        if (!this.success.equals(that.success))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      int hashCode = 1;
-
-      hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
-      if (isSetSuccess())
-        hashCode = hashCode * 8191 + success.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(getInstanceId_result other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      lastComparison = java.lang.Boolean.compare(isSetSuccess(), 
other.isSetSuccess());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetSuccess()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, 
other.success);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      return 0;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public _Fields fieldForId(int fieldId) {
-      return _Fields.findByThriftId(fieldId);
-    }
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol iprot) throws 
org.apache.thrift.TException {
-      scheme(iprot).read(iprot, this);
-    }
-
-    public void write(org.apache.thrift.protocol.TProtocol oprot) throws 
org.apache.thrift.TException {
-      scheme(oprot).write(oprot, this);
-      }
-
-    @Override
-    public java.lang.String toString() {
-      java.lang.StringBuilder sb = new 
java.lang.StringBuilder("getInstanceId_result(");
-      boolean first = true;
-
-      sb.append("success:");
-      if (this.success == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.success);
-      }
-      first = false;
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws org.apache.thrift.TException {
-      // check for required fields
-      // check for sub-struct validity
-    }
-
-    private void writeObject(java.io.ObjectOutputStream out) throws 
java.io.IOException {
-      try {
-        write(new org.apache.thrift.protocol.TCompactProtocol(new 
org.apache.thrift.transport.TIOStreamTransport(out)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private void readObject(java.io.ObjectInputStream in) throws 
java.io.IOException, java.lang.ClassNotFoundException {
-      try {
-        read(new org.apache.thrift.protocol.TCompactProtocol(new 
org.apache.thrift.transport.TIOStreamTransport(in)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private static class getInstanceId_resultStandardSchemeFactory implements 
org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getInstanceId_resultStandardScheme getScheme() {
-        return new getInstanceId_resultStandardScheme();
-      }
-    }
-
-    private static class getInstanceId_resultStandardScheme extends 
org.apache.thrift.scheme.StandardScheme<getInstanceId_result> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, 
getInstanceId_result struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 0: // SUCCESS
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) 
{
-                struct.success = iprot.readString();
-                struct.setSuccessIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, 
schemeField.type);
-              }
-              break;
-            default:
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, 
schemeField.type);
-          }
-          iprot.readFieldEnd();
-        }
-        iprot.readStructEnd();
-
-        // check for required fields of primitive type, which can't be checked 
in the validate method
-        struct.validate();
-      }
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol oprot, 
getInstanceId_result struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.success != null) {
-          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
-          oprot.writeString(struct.success);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class getInstanceId_resultTupleSchemeFactory implements 
org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getInstanceId_resultTupleScheme getScheme() {
-        return new getInstanceId_resultTupleScheme();
-      }
-    }
-
-    private static class getInstanceId_resultTupleScheme extends 
org.apache.thrift.scheme.TupleScheme<getInstanceId_result> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, 
getInstanceId_result struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol oprot = 
(org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet optionals = new java.util.BitSet();
-        if (struct.isSetSuccess()) {
-          optionals.set(0);
-        }
-        oprot.writeBitSet(optionals, 1);
-        if (struct.isSetSuccess()) {
-          oprot.writeString(struct.success);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, 
getInstanceId_result struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol iprot = 
(org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet incoming = iprot.readBitSet(1);
-        if (incoming.get(0)) {
-          struct.success = iprot.readString();
-          struct.setSuccessIsSet(true);
-        }
-      }
-    }
-
-    private static <S extends org.apache.thrift.scheme.IScheme> S 
scheme(org.apache.thrift.protocol.TProtocol proto) {
-      return 
(org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? 
STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
-    }
-  }
-
-  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-  public static class getZooKeepers_args implements 
org.apache.thrift.TBase<getZooKeepers_args, getZooKeepers_args._Fields>, 
java.io.Serializable, Cloneable, Comparable<getZooKeepers_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new 
org.apache.thrift.protocol.TStruct("getZooKeepers_args");
-
-
-    private static final org.apache.thrift.scheme.SchemeFactory 
STANDARD_SCHEME_FACTORY = new getZooKeepers_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory 
TUPLE_SCHEME_FACTORY = new getZooKeepers_argsTupleSchemeFactory();
-
-
-    /** The set of fields this struct contains, along with convenience methods 
for finding and manipulating them. */
-    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-;
-
-      private static final java.util.Map<java.lang.String, _Fields> byName = 
new java.util.HashMap<java.lang.String, _Fields>();
-
-      static {
-        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
-          byName.put(field.getFieldName(), field);
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, or null if its not 
found.
-       */
-      @org.apache.thrift.annotation.Nullable
-      public static _Fields findByThriftId(int fieldId) {
-        switch(fieldId) {
-          default:
-            return null;
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, throwing an exception
-       * if it is not found.
-       */
-      public static _Fields findByThriftIdOrThrow(int fieldId) {
-        _Fields fields = findByThriftId(fieldId);
-        if (fields == null) throw new 
java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
-        return fields;
-      }
-
-      /**
-       * Find the _Fields constant that matches name, or null if its not found.
-       */
-      @org.apache.thrift.annotation.Nullable
-      public static _Fields findByName(java.lang.String name) {
-        return byName.get(name);
-      }
-
-      private final short _thriftId;
-      private final java.lang.String _fieldName;
-
-      _Fields(short thriftId, java.lang.String fieldName) {
-        _thriftId = thriftId;
-        _fieldName = fieldName;
-      }
-
-      @Override
-      public short getThriftFieldId() {
-        return _thriftId;
-      }
-
-      @Override
-      public java.lang.String getFieldName() {
-        return _fieldName;
-      }
-    }
-    public static final java.util.Map<_Fields, 
org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-    static {
-      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap 
= new java.util.EnumMap<_Fields, 
org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getZooKeepers_args.class,
 metaDataMap);
-    }
-
-    public getZooKeepers_args() {
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public getZooKeepers_args(getZooKeepers_args other) {
-    }
-
-    @Override
-    public getZooKeepers_args deepCopy() {
-      return new getZooKeepers_args(this);
-    }
-
-    @Override
-    public void clear() {
-    }
-
-    @Override
-    public void setFieldValue(_Fields field, 
@org.apache.thrift.annotation.Nullable java.lang.Object value) {
-      switch (field) {
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public java.lang.Object getFieldValue(_Fields field) {
-      switch (field) {
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been 
assigned a value) and false otherwise */
-    @Override
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new java.lang.IllegalArgumentException();
-      }
-
-      switch (field) {
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof getZooKeepers_args)
-        return this.equals((getZooKeepers_args)that);
-      return false;
-    }
-
-    public boolean equals(getZooKeepers_args that) {
-      if (that == null)
-        return false;
-      if (this == that)
-        return true;
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      int hashCode = 1;
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(getZooKeepers_args other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      return 0;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public _Fields fieldForId(int fieldId) {
-      return _Fields.findByThriftId(fieldId);
-    }
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol iprot) throws 
org.apache.thrift.TException {
-      scheme(iprot).read(iprot, this);
-    }
-
-    @Override
-    public void write(org.apache.thrift.protocol.TProtocol oprot) throws 
org.apache.thrift.TException {
-      scheme(oprot).write(oprot, this);
-    }
-
-    @Override
-    public java.lang.String toString() {
-      java.lang.StringBuilder sb = new 
java.lang.StringBuilder("getZooKeepers_args(");
-      boolean first = true;
-
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws org.apache.thrift.TException {
-      // check for required fields
-      // check for sub-struct validity
-    }
-
-    private void writeObject(java.io.ObjectOutputStream out) throws 
java.io.IOException {
-      try {
-        write(new org.apache.thrift.protocol.TCompactProtocol(new 
org.apache.thrift.transport.TIOStreamTransport(out)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private void readObject(java.io.ObjectInputStream in) throws 
java.io.IOException, java.lang.ClassNotFoundException {
-      try {
-        read(new org.apache.thrift.protocol.TCompactProtocol(new 
org.apache.thrift.transport.TIOStreamTransport(in)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private static class getZooKeepers_argsStandardSchemeFactory implements 
org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getZooKeepers_argsStandardScheme getScheme() {
-        return new getZooKeepers_argsStandardScheme();
-      }
-    }
-
-    private static class getZooKeepers_argsStandardScheme extends 
org.apache.thrift.scheme.StandardScheme<getZooKeepers_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, 
getZooKeepers_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            default:
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, 
schemeField.type);
-          }
-          iprot.readFieldEnd();
-        }
-        iprot.readStructEnd();
-
-        // check for required fields of primitive type, which can't be checked 
in the validate method
-        struct.validate();
-      }
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol oprot, 
getZooKeepers_args struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class getZooKeepers_argsTupleSchemeFactory implements 
org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getZooKeepers_argsTupleScheme getScheme() {
-        return new getZooKeepers_argsTupleScheme();
-      }
-    }
-
-    private static class getZooKeepers_argsTupleScheme extends 
org.apache.thrift.scheme.TupleScheme<getZooKeepers_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, 
getZooKeepers_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol oprot = 
(org.apache.thrift.protocol.TTupleProtocol) prot;
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, 
getZooKeepers_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol iprot = 
(org.apache.thrift.protocol.TTupleProtocol) prot;
-      }
-    }
-
-    private static <S extends org.apache.thrift.scheme.IScheme> S 
scheme(org.apache.thrift.protocol.TProtocol proto) {
-      return 
(org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? 
STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
-    }
-  }
-
-  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-  public static class getZooKeepers_result implements 
org.apache.thrift.TBase<getZooKeepers_result, getZooKeepers_result._Fields>, 
java.io.Serializable, Cloneable, Comparable<getZooKeepers_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new 
org.apache.thrift.protocol.TStruct("getZooKeepers_result");
-
-    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC 
= new org.apache.thrift.protocol.TField("success", 
org.apache.thrift.protocol.TType.STRING, (short)0);
-
-    private static final org.apache.thrift.scheme.SchemeFactory 
STANDARD_SCHEME_FACTORY = new getZooKeepers_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory 
TUPLE_SCHEME_FACTORY = new getZooKeepers_resultTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable java.lang.String success; // 
required
-
-    /** The set of fields this struct contains, along with convenience methods 
for finding and manipulating them. */
-    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      SUCCESS((short)0, "success");
-
-      private static final java.util.Map<java.lang.String, _Fields> byName = 
new java.util.HashMap<java.lang.String, _Fields>();
-
-      static {
-        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
-          byName.put(field.getFieldName(), field);
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, or null if its not 
found.
-       */
-      @org.apache.thrift.annotation.Nullable
-      public static _Fields findByThriftId(int fieldId) {
-        switch(fieldId) {
-          case 0: // SUCCESS
-            return SUCCESS;
-          default:
-            return null;
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, throwing an exception
-       * if it is not found.
-       */
-      public static _Fields findByThriftIdOrThrow(int fieldId) {
-        _Fields fields = findByThriftId(fieldId);
-        if (fields == null) throw new 
java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
-        return fields;
-      }
-
-      /**
-       * Find the _Fields constant that matches name, or null if its not found.
-       */
-      @org.apache.thrift.annotation.Nullable
-      public static _Fields findByName(java.lang.String name) {
-        return byName.get(name);
-      }
-
-      private final short _thriftId;
-      private final java.lang.String _fieldName;
-
-      _Fields(short thriftId, java.lang.String fieldName) {
-        _thriftId = thriftId;
-        _fieldName = fieldName;
-      }
-
-      @Override
-      public short getThriftFieldId() {
-        return _thriftId;
-      }
-
-      @Override
-      public java.lang.String getFieldName() {
-        return _fieldName;
-      }
-    }
-
-    // isset id assignments
-    public static final java.util.Map<_Fields, 
org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-    static {
-      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap 
= new java.util.EnumMap<_Fields, 
org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new 
org.apache.thrift.meta_data.FieldMetaData("success", 
org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new 
org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getZooKeepers_result.class,
 metaDataMap);
-    }
-
-    public getZooKeepers_result() {
-    }
-
-    public getZooKeepers_result(
-      java.lang.String success)
-    {
-      this();
-      this.success = success;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public getZooKeepers_result(getZooKeepers_result other) {
-      if (other.isSetSuccess()) {
-        this.success = other.success;
-      }
-    }
-
-    @Override
-    public getZooKeepers_result deepCopy() {
-      return new getZooKeepers_result(this);
-    }
-
-    @Override
-    public void clear() {
-      this.success = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.lang.String getSuccess() {
-      return this.success;
-    }
-
-    public getZooKeepers_result 
setSuccess(@org.apache.thrift.annotation.Nullable java.lang.String success) {
-      this.success = success;
-      return this;
-    }
-
-    public void unsetSuccess() {
-      this.success = null;
-    }
-
-    /** Returns true if field success is set (has been assigned a value) and 
false otherwise */
-    public boolean isSetSuccess() {
-      return this.success != null;
-    }
-
-    public void setSuccessIsSet(boolean value) {
-      if (!value) {
-        this.success = null;
-      }
-    }
-
-    @Override
-    public void setFieldValue(_Fields field, 
@org.apache.thrift.annotation.Nullable java.lang.Object value) {
-      switch (field) {
-      case SUCCESS:
-        if (value == null) {
-          unsetSuccess();
-        } else {
-          setSuccess((java.lang.String)value);
-        }
-        break;
-
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public java.lang.Object getFieldValue(_Fields field) {
-      switch (field) {
-      case SUCCESS:
-        return getSuccess();
-
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been 
assigned a value) and false otherwise */
-    @Override
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new java.lang.IllegalArgumentException();
-      }
-
-      switch (field) {
-      case SUCCESS:
-        return isSetSuccess();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof getZooKeepers_result)
-        return this.equals((getZooKeepers_result)that);
-      return false;
-    }
-
-    public boolean equals(getZooKeepers_result that) {
-      if (that == null)
-        return false;
-      if (this == that)
-        return true;
-
-      boolean this_present_success = true && this.isSetSuccess();
-      boolean that_present_success = true && that.isSetSuccess();
-      if (this_present_success || that_present_success) {
-        if (!(this_present_success && that_present_success))
-          return false;
-        if (!this.success.equals(that.success))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      int hashCode = 1;
-
-      hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
-      if (isSetSuccess())
-        hashCode = hashCode * 8191 + success.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(getZooKeepers_result other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      lastComparison = java.lang.Boolean.compare(isSetSuccess(), 
other.isSetSuccess());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetSuccess()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, 
other.success);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      return 0;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public _Fields fieldForId(int fieldId) {
-      return _Fields.findByThriftId(fieldId);
-    }
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol iprot) throws 
org.apache.thrift.TException {
-      scheme(iprot).read(iprot, this);
-    }
-
-    public void write(org.apache.thrift.protocol.TProtocol oprot) throws 
org.apache.thrift.TException {
-      scheme(oprot).write(oprot, this);
-      }
-
-    @Override
-    public java.lang.String toString() {
-      java.lang.StringBuilder sb = new 
java.lang.StringBuilder("getZooKeepers_result(");
-      boolean first = true;
-
-      sb.append("success:");
-      if (this.success == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.success);
-      }
-      first = false;
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws org.apache.thrift.TException {
-      // check for required fields
-      // check for sub-struct validity
-    }
-
-    private void writeObject(java.io.ObjectOutputStream out) throws 
java.io.IOException {
-      try {
-        write(new org.apache.thrift.protocol.TCompactProtocol(new 
org.apache.thrift.transport.TIOStreamTransport(out)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private void readObject(java.io.ObjectInputStream in) throws 
java.io.IOException, java.lang.ClassNotFoundException {
-      try {
-        read(new org.apache.thrift.protocol.TCompactProtocol(new 
org.apache.thrift.transport.TIOStreamTransport(in)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private static class getZooKeepers_resultStandardSchemeFactory implements 
org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getZooKeepers_resultStandardScheme getScheme() {
-        return new getZooKeepers_resultStandardScheme();
-      }
-    }
-
-    private static class getZooKeepers_resultStandardScheme extends 
org.apache.thrift.scheme.StandardScheme<getZooKeepers_result> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, 
getZooKeepers_result struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 0: // SUCCESS
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) 
{
-                struct.success = iprot.readString();
-                struct.setSuccessIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, 
schemeField.type);
-              }
-              break;
-            default:
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, 
schemeField.type);
-          }
-          iprot.readFieldEnd();
-        }
-        iprot.readStructEnd();
-
-        // check for required fields of primitive type, which can't be checked 
in the validate method
-        struct.validate();
-      }
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol oprot, 
getZooKeepers_result struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.success != null) {
-          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
-          oprot.writeString(struct.success);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class getZooKeepers_resultTupleSchemeFactory implements 
org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getZooKeepers_resultTupleScheme getScheme() {
-        return new getZooKeepers_resultTupleScheme();
-      }
-    }
-
-    private static class getZooKeepers_resultTupleScheme extends 
org.apache.thrift.scheme.TupleScheme<getZooKeepers_result> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, 
getZooKeepers_result struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol oprot = 
(org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet optionals = new java.util.BitSet();
-        if (struct.isSetSuccess()) {
-          optionals.set(0);
-        }
-        oprot.writeBitSet(optionals, 1);
-        if (struct.isSetSuccess()) {
-          oprot.writeString(struct.success);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, 
getZooKeepers_result struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol iprot = 
(org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet incoming = iprot.readBitSet(1);
-        if (incoming.get(0)) {
-          struct.success = iprot.readString();
-          struct.setSuccessIsSet(true);
-        }
-      }
-    }
-
-    private static <S extends org.apache.thrift.scheme.IScheme> S 
scheme(org.apache.thrift.protocol.TProtocol proto) {
-      return 
(org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? 
STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
-    }
-  }
-
   @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
   public static class ping_args implements org.apache.thrift.TBase<ping_args, 
ping_args._Fields>, java.io.Serializable, Cloneable, Comparable<ping_args>   {
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new 
org.apache.thrift.protocol.TStruct("ping_args");
diff --git a/core/src/main/thrift/client.thrift 
b/core/src/main/thrift/client.thrift
index f62e914599..05c42e09e3 100644
--- a/core/src/main/thrift/client.thrift
+++ b/core/src/main/thrift/client.thrift
@@ -130,11 +130,6 @@ enum TTabletAvailability {
 
 service ClientService {
 
-  // system management methods
-  string getRootTabletLocation()
-  string getInstanceId()
-  string getZooKeepers()
-
   void ping(
     2:security.TCredentials credentials
   ) throws (
diff --git 
a/core/src/test/java/org/apache/accumulo/core/clientImpl/ClientTabletCacheImplTest.java
 
b/core/src/test/java/org/apache/accumulo/core/clientImpl/ClientTabletCacheImplTest.java
index 57350292ea..0b581c4134 100644
--- 
a/core/src/test/java/org/apache/accumulo/core/clientImpl/ClientTabletCacheImplTest.java
+++ 
b/core/src/test/java/org/apache/accumulo/core/clientImpl/ClientTabletCacheImplTest.java
@@ -200,6 +200,8 @@ public class ClientTabletCacheImplTest {
   private ClientContext context;
   private InstanceId iid;
 
+  private static String rootTabletLoc;
+
   @BeforeEach
   public void setUp() throws AccumuloException, TableNotFoundException {
     context = EasyMock.createMock(ClientContext.class);
@@ -218,9 +220,9 @@ public class ClientTabletCacheImplTest {
     EasyMock.expect(tops.isOnline("1")).andReturn(true).anyTimes();
     EasyMock.expect(tops.isOnline("tab1")).andReturn(true).anyTimes();
     iid = InstanceId.of("instance1");
-    
EasyMock.expect(context.getRootTabletLocation()).andReturn("tserver1").anyTimes();
     EasyMock.expect(context.getInstanceID()).andReturn(iid).anyTimes();
     replay(context, tops);
+    rootTabletLoc = "tserver1";
   }
 
   private void runTest(List<Range> ranges, ClientTabletCacheImpl 
tab1TabletCache,
@@ -613,8 +615,8 @@ public class ClientTabletCacheImplTest {
 
     @Override
     protected CachedTablet getRootTabletLocation(ClientContext context) {
-      return new CachedTablet(RootTable.EXTENT, 
context.getRootTabletLocation(), "1",
-          TabletAvailability.HOSTED, false);
+      return new CachedTablet(RootTable.EXTENT, rootTabletLoc, "1", 
TabletAvailability.HOSTED,
+          false);
     }
 
     @Override
@@ -811,7 +813,7 @@ public class ClientTabletCacheImplTest {
     
EasyMock.expect(context.getTableName(TableId.of("1"))).andReturn("1").anyTimes();
     
EasyMock.expect(context.getTableName(TableId.of("tab1"))).andReturn("tab1").anyTimes();
     iid = InstanceId.of("instance1");
-    
EasyMock.expect(context.getRootTabletLocation()).andReturn("tserver4").anyTimes();
+    rootTabletLoc = "tserver4";
     EasyMock.expect(context.getInstanceID()).andReturn(iid).anyTimes();
     replay(context, tops);
 
diff --git 
a/core/src/test/java/org/apache/accumulo/core/metadata/MetadataServicerTest.java
 
b/core/src/test/java/org/apache/accumulo/core/metadata/MetadataServicerTest.java
deleted file mode 100644
index dbb4e45d34..0000000000
--- 
a/core/src/test/java/org/apache/accumulo/core/metadata/MetadataServicerTest.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * 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
- *
- *   https://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.accumulo.core.metadata;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertNotEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.util.HashMap;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.data.TableId;
-import org.easymock.EasyMock;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.Test;
-
-public class MetadataServicerTest {
-
-  private static final String userTableName = "tableName";
-  private static final TableId userTableId = TableId.of("tableId");
-  private static ClientContext context;
-
-  @BeforeAll
-  public static void setupContext() {
-    HashMap<String,String> tableNameToIdMap = new HashMap<>();
-    tableNameToIdMap.put(AccumuloTable.ROOT.tableName(), 
AccumuloTable.ROOT.tableId().canonical());
-    tableNameToIdMap.put(AccumuloTable.METADATA.tableName(),
-        AccumuloTable.METADATA.tableId().canonical());
-    tableNameToIdMap.put(userTableName, userTableId.canonical());
-
-    context = EasyMock.createMock(ClientContext.class);
-    TableOperations tableOps = EasyMock.createMock(TableOperations.class);
-    
EasyMock.expect(tableOps.tableIdMap()).andReturn(tableNameToIdMap).anyTimes();
-    EasyMock.expect(context.tableOperations()).andReturn(tableOps).anyTimes();
-    EasyMock.replay(context, tableOps);
-  }
-
-  @Test
-  public void checkSystemTableIdentifiers() {
-    assertNotEquals(AccumuloTable.ROOT.tableId(), 
AccumuloTable.METADATA.tableId());
-    assertNotEquals(AccumuloTable.ROOT.tableName(), 
AccumuloTable.METADATA.tableName());
-  }
-
-  @Test
-  public void testGetCorrectServicer() throws AccumuloException, 
AccumuloSecurityException {
-    MetadataServicer ms = MetadataServicer.forTableId(context, 
AccumuloTable.ROOT.tableId());
-    assertTrue(ms instanceof ServicerForRootTable);
-    assertFalse(ms instanceof TableMetadataServicer);
-    assertEquals(AccumuloTable.ROOT.tableId(), ms.getServicedTableId());
-
-    ms = MetadataServicer.forTableId(context, 
AccumuloTable.METADATA.tableId());
-    assertTrue(ms instanceof ServicerForMetadataTable);
-    assertTrue(ms instanceof TableMetadataServicer);
-    assertEquals(AccumuloTable.ROOT.tableName(),
-        ((TableMetadataServicer) ms).getServicingTableName());
-    assertEquals(AccumuloTable.METADATA.tableId(), ms.getServicedTableId());
-
-    ms = MetadataServicer.forTableId(context, userTableId);
-    assertTrue(ms instanceof ServicerForUserTables);
-    assertTrue(ms instanceof TableMetadataServicer);
-    assertEquals(AccumuloTable.METADATA.tableName(),
-        ((TableMetadataServicer) ms).getServicingTableName());
-    assertEquals(userTableId, ms.getServicedTableId());
-
-    ms = MetadataServicer.forTableName(context, 
AccumuloTable.ROOT.tableName());
-    assertTrue(ms instanceof ServicerForRootTable);
-    assertFalse(ms instanceof TableMetadataServicer);
-    assertEquals(AccumuloTable.ROOT.tableId(), ms.getServicedTableId());
-
-    ms = MetadataServicer.forTableName(context, 
AccumuloTable.METADATA.tableName());
-    assertTrue(ms instanceof ServicerForMetadataTable);
-    assertTrue(ms instanceof TableMetadataServicer);
-    assertEquals(AccumuloTable.ROOT.tableName(),
-        ((TableMetadataServicer) ms).getServicingTableName());
-    assertEquals(AccumuloTable.METADATA.tableId(), ms.getServicedTableId());
-
-    ms = MetadataServicer.forTableName(context, userTableName);
-    assertTrue(ms instanceof ServicerForUserTables);
-    assertTrue(ms instanceof TableMetadataServicer);
-    assertEquals(AccumuloTable.METADATA.tableName(),
-        ((TableMetadataServicer) ms).getServicingTableName());
-    assertEquals(userTableId, ms.getServicedTableId());
-  }
-}
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
 
b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
index 271f86fe93..3f2ffb52e6 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
@@ -107,21 +107,6 @@ public class ClientServiceHandler implements 
ClientService.Iface {
     return namespaceId;
   }
 
-  @Override
-  public String getInstanceId() {
-    return context.getInstanceID().canonical();
-  }
-
-  @Override
-  public String getRootTabletLocation() {
-    return context.getRootTabletLocation();
-  }
-
-  @Override
-  public String getZooKeepers() {
-    return context.getZooKeepers();
-  }
-
   @Override
   public void ping(TCredentials credentials) {
     // anybody can call this; no authentication check
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
 
b/server/base/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
index a83f746bec..98c803b883 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
@@ -31,9 +31,6 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.clientImpl.thrift.TInfo;
 import org.apache.accumulo.core.data.Range;
@@ -45,7 +42,7 @@ import 
org.apache.accumulo.core.dataImpl.thrift.MultiScanResult;
 import org.apache.accumulo.core.dataImpl.thrift.TColumn;
 import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
 import org.apache.accumulo.core.dataImpl.thrift.TRange;
-import org.apache.accumulo.core.metadata.MetadataServicer;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
 import org.apache.accumulo.core.security.Authorizations;
@@ -81,8 +78,7 @@ public class VerifyTabletAssignments {
   }
 
   private static void checkTable(final ClientContext context, final boolean 
verbose,
-      String tableName, HashSet<KeyExtent> check) throws AccumuloException,
-      AccumuloSecurityException, TableNotFoundException, InterruptedException {
+      String tableName, HashSet<KeyExtent> check) throws InterruptedException {
 
     if (check == null) {
       System.out.println("Checking table " + tableName);
@@ -90,31 +86,31 @@ public class VerifyTabletAssignments {
       System.out.println("Checking table " + tableName + " again, failures " + 
check.size());
     }
 
-    TreeMap<KeyExtent,String> tabletLocations = new TreeMap<>();
-
     TableId tableId = context.getTableNameToIdMap().get(tableName);
-    MetadataServicer.forTableId(context, 
tableId).getTabletLocations(tabletLocations);
 
     final HashSet<KeyExtent> failures = new HashSet<>();
 
     Map<HostAndPort,List<KeyExtent>> extentsPerServer = new TreeMap<>(new 
HostAndPortComparator());
 
-    for (Entry<KeyExtent,String> entry : tabletLocations.entrySet()) {
-      KeyExtent keyExtent = entry.getKey();
-      String loc = entry.getValue();
-      if (loc == null) {
-        System.out.println(" Tablet " + keyExtent + " has no location");
-      } else if (verbose) {
-        System.out.println(" Tablet " + keyExtent + " is located at " + loc);
-      }
+    try (var tabletsMeta = context.getAmple().readTablets().forTable(tableId)
+        .fetch(TabletMetadata.ColumnType.LOCATION).checkConsistency().build()) 
{
+      for (var tabletMeta : tabletsMeta) {
+        var loc = tabletMeta.getLocation();
+        var keyExtent = tabletMeta.getExtent();
+        if (loc == null || loc.getType() != 
TabletMetadata.LocationType.CURRENT) {
+          System.out.println(" Tablet " + keyExtent + " has no location");
+        } else if (verbose) {
+          System.out.println(" Tablet " + keyExtent + " is located at " + loc);
+        }
 
-      if (loc != null) {
-        final HostAndPort parsedLoc = HostAndPort.fromString(loc);
-        List<KeyExtent> extentList =
-            extentsPerServer.computeIfAbsent(parsedLoc, k -> new 
ArrayList<>());
+        if (loc != null) {
+          final HostAndPort parsedLoc = loc.getHostAndPort();
+          List<KeyExtent> extentList =
+              extentsPerServer.computeIfAbsent(parsedLoc, k -> new 
ArrayList<>());
 
-        if (check == null || check.contains(keyExtent)) {
-          extentList.add(keyExtent);
+          if (check == null || check.contains(keyExtent)) {
+            extentList.add(keyExtent);
+          }
         }
       }
     }
diff --git 
a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tables/TablesResource.java
 
b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tables/TablesResource.java
index c6161e0119..8946bdd728 100644
--- 
a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tables/TablesResource.java
+++ 
b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tables/TablesResource.java
@@ -42,6 +42,7 @@ import 
org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo;
 import org.apache.accumulo.core.manager.thrift.TableInfo;
 import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
 import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.metadata.schema.Ample;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
@@ -125,7 +126,6 @@ public class TablesResource {
   @GET
   public TabletServers getParticipatingTabletServers(@PathParam("tableId") 
@NotNull @Pattern(
       regexp = ALPHA_NUM_REGEX_TABLE_ID) String tableIdStr) {
-    String rootTabletLocation = monitor.getContext().getRootTabletLocation();
     TableId tableId = TableId.of(tableIdStr);
     ManagerMonitorInfo mmi = monitor.getMmi();
     // fail fast if unable to get monitor info
@@ -141,7 +141,10 @@ public class TablesResource {
 
     TreeSet<String> locs = new TreeSet<>();
     if (AccumuloTable.ROOT.tableId().equals(tableId)) {
-      locs.add(rootTabletLocation);
+      var rootLoc = 
monitor.getContext().getAmple().readTablet(RootTable.EXTENT).getLocation();
+      if (rootLoc != null && rootLoc.getType() == 
TabletMetadata.LocationType.CURRENT) {
+        locs.add(rootLoc.getHostPort());
+      }
     } else {
       var level = Ample.DataLevel.of(tableId);
       try (TabletsMetadata tablets = 
monitor.getContext().getAmple().readTablets().forLevel(level)
diff --git 
a/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
 
b/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
index 4655b06ce9..80e2cc04b6 100644
--- 
a/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
+++ 
b/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
@@ -63,9 +63,9 @@ import 
org.apache.accumulo.core.iteratorsImpl.system.DeletingIterator.Behavior;
 import org.apache.accumulo.core.iteratorsImpl.system.MultiIterator;
 import org.apache.accumulo.core.iteratorsImpl.system.SortedMapIterator;
 import org.apache.accumulo.core.iteratorsImpl.system.VisibilityFilter;
-import org.apache.accumulo.core.metadata.MetadataServicer;
 import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.TabletFile;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.spi.crypto.NoCryptoServiceFactory;
 import org.apache.accumulo.core.util.Stat;
@@ -82,7 +82,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.beust.jcommander.Parameter;
-import com.google.common.net.HostAndPort;
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
@@ -356,22 +355,25 @@ public class CollectTabletStats {
       String tableName, SortedMap<KeyExtent,String> tabletLocations) throws 
Exception {
 
     TableId tableId = context.getTableId(tableName);
-    MetadataServicer.forTableId(context, 
tableId).getTabletLocations(tabletLocations);
 
     InetAddress localaddress = InetAddress.getLocalHost();
 
     List<KeyExtent> candidates = new ArrayList<>();
 
-    for (Entry<KeyExtent,String> entry : tabletLocations.entrySet()) {
-      String loc = entry.getValue();
-      if (loc != null) {
-        boolean isLocal =
-            
HostAndPort.fromString(entry.getValue()).getHost().equals(localaddress.getHostName());
-
-        if (selectLocalTablets && isLocal) {
-          candidates.add(entry.getKey());
-        } else if (!selectLocalTablets && !isLocal) {
-          candidates.add(entry.getKey());
+    try (var tabletsMeta = context.getAmple().readTablets().forTable(tableId)
+        .fetch(TabletMetadata.ColumnType.LOCATION).checkConsistency().build()) 
{
+      for (var tabletMeta : tabletsMeta) {
+        var loc = tabletMeta.getLocation();
+        if (loc != null && loc.getType() == 
TabletMetadata.LocationType.CURRENT) {
+          boolean isLocal = loc.getHost().equals(localaddress.getHostName());
+
+          if (selectLocalTablets && isLocal) {
+            candidates.add(tabletMeta.getExtent());
+            tabletLocations.put(tabletMeta.getExtent(), loc.getHostPort());
+          } else if (!selectLocalTablets && !isLocal) {
+            candidates.add(tabletMeta.getExtent());
+            tabletLocations.put(tabletMeta.getExtent(), loc.getHostPort());
+          }
         }
       }
     }


Reply via email to