diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java index f58dfba405a8..64dcb24110c3 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java @@ -32,6 +32,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; @@ -83,7 +84,7 @@ class AsyncConnectionImpl implements AsyncConnection { private final User user; - final AsyncRegistry registry; + final ConnectionRegistry registry; private final int rpcTimeout; @@ -118,7 +119,7 @@ class AsyncConnectionImpl implements AsyncConnection { private final ClusterStatusListener clusterStatusListener; - public AsyncConnectionImpl(Configuration conf, AsyncRegistry registry, String clusterId, + public AsyncConnectionImpl(Configuration conf, ConnectionRegistry registry, String clusterId, User user) { this.conf = conf; this.user = user; @@ -248,7 +249,7 @@ AdminService.Interface getAdminStub(ServerName serverName) throws IOException { CompletableFuture getMasterStub() { return ConnectionUtils.getOrFetch(masterStub, masterStubMakeFuture, false, () -> { CompletableFuture future = new CompletableFuture<>(); - addListener(registry.getMasterAddress(), (addr, error) -> { + addListener(registry.getActiveMaster(), (addr, error) -> { if (error != null) { future.completeExceptionally(error); } else if (addr == null) { @@ -268,6 +269,15 @@ CompletableFuture getMasterStub() { }, stub -> true, "master stub"); } + String getClusterId() { + try { + return registry.getClusterId().get(); + } catch (InterruptedException | ExecutionException e) { + LOG.error("Error fetching cluster ID: ", e); + } + return null; + } + void clearMasterStubCache(MasterService.Interface stub) { masterStub.compareAndSet(stub, null); } @@ -342,7 +352,7 @@ public AsyncBufferedMutatorBuilder getBufferedMutatorBuilder(TableName tableName @Override public CompletableFuture getHbck() { CompletableFuture future = new CompletableFuture<>(); - addListener(registry.getMasterAddress(), (sn, error) -> { + addListener(registry.getActiveMaster(), (sn, error) -> { if (error != null) { future.completeExceptionally(error); } else { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java index 50dab24f2925..9df8efb8a63d 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java @@ -36,14 +36,14 @@ @InterfaceAudience.Private class AsyncMetaRegionLocator { - private final AsyncRegistry registry; + private final ConnectionRegistry registry; private final AtomicReference metaRegionLocations = new AtomicReference<>(); private final AtomicReference> metaRelocateFuture = new AtomicReference<>(); - AsyncMetaRegionLocator(AsyncRegistry registry) { + AsyncMetaRegionLocator(ConnectionRegistry registry) { this.registry = registry; } @@ -58,7 +58,7 @@ class AsyncMetaRegionLocator { */ CompletableFuture getRegionLocations(int replicaId, boolean reload) { return ConnectionUtils.getOrFetch(metaRegionLocations, metaRelocateFuture, reload, - registry::getMetaRegionLocation, locs -> isGood(locs, replicaId), "meta region location"); + registry::getMetaRegionLocations, locs -> isGood(locs, replicaId), "meta region location"); } private HRegionLocation getCacheLocation(HRegionLocation loc) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java index 0e116ab6f3d7..fa3ea1ca4dfa 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java @@ -55,7 +55,7 @@ public CompletableFuture getRegionLocation(byte[] row, int repl @Override public CompletableFuture> getAllRegionLocations() { if (TableName.isMetaTableName(tableName)) { - return conn.registry.getMetaRegionLocation() + return conn.registry.getMetaRegionLocations() .thenApply(locs -> Arrays.asList(locs.getRegionLocations())); } return AsyncMetaTableAccessor.getTableHRegionLocations(conn.getTable(TableName.META_TABLE_NAME), diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java index 90891f411baf..ad86048ab531 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java @@ -193,6 +193,11 @@ default Table getTable(TableName tableName, ExecutorService pool) throws IOExcep */ TableBuilder getTableBuilder(TableName tableName, ExecutorService pool); + /** + * @return the cluster ID unique to this HBase cluster. + */ + String getClusterId(); + /** * Retrieve an Hbck implementation to fix an HBase cluster. * The returned Hbck is not guaranteed to be thread-safe. A new instance should be created by diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java index b36485fdf011..f91b2107c6b0 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java @@ -281,7 +281,7 @@ public static CompletableFuture createAsyncConnection(Configura public static CompletableFuture createAsyncConnection(Configuration conf, final User user) { CompletableFuture future = new CompletableFuture<>(); - AsyncRegistry registry = AsyncRegistryFactory.getRegistry(conf); + ConnectionRegistry registry = ConnectionRegistryFactory.getRegistry(conf); addListener(registry.getClusterId(), (clusterId, error) -> { if (error != null) { registry.close(); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java index 7bdda26e05be..0b79b4f5f3d2 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java @@ -217,7 +217,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable { /** * Cluster registry of basic info such as clusterid and meta region location. */ - private final AsyncRegistry registry; + private final ConnectionRegistry registry; private final ClientBackoffPolicy backoffPolicy; @@ -303,7 +303,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable { this.conf.get(BufferedMutator.CLASSNAME_KEY); try { - this.registry = AsyncRegistryFactory.getRegistry(conf); + this.registry = ConnectionRegistryFactory.getRegistry(conf); retrieveClusterId(); this.rpcClient = RpcClientFactory.createClient(this.conf, this.clusterId, this.metrics); @@ -378,6 +378,16 @@ public Table build() { }; } + @Override + public String getClusterId() { + try { + return registry.getClusterId().get(); + } catch (Exception e) { + LOG.error("Error fetching cluster ID: ", e); + } + return null; + } + @Override public BufferedMutator getBufferedMutator(BufferedMutatorParams params) { if (params.getTableName() == null) { @@ -434,7 +444,7 @@ public Admin getAdmin() throws IOException { @Override public Hbck getHbck() throws IOException { - return getHbck(get(registry.getMasterAddress())); + return getHbck(get(registry.getActiveMaster())); } @Override @@ -811,7 +821,7 @@ private RegionLocations locateMeta(final TableName tableName, } // Look up from zookeeper - locations = get(this.registry.getMetaRegionLocation()); + locations = get(this.registry.getMetaRegionLocations()); if (locations != null) { cacheLocation(tableName, locations); } @@ -1162,7 +1172,7 @@ private void isMasterRunning(MasterProtos.MasterService.BlockingInterface stub) */ private MasterProtos.MasterService.BlockingInterface makeStubNoRetries() throws IOException, KeeperException { - ServerName sn = get(registry.getMasterAddress()); + ServerName sn = get(registry.getActiveMaster()); if (sn == null) { String msg = "ZooKeeper available but no active master location found"; LOG.info(msg); @@ -1211,7 +1221,7 @@ MasterProtos.MasterService.BlockingInterface makeStub() throws IOException { @Override public AdminProtos.AdminService.BlockingInterface getAdminForMaster() throws IOException { - return getAdmin(get(registry.getMasterAddress())); + return getAdmin(get(registry.getActiveMaster())); } @Override diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistry.java similarity index 77% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegistry.java rename to hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistry.java index 9537777db1ad..cd22d7861b4c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegistry.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistry.java @@ -24,16 +24,17 @@ import org.apache.yetus.audience.InterfaceAudience; /** - * Implementations hold cluster information such as this cluster's id, location of hbase:meta, etc.. + * Registry for meta information needed for connection setup to a HBase cluster. Implementations + * hold cluster information such as this cluster's id, location of hbase:meta, etc.. * Internal use only. */ @InterfaceAudience.Private -interface AsyncRegistry extends Closeable { +interface ConnectionRegistry extends Closeable { /** - * Get the location of meta region. + * Get the location of meta region(s). */ - CompletableFuture getMetaRegionLocation(); + CompletableFuture getMetaRegionLocations(); /** * Should only be called once. @@ -43,9 +44,9 @@ interface AsyncRegistry extends Closeable { CompletableFuture getClusterId(); /** - * Get the address of HMaster. + * Get the address of active HMaster. */ - CompletableFuture getMasterAddress(); + CompletableFuture getActiveMaster(); /** * Closes this instance and releases any system resources associated with it diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegistryFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java similarity index 67% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegistryFactory.java rename to hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java index 28726ae5dd20..93084437a427 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegistryFactory.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -17,27 +17,27 @@ */ package org.apache.hadoop.hbase.client; +import static org.apache.hadoop.hbase.HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY; import org.apache.hadoop.conf.Configuration; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.util.ReflectionUtils; +import org.apache.yetus.audience.InterfaceAudience; /** - * Get instance of configured Registry. + * Factory class to get the instance of configured connection registry. */ @InterfaceAudience.Private -final class AsyncRegistryFactory { - - static final String REGISTRY_IMPL_CONF_KEY = "hbase.client.registry.impl"; +final class ConnectionRegistryFactory { - private AsyncRegistryFactory() { + private ConnectionRegistryFactory() { } /** - * @return The cluster registry implementation to use. + * @return The connection registry implementation to use. */ - static AsyncRegistry getRegistry(Configuration conf) { - Class clazz = - conf.getClass(REGISTRY_IMPL_CONF_KEY, ZKAsyncRegistry.class, AsyncRegistry.class); + static ConnectionRegistry getRegistry(Configuration conf) { + Class clazz = conf.getClass( + CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, ZKConnectionRegistry.class, + ConnectionRegistry.class); return ReflectionUtils.newInstance(clazz, conf); } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java new file mode 100644 index 000000000000..5680847ec37a --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java @@ -0,0 +1,226 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.client; + +import static org.apache.hadoop.hbase.HConstants.MASTER_ADDRS_DEFAULT; +import static org.apache.hadoop.hbase.HConstants.MASTER_ADDRS_KEY; +import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT; +import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.function.Function; +import java.util.function.Predicate; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.RegionLocations; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.exceptions.MasterRegistryFetchException; +import org.apache.hadoop.hbase.ipc.HBaseRpcController; +import org.apache.hadoop.hbase.ipc.RpcClient; +import org.apache.hadoop.hbase.ipc.RpcClientFactory; +import org.apache.hadoop.hbase.ipc.RpcControllerFactory; +import org.apache.hadoop.hbase.security.User; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; +import org.apache.hbase.thirdparty.com.google.common.net.HostAndPort; +import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClientMetaService; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetActiveMasterRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetActiveMasterResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterIdRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterIdResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetMetaRegionLocationsRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetMetaRegionLocationsResponse; + +/** + * Master based registry implementation. Makes RPCs to the configured master addresses from config + * {@value org.apache.hadoop.hbase.HConstants#MASTER_ADDRS_KEY}. + * + * It supports hedged reads, which can be enabled by setting + * {@value org.apache.hadoop.hbase.HConstants#MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY} to True. Fan + * out the requests batch is controlled by + * {@value org.apache.hadoop.hbase.HConstants#HBASE_RPCS_HEDGED_REQS_FANOUT_KEY}. + * + * TODO: Handle changes to the configuration dynamically without having to restart the client. + */ +@InterfaceAudience.Private +public class MasterRegistry implements ConnectionRegistry { + private static final String MASTER_ADDRS_CONF_SEPARATOR = ","; + + // Configured list of masters to probe the meta information from. + private final Set masterServers; + + // RPC client used to talk to the masters. + private final RpcClient rpcClient; + private final RpcControllerFactory rpcControllerFactory; + private final int rpcTimeoutMs; + + MasterRegistry(Configuration conf) { + boolean hedgedReadsEnabled = conf.getBoolean(MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY, + MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT); + Configuration finalConf; + if (!hedgedReadsEnabled) { + // If hedged reads are disabled, it is equivalent to setting a fan out of 1. We make a copy of + // the configuration so that other places reusing this reference is not affected. + finalConf = new Configuration(conf); + finalConf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, 1); + } else { + finalConf = conf; + } + rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE, conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, + HConstants.DEFAULT_HBASE_RPC_TIMEOUT)); + masterServers = new HashSet<>(); + parseMasterAddrs(finalConf); + rpcClient = RpcClientFactory.createClient(finalConf, HConstants.CLUSTER_ID_DEFAULT); + rpcControllerFactory = RpcControllerFactory.instantiate(finalConf); + } + + /** + * @return Stub needed to make RPC using a hedged channel to the master end points. + */ + private ClientMetaService.Interface getMasterStub() throws IOException { + return ClientMetaService.newStub( + rpcClient.createHedgedRpcChannel(masterServers, User.getCurrent(), rpcTimeoutMs)); + } + + /** + * Parses the list of master addresses from the provided configuration. Supported format is + * comma separated host[:port] values. If no port number if specified, default master port is + * assumed. + * @param conf Configuration to parse from. + */ + private void parseMasterAddrs(Configuration conf) { + String configuredMasters = conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT); + for (String masterAddr: configuredMasters.split(MASTER_ADDRS_CONF_SEPARATOR)) { + HostAndPort masterHostPort = + HostAndPort.fromString(masterAddr.trim()).withDefaultPort(HConstants.DEFAULT_MASTER_PORT); + masterServers.add(ServerName.valueOf(masterHostPort.toString(), ServerName.NON_STARTCODE)); + } + Preconditions.checkArgument(!masterServers.isEmpty(), "At least one master address is needed"); + } + + @VisibleForTesting + public Set getParsedMasterServers() { + return Collections.unmodifiableSet(masterServers); + } + + /** + * Returns a call back that can be passed along to the non-blocking rpc call. It is invoked once + * the rpc finishes and the response is propagated to the passed future. + * @param future Result future to which the rpc response is propagated. + * @param isValidResp Checks if the rpc response has a valid result. + * @param transformResult Transforms the result to a different form as expected by callers. + * @param hrc RpcController instance for this rpc. + * @param debug Debug message passed along to the caller in case of exceptions. + * @param RPC result type. + * @param Transformed type of the result. + * @return A call back that can be embedded in the non-blocking rpc call. + */ + private RpcCallback getRpcCallBack(CompletableFuture future, + Predicate isValidResp, Function transformResult, HBaseRpcController hrc, + final String debug) { + return rpcResult -> { + if (rpcResult == null) { + future.completeExceptionally( + new MasterRegistryFetchException(masterServers, hrc.getFailed())); + } + if (!isValidResp.test(rpcResult)) { + // Rpc returned ok, but result was malformed. + future.completeExceptionally(new IOException( + String.format("Invalid result for request %s. Will be retried", debug))); + + } + future.complete(transformResult.apply(rpcResult)); + }; + } + + /** + * Simple helper to transform the result of getMetaRegionLocations() rpc. + */ + private RegionLocations transformMetaRegionLocations(GetMetaRegionLocationsResponse resp) { + List regionLocations = new ArrayList<>(); + resp.getMetaLocationsList().forEach( + location -> regionLocations.add(ProtobufUtil.toRegionLocation(location))); + return new RegionLocations(regionLocations); + } + + @Override + public CompletableFuture getMetaRegionLocations() { + CompletableFuture result = new CompletableFuture<>(); + HBaseRpcController hrc = rpcControllerFactory.newController(); + RpcCallback callback = getRpcCallBack(result, + (rpcResp) -> rpcResp.getMetaLocationsCount() != 0, this::transformMetaRegionLocations, hrc, + "getMetaRegionLocations()"); + try { + getMasterStub().getMetaRegionLocations( + hrc, GetMetaRegionLocationsRequest.getDefaultInstance(), callback); + } catch (IOException e) { + result.completeExceptionally(e); + } + return result; + } + + @Override + public CompletableFuture getClusterId() { + CompletableFuture result = new CompletableFuture<>(); + HBaseRpcController hrc = rpcControllerFactory.newController(); + RpcCallback callback = getRpcCallBack(result, + GetClusterIdResponse::hasClusterId, GetClusterIdResponse::getClusterId, hrc, + "getClusterId()"); + try { + getMasterStub().getClusterId(hrc, GetClusterIdRequest.getDefaultInstance(), callback); + } catch (IOException e) { + result.completeExceptionally(e); + } + return result; + } + + private ServerName transformServerName(GetActiveMasterResponse resp) { + return ProtobufUtil.toServerName(resp.getServerName()); + } + + @Override + public CompletableFuture getActiveMaster() { + CompletableFuture result = new CompletableFuture<>(); + HBaseRpcController hrc = rpcControllerFactory.newController(); + RpcCallback callback = getRpcCallBack(result, + GetActiveMasterResponse::hasServerName, this::transformServerName, hrc, + "getActiveMaster()"); + try { + getMasterStub().getActiveMaster(hrc, GetActiveMasterRequest.getDefaultInstance(), callback); + } catch (IOException e) { + result.completeExceptionally(e); + } + return result; + } + + @Override + public void close() { + if (rpcClient != null) { + rpcClient.close(); + } + } +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java index e48438390832..ee32e429eea8 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -21,7 +21,6 @@ import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME; import static org.apache.hadoop.hbase.util.FutureUtils.addListener; import static org.apache.hadoop.hbase.util.FutureUtils.unwrapCompletionException; - import com.google.protobuf.Message; import com.google.protobuf.RpcChannel; import java.io.IOException; @@ -46,7 +45,6 @@ import java.util.regex.Pattern; import java.util.stream.Collectors; import java.util.stream.Stream; -import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.AsyncMetaTableAccessor; import org.apache.hadoop.hbase.CacheEvictionStats; @@ -98,14 +96,12 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback; import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer; import org.apache.hbase.thirdparty.io.netty.util.Timeout; import org.apache.hbase.thirdparty.io.netty.util.TimerTask; - import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos; @@ -722,7 +718,7 @@ public CompletableFuture isTableAvailable(TableName tableName, byte[][] private CompletableFuture isTableAvailable(TableName tableName, Optional splitKeys) { if (TableName.isMetaTableName(tableName)) { - return connection.registry.getMetaRegionLocation().thenApply(locs -> Stream + return connection.registry.getMetaRegionLocations().thenApply(locs -> Stream .of(locs.getRegionLocations()).allMatch(loc -> loc != null && loc.getServerName() != null)); } CompletableFuture future = new CompletableFuture<>(); @@ -784,7 +780,8 @@ private boolean compareRegionsWithSplitKeys(List locations, byt } @Override - public CompletableFuture addColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily) { + public CompletableFuture addColumnFamily( + TableName tableName, ColumnFamilyDescriptor columnFamily) { return this. procedureCall(tableName, RequestConverter.buildAddColumnRequest(tableName, columnFamily, ng.getNonceGroup(), ng.newNonce()), (s, c, req, done) -> s.addColumn(c, req, done), (resp) -> resp.getProcId(), @@ -838,10 +835,10 @@ public CompletableFuture getNamespaceDescriptor(String name . newMasterCaller() .action( (controller, stub) -> this - . call( - controller, stub, RequestConverter.buildGetNamespaceDescriptorRequest(name), (s, c, - req, done) -> s.getNamespaceDescriptor(c, req, done), (resp) -> ProtobufUtil - .toNamespaceDescriptor(resp.getNamespaceDescriptor()))).call(); + . + call(controller, stub, RequestConverter.buildGetNamespaceDescriptorRequest(name), + (s, c, req, done) -> s.getNamespaceDescriptor(c, req, done), (resp) + -> ProtobufUtil.toNamespaceDescriptor(resp.getNamespaceDescriptor()))).call(); } @Override @@ -859,13 +856,12 @@ public CompletableFuture> listNamespaces() { @Override public CompletableFuture> listNamespaceDescriptors() { return this - .> newMasterCaller() - .action( - (controller, stub) -> this - .> call( - controller, stub, ListNamespaceDescriptorsRequest.newBuilder().build(), (s, c, req, - done) -> s.listNamespaceDescriptors(c, req, done), (resp) -> ProtobufUtil - .toNamespaceDescriptorList(resp))).call(); + .> newMasterCaller().action((controller, stub) -> this + .> call(controller, stub, + ListNamespaceDescriptorsRequest.newBuilder().build(), (s, c, req, done) -> + s.listNamespaceDescriptors(c, req, done), + (resp) -> ProtobufUtil.toNamespaceDescriptorList(resp))).call(); } @Override @@ -882,7 +878,7 @@ public CompletableFuture> getRegions(ServerName serverName) { @Override public CompletableFuture> getRegions(TableName tableName) { if (tableName.equals(META_TABLE_NAME)) { - return connection.registry.getMetaRegionLocation() + return connection.registry.getMetaRegionLocations() .thenApply(locs -> Stream.of(locs.getRegionLocations()).map(HRegionLocation::getRegion) .collect(Collectors.toList())); } else { @@ -1097,9 +1093,7 @@ private CompletableFuture compactRegion(byte[] regionName, byte[] columnFa private CompletableFuture> getTableHRegionLocations(TableName tableName) { if (TableName.META_TABLE_NAME.equals(tableName)) { CompletableFuture> future = new CompletableFuture<>(); - // For meta table, we use zk to fetch all locations. - AsyncRegistry registry = AsyncRegistryFactory.getRegistry(connection.getConfiguration()); - addListener(registry.getMetaRegionLocation(), (metaRegions, err) -> { + addListener(connection.registry.getMetaRegionLocations(), (metaRegions, err) -> { if (err != null) { future.completeExceptionally(err); } else if (metaRegions == null || metaRegions.isEmpty() || @@ -1108,8 +1102,6 @@ private CompletableFuture> getTableHRegionLocations(TableN } else { future.complete(Collections.singletonList(metaRegions.getDefaultRegionLocation())); } - // close the registry. - IOUtils.closeQuietly(registry); }); return future; } else { @@ -1127,7 +1119,7 @@ private CompletableFuture compact(TableName tableName, byte[] columnFamily switch (compactType) { case MOB: - addListener(connection.registry.getMasterAddress(), (serverName, err) -> { + addListener(connection.registry.getActiveMaster(), (serverName, err) -> { if (err != null) { future.completeExceptionally(err); return; @@ -1792,11 +1784,8 @@ private CompletableFuture> listReplicationPeers return this .> newMasterCaller() .action( - (controller, stub) -> this - .> call( - controller, - stub, - request, + (controller, stub) -> this.> call(controller, stub, request, (s, c, req, done) -> s.listReplicationPeers(c, req, done), (resp) -> resp.getPeerDescList().stream() .map(ReplicationPeerConfigUtil::toReplicationPeerDescription) @@ -2307,11 +2296,13 @@ public CompletableFuture getLocks() { } @Override - public CompletableFuture decommissionRegionServers(List servers, boolean offload) { + public CompletableFuture decommissionRegionServers( + List servers, boolean offload) { return this. newMasterCaller() .action((controller, stub) -> this . call( - controller, stub, RequestConverter.buildDecommissionRegionServersRequest(servers, offload), + controller, stub, + RequestConverter.buildDecommissionRegionServersRequest(servers, offload), (s, c, req, done) -> s.decommissionRegionServers(c, req, done), resp -> null)) .call(); } @@ -2333,11 +2324,11 @@ List> call( public CompletableFuture recommissionRegionServer(ServerName server, List encodedRegionNames) { return this. newMasterCaller() - .action((controller, stub) -> this - . call(controller, - stub, RequestConverter.buildRecommissionRegionServerRequest(server, encodedRegionNames), - (s, c, req, done) -> s.recommissionRegionServer(c, req, done), resp -> null)) - .call(); + .action((controller, stub) -> + this. call( + controller, stub, RequestConverter.buildRecommissionRegionServerRequest( + server, encodedRegionNames), (s, c, req, done) -> s.recommissionRegionServer( + c, req, done), resp -> null)).call(); } /** @@ -2358,7 +2349,7 @@ CompletableFuture getRegionLocation(byte[] regionNameOrEncodedR String encodedName = Bytes.toString(regionNameOrEncodedRegionName); if (encodedName.length() < RegionInfo.MD5_HEX_LENGTH) { // old format encodedName, should be meta region - future = connection.registry.getMetaRegionLocation() + future = connection.registry.getMetaRegionLocations() .thenApply(locs -> Stream.of(locs.getRegionLocations()) .filter(loc -> loc.getRegion().getEncodedName().equals(encodedName)).findFirst()); } else { @@ -2369,7 +2360,7 @@ CompletableFuture getRegionLocation(byte[] regionNameOrEncodedR RegionInfo regionInfo = MetaTableAccessor.parseRegionInfoFromRegionName(regionNameOrEncodedRegionName); if (regionInfo.isMetaRegion()) { - future = connection.registry.getMetaRegionLocation() + future = connection.registry.getMetaRegionLocations() .thenApply(locs -> Stream.of(locs.getRegionLocations()) .filter(loc -> loc.getRegion().getReplicaId() == regionInfo.getReplicaId()) .findFirst()); @@ -2403,7 +2394,6 @@ CompletableFuture getRegionLocation(byte[] regionNameOrEncodedR * Get the region info for the passed region name. The region name may be a full region name or * encoded region name. If the region does not found, then it'll throw an UnknownRegionException * wrapped by a {@link CompletableFuture} - * @param regionNameOrEncodedRegionName * @return region info, wrapped by a {@link CompletableFuture} */ private CompletableFuture getRegionInfo(byte[] regionNameOrEncodedRegionName) { @@ -2894,10 +2884,11 @@ public CompletableFuture> getSecurityCapabilities() { .> newMasterCaller() .action( (controller, stub) -> this - .> call( - controller, stub, SecurityCapabilitiesRequest.newBuilder().build(), (s, c, req, - done) -> s.getSecurityCapabilities(c, req, done), (resp) -> ProtobufUtil - .toSecurityCapabilityList(resp.getCapabilitiesList()))).call(); + .> + call(controller, stub, SecurityCapabilitiesRequest.newBuilder().build(), + (s, c, req, done) -> s.getSecurityCapabilities(c, req, done), + (resp) -> ProtobufUtil.toSecurityCapabilityList(resp.getCapabilitiesList()))) + .call(); } @Override @@ -2942,7 +2933,7 @@ public CompletableFuture getCompactionState(TableName tableName switch (compactType) { case MOB: - addListener(connection.registry.getMasterAddress(), (serverName, err) -> { + addListener(connection.registry.getActiveMaster(), (serverName, err) -> { if (err != null) { future.completeExceptionally(err); return; @@ -3074,14 +3065,10 @@ public CompletableFuture> getLastMajorCompactionTimestamp(TableNa MajorCompactionTimestampRequest request = MajorCompactionTimestampRequest.newBuilder() .setTableName(ProtobufUtil.toProtoTableName(tableName)).build(); - return this - .> newMasterCaller() - .action( - (controller, stub) -> this - .> call( - controller, stub, request, - (s, c, req, done) -> s.getLastMajorCompactionTimestamp(c, req, done), - ProtobufUtil::toOptionalTimestamp)).call(); + return this.> newMasterCaller().action((controller, stub) -> + this.> + call(controller, stub, request, (s, c, req, done) -> s.getLastMajorCompactionTimestamp( + c, req, done), ProtobufUtil::toOptionalTimestamp)).call(); } @Override @@ -3221,11 +3208,10 @@ public CompletableFuture balance(boolean forcible) { public CompletableFuture isBalancerEnabled() { return this . newMasterCaller() - .action( - (controller, stub) -> this. call( - controller, stub, RequestConverter.buildIsBalancerEnabledRequest(), - (s, c, req, done) -> s.isBalancerEnabled(c, req, done), (resp) -> resp.getEnabled())) - .call(); + .action((controller, stub) -> + this. call(controller, + stub, RequestConverter.buildIsBalancerEnabledRequest(), (s, c, req, done) + -> s.isBalancerEnabled(c, req, done), (resp) -> resp.getEnabled())).call(); } @Override diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java similarity index 92% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java rename to hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java index b6bacc03e60b..42a418859f18 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java @@ -50,15 +50,15 @@ * Zookeeper based registry implementation. */ @InterfaceAudience.Private -class ZKAsyncRegistry implements AsyncRegistry { +class ZKConnectionRegistry implements ConnectionRegistry { - private static final Logger LOG = LoggerFactory.getLogger(ZKAsyncRegistry.class); + private static final Logger LOG = LoggerFactory.getLogger(ZKConnectionRegistry.class); private final ReadOnlyZKClient zk; private final ZNodePaths znodePaths; - ZKAsyncRegistry(Configuration conf) { + ZKConnectionRegistry(Configuration conf) { this.znodePaths = new ZNodePaths(conf); this.zk = new ReadOnlyZKClient(conf); } @@ -93,7 +93,7 @@ private static String getClusterId(byte[] data) throws DeserializationException @Override public CompletableFuture getClusterId() { - return getAndConvert(znodePaths.clusterIdZNode, ZKAsyncRegistry::getClusterId); + return getAndConvert(znodePaths.clusterIdZNode, ZKConnectionRegistry::getClusterId); } @VisibleForTesting @@ -144,7 +144,7 @@ private void getMetaRegionLocation(CompletableFuture future, int replicaId = znodePaths.getMetaReplicaIdFromZnode(metaReplicaZNode); String path = ZNodePaths.joinZNode(znodePaths.baseZNode, metaReplicaZNode); if (replicaId == DEFAULT_REPLICA_ID) { - addListener(getAndConvert(path, ZKAsyncRegistry::getMetaProto), (proto, error) -> { + addListener(getAndConvert(path, ZKConnectionRegistry::getMetaProto), (proto, error) -> { if (error != null) { future.completeExceptionally(error); return; @@ -162,7 +162,7 @@ private void getMetaRegionLocation(CompletableFuture future, tryComplete(remaining, locs, future); }); } else { - addListener(getAndConvert(path, ZKAsyncRegistry::getMetaProto), (proto, error) -> { + addListener(getAndConvert(path, ZKConnectionRegistry::getMetaProto), (proto, error) -> { if (future.isDone()) { return; } @@ -191,7 +191,7 @@ private void getMetaRegionLocation(CompletableFuture future, } @Override - public CompletableFuture getMetaRegionLocation() { + public CompletableFuture getMetaRegionLocations() { CompletableFuture future = new CompletableFuture<>(); addListener( zk.list(znodePaths.baseZNode) @@ -217,8 +217,8 @@ private static ZooKeeperProtos.Master getMasterProto(byte[] data) throws IOExcep } @Override - public CompletableFuture getMasterAddress() { - return getAndConvert(znodePaths.masterAddressZNode, ZKAsyncRegistry::getMasterProto) + public CompletableFuture getActiveMaster() { + return getAndConvert(znodePaths.masterAddressZNode, ZKConnectionRegistry::getMasterProto) .thenApply(proto -> { if (proto == null) { return null; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/MasterRegistryFetchException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/MasterRegistryFetchException.java new file mode 100644 index 000000000000..18871befef87 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/MasterRegistryFetchException.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.exceptions; + +import java.util.Set; +import org.apache.hadoop.hbase.HBaseIOException; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.util.PrettyPrinter; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Exception thrown when an master registry RPC fails in client. The exception includes the list of + * masters to which RPC was attempted and the last exception encountered. Prior exceptions are + * included in the logs. + */ +@InterfaceAudience.Private +public class MasterRegistryFetchException extends HBaseIOException { + public MasterRegistryFetchException(Set masters, Throwable failure) { + super(String.format("Exception making rpc to masters %s", PrettyPrinter.toString(masters)), + failure); + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java index d4187203377c..b0baff23d512 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java @@ -20,39 +20,20 @@ import static org.apache.hadoop.hbase.ipc.IPCUtil.toIOE; import static org.apache.hadoop.hbase.ipc.IPCUtil.wrapException; - -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; -import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder; -import org.apache.hbase.thirdparty.com.google.common.cache.CacheLoader; -import org.apache.hbase.thirdparty.com.google.common.cache.LoadingCache; -import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel; -import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors; -import org.apache.hbase.thirdparty.com.google.protobuf.Message; -import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel; -import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; -import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; -import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback; - -import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer; - import java.io.IOException; import java.net.InetSocketAddress; import java.net.SocketAddress; import java.net.UnknownHostException; import java.util.Collection; +import java.util.Set; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.hadoop.hbase.client.MetricsConnection; import org.apache.hadoop.hbase.codec.Codec; import org.apache.hadoop.hbase.codec.KeyValueCodec; @@ -63,7 +44,22 @@ import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.ipc.RemoteException; - +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; +import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder; +import org.apache.hbase.thirdparty.com.google.common.cache.CacheLoader; +import org.apache.hbase.thirdparty.com.google.common.cache.LoadingCache; +import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel; +import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors; +import org.apache.hbase.thirdparty.com.google.protobuf.Message; +import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback; +import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel; +import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; +import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; +import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; /** @@ -203,7 +199,9 @@ private void cleanupIdleConnections() { // have some pending calls on connection so we should not shutdown the connection outside. // The connection itself will disconnect if there is no pending call for maxIdleTime. if (conn.getLastTouched() < closeBeforeTime && !conn.isActive()) { - if (LOG.isTraceEnabled()) LOG.trace("Cleanup idle connection to " + conn.remoteId().address); + if (LOG.isTraceEnabled()) { + LOG.trace("Cleanup idle connection to {}", conn.remoteId().address); + } connections.removeValue(conn.remoteId(), conn); conn.cleanupConnection(); } @@ -384,7 +382,7 @@ private void onCallFinished(Call call, HBaseRpcController hrc, InetSocketAddress } } - private void callMethod(final Descriptors.MethodDescriptor md, final HBaseRpcController hrc, + Call callMethod(final Descriptors.MethodDescriptor md, final HBaseRpcController hrc, final Message param, Message returnType, final User ticket, final InetSocketAddress addr, final RpcCallback callback) { final MetricsConnection.CallStats cs = MetricsConnection.newCallStats(); @@ -421,9 +419,10 @@ public void run(Call call) { } catch (Exception e) { call.setException(toIOE(e)); } + return call; } - private InetSocketAddress createAddr(ServerName sn) throws UnknownHostException { + InetSocketAddress createAddr(ServerName sn) throws UnknownHostException { InetSocketAddress addr = new InetSocketAddress(sn.getHostname(), sn.getPort()); if (addr.isUnresolved()) { throw new UnknownHostException("can not resolve " + sn.getServerName()); @@ -513,6 +512,13 @@ public RpcChannel createRpcChannel(ServerName sn, User user, int rpcTimeout) return new RpcChannelImplementation(this, createAddr(sn), user, rpcTimeout); } + @Override + public RpcChannel createHedgedRpcChannel(Set sns, User user, int rpcTimeout) + throws UnknownHostException { + // Check HedgedRpcChannel implementation for detailed comments. + throw new UnsupportedOperationException("Hedging not supported for this implementation."); + } + private static class AbstractRpcChannel { protected final InetSocketAddress addr; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java index f84c308715b2..22eca535e958 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java @@ -17,18 +17,15 @@ */ package org.apache.hadoop.hbase.ipc; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - import java.io.IOException; import java.net.SocketAddress; - import javax.net.SocketFactory; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.MetricsConnection; import org.apache.hadoop.net.NetUtils; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; /** * Does RPC against a cluster. Manages connections per regionserver in the cluster. diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java new file mode 100644 index 000000000000..7b681e079bdf --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java @@ -0,0 +1,274 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.ipc; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.util.PrettyPrinter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; +import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors; +import org.apache.hbase.thirdparty.com.google.protobuf.Message; +import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback; +import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel; +import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; + +/** + * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points. + * First received response is returned to the caller. This abstracts out the logic needed to batch + * requests to multiple end points underneath and presents itself as a single logical RpcChannel to + * the client. + * + * Hedging Details: + * --------------- + * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address + * end points to make the call to. We do multiple iterations until we get a proper response to the + * rpc call or all the service addresses are exhausted, which ever happens first. Size of each is + * configurable and is also known as 'fanOutSize'. + * + * - We randomize the addresses up front so that the batch order per client is non deterministic. + * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'. + * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the + * load on the client and server side when configuring the fan out. + * + * - In a happy case, once we receive a response from one end point, we cancel all the + * other inflight rpcs in the same batch and return the response to the caller. If we do not get a + * valid response from any address end point, we propagate the error back to the caller. + * + * - Rpc timeouts are applied to every hedged rpc. + * + * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can + * be hedged (for example: cluster state changing rpcs). + * + * (TODO) Retries and Adaptive hedging policy: + * ------------------------------------------ + * + * - No retries are handled at the channel level. Retries can be built in upper layers. However the + * question is, do we even need retries? Hedging in fact is a substitute for retries. + * + * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging + * policy more adaptive. In most happy cases, the rpcs from the first few end points should return + * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging + * is not needed. So, the idea is to make this request pattern pluggable so that the requests are + * hedged only when needed. + */ +class HedgedRpcChannel implements RpcChannel { + private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class); + + /** + * Currently hedging is only supported for non-blocking connection implementation types because + * the channel implementation inherently relies on the connection implementation being async. + * Refer to the comments in doCallMethod(). + */ + private final NettyRpcClient rpcClient; + // List of service addresses to hedge the requests to. + private final List addrs; + private final User ticket; + private final int rpcTimeout; + // Controls the size of request fan out (number of rpcs per a single batch). + private final int fanOutSize; + + /** + * A simple rpc call back implementation to notify the batch context if any rpc is successful. + */ + private static class BatchRpcCtxCallBack implements RpcCallback { + private final BatchRpcCtx batchRpcCtx; + private final HBaseRpcController rpcController; + BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) { + this.batchRpcCtx = batchRpcCtx; + this.rpcController = rpcController; + } + @Override + public void run(Message result) { + batchRpcCtx.setResultIfNotSet(result, rpcController); + } + } + + /** + * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to + * synchronize on multiple RPCs to different end points fetching the result. All the methods are + * thread-safe. + */ + private static class BatchRpcCtx { + // Result set by the thread finishing first. Set only once. + private final AtomicReference result = new AtomicReference<>(); + // Caller waits on this latch being set. + // We set this to 1, so that the first successful RPC result is returned to the client. + private CountDownLatch resultsReady = new CountDownLatch(1); + // Failed rpc book-keeping. + private AtomicInteger failedRpcCount = new AtomicInteger(); + // All the call handles for this batch. + private final List callsInFlight = Collections.synchronizedList(new ArrayList<>()); + + // Target addresses. + private final List addresses; + // Called when the result is ready. + private final RpcCallback callBack; + // Last failed rpc's exception. Used to propagate the reason to the controller. + private IOException lastFailedRpcReason; + + + BatchRpcCtx(List addresses, RpcCallback callBack) { + this.addresses = addresses; + this.callBack = Preconditions.checkNotNull(callBack); + } + + /** + * Sets the result only if it is not already set by another thread. Thread that successfully + * sets the result also count downs the latch. + * @param result Result to be set. + */ + public void setResultIfNotSet(Message result, HBaseRpcController rpcController) { + if (rpcController.failed()) { + incrementFailedRpcs(rpcController.getFailed()); + return; + } + if (this.result.compareAndSet(null, result)) { + resultsReady.countDown(); + // Cancel all pending in flight calls. + for (Call call: callsInFlight) { + // It is ok to do it for all calls as it is a no-op if the call is already done. + final String exceptionMsg = String.format("%s canceled because another hedged attempt " + + "for the same rpc already succeeded. This is not needed anymore.", call); + call.setException(new CallCancelledException(exceptionMsg)); + } + } + } + + /** + * Waits until the results are populated and calls the callback if the call is successful. + * @return true for successful rpc and false otherwise. + */ + public boolean waitForResults() { + try { + // We do not set a timeout on await() because we rely on the underlying RPCs to timeout if + // something on the remote is broken. Worst case we should wait for rpc time out to kick in. + resultsReady.await(); + } catch (InterruptedException e) { + LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e); + } + Message message = result.get(); + if (message != null) { + callBack.run(message); + return true; + } + return false; + } + + public void addCallInFlight(Call c) { + callsInFlight.add(c); + } + + public void incrementFailedRpcs(IOException reason) { + if (failedRpcCount.incrementAndGet() == addresses.size()) { + lastFailedRpcReason = reason; + // All the rpcs in this batch have failed. Invoke the waiting threads. + resultsReady.countDown(); + } + } + + public IOException getLastFailedRpcReason() { + return lastFailedRpcReason; + } + + @Override + public String toString() { + return String.format("Batched rpc for target(s) %s", PrettyPrinter.toString(addresses)); + } + } + + public HedgedRpcChannel(NettyRpcClient rpcClient, Set addrs, + User ticket, int rpcTimeout, int fanOutSize) { + this.rpcClient = rpcClient; + this.addrs = new ArrayList<>(Preconditions.checkNotNull(addrs)); + Preconditions.checkArgument(this.addrs.size() >= 1); + // For non-deterministic client query pattern. Not all clients want to hedge RPCs in the same + // order, creating hot spots on the service end points. + Collections.shuffle(this.addrs); + this.ticket = ticket; + this.rpcTimeout = rpcTimeout; + // fanOutSize controls the number of hedged RPCs per batch. + this.fanOutSize = fanOutSize; + } + + private HBaseRpcController applyRpcTimeout(RpcController controller) { + HBaseRpcController hBaseRpcController = (HBaseRpcController) controller; + int rpcTimeoutToSet = + hBaseRpcController.hasCallTimeout() ? hBaseRpcController.getCallTimeout() : rpcTimeout; + HBaseRpcController response = new HBaseRpcControllerImpl(); + response.setCallTimeout(rpcTimeoutToSet); + return response; + } + + private void doCallMethod(Descriptors.MethodDescriptor method, HBaseRpcController controller, + Message request, Message responsePrototype, RpcCallback done) { + int i = 0; + BatchRpcCtx lastBatchCtx = null; + while (i < addrs.size()) { + // Each iteration picks fanOutSize addresses to run as batch. + int batchEnd = Math.min(addrs.size(), i + fanOutSize); + List addrSubList = addrs.subList(i, batchEnd); + BatchRpcCtx batchRpcCtx = new BatchRpcCtx(addrSubList, done); + lastBatchCtx = batchRpcCtx; + LOG.debug("Attempting request {}, {}", method.getName(), batchRpcCtx); + for (InetSocketAddress address : addrSubList) { + HBaseRpcController rpcController = applyRpcTimeout(controller); + // ** WARN ** This is a blocking call if the underlying connection for the rpc client is + // a blocking implementation (ex: BlockingRpcConnection). That essentially serializes all + // the write calls. Handling blocking connection means that this should be run in a separate + // thread and hence more code complexity. Is it ok to handle only non-blocking connections? + batchRpcCtx.addCallInFlight(rpcClient.callMethod(method, rpcController, request, + responsePrototype, ticket, address, + new BatchRpcCtxCallBack(batchRpcCtx, rpcController))); + } + if (batchRpcCtx.waitForResults()) { + return; + } + // Entire batch has failed, lets try the next batch. + LOG.debug("Failed request {}, {}.", method.getName(), batchRpcCtx); + i = batchEnd; + } + Preconditions.checkNotNull(lastBatchCtx); + // All the batches failed, mark it a failed rpc. + // Propagate the failure reason. We propagate the last batch's last failing rpc reason. + // Can we do something better? + controller.setFailed(lastBatchCtx.getLastFailedRpcReason()); + done.run(null); + } + + @Override + public void callMethod(Descriptors.MethodDescriptor method, RpcController controller, + Message request, Message responsePrototype, RpcCallback done) { + // There is no reason to use any other implementation of RpcController. + Preconditions.checkState(controller instanceof HBaseRpcController); + // To make the channel non-blocking, we run the actual doCalMethod() async. The call back is + // called once the hedging finishes. + CompletableFuture.runAsync( + () -> doCallMethod(method, (HBaseRpcController)controller, request, responsePrototype, done)); + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java index 61dedbb5c124..c4f70b05aaa1 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java @@ -17,21 +17,26 @@ */ package org.apache.hadoop.hbase.ipc; -import org.apache.hbase.thirdparty.io.netty.channel.Channel; -import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup; -import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup; -import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel; -import org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultThreadFactory; - import java.io.IOException; +import java.net.InetSocketAddress; import java.net.SocketAddress; - +import java.net.UnknownHostException; +import java.util.HashSet; +import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HConstants; -import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.MetricsConnection; +import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Pair; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel; +import org.apache.hbase.thirdparty.io.netty.channel.Channel; +import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup; +import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup; +import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel; +import org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultThreadFactory; /** * Netty client for the requests and responses. @@ -74,6 +79,19 @@ protected NettyRpcConnection createConnection(ConnectionId remoteId) throws IOEx return new NettyRpcConnection(this, remoteId); } + @Override + public RpcChannel createHedgedRpcChannel(Set sns, User user, int rpcTimeout) + throws UnknownHostException { + final int hedgedRpcFanOut = conf.getInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, + HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_DEFAULT); + Set addresses = new HashSet<>(); + for (ServerName sn: sns) { + addresses.add(createAddr(sn)); + } + return new HedgedRpcChannel(this, addresses, user, rpcTimeout, + hedgedRpcFanOut); + } + @Override protected void closeInternal() { if (shutdownGroupWhenClose) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java index 0e006956d249..558fceeb78bb 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java @@ -17,15 +17,14 @@ */ package org.apache.hadoop.hbase.ipc; -import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel; -import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel; - import java.io.Closeable; import java.io.IOException; - +import java.util.Set; import org.apache.hadoop.hbase.ServerName; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.security.User; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel; +import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel; /** * Interface for RpcClient implementations so ConnectionManager can handle it. @@ -83,6 +82,16 @@ BlockingRpcChannel createBlockingRpcChannel(ServerName sn, User user, int rpcTim RpcChannel createRpcChannel(final ServerName sn, final User user, int rpcTimeout) throws IOException; + /** + * Creates a channel that can hedge request to multiple underlying channels. + * @param sns Set of servers for underlying channels. + * @param user user for the connection. + * @param rpcTimeout rpc timeout to use. + * @return A hedging rpc channel for this rpc client instance. + */ + RpcChannel createHedgedRpcChannel(final Set sns, final User user, int rpcTimeout) + throws IOException; + /** * Interrupt the connections to the given server. This should be called if the server * is known as actually dead. This will not prevent current operation to be retried, and, diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingAsyncRegistry.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingConnectionRegistry.java similarity index 89% rename from hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingAsyncRegistry.java rename to hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingConnectionRegistry.java index 8c7b07384c72..4bd66877b1b4 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingAsyncRegistry.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingConnectionRegistry.java @@ -27,13 +27,13 @@ * Registry that does nothing. Otherwise, default Registry wants zookeeper up and running. */ @InterfaceAudience.Private -class DoNothingAsyncRegistry implements AsyncRegistry { +class DoNothingConnectionRegistry implements ConnectionRegistry { - public DoNothingAsyncRegistry(Configuration conf) { + public DoNothingConnectionRegistry(Configuration conf) { } @Override - public CompletableFuture getMetaRegionLocation() { + public CompletableFuture getMetaRegionLocations() { return CompletableFuture.completedFuture(null); } @@ -43,7 +43,7 @@ public CompletableFuture getClusterId() { } @Override - public CompletableFuture getMasterAddress() { + public CompletableFuture getActiveMaster() { return CompletableFuture.completedFuture(null); } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java index 46d786e77e94..ce5c321767fc 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java @@ -142,7 +142,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { }).when(adminStub).stopServer(any(HBaseRpcController.class), any(StopServerRequest.class), any()); - conn = new AsyncConnectionImpl(CONF, new DoNothingAsyncRegistry(CONF), "test", + conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF), "test", UserProvider.instantiate(CONF).getCurrent()) { @Override diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocatorFailFast.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocatorFailFast.java index f29c3bfcc8e2..b306500c8b13 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocatorFailFast.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocatorFailFast.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -43,21 +43,21 @@ public class TestAsyncMetaRegionLocatorFailFast { private static AsyncMetaRegionLocator LOCATOR; - private static final class FaultyAsyncRegistry extends DoNothingAsyncRegistry { + private static final class FaultyConnectionRegistry extends DoNothingConnectionRegistry { - public FaultyAsyncRegistry(Configuration conf) { + public FaultyConnectionRegistry(Configuration conf) { super(conf); } @Override - public CompletableFuture getMetaRegionLocation() { + public CompletableFuture getMetaRegionLocations() { return FutureUtils.failedFuture(new DoNotRetryRegionException("inject error")); } } @BeforeClass public static void setUp() { - LOCATOR = new AsyncMetaRegionLocator(new FaultyAsyncRegistry(CONF)); + LOCATOR = new AsyncMetaRegionLocator(new FaultyConnectionRegistry(CONF)); } @Test(expected = DoNotRetryIOException.class) diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java index 6afe0b59b82e..f94bd2b89280 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java @@ -462,7 +462,7 @@ private static interface ResponseGenerator { * Returns our async process. */ static class MyConnectionImpl extends ConnectionImplementation { - public static class TestRegistry extends DoNothingAsyncRegistry { + public static class TestRegistry extends DoNothingConnectionRegistry { public TestRegistry(Configuration conf) { super(conf); @@ -481,8 +481,8 @@ protected MyConnectionImpl(Configuration conf) throws IOException { } private static Configuration setupConf(Configuration conf) { - conf.setClass(AsyncRegistryFactory.REGISTRY_IMPL_CONF_KEY, TestRegistry.class, - AsyncRegistry.class); + conf.setClass(HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, + TestRegistry.class, ConnectionRegistry.class); return conf; } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java index 56dcf10beefa..8c22946649dc 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java @@ -175,7 +175,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { return null; } }).when(stub).get(any(HBaseRpcController.class), any(GetRequest.class), any()); - conn = new AsyncConnectionImpl(CONF, new DoNothingAsyncRegistry(CONF), "test", + conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF), "test", UserProvider.instantiate(CONF).getCurrent()) { @Override diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutator.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutator.java index f8e12954b1e9..c9993ee2dd1f 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutator.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutator.java @@ -23,6 +23,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -58,7 +59,8 @@ public static class MyBufferedMutator extends BufferedMutatorImpl { public void testAlternateBufferedMutatorImpl() throws IOException { BufferedMutatorParams params = new BufferedMutatorParams(TableName.valueOf(name.getMethodName())); Configuration conf = HBaseConfiguration.create(); - conf.set(AsyncRegistryFactory.REGISTRY_IMPL_CONF_KEY, DoNothingAsyncRegistry.class.getName()); + conf.set(HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, + DoNothingConnectionRegistry.class.getName()); try (Connection connection = ConnectionFactory.createConnection(conf)) { BufferedMutator bm = connection.getBufferedMutator(params); // Assert we get default BM if nothing specified. diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java index 19f98f1d8940..ff66924f5488 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java @@ -121,13 +121,14 @@ public void setUp() throws Exception { // Run my Connection overrides. Use my little ConnectionImplementation below which // allows me insert mocks and also use my Registry below rather than the default zk based // one so tests run faster and don't have zk dependency. - this.conf.set("hbase.client.registry.impl", SimpleRegistry.class.getName()); + this.conf.set(HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, + SimpleRegistry.class.getName()); } /** * Simple cluster registry inserted in place of our usual zookeeper based one. */ - static class SimpleRegistry extends DoNothingAsyncRegistry { + static class SimpleRegistry extends DoNothingConnectionRegistry { final ServerName META_HOST = META_SERVERNAME; public SimpleRegistry(Configuration conf) { @@ -135,7 +136,7 @@ public SimpleRegistry(Configuration conf) { } @Override - public CompletableFuture getMetaRegionLocation() { + public CompletableFuture getMetaRegionLocations() { return CompletableFuture.completedFuture(new RegionLocations( new HRegionLocation(RegionInfoBuilder.FIRST_META_REGIONINFO, META_HOST))); } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegistryLeak.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionRegistryLeak.java similarity index 82% rename from hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegistryLeak.java rename to hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionRegistryLeak.java index 7c210756d57e..561b1f5715fd 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegistryLeak.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionRegistryLeak.java @@ -29,6 +29,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.FutureUtils; @@ -38,17 +39,17 @@ import org.junit.experimental.categories.Category; @Category({ ClientTests.class, SmallTests.class }) -public class TestAsyncRegistryLeak { +public class TestConnectionRegistryLeak { @ClassRule public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestAsyncRegistryLeak.class); + HBaseClassTestRule.forClass(TestConnectionRegistryLeak.class); - public static final class AsyncRegistryForTest extends DoNothingAsyncRegistry { + public static final class ConnectionRegistryForTest extends DoNothingConnectionRegistry { private boolean closed = false; - public AsyncRegistryForTest(Configuration conf) { + public ConnectionRegistryForTest(Configuration conf) { super(conf); CREATED.add(this); } @@ -64,14 +65,14 @@ public void close() { } } - private static final List CREATED = new ArrayList<>(); + private static final List CREATED = new ArrayList<>(); private static Configuration CONF = HBaseConfiguration.create(); @BeforeClass public static void setUp() { - CONF.setClass(AsyncRegistryFactory.REGISTRY_IMPL_CONF_KEY, AsyncRegistryForTest.class, - AsyncRegistry.class); + CONF.setClass(HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, + ConnectionRegistryForTest.class, ConnectionRegistry.class); } @Test diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index 6da186e50206..11ec8409ec24 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -184,10 +184,17 @@ public enum OperationStatusCode { public static final String MASTER_INFO_PORT = "hbase.master.info.port"; /** Configuration key for the list of master host:ports **/ - public static final String MASTER_ADDRS_KEY = "hbase.master.addrs"; + public static final String MASTER_ADDRS_KEY = "hbase.masters"; public static final String MASTER_ADDRS_DEFAULT = "localhost:" + DEFAULT_MASTER_PORT; + /** Configuration to enable hedged reads on master registry **/ + public static final String MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY = + "hbase.client.master_registry.enable_hedged_reads"; + + /** Default value for enabling hedging reads on master registry **/ + public static final boolean MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT = false; + /** Parameter name for the master type being backup (waits for primary to go inactive). */ public static final String MASTER_TYPE_BACKUP = "hbase.master.backup"; @@ -930,6 +937,12 @@ public enum OperationStatusCode { */ public static final String HBASE_RPC_TIMEOUT_KEY = "hbase.rpc.timeout"; + /** Configuration key that controls the fan out of requests in hedged channel implementation. **/ + public static final String HBASE_RPCS_HEDGED_REQS_FANOUT_KEY = "hbase.rpc.hedged.fanout"; + + /** Default value for the fan out of hedged requests. **/ + public static final int HBASE_RPCS_HEDGED_REQS_FANOUT_DEFAULT = 2; + /** * timeout for each read RPC */ @@ -961,6 +974,11 @@ public enum OperationStatusCode { */ public static final long NO_SEQNUM = -1; + /** + * Registry implementation to be used on the client side. + */ + public static final String CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY = + "hbase.client.registry.impl"; /* * cluster replication constants. diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PrettyPrinter.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PrettyPrinter.java index 147e9160f910..ff7064b11430 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PrettyPrinter.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PrettyPrinter.java @@ -19,9 +19,12 @@ package org.apache.hadoop.hbase.util; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Objects; import java.util.regex.Matcher; import java.util.regex.Pattern; - import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.exceptions.HBaseException; import org.apache.yetus.audience.InterfaceAudience; @@ -29,7 +32,7 @@ import org.slf4j.LoggerFactory; @InterfaceAudience.Private -public class PrettyPrinter { +public final class PrettyPrinter { private static final Logger LOG = LoggerFactory.getLogger(PrettyPrinter.class); @@ -117,7 +120,7 @@ private static String humanReadableTTL(final long interval){ sb.append(" DAY").append(days == 1 ? "" : "S"); } - if (hours > 0 ) { + if (hours > 0) { sb.append(days > 0 ? " " : ""); sb.append(hours); sb.append(" HOUR").append(hours == 1 ? "" : "S"); @@ -188,4 +191,18 @@ private static long humanReadableIntervalToSec(final String humanReadableInterva return ttl; } + /** + * Pretty prints a collection of any type to a string. Relies on toString() implementation of the + * object type. + * @param collection collection to pretty print. + * @return Pretty printed string for the collection. + */ + public static String toString(Collection collection) { + List stringList = new ArrayList<>(); + for (Object o: collection) { + stringList.add(Objects.toString(o)); + } + return "[" + String.join(",", stringList) + "]"; + } + } diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java index 00374c115abb..0880ad08ba81 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -17,10 +17,16 @@ */ package org.apache.hadoop.hbase; +import edu.umd.cs.findbugs.annotations.NonNull; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.lang.reflect.Modifier; +import java.util.Arrays; +import java.util.Collection; import java.util.Collections; +import java.util.List; import java.util.Set; import java.util.concurrent.TimeUnit; - import org.apache.hadoop.hbase.testclassification.IntegrationTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -30,8 +36,14 @@ import org.junit.rules.TestRule; import org.junit.rules.Timeout; import org.junit.runner.Description; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; import org.junit.runners.model.Statement; - +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; import org.apache.hbase.thirdparty.com.google.common.collect.Sets; /** @@ -43,9 +55,13 @@ */ @InterfaceAudience.Private public final class HBaseClassTestRule implements TestRule { + private static final Logger LOG = LoggerFactory.getLogger(HBaseClassTestRule.class); public static final Set> UNIT_TEST_CLASSES = Collections.unmodifiableSet( Sets.> newHashSet(SmallTests.class, MediumTests.class, LargeTests.class)); + // Each unit test has this timeout. + private static long PER_UNIT_TEST_TIMEOUT_MINS = 13; + private final Class clazz; private final Timeout timeout; @@ -65,13 +81,16 @@ public Class getClazz() { private static long getTimeoutInSeconds(Class clazz) { Category[] categories = clazz.getAnnotationsByType(Category.class); - + // Starting JUnit 4.13, it appears that the timeout is applied across all the parameterized + // runs. So the timeout is multiplied by number of parameterized runs. + int numParams = getNumParameters(clazz); // @Category is not repeatable -- it is only possible to get an array of length zero or one. if (categories.length == 1) { for (Class c : categories[0].value()) { if (UNIT_TEST_CLASSES.contains(c)) { - // All tests have a 13 minutes timeout. - return TimeUnit.MINUTES.toSeconds(13); + long timeout = numParams * PER_UNIT_TEST_TIMEOUT_MINS; + LOG.info("Test {} timeout: {} mins", clazz, timeout); + return TimeUnit.MINUTES.toSeconds(timeout); } if (c == IntegrationTests.class) { return TimeUnit.MINUTES.toSeconds(Long.MAX_VALUE); @@ -82,6 +101,59 @@ private static long getTimeoutInSeconds(Class clazz) { clazz.getName() + " does not have SmallTests/MediumTests/LargeTests in @Category"); } + /** + * @param clazz Test class that is running. + * @return the number of parameters for this given test class. If the test is not parameterized or + * if there is any issue determining the number of parameters, returns 1. + */ + @VisibleForTesting + static int getNumParameters(Class clazz) { + RunWith[] runWiths = clazz.getAnnotationsByType(RunWith.class); + boolean testParameterized = runWiths != null && Arrays.stream(runWiths).anyMatch( + (r) -> r.value().equals(Parameterized.class)); + if (!testParameterized) { + return 1; + } + for (Method method : clazz.getMethods()) { + if (!isParametersMethod(method)) { + continue; + } + // Found the parameters method. Figure out the number of parameters. + Object parameters; + try { + parameters = method.invoke(clazz); + } catch (IllegalAccessException | InvocationTargetException e) { + LOG.warn("Error invoking parameters method {} in test class {}", + method.getName(), clazz, e); + continue; + } + if (parameters instanceof List) { + return ((List) parameters).size(); + } else if (parameters instanceof Collection) { + return ((Collection) parameters).size(); + } else if (parameters instanceof Iterable) { + return Iterables.size((Iterable) parameters); + } else if (parameters instanceof Object[]) { + return ((Object[]) parameters).length; + } + } + LOG.warn("Unable to determine parameters size. Returning the default of 1."); + return 1; + } + + /** + * Helper method that checks if the input method is a valid JUnit @Parameters method. + * @param method Input method. + * @return true if the method is a valid JUnit parameters method, false otherwise. + */ + private static boolean isParametersMethod(@NonNull Method method) { + // A valid parameters method is public static and with @Parameters annotation. + boolean methodPublicStatic = Modifier.isPublic(method.getModifiers()) && + Modifier.isStatic(method.getModifiers()); + Parameters[] params = method.getAnnotationsByType(Parameters.class); + return methodPublicStatic && (params != null && params.length > 0); + } + public static HBaseClassTestRule forClass(Class clazz) { return new HBaseClassTestRule(clazz, Timeout.builder().withLookingForStuckThread(true) .withTimeout(getTimeoutInSeconds(clazz), TimeUnit.SECONDS).build()); diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TableNameTestRule.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TableNameTestRule.java index b5654265efac..626c67ee2422 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TableNameTestRule.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TableNameTestRule.java @@ -29,7 +29,21 @@ public class TableNameTestRule extends TestWatcher { @Override protected void starting(Description description) { - tableName = TableName.valueOf(description.getMethodName()); + tableName = TableName.valueOf(cleanUpTestName(description.getMethodName())); + } + + /** + * Helper to handle parameterized method names. Unlike regular test methods, parameterized method + * names look like 'foo[x]'. This is problematic for tests that use this name for HBase tables. + * This helper strips out the parameter suffixes. + * @return current test method name with out parameterized suffixes. + */ + private static String cleanUpTestName(String methodName) { + int index = methodName.indexOf('['); + if (index == -1) { + return methodName; + } + return methodName.substring(0, index); } public TableName getTableName() { diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestHBaseClassTestRule.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestHBaseClassTestRule.java new file mode 100644 index 000000000000..78853e6aed76 --- /dev/null +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestHBaseClassTestRule.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase; + +import static junit.framework.TestCase.assertEquals; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; +import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; + +/** + * Tests HBaseClassTestRule. + */ +@Category(SmallTests.class) +public class TestHBaseClassTestRule { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass( + TestHBaseClassTestRule.class); + + // Test input classes of various kinds. + private static class NonParameterizedClass { + void dummy() { + } + int dummy(int a) { + return 0; + } + } + + @RunWith(Parameterized.class) + private static class ParameterizedClassWithNoParametersMethod { + void dummy() { + } + } + + @RunWith(Parameterized.class) + private static class InValidParameterizedClass { + // Not valid because parameters method is private. + @Parameters + private static List parameters() { + return Arrays.asList(1, 2, 3, 4); + } + int dummy(int a) { + return 0; + } + } + + @RunWith(Parameterized.class) + private static class ValidParameterizedClass1 { + @Parameters + public static List parameters() { + return Arrays.asList(1, 2, 3, 4, 5); + } + int dummy(int a) { + return 0; + } + } + + @RunWith(Parameterized.class) + private static class ValidParameterizedClass2 { + @Parameters + public static Object[] parameters() { + return new Integer[] {1, 2, 3, 4, 5, 6}; + } + } + + @RunWith(Parameterized.class) + private static class ValidParameterizedClass3 { + @Parameters + public static Iterable parameters() { + return Arrays.asList(1, 2, 3, 4, 5, 6, 7); + } + } + + @RunWith(Parameterized.class) + private static class ValidParameterizedClass4 { + @Parameters + public static Collection parameters() { + return Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9); + } + } + + + @RunWith(Parameterized.class) + private static class ExtendedParameterizedClass1 extends ValidParameterizedClass1 { + // Should be inferred from the parent class. + int dummy2(int a) { + return 0; + } + } + + @RunWith(Parameterized.class) + private static class ExtendedParameterizedClass2 extends ValidParameterizedClass1 { + // Should override the parent parameters class. + @Parameters + public static List parameters() { + return Arrays.asList(1, 2, 3); + } + } + + @Test + public void testNumParameters() { + // Invalid cases, expected to return 1. + assertEquals(HBaseClassTestRule.getNumParameters(NonParameterizedClass.class), 1); + assertEquals(HBaseClassTestRule.getNumParameters( + ParameterizedClassWithNoParametersMethod.class), 1); + assertEquals(HBaseClassTestRule.getNumParameters(InValidParameterizedClass.class), 1); + // Valid parameterized classes. + assertEquals(HBaseClassTestRule.getNumParameters(ValidParameterizedClass1.class), + ValidParameterizedClass1.parameters().size()); + assertEquals(HBaseClassTestRule.getNumParameters(ValidParameterizedClass2.class), + ValidParameterizedClass2.parameters().length); + assertEquals(HBaseClassTestRule.getNumParameters(ValidParameterizedClass3.class), + Iterables.size(ValidParameterizedClass3.parameters())); + assertEquals(HBaseClassTestRule.getNumParameters(ValidParameterizedClass4.class), + ValidParameterizedClass4.parameters().size()); + // Testing inheritance. + assertEquals(HBaseClassTestRule.getNumParameters(ExtendedParameterizedClass1.class), + ValidParameterizedClass1.parameters().size()); + assertEquals(HBaseClassTestRule.getNumParameters(ExtendedParameterizedClass2.class), + ExtendedParameterizedClass2.parameters().size()); + } +} diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java index 906abca05c9d..cb78e37fbb4c 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java @@ -240,5 +240,10 @@ public TableBuilder getTableBuilder(TableName tableName, ExecutorService pool) { @Override public void clearRegionLocationCache() { } + + @Override + public String getClusterId() { + return null; + } } } diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java index 4436ee109188..9faade12d93c 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java @@ -290,5 +290,10 @@ public TableBuilder getTableBuilder(TableName tableName, ExecutorService pool) { @Override public void clearRegionLocationCache() { } + + @Override + public String getClusterId() { + return null; + } } } diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplication.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplication.java index 7771c0a42993..311e55c6b267 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplication.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplication.java @@ -27,6 +27,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -689,7 +690,7 @@ public void testVerifyRepJobWithPeerTableNameAndSnapshotSupport() throws Excepti @AfterClass public static void tearDownAfterClass() throws Exception { - htable3.close(); + IOUtils.closeQuietly(htable3); TestReplicationBase.tearDownAfterClass(); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/SharedConnection.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/SharedConnection.java index de0c39b422dd..1faa2791ea17 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/SharedConnection.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/SharedConnection.java @@ -105,4 +105,9 @@ public Hbck getHbck() throws IOException { public Hbck getHbck(ServerName masterServer) throws IOException { return conn.getHbck(masterServer); } + + @Override + public String getClusterId() { + return conn.getClusterId(); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index 6e5799eba2b1..a119a0bc563a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -20,7 +20,6 @@ import static org.apache.hadoop.hbase.HConstants.DEFAULT_HBASE_SPLIT_COORDINATED_BY_ZK; import static org.apache.hadoop.hbase.HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS; import static org.apache.hadoop.hbase.HConstants.HBASE_SPLIT_WAL_COORDINATED_BY_ZK; - import com.google.protobuf.Descriptors; import com.google.protobuf.Service; import java.io.IOException; @@ -56,8 +55,10 @@ import javax.servlet.http.HttpServlet; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; +import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.ChoreService; import org.apache.hadoop.hbase.ClusterId; @@ -215,11 +216,9 @@ import org.eclipse.jetty.webapp.WebAppContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hbase.thirdparty.com.google.common.collect.Maps; - import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState; import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription; @@ -906,8 +905,15 @@ private void finishActiveMasterInitialization(MonitoredTask status) // hbck1s against an hbase2 cluster; it could do damage. To skip this behavior, set // hbase.write.hbck1.lock.file to false. if (this.conf.getBoolean("hbase.write.hbck1.lock.file", true)) { - HBaseFsck.checkAndMarkRunningHbck(this.conf, - HBaseFsck.createLockRetryCounterFactory(this.conf).create()); + Pair result = null; + try { + result = HBaseFsck.checkAndMarkRunningHbck(this.conf, + HBaseFsck.createLockRetryCounterFactory(this.conf).create()); + } finally { + if (result != null) { + IOUtils.closeQuietly(result.getSecond()); + } + } } status.setStatus("Initialize ServerManager and schedule SCP for crash servers"); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java index f540b536ee2d..ca53b880288b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java @@ -25,18 +25,14 @@ import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos; import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.zookeeper.ZKClusterId; -import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.security.token.Token; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - /** * Utility methods for obtaining authentication tokens. */ @@ -206,7 +202,7 @@ public static void obtainTokenForJob(final Connection conn, final JobConf job, U public static void addTokenForJob(final Connection conn, final JobConf job, User user) throws IOException, InterruptedException { - Token token = getAuthToken(conn.getConfiguration(), user); + Token token = getAuthToken(conn, user); if (token == null) { token = ClientTokenUtil.obtainToken(conn, user); } @@ -225,7 +221,7 @@ public static void addTokenForJob(final Connection conn, final JobConf job, User */ public static void addTokenForJob(final Connection conn, User user, Job job) throws IOException, InterruptedException { - Token token = getAuthToken(conn.getConfiguration(), user); + Token token = getAuthToken(conn, user); if (token == null) { token = ClientTokenUtil.obtainToken(conn, user); } @@ -244,7 +240,7 @@ public static void addTokenForJob(final Connection conn, User user, Job job) */ public static boolean addTokenIfMissing(Connection conn, User user) throws IOException, InterruptedException { - Token token = getAuthToken(conn.getConfiguration(), user); + Token token = getAuthToken(conn, user); if (token == null) { token = ClientTokenUtil.obtainToken(conn, user); user.getUGI().addToken(token.getService(), token); @@ -257,19 +253,12 @@ public static boolean addTokenIfMissing(Connection conn, User user) * Get the authentication token of the user for the cluster specified in the configuration * @return null if the user does not have the token, otherwise the auth token for the cluster. */ - private static Token getAuthToken(Configuration conf, User user) - throws IOException, InterruptedException { - ZKWatcher zkw = new ZKWatcher(conf, "TokenUtil-getAuthToken", null); - try { - String clusterId = ZKClusterId.readClusterIdZNode(zkw); - if (clusterId == null) { - throw new IOException("Failed to get cluster ID"); - } - return new AuthenticationTokenSelector().selectToken(new Text(clusterId), user.getTokens()); - } catch (KeeperException e) { - throw new IOException(e); - } finally { - zkw.close(); + private static Token getAuthToken(Connection conn, User user) + throws IOException { + final String clusterId = conn.getClusterId(); + if (clusterId == null) { + throw new IOException("Failed to get cluster ID"); } + return new AuthenticationTokenSelector().selectToken(new Text(clusterId), user.getTokens()); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ConnectionCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ConnectionCache.java index 7b9f021313a6..4559d783729c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ConnectionCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ConnectionCache.java @@ -202,6 +202,19 @@ public boolean updateConnectionAccessTime() { return false; } + /** + * @return Cluster ID for the HBase cluster or null if there is an err making the connection. + */ + public String getClusterId() { + try { + ConnectionInfo connInfo = getCurrentConnection(); + return connInfo.connection.getClusterId(); + } catch (IOException e) { + LOG.error("Error getting connection: ", e); + } + return null; + } + class ConnectionInfo { final Connection connection; final String userName; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java index 5c6ad955fe26..cfa6f75faea0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java @@ -26,12 +26,14 @@ import java.util.concurrent.TimeUnit; import java.util.function.Supplier; +import org.apache.hadoop.hbase.HConstants; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; /** * Utility used running a cluster all in the one JVM. @@ -136,6 +138,11 @@ public static JVMClusterUtil.MasterThread createMasterThread(final Configuration } catch (Exception e) { throw new IOException(e); } + // Needed if a master based registry is configured for internal cluster connections. Here, we + // just add the current master host port since we do not know other master addresses up front + // in mini cluster tests. + c.set(HConstants.MASTER_ADDRS_KEY, + Preconditions.checkNotNull(server.getServerName().getAddress()).toString()); return new JVMClusterUtil.MasterThread(server, index); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index 95a50acc0c87..9dbbef49a8ad 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java @@ -1144,6 +1144,9 @@ public MiniHBaseCluster startMiniHBaseCluster(StartMiniClusterOption option) this.hbaseCluster = new MiniHBaseCluster(c, option.getNumMasters(), option.getNumRegionServers(), option.getRsPorts(), option.getMasterClass(), option.getRsClass()); + // Populate the master address configuration from mini cluster configuration. + conf.set(HConstants.MASTER_ADDRS_KEY, + c.get(HConstants.MASTER_ADDRS_KEY, HConstants.MASTER_ADDRS_DEFAULT)); // Don't leave here till we've done a successful scan of the hbase:meta Table t = getConnection().getTable(TableName.META_TABLE_NAME); ResultScanner s = t.getScanner(new Scan()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java index 3d7228ce00d0..473e23309cb2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java @@ -21,7 +21,6 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.mock; - import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -90,6 +89,7 @@ public class TestZooKeeperTableArchiveClient { private static RegionServerServices rss; private static DirScanPool POOL; + /** * Setup the config for the cluster */ @@ -132,9 +132,13 @@ public void tearDown() throws Exception { @AfterClass public static void cleanupTest() throws Exception { - CONNECTION.close(); + if (CONNECTION != null) { + CONNECTION.close(); + } UTIL.shutdownMiniZKCluster(); - POOL.shutdownNow(); + if (POOL != null) { + POOL.shutdownNow(); + } } /** @@ -338,6 +342,7 @@ private HFileCleaner setupAndCreateCleaner(Configuration conf, FileSystem fs, Pa * @throws IOException on failure * @throws KeeperException on failure */ + @SuppressWarnings("checkstyle:EmptyBlock") private List turnOnArchiving(String tableName, HFileCleaner cleaner) throws IOException, KeeperException { // turn on hfile retention diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java index 4668d150d5f4..f96daf66e128 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java @@ -58,7 +58,8 @@ protected static void startClusterAndCreateTable() throws Exception { } UTIL.getAdmin().createTable(td, SPLIT_KEYS); UTIL.waitTableAvailable(TABLE_NAME); - try (AsyncRegistry registry = AsyncRegistryFactory.getRegistry(UTIL.getConfiguration())) { + try (ConnectionRegistry registry = + ConnectionRegistryFactory.getRegistry(UTIL.getConfiguration())) { RegionReplicaTestHelper.waitUntilAllMetaReplicasHavingRegionLocation(UTIL.getConfiguration(), registry, REGION_REPLICATION); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java index ba3e74c278d3..73924a38393f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java @@ -44,7 +44,7 @@ private RegionReplicaTestHelper() { // waits for all replicas to have region location static void waitUntilAllMetaReplicasHavingRegionLocation(Configuration conf, - AsyncRegistry registry, int regionReplication) throws IOException { + ConnectionRegistry registry, int regionReplication) throws IOException { Waiter.waitFor(conf, conf.getLong("hbase.client.sync.wait.timeout.msec", 60000), 200, true, new ExplainingPredicate() { @Override @@ -55,7 +55,7 @@ public String explainFailure() throws IOException { @Override public boolean evaluate() throws IOException { try { - RegionLocations locs = registry.getMetaRegionLocation().get(); + RegionLocations locs = registry.getMetaRegionLocations().get(); if (locs.size() < regionReplication) { return false; } @@ -66,7 +66,7 @@ public boolean evaluate() throws IOException { } return true; } catch (Exception e) { - TestZKAsyncRegistry.LOG.warn("Failed to get meta region locations", e); + TestZKConnectionRegistry.LOG.warn("Failed to get meta region locations", e); return false; } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java index 1401afeae8f4..6d7d36839824 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java @@ -53,7 +53,8 @@ public class TestAsyncAdminWithRegionReplicas extends TestAsyncAdminBase { public static void setUpBeforeClass() throws Exception { TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 3); TestAsyncAdminBase.setUpBeforeClass(); - try (AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration())) { + try (ConnectionRegistry registry = + ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration())) { RegionReplicaTestHelper .waitUntilAllMetaReplicasHavingRegionLocation(TEST_UTIL.getConfiguration(), registry, 3); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java index 1f0d40b5f4f7..609a12926206 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java @@ -45,7 +45,7 @@ public class TestAsyncMetaRegionLocator { private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private static AsyncRegistry REGISTRY; + private static ConnectionRegistry REGISTRY; private static AsyncMetaRegionLocator LOCATOR; @@ -54,7 +54,7 @@ public static void setUp() throws Exception { TEST_UTIL.getConfiguration().set(BaseLoadBalancer.TABLES_ON_MASTER, "none"); TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 3); TEST_UTIL.startMiniCluster(3); - REGISTRY = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration()); + REGISTRY = ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration()); RegionReplicaTestHelper .waitUntilAllMetaReplicasHavingRegionLocation(TEST_UTIL.getConfiguration(), REGISTRY, 3); TEST_UTIL.getAdmin().balancerSwitch(false, true); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java index 6bc024ac1212..70f867a5bd68 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java @@ -79,7 +79,8 @@ public class TestAsyncNonMetaRegionLocator { public static void setUp() throws Exception { TEST_UTIL.startMiniCluster(3); TEST_UTIL.getAdmin().balancerSwitch(false, true); - AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration()); + ConnectionRegistry registry = + ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration()); CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry, registry.getClusterId().get(), User.getCurrent()); LOCATOR = new AsyncNonMetaRegionLocator(CONN); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java index 8cdb4a9363f7..52242f176297 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java @@ -123,7 +123,8 @@ public static void setUp() throws Exception { conf.setInt(MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE, MAX_ALLOWED); TEST_UTIL.startMiniCluster(3); TEST_UTIL.getAdmin().balancerSwitch(false, true); - AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration()); + ConnectionRegistry registry = + ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration()); CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry, registry.getClusterId().get(), User.getCurrent()); LOCATOR = new AsyncNonMetaRegionLocator(CONN); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java index 0a94def619f5..677dc93aec91 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java @@ -98,7 +98,8 @@ public static void setUp() throws Exception { TEST_UTIL.startMiniCluster(1); TEST_UTIL.createTable(TABLE_NAME, FAMILY); TEST_UTIL.waitTableAvailable(TABLE_NAME); - AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration()); + ConnectionRegistry registry = + ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration()); CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry, registry.getClusterId().get(), User.getCurrent()); LOCATOR = CONN.getLocator(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java index 8959c1d4f3bc..4bf6bb05c1eb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java @@ -71,7 +71,8 @@ public static void setUpBeforeClass() throws Exception { TEST_UTIL.getAdmin().balancerSwitch(false, true); TEST_UTIL.createTable(TABLE_NAME, FAMILY); TEST_UTIL.waitTableAvailable(TABLE_NAME); - AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration()); + ConnectionRegistry registry = + ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration()); CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry, registry.getClusterId().get(), User.getCurrent()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java index e74d3fa291f5..ed6c66f8e5dc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java @@ -92,7 +92,7 @@ public static void setUp() throws Exception { conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, FailPrimaryMetaScanCp.class.getName()); UTIL.startMiniCluster(3); - try (AsyncRegistry registry = AsyncRegistryFactory.getRegistry(conf)) { + try (ConnectionRegistry registry = ConnectionRegistryFactory.getRegistry(conf)) { RegionReplicaTestHelper.waitUntilAllMetaReplicasHavingRegionLocation(conf, registry, 3); } try (Table table = UTIL.createTable(TABLE_NAME, FAMILY)) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java index 5cb9ffd188d9..67fe82dde55d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -25,10 +25,10 @@ import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; - import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.EnumSet; import java.util.HashMap; import java.util.HashSet; @@ -63,7 +63,9 @@ import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.StartMiniClusterOption; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.metrics.ScanMetrics; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; @@ -106,23 +108,28 @@ import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.TableDescriptorChecker; import org.junit.AfterClass; -import org.junit.BeforeClass; +import org.junit.Assume; import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; /** * Run tests that use the HBase clients; {@link Table}. * Sets up the HBase mini cluster once at start and runs through all client tests. * Each creates a table named for the method and does its stuff against that. + * + * Parameterized to run with different registry implementations. */ @Category({LargeTests.class, ClientTests.class}) @SuppressWarnings ("deprecation") +@RunWith(Parameterized.class) public class TestFromClientSide { @ClassRule @@ -131,7 +138,7 @@ public class TestFromClientSide { // NOTE: Increment tests were moved to their own class, TestIncrementsFromClientSide. private static final Logger LOG = LoggerFactory.getLogger(TestFromClientSide.class); - protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + protected static HBaseTestingUtility TEST_UTIL; private static byte [] ROW = Bytes.toBytes("testRow"); private static byte [] FAMILY = Bytes.toBytes("testFamily"); private static final byte[] INVALID_FAMILY = Bytes.toBytes("invalidTestFamily"); @@ -139,10 +146,54 @@ public class TestFromClientSide { private static byte [] VALUE = Bytes.toBytes("testValue"); protected static int SLAVES = 3; - @Rule - public TestName name = new TestName(); + @Rule public TableNameTestRule name = new TableNameTestRule(); + + // To keep the child classes happy. + TestFromClientSide() {} + + public TestFromClientSide(Class registry, int numHedgedReqs) throws Exception { + initialize(registry, numHedgedReqs, MultiRowMutationEndpoint.class); + } + + @Parameterized.Parameters + public static Collection parameters() { + return Arrays.asList(new Object[][] { + { MasterRegistry.class, 1}, + { MasterRegistry.class, 2}, + { ZKConnectionRegistry.class, 1} + }); + } + + /** + * JUnit does not provide an easy way to run a hook after each parameterized run. Without that + * there is no easy way to restart the test cluster after each parameterized run. Annotation + * BeforeParam does not work either because it runs before parameterization and hence does not + * have access to the test parameters (which is weird). + * + * This *hack* checks if the current instance of test cluster configuration has the passed + * parameterized configs. In such a case, we can just reuse the cluster for test and do not need + * to initialize from scratch. While this is a hack, it saves a ton of time for the full + * test and de-flakes it. + */ + private static boolean isSameParameterizedCluster(Class registryImpl, int numHedgedReqs) { + if (TEST_UTIL == null) { + return false; + } + Configuration conf = TEST_UTIL.getConfiguration(); + Class confClass = conf.getClass( + HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, ZKConnectionRegistry.class); + int hedgedReqConfig = conf.getInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, + HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_DEFAULT); + return confClass.getName().equals(registryImpl.getName()) && numHedgedReqs == hedgedReqConfig; + } - protected static final void initialize(Class... cps) throws Exception { + protected static final void initialize(Class registryImpl, int numHedgedReqs, Class... cps) + throws Exception { + // initialize() is called for every unit test, however we only want to reset the cluster state + // at the end of every parameterized run. + if (isSameParameterizedCluster(registryImpl, numHedgedReqs)) { + return; + } // Uncomment the following lines if more verbosity is needed for // debugging (see HBASE-12285 for details). // ((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL); @@ -150,22 +201,35 @@ protected static final void initialize(Class... cps) throws Exception { // ((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL); // make sure that we do not get the same ts twice, see HBASE-19731 for more details. EnvironmentEdgeManager.injectEdge(new NonRepeatedEnvironmentEdge()); + if (TEST_UTIL != null) { + // We reached end of a parameterized run, clean up. + TEST_UTIL.shutdownMiniCluster(); + } + TEST_UTIL = new HBaseTestingUtility(); Configuration conf = TEST_UTIL.getConfiguration(); conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, Arrays.stream(cps).map(Class::getName).toArray(String[]::new)); conf.setBoolean(TableDescriptorChecker.TABLE_SANITY_CHECKS, true); // enable for below tests - // We need more than one region server in this test - TEST_UTIL.startMiniCluster(SLAVES); - } - - @BeforeClass - public static void setUpBeforeClass() throws Exception { - initialize(MultiRowMutationEndpoint.class); + conf.setClass(HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, registryImpl, + ConnectionRegistry.class); + if (numHedgedReqs == 1) { + conf.setBoolean(HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY, false); + } else { + Preconditions.checkArgument(numHedgedReqs > 1); + conf.setBoolean(HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY, true); + } + conf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, numHedgedReqs); + StartMiniClusterOption.Builder builder = StartMiniClusterOption.builder(); + // Multiple masters needed only when hedged reads for master registry are enabled. + builder.numMasters(numHedgedReqs > 1 ? 3 : 1).numRegionServers(SLAVES); + TEST_UTIL.startMiniCluster(builder.build()); } @AfterClass public static void tearDownAfterClass() throws Exception { - TEST_UTIL.shutdownMiniCluster(); + if (TEST_UTIL != null) { + TEST_UTIL.shutdownMiniCluster(); + } } /** @@ -173,7 +237,7 @@ public static void tearDownAfterClass() throws Exception { */ @Test public void testDuplicateAppend() throws Exception { - HTableDescriptor hdt = TEST_UTIL.createTableDescriptor(name.getMethodName()); + HTableDescriptor hdt = TEST_UTIL.createTableDescriptor(name.getTableName()); Map kvs = new HashMap<>(); kvs.put(HConnectionTestingUtility.SleepAtFirstRpcCall.SLEEP_TIME_CONF_KEY, "2000"); hdt.addCoprocessor(HConnectionTestingUtility.SleepAtFirstRpcCall.class.getName(), null, 1, kvs); @@ -181,11 +245,11 @@ public void testDuplicateAppend() throws Exception { Configuration c = new Configuration(TEST_UTIL.getConfiguration()); c.setInt(HConstants.HBASE_CLIENT_PAUSE, 50); - // Client will retry beacuse rpc timeout is small than the sleep time of first rpc call + // Client will retry because rpc timeout is small than the sleep time of first rpc call c.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 1500); try (Connection connection = ConnectionFactory.createConnection(c)) { - try (Table t = connection.getTable(TableName.valueOf(name.getMethodName()))) { + try (Table t = connection.getTable(name.getTableName())) { if (t instanceof HTable) { HTable table = (HTable) t; table.setOperationTimeout(3 * 1000); @@ -214,7 +278,7 @@ public void testDuplicateAppend() throws Exception { */ @Test public void testKeepDeletedCells() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); final byte[] FAMILY = Bytes.toBytes("family"); final byte[] C0 = Bytes.toBytes("c0"); @@ -280,7 +344,7 @@ public void testKeepDeletedCells() throws Exception { */ @Test public void testPurgeFutureDeletes() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); final byte[] ROW = Bytes.toBytes("row"); final byte[] FAMILY = Bytes.toBytes("family"); final byte[] COLUMN = Bytes.toBytes("column"); @@ -333,7 +397,7 @@ public boolean evaluate() throws IOException { */ @Test public void testGetConfiguration() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); byte[][] FAMILIES = new byte[][] { Bytes.toBytes("foo") }; Configuration conf = TEST_UTIL.getConfiguration(); try (Table table = TEST_UTIL.createTable(tableName, FAMILIES)) { @@ -347,7 +411,7 @@ public void testGetConfiguration() throws Exception { */ @Test public void testWeirdCacheBehaviour() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); byte [][] FAMILIES = new byte[][] { Bytes.toBytes("trans-blob"), Bytes.toBytes("trans-type"), Bytes.toBytes("trans-date"), Bytes.toBytes("trans-tags"), Bytes.toBytes("trans-group") }; @@ -388,7 +452,7 @@ public void testWeirdCacheBehaviour() throws Exception { } private void deleteColumns(Table ht, String value, String keyPrefix) - throws IOException { + throws IOException { ResultScanner scanner = buildScanner(keyPrefix, value, ht); Iterator it = scanner.iterator(); int count = 0; @@ -473,8 +537,8 @@ private void putRows(Table ht, int numRows, String value, String key) */ @Test public void testFilterAcrossMultipleRegions() - throws IOException, InterruptedException { - final TableName tableName = TableName.valueOf(name.getMethodName()); + throws IOException, InterruptedException { + final TableName tableName = name.getTableName(); try (Table t = TEST_UTIL.createTable(tableName, FAMILY)) { int rowCount = TEST_UTIL.loadTable(t, FAMILY, false); assertRowCount(t, rowCount); @@ -563,8 +627,7 @@ private void assertRowCount(final Table t, final int expected) * @return Map of regions to servers. * @throws IOException */ - private List splitTable(final Table t) - throws IOException, InterruptedException { + private List splitTable(final Table t) throws IOException { // Split this table in two. try (Admin admin = TEST_UTIL.getAdmin()) { admin.split(t.getName()); @@ -580,8 +643,7 @@ private List splitTable(final Table t) * @param t * @return Map of table regions; caller needs to check table actually split. */ - private List waitOnSplit(final Table t) - throws IOException { + private List waitOnSplit(final Table t) throws IOException { try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(t.getName())) { List regions = locator.getAllRegionLocations(); int originalCount = regions.size(); @@ -593,8 +655,9 @@ private List waitOnSplit(final Table t) e.printStackTrace(); } regions = locator.getAllRegionLocations(); - if (regions.size() > originalCount) + if (regions.size() > originalCount) { break; + } } return regions; } @@ -602,7 +665,7 @@ private List waitOnSplit(final Table t) @Test public void testSuperSimple() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { Put put = new Put(ROW); put.addColumn(FAMILY, QUALIFIER, VALUE); @@ -618,7 +681,7 @@ public void testSuperSimple() throws Exception { @Test public void testMaxKeyValueSize() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); Configuration conf = TEST_UTIL.getConfiguration(); String oldMaxSize = conf.get(ConnectionConfiguration.MAX_KEYVALUE_SIZE_KEY); try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { @@ -648,7 +711,7 @@ public void testMaxKeyValueSize() throws Exception { @Test public void testFilters() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { byte[][] ROWS = makeN(ROW, 10); byte[][] QUALIFIERS = { @@ -685,7 +748,7 @@ public void testFilters() throws Exception { @Test public void testFilterWithLongCompartor() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { byte[][] ROWS = makeN(ROW, 10); byte[][] values = new byte[10][]; @@ -717,7 +780,7 @@ public void testFilterWithLongCompartor() throws Exception { @Test public void testKeyOnlyFilter() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { byte[][] ROWS = makeN(ROW, 10); byte[][] QUALIFIERS = { @@ -755,7 +818,7 @@ public void testKeyOnlyFilter() throws Exception { */ @Test public void testSimpleMissing() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { byte[][] ROWS = makeN(ROW, 4); @@ -866,7 +929,7 @@ public void testSimpleMissing() throws Exception { */ @Test public void testSingleRowMultipleFamily() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); byte [][] ROWS = makeN(ROW, 3); byte [][] FAMILIES = makeNAscii(FAMILY, 10); byte [][] QUALIFIERS = makeN(QUALIFIER, 10); @@ -1174,7 +1237,7 @@ public void testNullTableName() throws IOException { @Test(expected = IllegalArgumentException.class) public void testNullFamilyName() throws IOException { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); // Null family (should NOT work) TEST_UTIL.createTable(tableName, new byte[][]{null}); @@ -1183,7 +1246,7 @@ public void testNullFamilyName() throws IOException { @Test public void testNullRowAndQualifier() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { @@ -1219,7 +1282,7 @@ public void testNullRowAndQualifier() throws Exception { @Test public void testNullEmptyQualifier() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { @@ -1257,7 +1320,7 @@ public void testNullEmptyQualifier() throws Exception { @Test public void testNullValue() throws IOException { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { // Null value @@ -1292,7 +1355,7 @@ public void testNullValue() throws IOException { @Test public void testNullQualifier() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) { // Work for Put @@ -1350,7 +1413,7 @@ public void testNullQualifier() throws Exception { @Test public void testVersions() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); long [] STAMPS = makeStamps(20); byte [][] VALUES = makeNAscii(VALUE, 20); @@ -1577,7 +1640,7 @@ public void testVersions() throws Exception { @Test @SuppressWarnings("checkstyle:MethodLength") public void testVersionLimits() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); byte [][] FAMILIES = makeNAscii(FAMILY, 3); int [] LIMITS = {1,3,5}; long [] STAMPS = makeStamps(10); @@ -1772,7 +1835,7 @@ public void testVersionLimits() throws Exception { @Test public void testDeleteFamilyVersion() throws Exception { try (Admin admin = TEST_UTIL.getAdmin()) { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); byte[][] QUALIFIERS = makeNAscii(QUALIFIER, 1); byte[][] VALUES = makeN(VALUE, 5); @@ -1812,7 +1875,7 @@ public void testDeleteFamilyVersion() throws Exception { @Test public void testDeleteFamilyVersionWithOtherDeletes() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); byte [][] QUALIFIERS = makeNAscii(QUALIFIER, 5); byte [][] VALUES = makeN(VALUE, 5); @@ -1930,7 +1993,7 @@ public void testDeleteFamilyVersionWithOtherDeletes() throws Exception { @Test public void testDeleteWithFailed() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); byte [][] FAMILIES = makeNAscii(FAMILY, 3); byte [][] VALUES = makeN(VALUE, 5); @@ -1956,7 +2019,7 @@ public void testDeleteWithFailed() throws Exception { @Test public void testDeletes() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); byte [][] ROWS = makeNAscii(ROW, 6); byte [][] FAMILIES = makeNAscii(FAMILY, 3); @@ -2262,7 +2325,7 @@ public void testDeletes() throws Exception { */ @Test public void testBatchOperationsWithErrors() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try (Table foo = TEST_UTIL.createTable(tableName, new byte[][] {FAMILY}, 10)) { int NUM_OPS = 100; @@ -2436,7 +2499,7 @@ public void testJiraTest867() throws Exception { int numRows = 10; int numColsPerRow = 2000; - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); byte [][] ROWS = makeN(ROW, numRows); byte [][] QUALIFIERS = makeN(QUALIFIER, numColsPerRow); @@ -2519,7 +2582,7 @@ public void testJiraTest867() throws Exception { */ @Test public void testJiraTest861() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); byte [][] VALUES = makeNAscii(VALUE, 7); long [] STAMPS = makeStamps(7); @@ -2582,7 +2645,7 @@ public void testJiraTest861() throws Exception { */ @Test public void testJiraTest33() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); byte [][] VALUES = makeNAscii(VALUE, 7); long [] STAMPS = makeStamps(7); @@ -2630,7 +2693,7 @@ public void testJiraTest33() throws Exception { */ @Test public void testJiraTest1014() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try (Table ht = TEST_UTIL.createTable(tableName, FAMILY, 10)) { @@ -2654,7 +2717,7 @@ public void testJiraTest1014() throws Exception { */ @Test public void testJiraTest1182() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); byte [][] VALUES = makeNAscii(VALUE, 7); long [] STAMPS = makeStamps(7); @@ -2698,7 +2761,7 @@ public void testJiraTest1182() throws Exception { */ @Test public void testJiraTest52() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); byte [][] VALUES = makeNAscii(VALUE, 7); long [] STAMPS = makeStamps(7); @@ -2734,8 +2797,7 @@ public void testJiraTest52() throws Exception { private void getVersionRangeAndVerifyGreaterThan(Table ht, byte [] row, byte [] family, byte [] qualifier, long [] stamps, byte [][] values, - int start, int end) - throws IOException { + int start, int end) throws IOException { Get get = new Get(row); get.addColumn(family, qualifier); get.setMaxVersions(Integer.MAX_VALUE); @@ -2745,8 +2807,7 @@ private void getVersionRangeAndVerifyGreaterThan(Table ht, byte [] row, } private void getVersionRangeAndVerify(Table ht, byte [] row, byte [] family, - byte [] qualifier, long [] stamps, byte [][] values, int start, int end) - throws IOException { + byte [] qualifier, long [] stamps, byte [][] values, int start, int end) throws IOException { Get get = new Get(row); get.addColumn(family, qualifier); get.setMaxVersions(Integer.MAX_VALUE); @@ -2756,8 +2817,7 @@ private void getVersionRangeAndVerify(Table ht, byte [] row, byte [] family, } private void getAllVersionsAndVerify(Table ht, byte [] row, byte [] family, - byte [] qualifier, long [] stamps, byte [][] values, int start, int end) - throws IOException { + byte [] qualifier, long [] stamps, byte [][] values, int start, int end) throws IOException { Get get = new Get(row); get.addColumn(family, qualifier); get.setMaxVersions(Integer.MAX_VALUE); @@ -2767,8 +2827,7 @@ private void getAllVersionsAndVerify(Table ht, byte [] row, byte [] family, private void scanVersionRangeAndVerifyGreaterThan(Table ht, byte [] row, byte [] family, byte [] qualifier, long [] stamps, byte [][] values, - int start, int end) - throws IOException { + int start, int end) throws IOException { Scan scan = new Scan(row); scan.addColumn(family, qualifier); scan.setMaxVersions(Integer.MAX_VALUE); @@ -2778,8 +2837,7 @@ private void scanVersionRangeAndVerifyGreaterThan(Table ht, byte [] row, } private void scanVersionRangeAndVerify(Table ht, byte [] row, byte [] family, - byte [] qualifier, long [] stamps, byte [][] values, int start, int end) - throws IOException { + byte [] qualifier, long [] stamps, byte [][] values, int start, int end) throws IOException { Scan scan = new Scan(row); scan.addColumn(family, qualifier); scan.setMaxVersions(Integer.MAX_VALUE); @@ -2789,8 +2847,7 @@ private void scanVersionRangeAndVerify(Table ht, byte [] row, byte [] family, } private void scanAllVersionsAndVerify(Table ht, byte [] row, byte [] family, - byte [] qualifier, long [] stamps, byte [][] values, int start, int end) - throws IOException { + byte [] qualifier, long [] stamps, byte [][] values, int start, int end) throws IOException { Scan scan = new Scan(row); scan.addColumn(family, qualifier); scan.setMaxVersions(Integer.MAX_VALUE); @@ -2799,8 +2856,7 @@ private void scanAllVersionsAndVerify(Table ht, byte [] row, byte [] family, } private void getVersionAndVerify(Table ht, byte [] row, byte [] family, - byte [] qualifier, long stamp, byte [] value) - throws Exception { + byte [] qualifier, long stamp, byte [] value) throws Exception { Get get = new Get(row); get.addColumn(family, qualifier); get.setTimestamp(stamp); @@ -2810,8 +2866,7 @@ private void getVersionAndVerify(Table ht, byte [] row, byte [] family, } private void getVersionAndVerifyMissing(Table ht, byte [] row, byte [] family, - byte [] qualifier, long stamp) - throws Exception { + byte [] qualifier, long stamp) throws Exception { Get get = new Get(row); get.addColumn(family, qualifier); get.setTimestamp(stamp); @@ -2821,8 +2876,7 @@ private void getVersionAndVerifyMissing(Table ht, byte [] row, byte [] family, } private void scanVersionAndVerify(Table ht, byte [] row, byte [] family, - byte [] qualifier, long stamp, byte [] value) - throws Exception { + byte [] qualifier, long stamp, byte [] value) throws Exception { Scan scan = new Scan(row); scan.addColumn(family, qualifier); scan.setTimestamp(stamp); @@ -2832,8 +2886,7 @@ private void scanVersionAndVerify(Table ht, byte [] row, byte [] family, } private void scanVersionAndVerifyMissing(Table ht, byte [] row, - byte [] family, byte [] qualifier, long stamp) - throws Exception { + byte [] family, byte [] qualifier, long stamp) throws Exception { Scan scan = new Scan(row); scan.addColumn(family, qualifier); scan.setTimestamp(stamp); @@ -2842,10 +2895,7 @@ private void scanVersionAndVerifyMissing(Table ht, byte [] row, assertNullResult(result); } - private void getTestNull(Table ht, byte [] row, byte [] family, - byte [] value) - throws Exception { - + private void getTestNull(Table ht, byte [] row, byte [] family, byte [] value) throws Exception { Get get = new Get(row); get.addColumn(family, null); Result result = ht.get(get); @@ -2922,9 +2972,7 @@ private void scanTestNull(Table ht, byte[] row, byte[] family, byte[] value, } private void singleRowGetTest(Table ht, byte [][] ROWS, byte [][] FAMILIES, - byte [][] QUALIFIERS, byte [][] VALUES) - throws Exception { - + byte [][] QUALIFIERS, byte [][] VALUES) throws Exception { // Single column from memstore Get get = new Get(ROWS[0]); get.addColumn(FAMILIES[4], QUALIFIERS[0]); @@ -2979,7 +3027,7 @@ private void singleRowGetTest(Table ht, byte [][] ROWS, byte [][] FAMILIES, assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES, new int [][] { {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7} - }); + }); // Multiple columns from everywhere storefile, many family, wildcard get = new Get(ROWS[0]); @@ -2995,7 +3043,7 @@ private void singleRowGetTest(Table ht, byte [][] ROWS, byte [][] FAMILIES, assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES, new int [][] { {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7} - }); + }); // Everything get = new Get(ROWS[0]); @@ -3003,7 +3051,7 @@ private void singleRowGetTest(Table ht, byte [][] ROWS, byte [][] FAMILIES, assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES, new int [][] { {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}, {9, 0, 0} - }); + }); // Get around inserted columns @@ -3020,9 +3068,7 @@ private void singleRowGetTest(Table ht, byte [][] ROWS, byte [][] FAMILIES, } private void singleRowScanTest(Table ht, byte [][] ROWS, byte [][] FAMILIES, - byte [][] QUALIFIERS, byte [][] VALUES) - throws Exception { - + byte [][] QUALIFIERS, byte [][] VALUES) throws Exception { // Single column from memstore Scan scan = new Scan(); scan.addColumn(FAMILIES[4], QUALIFIERS[0]); @@ -3077,7 +3123,7 @@ private void singleRowScanTest(Table ht, byte [][] ROWS, byte [][] FAMILIES, assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES, new int [][] { {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7} - }); + }); // Multiple columns from everywhere storefile, many family, wildcard scan = new Scan(); @@ -3093,7 +3139,7 @@ private void singleRowScanTest(Table ht, byte [][] ROWS, byte [][] FAMILIES, assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES, new int [][] { {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7} - }); + }); // Everything scan = new Scan(); @@ -3101,7 +3147,7 @@ private void singleRowScanTest(Table ht, byte [][] ROWS, byte [][] FAMILIES, assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES, new int [][] { {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}, {9, 0, 0} - }); + }); // Scan around inserted columns @@ -3121,13 +3167,9 @@ private void singleRowScanTest(Table ht, byte [][] ROWS, byte [][] FAMILIES, * Expects family and qualifier arrays to be valid for at least * the range: idx-2 < idx < idx+2 */ - private void getVerifySingleColumn(Table ht, - byte [][] ROWS, int ROWIDX, - byte [][] FAMILIES, int FAMILYIDX, - byte [][] QUALIFIERS, int QUALIFIERIDX, - byte [][] VALUES, int VALUEIDX) - throws Exception { - + private void getVerifySingleColumn(Table ht, byte [][] ROWS, int ROWIDX, byte [][] FAMILIES, + int FAMILYIDX, byte [][] QUALIFIERS, int QUALIFIERIDX, byte [][] VALUES, int VALUEIDX) + throws Exception { Get get = new Get(ROWS[ROWIDX]); Result result = ht.get(get); assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], @@ -3179,13 +3221,9 @@ private void getVerifySingleColumn(Table ht, * the range: idx-2 to idx+2 * Expects row array to be valid for at least idx to idx+2 */ - private void scanVerifySingleColumn(Table ht, - byte [][] ROWS, int ROWIDX, - byte [][] FAMILIES, int FAMILYIDX, - byte [][] QUALIFIERS, int QUALIFIERIDX, - byte [][] VALUES, int VALUEIDX) - throws Exception { - + private void scanVerifySingleColumn(Table ht, byte [][] ROWS, int ROWIDX, byte [][] FAMILIES, + int FAMILYIDX, byte [][] QUALIFIERS, int QUALIFIERIDX, byte [][] VALUES, int VALUEIDX) + throws Exception { Scan scan = new Scan(); Result result = getSingleScanResult(ht, scan); assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], @@ -3239,12 +3277,8 @@ private void scanVerifySingleColumn(Table ht, * Verify we do not read any values by accident around a single column * Same requirements as getVerifySingleColumn */ - private void getVerifySingleEmpty(Table ht, - byte [][] ROWS, int ROWIDX, - byte [][] FAMILIES, int FAMILYIDX, - byte [][] QUALIFIERS, int QUALIFIERIDX) - throws Exception { - + private void getVerifySingleEmpty(Table ht, byte [][] ROWS, int ROWIDX, byte [][] FAMILIES, + int FAMILYIDX, byte [][] QUALIFIERS, int QUALIFIERIDX) throws Exception { Get get = new Get(ROWS[ROWIDX]); get.addFamily(FAMILIES[4]); get.addColumn(FAMILIES[4], QUALIFIERS[1]); @@ -3270,12 +3304,8 @@ private void getVerifySingleEmpty(Table ht, } - private void scanVerifySingleEmpty(Table ht, - byte [][] ROWS, int ROWIDX, - byte [][] FAMILIES, int FAMILYIDX, - byte [][] QUALIFIERS, int QUALIFIERIDX) - throws Exception { - + private void scanVerifySingleEmpty(Table ht, byte [][] ROWS, int ROWIDX, byte [][] FAMILIES, + int FAMILYIDX, byte [][] QUALIFIERS, int QUALIFIERIDX) throws Exception { Scan scan = new Scan(ROWS[ROWIDX+1]); Result result = getSingleScanResult(ht, scan); assertNullResult(result); @@ -3300,9 +3330,7 @@ private void scanVerifySingleEmpty(Table ht, // Verifiers // - private void assertKey(Cell key, byte [] row, byte [] family, - byte [] qualifier, byte [] value) - throws Exception { + private void assertKey(Cell key, byte [] row, byte [] family, byte [] qualifier, byte [] value) { assertTrue("Expected row [" + Bytes.toString(row) + "] " + "Got row [" + Bytes.toString(CellUtil.cloneRow(key)) +"]", equals(row, CellUtil.cloneRow(key))); @@ -3318,8 +3346,7 @@ private void assertKey(Cell key, byte [] row, byte [] family, } static void assertIncrementKey(Cell key, byte [] row, byte [] family, - byte [] qualifier, long value) - throws Exception { + byte [] qualifier, long value) { assertTrue("Expected row [" + Bytes.toString(row) + "] " + "Got row [" + Bytes.toString(CellUtil.cloneRow(key)) +"]", equals(row, CellUtil.cloneRow(key))); @@ -3340,9 +3367,7 @@ private void assertNumKeys(Result result, int n) throws Exception { } private void assertNResult(Result result, byte [] row, - byte [][] families, byte [][] qualifiers, byte [][] values, - int [][] idxs) - throws Exception { + byte [][] families, byte [][] qualifiers, byte [][] values, int [][] idxs) { assertTrue("Expected row [" + Bytes.toString(row) + "] " + "Got row [" + Bytes.toString(result.getRow()) +"]", equals(row, result.getRow())); @@ -3374,8 +3399,7 @@ private void assertNResult(Result result, byte [] row, private void assertNResult(Result result, byte [] row, byte [] family, byte [] qualifier, long [] stamps, byte [][] values, - int start, int end) - throws IOException { + int start, int end) { assertTrue("Expected row [" + Bytes.toString(row) + "] " + "Got row [" + Bytes.toString(result.getRow()) +"]", equals(row, result.getRow())); @@ -3409,8 +3433,7 @@ private void assertNResult(Result result, byte [] row, */ private void assertDoubleResult(Result result, byte [] row, byte [] familyA, byte [] qualifierA, byte [] valueA, - byte [] familyB, byte [] qualifierB, byte [] valueB) - throws Exception { + byte [] familyB, byte [] qualifierB, byte [] valueB) { assertTrue("Expected row [" + Bytes.toString(row) + "] " + "Got row [" + Bytes.toString(result.getRow()) +"]", equals(row, result.getRow())); @@ -3440,8 +3463,7 @@ private void assertDoubleResult(Result result, byte [] row, } private void assertSingleResult(Result result, byte [] row, byte [] family, - byte [] qualifier, byte [] value) - throws Exception { + byte [] qualifier, byte [] value) { assertTrue("Expected row [" + Bytes.toString(row) + "] " + "Got row [" + Bytes.toString(result.getRow()) +"]", equals(row, result.getRow())); @@ -3479,8 +3501,7 @@ private void assertSingleResult(Result result, byte[] row, byte[] family, byte[] } private void assertSingleResult(Result result, byte [] row, byte [] family, - byte [] qualifier, long ts, byte [] value) - throws Exception { + byte [] qualifier, long ts, byte [] value) { assertTrue("Expected row [" + Bytes.toString(row) + "] " + "Got row [" + Bytes.toString(result.getRow()) +"]", equals(row, result.getRow())); @@ -3571,7 +3592,7 @@ static boolean equals(byte [] left, byte [] right) { @Test public void testDuplicateVersions() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); long [] STAMPS = makeStamps(20); byte [][] VALUES = makeNAscii(VALUE, 20); @@ -3794,7 +3815,7 @@ public void testDuplicateVersions() throws Exception { @Test public void testUpdates() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try (Table hTable = TEST_UTIL.createTable(tableName, FAMILY, 10)) { // Write a column with values at timestamp 1, 2 and 3 @@ -3844,7 +3865,7 @@ public void testUpdates() throws Exception { @Test public void testUpdatesWithMajorCompaction() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try (Table hTable = TEST_UTIL.createTable(tableName, FAMILY, 10); Admin admin = TEST_UTIL.getAdmin()) { @@ -3905,7 +3926,7 @@ public void testUpdatesWithMajorCompaction() throws Exception { @Test public void testMajorCompactionBetweenTwoUpdates() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try (Table hTable = TEST_UTIL.createTable(tableName, FAMILY, 10); Admin admin = TEST_UTIL.getAdmin()) { @@ -3972,7 +3993,7 @@ public void testMajorCompactionBetweenTwoUpdates() throws Exception { @Test public void testGet_EmptyTable() throws IOException { - try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) { + try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) { Get get = new Get(ROW); get.addFamily(FAMILY); Result r = table.get(get); @@ -3982,7 +4003,7 @@ public void testGet_EmptyTable() throws IOException { @Test public void testGet_NullQualifier() throws IOException { - try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) { + try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) { Put put = new Put(ROW); put.addColumn(FAMILY, QUALIFIER, VALUE); table.put(put); @@ -4006,7 +4027,7 @@ public void testGet_NullQualifier() throws IOException { @Test public void testGet_NonExistentRow() throws IOException { - try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) { + try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) { Put put = new Put(ROW); put.addColumn(FAMILY, QUALIFIER, VALUE); table.put(put); @@ -4034,7 +4055,7 @@ public void testPut() throws IOException { final byte [] row1 = Bytes.toBytes("row1"); final byte [] row2 = Bytes.toBytes("row2"); final byte [] value = Bytes.toBytes("abcd"); - try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), + try (Table table = TEST_UTIL.createTable(name.getTableName(), new byte[][] { CONTENTS_FAMILY, SMALL_FAMILY })) { Put put = new Put(row1); put.addColumn(CONTENTS_FAMILY, null, value); @@ -4073,7 +4094,7 @@ public void testPut() throws IOException { public void testPutNoCF() throws IOException { final byte[] BAD_FAM = Bytes.toBytes("BAD_CF"); final byte[] VAL = Bytes.toBytes(100); - try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) { + try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) { boolean caughtNSCFE = false; try { @@ -4093,7 +4114,7 @@ public void testRowsPut() throws IOException { final byte[] SMALL_FAMILY = Bytes.toBytes("smallfam"); final int NB_BATCH_ROWS = 10; final byte[] value = Bytes.toBytes("abcd"); - try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), + try (Table table = TEST_UTIL.createTable(name.getTableName(), new byte[][] {CONTENTS_FAMILY, SMALL_FAMILY })) { ArrayList rowsUpdate = new ArrayList(); for (int i = 0; i < NB_BATCH_ROWS; i++) { @@ -4123,7 +4144,7 @@ public void testRowsPutBufferedManyManyFlushes() throws IOException { final byte[] SMALL_FAMILY = Bytes.toBytes("smallfam"); final byte[] value = Bytes.toBytes("abcd"); final int NB_BATCH_ROWS = 10; - try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), + try (Table table = TEST_UTIL.createTable(name.getTableName(), new byte[][] { CONTENTS_FAMILY, SMALL_FAMILY })) { ArrayList rowsUpdate = new ArrayList(); for (int i = 0; i < NB_BATCH_ROWS * 10; i++) { @@ -4186,7 +4207,7 @@ public void testHBase737 () throws IOException { final byte [] FAM1 = Bytes.toBytes("fam1"); final byte [] FAM2 = Bytes.toBytes("fam2"); // Open table - try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), + try (Table table = TEST_UTIL.createTable(name.getTableName(), new byte [][] {FAM1, FAM2})) { // Insert some values Put put = new Put(ROW); @@ -4269,9 +4290,10 @@ public void testHBase737 () throws IOException { @Test public void testListTables() throws IOException, InterruptedException { - final TableName tableName1 = TableName.valueOf(name.getMethodName() + "1"); - final TableName tableName2 = TableName.valueOf(name.getMethodName() + "2"); - final TableName tableName3 = TableName.valueOf(name.getMethodName() + "3"); + final String testTableName = name.getTableName().toString(); + final TableName tableName1 = TableName.valueOf(testTableName + "1"); + final TableName tableName2 = TableName.valueOf(testTableName + "2"); + final TableName tableName3 = TableName.valueOf(testTableName + "3"); TableName [] tables = new TableName[] { tableName1, tableName2, tableName3 }; for (int i = 0; i < tables.length; i++) { TEST_UTIL.createTable(tables[i], FAMILY); @@ -4300,7 +4322,7 @@ public void testListTables() throws IOException, InterruptedException { */ @Test public void testUnmanagedHConnection() throws IOException { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY); try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()); Table t = conn.getTable(tableName); @@ -4316,7 +4338,13 @@ public void testUnmanagedHConnection() throws IOException { */ @Test public void testUnmanagedHConnectionReconnect() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + Configuration conf = TEST_UTIL.getConfiguration(); + Class registryImpl = conf.getClass( + HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, ZKConnectionRegistry.class); + // This test does not make sense for MasterRegistry since it stops the only master in the + // cluster and starts a new master without populating the underlying config for the connection. + Assume.assumeFalse(registryImpl.equals(MasterRegistry.class)); + final TableName tableName = name.getTableName(); TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY); try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) { try (Table t = conn.getTable(tableName); @@ -4347,8 +4375,9 @@ public void testUnmanagedHConnectionReconnect() throws Exception { @Test public void testMiscHTableStuff() throws IOException { - final TableName tableAname = TableName.valueOf(name.getMethodName() + "A"); - final TableName tableBname = TableName.valueOf(name.getMethodName() + "B"); + final String testTableName = name.getTableName().toString(); + final TableName tableAname = TableName.valueOf(testTableName + "A"); + final TableName tableBname = TableName.valueOf(testTableName + "B"); final byte[] attrName = Bytes.toBytes("TESTATTR"); final byte[] attrValue = Bytes.toBytes("somevalue"); byte[] value = Bytes.toBytes("value"); @@ -4405,7 +4434,6 @@ public void testMiscHTableStuff() throws IOException { // enable the table admin.enableTable(tableAname); } - // Test that attribute changes were applied HTableDescriptor desc = a.getTableDescriptor(); assertEquals("wrong table descriptor returned", desc.getTableName(), tableAname); @@ -4426,7 +4454,7 @@ public void testMiscHTableStuff() throws IOException { @Test public void testGetClosestRowBefore() throws IOException, InterruptedException { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); final byte[] firstRow = Bytes.toBytes("row111"); final byte[] secondRow = Bytes.toBytes("row222"); final byte[] thirdRow = Bytes.toBytes("row333"); @@ -4553,7 +4581,7 @@ public void testScanVariableReuse() throws Exception { @Test public void testMultiRowMutation() throws Exception { LOG.info("Starting testMultiRowMutation"); - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); final byte [] ROW1 = Bytes.toBytes("testRow1"); try (Table t = TEST_UTIL.createTable(tableName, FAMILY)) { @@ -4585,7 +4613,7 @@ public void testMultiRowMutation() throws Exception { @Test public void testRowMutation() throws Exception { LOG.info("Starting testRowMutation"); - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try (Table t = TEST_UTIL.createTable(tableName, FAMILY)) { byte[][] QUALIFIERS = new byte[][]{ Bytes.toBytes("a"), Bytes.toBytes("b") @@ -4635,7 +4663,7 @@ public void testRowMutation() throws Exception { @Test public void testBatchAppendWithReturnResultFalse() throws Exception { LOG.info("Starting testBatchAppendWithReturnResultFalse"); - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) { Append append1 = new Append(Bytes.toBytes("row1")); append1.setReturnResults(false); @@ -4659,7 +4687,7 @@ public void testBatchAppendWithReturnResultFalse() throws Exception { @Test public void testAppend() throws Exception { LOG.info("Starting testAppend"); - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try (Table t = TEST_UTIL.createTable(tableName, FAMILY)) { byte[] v1 = Bytes.toBytes("42"); byte[] v2 = Bytes.toBytes("23"); @@ -4754,7 +4782,7 @@ public void testAppendWithoutWAL() throws Exception { @Test public void testClientPoolRoundRobin() throws IOException { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); int poolSize = 3; int numVersions = poolSize * 2; @@ -4792,7 +4820,7 @@ public void testClientPoolRoundRobin() throws IOException { @Ignore ("Flakey: HBASE-8989") @Test public void testClientPoolThreadLocal() throws IOException { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); int poolSize = Integer.MAX_VALUE; int numVersions = 3; @@ -4878,7 +4906,7 @@ public void testCheckAndPut() throws IOException { final byte [] anotherrow = Bytes.toBytes("anotherrow"); final byte [] value2 = Bytes.toBytes("abcd"); - try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) { + try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) { Put put1 = new Put(ROW); put1.addColumn(FAMILY, QUALIFIER, VALUE); @@ -4916,7 +4944,7 @@ public void testCheckAndPut() throws IOException { @Test public void testCheckAndMutateWithTimeRange() throws IOException { - try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) { + try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) { final long ts = System.currentTimeMillis() / 2; Put put = new Put(ROW); put.addColumn(FAMILY, QUALIFIER, ts, VALUE); @@ -5012,7 +5040,7 @@ public void testCheckAndPutWithCompareOp() throws IOException { final byte [] value3 = Bytes.toBytes("cccc"); final byte [] value4 = Bytes.toBytes("dddd"); - try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) { + try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) { Put put2 = new Put(ROW); put2.addColumn(FAMILY, QUALIFIER, value2); @@ -5094,7 +5122,7 @@ public void testCheckAndPutWithCompareOp() throws IOException { public void testCheckAndDelete() throws IOException { final byte [] value1 = Bytes.toBytes("aaaa"); - try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), + try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) { Put put = new Put(ROW); @@ -5117,7 +5145,7 @@ public void testCheckAndDeleteWithCompareOp() throws IOException { final byte [] value3 = Bytes.toBytes("cccc"); final byte [] value4 = Bytes.toBytes("dddd"); - try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), + try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) { Put put2 = new Put(ROW); @@ -5207,9 +5235,9 @@ public void testCheckAndDeleteWithCompareOp() throws IOException { * Test ScanMetrics */ @Test - @SuppressWarnings ("unused") + @SuppressWarnings({"unused", "checkstyle:EmptyBlock"}) public void testScanMetrics() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); // Set up test table: // Create table: @@ -5261,11 +5289,10 @@ public void testScanMetrics() throws Exception { // the end of the scanner. So this is asking for 2 of the 3 rows we inserted. for (Result result : scanner.next(numRecords - 1)) { } + ScanMetrics scanMetrics = scanner.getScanMetrics(); + assertEquals("Did not access all the regions in the table", numOfRegions, + scanMetrics.countOfRegions.get()); } - ScanMetrics scanMetrics = scan2.getScanMetrics(); - assertEquals("Did not access all the regions in the table", numOfRegions, - scanMetrics.countOfRegions.get()); - // check byte counters scan2 = new Scan(); scan2.setScanMetricsEnabled(true); @@ -5277,7 +5304,7 @@ public void testScanMetrics() throws Exception { numBytes += PrivateCellUtil.estimatedSerializedSizeOf(cell); } } - scanMetrics = scanner.getScanMetrics(); + ScanMetrics scanMetrics = scanner.getScanMetrics(); assertEquals("Did not count the result bytes", numBytes, scanMetrics.countOfBytesInResults.get()); } @@ -5294,7 +5321,7 @@ public void testScanMetrics() throws Exception { numBytes += PrivateCellUtil.estimatedSerializedSizeOf(cell); } } - scanMetrics = scanner.getScanMetrics(); + ScanMetrics scanMetrics = scanner.getScanMetrics(); assertEquals("Did not count the result bytes", numBytes, scanMetrics.countOfBytesInResults.get()); } @@ -5347,7 +5374,7 @@ private ScanMetrics getScanMetrics(Scan scan) throws Exception { */ @Test public void testCacheOnWriteEvictOnClose() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); byte [] data = Bytes.toBytes("data"); try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) { try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) { @@ -5473,7 +5500,7 @@ private void waitForStoreFileCount(HStore store, int count, int timeout) */ public void testNonCachedGetRegionLocation() throws Exception { // Test Initialization. - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); byte [] family1 = Bytes.toBytes("f1"); byte [] family2 = Bytes.toBytes("f2"); try (Table table = TEST_UTIL.createTable(tableName, new byte[][] {family1, family2}, 10); @@ -5522,7 +5549,7 @@ public void testGetRegionsInRange() throws Exception { // Test Initialization. byte [] startKey = Bytes.toBytes("ddc"); byte [] endKey = Bytes.toBytes("mmm"); - TableName tableName = TableName.valueOf(name.getMethodName()); + TableName tableName = name.getTableName(); TEST_UTIL.createMultiRegionTable(tableName, new byte[][] { FAMILY }, 10); int numOfRegions = -1; @@ -5592,7 +5619,7 @@ private List getRegionsInRange(TableName tableName, byte[] star @Test public void testJira6912() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try (Table foo = TEST_UTIL.createTable(tableName, new byte[][] {FAMILY}, 10)) { List puts = new ArrayList(); @@ -5621,7 +5648,7 @@ public void testJira6912() throws Exception { @Test public void testScan_NullQualifier() throws IOException { - try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) { + try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) { Put put = new Put(ROW); put.addColumn(FAMILY, QUALIFIER, VALUE); table.put(put); @@ -5651,7 +5678,7 @@ public void testScan_NullQualifier() throws IOException { @Test public void testNegativeTimestamp() throws IOException { - try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) { + try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) { try { Put put = new Put(ROW, -1); @@ -5712,7 +5739,7 @@ public void testNegativeTimestamp() throws IOException { @Test public void testRawScanRespectsVersions() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) { byte[] row = Bytes.toBytes("row"); @@ -5786,7 +5813,7 @@ public void testRawScanRespectsVersions() throws Exception { @Test public void testEmptyFilterList() throws Exception { // Test Initialization. - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) { // Insert one row each region @@ -5826,7 +5853,7 @@ public void testEmptyFilterList() throws Exception { @Test public void testSmallScan() throws Exception { // Test Initialization. - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) { // Insert one row each region @@ -5864,7 +5891,7 @@ public void testSmallScan() throws Exception { @Test public void testSuperSimpleWithReverseScan() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { Put put = new Put(Bytes.toBytes("0-b11111-0000000000000000000")); put.addColumn(FAMILY, QUALIFIER, VALUE); @@ -5909,7 +5936,7 @@ public void testSuperSimpleWithReverseScan() throws Exception { @Test public void testFiltersWithReverseScan() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { byte[][] ROWS = makeN(ROW, 10); byte[][] QUALIFIERS = {Bytes.toBytes("col0--"), @@ -5952,7 +5979,7 @@ public void testFiltersWithReverseScan() throws Exception { @Test public void testKeyOnlyFilterWithReverseScan() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { byte[][] ROWS = makeN(ROW, 10); byte[][] QUALIFIERS = {Bytes.toBytes("col0--"), @@ -5993,7 +6020,7 @@ public void testKeyOnlyFilterWithReverseScan() throws Exception { */ @Test public void testSimpleMissingWithReverseScan() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { byte[][] ROWS = makeN(ROW, 4); @@ -6058,7 +6085,7 @@ public void testSimpleMissingWithReverseScan() throws Exception { @Test public void testNullWithReverseScan() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) { // Null qualifier (should work) Put put = new Put(ROW); @@ -6071,7 +6098,8 @@ public void testNullWithReverseScan() throws Exception { } // Use a new table - try (Table ht = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName() + "2"), FAMILY)) { + try (Table ht = + TEST_UTIL.createTable(TableName.valueOf(name.getTableName().toString() + "2"), FAMILY)) { // Empty qualifier, byte[0] instead of null (should work) Put put = new Put(ROW); put.addColumn(FAMILY, HConstants.EMPTY_BYTE_ARRAY, VALUE); @@ -6097,7 +6125,7 @@ public void testNullWithReverseScan() throws Exception { @Test @SuppressWarnings("checkstyle:MethodLength") public void testDeletesWithReverseScan() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); byte[][] ROWS = makeNAscii(ROW, 6); byte[][] FAMILIES = makeNAscii(FAMILY, 3); byte[][] VALUES = makeN(VALUE, 5); @@ -6283,7 +6311,7 @@ public void testDeletesWithReverseScan() throws Exception { @Test public void testReversedScanUnderMultiRegions() throws Exception { // Test Initialization. - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); byte[] maxByteArray = ConnectionUtils.MAX_BYTE_ARRAY; byte[][] splitRows = new byte[][] { Bytes.toBytes("005"), Bytes.add(Bytes.toBytes("005"), Bytes.multiple(maxByteArray, 16)), @@ -6344,7 +6372,7 @@ public void testReversedScanUnderMultiRegions() throws Exception { @Test public void testSmallReversedScanUnderMultiRegions() throws Exception { // Test Initialization. - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); byte[][] splitRows = new byte[][]{ Bytes.toBytes("000"), Bytes.toBytes("002"), Bytes.toBytes("004"), Bytes.toBytes("006"), Bytes.toBytes("008"), Bytes.toBytes("010")}; @@ -6540,7 +6568,7 @@ public void testFilterAllRecords() throws IOException { @Test public void testRegionCache() throws IOException { - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName())); + HTableDescriptor htd = new HTableDescriptor(name.getTableName()); HColumnDescriptor fam = new HColumnDescriptor(FAMILY); htd.addFamily(fam); byte[][] KEYS = HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE; @@ -6601,7 +6629,7 @@ public void testCellSizeLimit() throws IOException { @Test public void testDeleteSpecifiedVersionOfSpecifiedColumn() throws Exception { try (Admin admin = TEST_UTIL.getAdmin()) { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); byte[][] VALUES = makeN(VALUE, 5); long[] ts = {1000, 2000, 3000, 4000, 5000}; @@ -6647,7 +6675,7 @@ public void testDeleteSpecifiedVersionOfSpecifiedColumn() throws Exception { @Test public void testDeleteLatestVersionOfSpecifiedColumn() throws Exception { try (Admin admin = TEST_UTIL.getAdmin()) { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); byte[][] VALUES = makeN(VALUE, 5); long[] ts = {1000, 2000, 3000, 4000, 5000}; @@ -6710,7 +6738,7 @@ public void testDeleteLatestVersionOfSpecifiedColumn() throws Exception { @Test public void testReadWithFilter() throws Exception { try (Admin admin = TEST_UTIL.getAdmin()) { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try (Table table = TEST_UTIL.createTable(tableName, FAMILY, 3)) { byte[] VALUEA = Bytes.toBytes("value-a"); @@ -6797,7 +6825,7 @@ public void testReadWithFilter() throws Exception { @Test public void testCellUtilTypeMethods() throws IOException { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) { final byte[] row = Bytes.toBytes("p"); @@ -6851,7 +6879,7 @@ public void testCellUtilTypeMethods() throws IOException { @Test(expected = DoNotRetryIOException.class) public void testCreateTableWithZeroRegionReplicas() throws Exception { - TableName tableName = TableName.valueOf(name.getMethodName()); + TableName tableName = name.getTableName(); TableDescriptor desc = TableDescriptorBuilder.newBuilder(tableName) .setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("cf"))) .setRegionReplication(0) @@ -6862,7 +6890,7 @@ public void testCreateTableWithZeroRegionReplicas() throws Exception { @Test(expected = DoNotRetryIOException.class) public void testModifyTableWithZeroRegionReplicas() throws Exception { - TableName tableName = TableName.valueOf(name.getMethodName()); + TableName tableName = name.getTableName(); TableDescriptor desc = TableDescriptorBuilder.newBuilder(tableName) .setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("cf"))) .build(); @@ -6877,13 +6905,13 @@ public void testModifyTableWithZeroRegionReplicas() throws Exception { @Test(timeout = 60000) public void testModifyTableWithMemstoreData() throws Exception { - TableName tableName = TableName.valueOf(name.getMethodName()); + TableName tableName = name.getTableName(); createTableAndValidateTableSchemaModification(tableName, true); } @Test(timeout = 60000) public void testDeleteCFWithMemstoreData() throws Exception { - TableName tableName = TableName.valueOf(name.getMethodName()); + TableName tableName = name.getTableName(); createTableAndValidateTableSchemaModification(tableName, false); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java index 37d0135fb76f..8845f9adacb3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java @@ -17,14 +17,16 @@ */ package org.apache.hadoop.hbase.client; +import java.util.Arrays; +import java.util.Collection; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint; import org.apache.hadoop.hbase.regionserver.NoOpScanPolicyObserver; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; -import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.experimental.categories.Category; +import org.junit.runners.Parameterized; /** * Test all client operations with a coprocessor that just implements the default flush/compact/scan @@ -32,13 +34,24 @@ */ @Category({ LargeTests.class, ClientTests.class }) public class TestFromClientSideWithCoprocessor extends TestFromClientSide { - @ClassRule public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestFromClientSideWithCoprocessor.class); - @BeforeClass - public static void setUpBeforeClass() throws Exception { - initialize(MultiRowMutationEndpoint.class, NoOpScanPolicyObserver.class); + // Override the parameters from the parent class. We just want to run it for the default + // param combination. + @Parameterized.Parameters + public static Collection parameters() { + return Arrays.asList(new Object[][] { + { ZKConnectionRegistry.class, 1} + }); + } + + public TestFromClientSideWithCoprocessor(Class registry, int numHedgedReqs) throws Exception { + if (TEST_UTIL == null) { + // It is ok to initialize once because the test is parameterized for a single dimension. + initialize(registry, numHedgedReqs, NoOpScanPolicyObserver.class, + MultiRowMutationEndpoint.class); + } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java new file mode 100644 index 000000000000..335f96814e0c --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.client; + +import static org.apache.hadoop.hbase.HConstants.META_REPLICAS_NUM; +import static org.junit.Assert.assertEquals; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.StartMiniClusterOption; +import org.apache.hadoop.hbase.master.HMaster; +import org.apache.hadoop.hbase.testclassification.ClientTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ MediumTests.class, ClientTests.class }) +public class TestMasterRegistry { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMasterRegistry.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + + @BeforeClass + public static void setUp() throws Exception { + TEST_UTIL.getConfiguration().setInt(META_REPLICAS_NUM, 3); + StartMiniClusterOption.Builder builder = StartMiniClusterOption.builder(); + builder.numMasters(3).numRegionServers(3); + TEST_UTIL.startMiniCluster(builder.build()); + } + + @AfterClass + public static void tearDown() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + /** + * Generates a string of dummy master addresses in host:port format. Every other hostname won't + * have a port number. + */ + private static String generateDummyMastersList(int size) { + List masters = new ArrayList<>(); + for (int i = 0; i < size; i++) { + masters.add(" localhost" + (i % 2 == 0 ? ":" + (1000 + i) : "")); + } + return String.join(",", masters); + } + + /** + * Makes sure the master registry parses the master end points in the configuration correctly. + */ + @Test public void testMasterAddressParsing() { + Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); + int numMasters = 10; + conf.set(HConstants.MASTER_ADDRS_KEY, generateDummyMastersList(numMasters)); + try (MasterRegistry registry = new MasterRegistry(conf)) { + List parsedMasters = new ArrayList<>(registry.getParsedMasterServers()); + // Half of them would be without a port, duplicates are removed. + assertEquals(numMasters/2 + 1, parsedMasters.size()); + // Sort in the increasing order of port numbers. + Collections.sort(parsedMasters, Comparator.comparingInt(ServerName::getPort)); + for (int i = 0; i < parsedMasters.size(); i++) { + ServerName sn = parsedMasters.get(i); + assertEquals("localhost", sn.getHostname()); + if (i == parsedMasters.size() - 1) { + // Last entry should be the one with default port. + assertEquals(HConstants.DEFAULT_MASTER_PORT, sn.getPort()); + } else { + assertEquals(1000 + (2 * i), sn.getPort()); + } + } + } + } + + @Test public void testRegistryRPCs() throws Exception { + Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); + HMaster activeMaster = TEST_UTIL.getHBaseCluster().getMaster(); + for (int numHedgedReqs = 1; numHedgedReqs <=3; numHedgedReqs++) { + if (numHedgedReqs == 1) { + conf.setBoolean(HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY, false); + } else { + conf.setBoolean(HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY, true); + } + conf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, numHedgedReqs); + try (MasterRegistry registry = new MasterRegistry(conf)) { + assertEquals(registry.getClusterId().get(), activeMaster.getClusterId()); + assertEquals(registry.getActiveMaster().get(), activeMaster.getServerName()); + List metaLocations = + Arrays.asList(registry.getMetaRegionLocations().get().getRegionLocations()); + List actualMetaLocations = activeMaster.getMetaRegionLocationCache() + .getMetaRegionLocations().get(); + Collections.sort(metaLocations); + Collections.sort(actualMetaLocations); + assertEquals(actualMetaLocations, metaLocations); + } + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java index 02236a67f44a..1205b05a996c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java @@ -53,13 +53,13 @@ public class TestMetaRegionLocationCache { HBaseClassTestRule.forClass(TestMetaRegionLocationCache.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private static AsyncRegistry REGISTRY; + private static ConnectionRegistry REGISTRY; @BeforeClass public static void setUp() throws Exception { TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 3); TEST_UTIL.startMiniCluster(3); - REGISTRY = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration()); + REGISTRY = ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration()); RegionReplicaTestHelper.waitUntilAllMetaReplicasHavingRegionLocation( TEST_UTIL.getConfiguration(), REGISTRY, 3); TEST_UTIL.getAdmin().balancerSwitch(false, true); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java index ad73f89052c1..8eb155d38a7b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java @@ -27,9 +27,10 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; - import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; import java.util.List; import java.util.concurrent.TimeUnit; import java.util.function.Consumer; @@ -46,20 +47,21 @@ import org.apache.hadoop.hbase.HTestConst; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.MiniHBaseCluster; -import org.apache.hadoop.hbase.exceptions.DeserializationException; -import org.apache.hadoop.hbase.filter.FilterBase; -import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.StartMiniClusterOption; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.filter.BinaryComparator; import org.apache.hadoop.hbase.filter.ColumnPrefixFilter; import org.apache.hadoop.hbase.filter.ColumnRangeFilter; +import org.apache.hadoop.hbase.filter.FilterBase; import org.apache.hadoop.hbase.filter.QualifierFilter; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.testclassification.ClientTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; @@ -67,14 +69,18 @@ import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; /** - * A client-side test, mostly testing scanners with various parameters. + * A client-side test, mostly testing scanners with various parameters. Parameterized on different + * registry implementations. */ @Category({MediumTests.class, ClientTests.class}) +@RunWith(Parameterized.class) public class TestScannersFromClientSide { @ClassRule @@ -83,15 +89,12 @@ public class TestScannersFromClientSide { private static final Logger LOG = LoggerFactory.getLogger(TestScannersFromClientSide.class); - private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static byte [] ROW = Bytes.toBytes("testRow"); private static byte [] FAMILY = Bytes.toBytes("testFamily"); private static byte [] QUALIFIER = Bytes.toBytes("testQualifier"); private static byte [] VALUE = Bytes.toBytes("testValue"); - @Rule - public TestName name = new TestName(); - /** * @throws java.lang.Exception */ @@ -102,12 +105,16 @@ public static void setUpBeforeClass() throws Exception { TEST_UTIL.startMiniCluster(3); } + @Rule public TableNameTestRule name = new TableNameTestRule(); + /** * @throws java.lang.Exception */ @AfterClass public static void tearDownAfterClass() throws Exception { - TEST_UTIL.shutdownMiniCluster(); + if (TEST_UTIL != null) { + TEST_UTIL.shutdownMiniCluster(); + } } /** @@ -118,12 +125,64 @@ public void setUp() throws Exception { // Nothing to do. } + @Parameterized.Parameters + public static Collection parameters() { + return Arrays.asList(new Object[][]{ + {MasterRegistry.class, 1}, + {MasterRegistry.class, 2}, + {ZKConnectionRegistry.class, 1} + }); + } + /** - * @throws java.lang.Exception + * JUnit does not provide an easy way to run a hook after each parameterized run. Without that + * there is no easy way to restart the test cluster after each parameterized run. Annotation + * BeforeParam does not work either because it runs before parameterization and hence does not + * have access to the test parameters (which is weird). + * + * This *hack* checks if the current instance of test cluster configuration has the passed + * parameterized configs. In such a case, we can just reuse the cluster for test and do not need + * to initialize from scratch. While this is a hack, it saves a ton of time for the full + * test and de-flakes it. */ - @After - public void tearDown() throws Exception { - // Nothing to do. + private static boolean isSameParameterizedCluster(Class registryImpl, int numHedgedReqs) { + // initialize() is called for every unit test, however we only want to reset the cluster state + // at the end of every parameterized run. + if (TEST_UTIL == null) { + return false; + } + Configuration conf = TEST_UTIL.getConfiguration(); + Class confClass = conf.getClass( + HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, ZKConnectionRegistry.class); + int hedgedReqConfig = conf.getInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, + HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_DEFAULT); + return confClass.getName().equals(registryImpl.getName()) && numHedgedReqs == hedgedReqConfig; + } + + public TestScannersFromClientSide(Class registryImpl, int numHedgedReqs) throws Exception { + if (isSameParameterizedCluster(registryImpl, numHedgedReqs)) { + return; + } + if (TEST_UTIL != null) { + // We reached the end of a parameterized run, clean up the cluster. + TEST_UTIL.shutdownMiniCluster(); + } + TEST_UTIL = new HBaseTestingUtility(); + Configuration conf = TEST_UTIL.getConfiguration(); + conf.setLong(HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY, 10 * 1024 * 1024); + conf.setClass(HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, registryImpl, + ConnectionRegistry.class); + if (numHedgedReqs == 1) { + conf.setBoolean(HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY, false); + } else { + Preconditions.checkArgument(numHedgedReqs > 1); + conf.setBoolean(HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY, true); + } + conf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, numHedgedReqs); + StartMiniClusterOption.Builder builder = StartMiniClusterOption.builder(); + // Multiple masters needed only when hedged reads for master registry are enabled. + builder.numMasters(numHedgedReqs > 1 ? 3 : 1).numRegionServers(3); + TEST_UTIL.startMiniCluster(builder.build()); } /** @@ -133,7 +192,7 @@ public void tearDown() throws Exception { */ @Test public void testScanBatch() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 8); Table ht = TEST_UTIL.createTable(tableName, FAMILY); @@ -203,7 +262,7 @@ public void testScanBatch() throws Exception { @Test public void testMaxResultSizeIsSetToDefault() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); Table ht = TEST_UTIL.createTable(tableName, FAMILY); // The max result size we expect the scan to use by default. @@ -274,7 +333,7 @@ public void testScannerForNotExistingTable() { @Test public void testSmallScan() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); int numRows = 10; byte[][] ROWS = HTestConst.makeNAscii(ROW, numRows); @@ -312,7 +371,8 @@ public void testSmallScan() throws Exception { * @param columns * @throws Exception */ - private void testSmallScan(Table table, boolean reversed, int rows, int columns) throws Exception { + private void testSmallScan( + Table table, boolean reversed, int rows, int columns) throws Exception { Scan baseScan = new Scan(); baseScan.setReversed(reversed); baseScan.setSmall(true); @@ -356,7 +416,7 @@ private void verifyExpectedCounts(Table table, Scan scan, int expectedRowCount, */ @Test public void testGetMaxResults() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); byte [][] FAMILIES = HTestConst.makeNAscii(FAMILY, 3); byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 20); @@ -430,8 +490,8 @@ public void testGetMaxResults() throws Exception { kvListExp.add(new KeyValue(ROW, FAMILIES[1], QUALIFIERS[i], 1, VALUE)); } for (int i=0; i < 2; i++) { - kvListExp.add(new KeyValue(ROW, FAMILIES[2], QUALIFIERS[i], 1, VALUE)); - } + kvListExp.add(new KeyValue(ROW, FAMILIES[2], QUALIFIERS[i], 1, VALUE)); + } for (int i=10; i < 20; i++) { kvListExp.add(new KeyValue(ROW, FAMILIES[2], QUALIFIERS[i], 1, VALUE)); } @@ -476,7 +536,7 @@ public void testGetMaxResults() throws Exception { */ @Test public void testScanMaxResults() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); byte [][] ROWS = HTestConst.makeNAscii(ROW, 2); byte [][] FAMILIES = HTestConst.makeNAscii(FAMILY, 3); byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 10); @@ -526,7 +586,7 @@ public void testScanMaxResults() throws Exception { */ @Test public void testGetRowOffset() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); byte [][] FAMILIES = HTestConst.makeNAscii(FAMILY, 3); byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 20); @@ -545,7 +605,9 @@ public void testGetRowOffset() throws Exception { KeyValue kv = new KeyValue(ROW, FAMILIES[0], QUALIFIERS[i], 1, VALUE); put.add(kv); // skipping first two kvs - if (i < 2) continue; + if (i < 2) { + continue; + } kvListExp.add(kv); } ht.put(put); @@ -616,7 +678,7 @@ public void testGetRowOffset() throws Exception { @Test public void testScanRawDeleteFamilyVersion() throws Exception { - TableName tableName = TableName.valueOf(name.getMethodName()); + TableName tableName = name.getTableName(); TEST_UTIL.createTable(tableName, FAMILY); Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); conf.set(RPC_CODEC_CONF_KEY, ""); @@ -646,7 +708,7 @@ public void testScanRawDeleteFamilyVersion() throws Exception { */ @Test public void testScanOnReopenedRegion() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); + final TableName tableName = name.getTableName(); byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 2); Table ht = TEST_UTIL.createTable(tableName, FAMILY); @@ -717,7 +779,7 @@ public void testScanOnReopenedRegion() throws Exception { @Test public void testAsyncScannerWithSmallData() throws Exception { - testAsyncScanner(TableName.valueOf(name.getMethodName()), + testAsyncScanner(name.getTableName(), 2, 3, 10, @@ -727,7 +789,7 @@ public void testAsyncScannerWithSmallData() throws Exception { @Test public void testAsyncScannerWithManyRows() throws Exception { - testAsyncScanner(TableName.valueOf(name.getMethodName()), + testAsyncScanner(name.getTableName(), 30000, 1, 1, @@ -737,7 +799,7 @@ public void testAsyncScannerWithManyRows() throws Exception { @Test public void testAsyncScannerWithoutCaching() throws Exception { - testAsyncScanner(TableName.valueOf(name.getMethodName()), + testAsyncScanner(name.getTableName(), 5, 1, 1, @@ -840,8 +902,9 @@ static void verifyResult(Result result, List expKvList, boolean toLog, LOG.info(msg); LOG.info("Expected count: " + expKvList.size()); LOG.info("Actual count: " + result.size()); - if (expKvList.isEmpty()) + if (expKvList.isEmpty()) { return; + } int i = 0; for (Cell kv : result.rawCells()) { @@ -862,7 +925,7 @@ static void verifyResult(Result result, List expKvList, boolean toLog, @Test public void testReadExpiredDataForRawScan() throws IOException { - TableName tableName = TableName.valueOf(name.getMethodName()); + TableName tableName = name.getTableName(); long ts = System.currentTimeMillis() - 10000; byte[] value = Bytes.toBytes("expired"); try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) { @@ -882,7 +945,7 @@ public void testReadExpiredDataForRawScan() throws IOException { @Test public void testScanWithColumnsAndFilterAndVersion() throws IOException { - TableName tableName = TableName.valueOf(name.getMethodName()); + TableName tableName = name.getTableName(); try (Table table = TEST_UTIL.createTable(tableName, FAMILY, 4)) { for (int i = 0; i < 4; i++) { Put put = new Put(ROW); @@ -904,7 +967,7 @@ public void testScanWithColumnsAndFilterAndVersion() throws IOException { @Test public void testScanWithSameStartRowStopRow() throws IOException { - TableName tableName = TableName.valueOf(name.getMethodName()); + TableName tableName = name.getTableName(); try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) { table.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE)); @@ -941,7 +1004,7 @@ public void testScanWithSameStartRowStopRow() throws IOException { @Test public void testReverseScanWithFlush() throws Exception { - TableName tableName = TableName.valueOf(name.getMethodName()); + TableName tableName = name.getTableName(); final int BATCH_SIZE = 10; final int ROWS_TO_INSERT = 100; final byte[] LARGE_VALUE = generateHugeValue(128 * 1024); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java similarity index 89% rename from hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java rename to hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java index 88de0082e7f4..1336defeca12 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java @@ -48,16 +48,16 @@ import org.slf4j.LoggerFactory; @Category({ MediumTests.class, ClientTests.class }) -public class TestZKAsyncRegistry { +public class TestZKConnectionRegistry { @ClassRule public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestZKAsyncRegistry.class); + HBaseClassTestRule.forClass(TestZKConnectionRegistry.class); - static final Logger LOG = LoggerFactory.getLogger(TestZKAsyncRegistry.class); + static final Logger LOG = LoggerFactory.getLogger(TestZKConnectionRegistry.class); static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private static ZKAsyncRegistry REGISTRY; + private static ZKConnectionRegistry REGISTRY; @BeforeClass public static void setUp() throws Exception { @@ -67,7 +67,7 @@ public static void setUp() throws Exception { // make sure that we do not depend on this config when getting locations for meta replicas, see // HBASE-21658. conf.setInt(META_REPLICAS_NUM, 1); - REGISTRY = new ZKAsyncRegistry(conf); + REGISTRY = new ZKConnectionRegistry(conf); } @AfterClass @@ -84,10 +84,10 @@ public void test() throws InterruptedException, ExecutionException, IOException assertEquals("Expected " + expectedClusterId + ", found=" + clusterId, expectedClusterId, clusterId); assertEquals(TEST_UTIL.getHBaseCluster().getMaster().getServerName(), - REGISTRY.getMasterAddress().get()); + REGISTRY.getActiveMaster().get()); RegionReplicaTestHelper .waitUntilAllMetaReplicasHavingRegionLocation(TEST_UTIL.getConfiguration(), REGISTRY, 3); - RegionLocations locs = REGISTRY.getMetaRegionLocation().get(); + RegionLocations locs = REGISTRY.getMetaRegionLocations().get(); assertEquals(3, locs.getRegionLocations().length); IntStream.range(0, 3).forEach(i -> { HRegionLocation loc = locs.getRegionLocation(i); @@ -102,7 +102,7 @@ public void testIndependentZKConnections() throws IOException { try (ReadOnlyZKClient zk1 = REGISTRY.getZKClient()) { Configuration otherConf = new Configuration(TEST_UTIL.getConfiguration()); otherConf.set(HConstants.ZOOKEEPER_QUORUM, "localhost"); - try (ZKAsyncRegistry otherRegistry = new ZKAsyncRegistry(otherConf)) { + try (ZKConnectionRegistry otherRegistry = new ZKConnectionRegistry(otherConf)) { ReadOnlyZKClient zk2 = otherRegistry.getZKClient(); assertNotSame("Using a different configuration / quorum should result in different " + "backing zk connection.", zk1, zk2); @@ -119,9 +119,9 @@ public void testIndependentZKConnections() throws IOException { public void testNoMetaAvailable() throws InterruptedException { Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); conf.set("zookeeper.znode.metaserver", "whatever"); - try (ZKAsyncRegistry registry = new ZKAsyncRegistry(conf)) { + try (ZKConnectionRegistry registry = new ZKConnectionRegistry(conf)) { try { - registry.getMetaRegionLocation().get(); + registry.getMetaRegionLocations().get(); fail("Should have failed since we set an incorrect meta znode prefix"); } catch (ExecutionException e) { assertThat(e.getCause(), instanceOf(IOException.class)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java index ac0d35613dca..2797df34b81c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java @@ -30,22 +30,31 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; import static org.mockito.internal.verification.VerificationModeFactory.times; - import java.io.IOException; import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.io.compress.GzipCodec; +import org.apache.hadoop.util.StringUtils; +import org.junit.Assume; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; +import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos; import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto; import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto; import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto; @@ -54,14 +63,6 @@ import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface; import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.Interface; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.io.compress.GzipCodec; -import org.apache.hadoop.util.StringUtils; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; /** * Some basic ipc tests. @@ -232,7 +233,6 @@ public void testRpcMaxRequestSize() throws IOException, ServiceException { /** * Tests that the RpcServer creates & dispatches CallRunner object to scheduler with non-null * remoteAddress set to its Call Object - * @throws ServiceException */ @Test public void testRpcServerForNotNullRemoteAddressInCallObject() @@ -363,6 +363,104 @@ public void testAsyncEcho() throws IOException { } } + /** + * Tests the various request fan out values using a simple RPC hedged across a mix of running and + * failing servers. + */ + @Test + public void testHedgedAsyncEcho() throws Exception { + // Hedging is not supported for blocking connection types. + Assume.assumeFalse(this instanceof TestBlockingIPC); + List rpcServers = new ArrayList<>(); + List addresses = new ArrayList<>(); + // Create a mix of running and failing servers. + final int numRunningServers = 5; + final int numFailingServers = 3; + final int numServers = numRunningServers + numFailingServers; + for (int i = 0; i < numRunningServers; i++) { + RpcServer rpcServer = createRpcServer(null, "testRpcServer" + i, + Lists.newArrayList(new RpcServer.BlockingServiceAndInterface( + SERVICE, null)), new InetSocketAddress("localhost", 0), CONF, + new FifoRpcScheduler(CONF, 1)); + rpcServer.start(); + addresses.add(rpcServer.getListenerAddress()); + rpcServers.add(rpcServer); + } + for (int i = 0; i < numFailingServers; i++) { + RpcServer rpcServer = createTestFailingRpcServer(null, "testFailingRpcServer" + i, + Lists.newArrayList(new RpcServer.BlockingServiceAndInterface( + SERVICE, null)), new InetSocketAddress("localhost", 0), CONF, + new FifoRpcScheduler(CONF, 1)); + rpcServer.start(); + addresses.add(rpcServer.getListenerAddress()); + rpcServers.add(rpcServer); + } + Configuration conf = HBaseConfiguration.create(); + try (AbstractRpcClient client = createRpcClient(conf)) { + // Try out various fan out values starting from 1 -> numServers. + for (int reqFanOut = 1; reqFanOut <= numServers; reqFanOut++) { + // Update the client's underlying conf, should be ok for the test. + LOG.debug("Testing with request fan out: " + reqFanOut); + conf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, reqFanOut); + Interface stub = newStub(client, addresses); + BlockingRpcCallback done = new BlockingRpcCallback<>(); + stub.echo(new HBaseRpcControllerImpl(), + EchoRequestProto.newBuilder().setMessage("hello").build(), done); + TestProtos.EchoResponseProto responseProto = done.get(); + assertNotNull(responseProto); + assertEquals("hello", responseProto.getMessage()); + LOG.debug("Ended test with request fan out: " + reqFanOut); + } + } finally { + for (RpcServer rpcServer: rpcServers) { + rpcServer.stop(); + } + } + } + + @Test + public void testHedgedAsyncTimeouts() throws Exception { + // Hedging is not supported for blocking connection types. + Assume.assumeFalse(this instanceof TestBlockingIPC); + List rpcServers = new ArrayList<>(); + List addresses = new ArrayList<>(); + final int numServers = 3; + for (int i = 0; i < numServers; i++) { + RpcServer rpcServer = createRpcServer(null, "testTimeoutRpcServer" + i, + Lists.newArrayList(new RpcServer.BlockingServiceAndInterface( + SERVICE, null)), new InetSocketAddress("localhost", 0), CONF, + new FifoRpcScheduler(CONF, 1)); + rpcServer.start(); + addresses.add(rpcServer.getListenerAddress()); + rpcServers.add(rpcServer); + } + Configuration conf = HBaseConfiguration.create(); + int timeout = 100; + int pauseTime = 1000; + try (AbstractRpcClient client = createRpcClient(conf)) { + // Try out various fan out values starting from 1 -> numServers. + for (int reqFanOut = 1; reqFanOut <= numServers; reqFanOut++) { + // Update the client's underlying conf, should be ok for the test. + LOG.debug("Testing with request fan out: " + reqFanOut); + conf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, reqFanOut); + Interface stub = newStub(client, addresses); + HBaseRpcController pcrc = new HBaseRpcControllerImpl(); + pcrc.setCallTimeout(timeout); + BlockingRpcCallback callback = new BlockingRpcCallback<>(); + stub.pause(pcrc, PauseRequestProto.newBuilder().setMs(pauseTime).build(), callback); + assertNull(callback.get()); + // Make sure the controller has the right exception propagated. + assertTrue(pcrc.getFailed() instanceof CallTimeoutException); + LOG.debug("Ended test with request fan out: " + reqFanOut); + } + } finally { + for (RpcServer rpcServer: rpcServers) { + rpcServer.stop(); + } + } + } + + @Test public void testAsyncRemoteError() throws IOException { AbstractRpcClient client = createRpcClient(CONF); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtobufRpcServiceImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtobufRpcServiceImpl.java index d8a2d348ff9c..6adfa4602ee4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtobufRpcServiceImpl.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtobufRpcServiceImpl.java @@ -17,21 +17,23 @@ */ package org.apache.hadoop.hbase.ipc; -import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService; -import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; -import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; - import java.io.IOException; import java.net.InetSocketAddress; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; - +import java.util.Set; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.util.Threads; import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService; +import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; +import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.AddrResponseProto; import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto; import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto; @@ -41,8 +43,6 @@ import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto; import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface; import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.Interface; -import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.util.Threads; @InterfaceAudience.Private public class TestProtobufRpcServiceImpl implements BlockingInterface { @@ -67,6 +67,17 @@ public static Interface newStub(RpcClient client, InetSocketAddress addr) throws User.getCurrent(), 0)); } + public static Interface newStub(RpcClient client, List addrs) + throws IOException { + Set serverNames = new HashSet<>(); + for (InetSocketAddress addr: addrs) { + serverNames.add(ServerName.valueOf( + addr.getHostName(), addr.getPort(), System.currentTimeMillis())); + } + return TestProtobufRpcProto.newStub(client.createHedgedRpcChannel( + serverNames, User.getCurrent(), 0)); + } + @Override public EmptyResponseProto ping(RpcController controller, EmptyRequestProto request) throws ServiceException { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java index 2787f44422cd..d07d0f80a3ce 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -19,7 +19,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; - import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -73,9 +72,7 @@ import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; - import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; /** @@ -550,6 +547,11 @@ public RegionLocator getRegionLocator() throws IOException { }; } + @Override + public String getClusterId() { + return null; + } + @Override public void clearRegionLocationCache() { } diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java index 973a03d15177..93a394f1606b 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java @@ -1276,6 +1276,11 @@ public TThriftServerType getThriftServerType() { return TThriftServerType.ONE; } + @Override + public String getClusterId() throws TException { + return connectionCache.getClusterId(); + } + private static IOError getIOError(Throwable throwable) { IOError error = new IOErrorWithCause(throwable); error.setMessage(Throwables.getStackTraceAsString(throwable)); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java index fb7fa11c20e3..3088ef0742c5 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java @@ -11,7 +11,7 @@ * An AlreadyExists exceptions signals that a table with the specified * name already exists */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class AlreadyExists extends org.apache.thrift.TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AlreadyExists"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java index a756c9cc9217..0f2b113fb241 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java @@ -10,7 +10,7 @@ /** * A BatchMutation object is used to apply a number of Mutations to a single row. */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class BatchMutation implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("BatchMutation"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java index 6aeb5e25a9df..6d120e505558 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java @@ -12,7 +12,7 @@ * such as the number of versions, compression settings, etc. It is * used as input when creating a table or adding a column. */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class ColumnDescriptor implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ColumnDescriptor"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java index 1530535ad3f9..f92fb318343d 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java @@ -7,7 +7,7 @@ package org.apache.hadoop.hbase.thrift.generated; @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class Hbase { public interface Iface { @@ -622,6 +622,11 @@ public interface Iface { */ public TThriftServerType getThriftServerType() throws org.apache.thrift.TException; + /** + * Returns the cluster ID for this cluster. + */ + public java.lang.String getClusterId() throws org.apache.thrift.TException; + } public interface AsyncIface { @@ -716,6 +721,8 @@ public interface AsyncIface { public void getThriftServerType(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void getClusterId(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + } public static class Client extends org.apache.thrift.TServiceClient implements Iface { @@ -1976,6 +1983,28 @@ public TThriftServerType recv_getThriftServerType() throws org.apache.thrift.TEx throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getThriftServerType failed: unknown result"); } + public java.lang.String getClusterId() throws org.apache.thrift.TException + { + send_getClusterId(); + return recv_getClusterId(); + } + + public void send_getClusterId() throws org.apache.thrift.TException + { + getClusterId_args args = new getClusterId_args(); + sendBase("getClusterId", args); + } + + public java.lang.String recv_getClusterId() throws org.apache.thrift.TException + { + getClusterId_result result = new getClusterId_result(); + receiveBase(result, "getClusterId"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getClusterId failed: unknown result"); + } + } public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface { public static class Factory implements org.apache.thrift.async.TAsyncClientFactory { @@ -3701,6 +3730,35 @@ public TThriftServerType getResult() throws org.apache.thrift.TException { } } + public void getClusterId(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + getClusterId_call method_call = new getClusterId_call(resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class getClusterId_call extends org.apache.thrift.async.TAsyncMethodCall { + public getClusterId_call(org.apache.thrift.async.AsyncMethodCallback 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); + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getClusterId", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getClusterId_args args = new getClusterId_args(); + args.write(prot); + prot.writeMessageEnd(); + } + + 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_getClusterId(); + } + } + } public static class Processor extends org.apache.thrift.TBaseProcessor implements org.apache.thrift.TProcessor { @@ -3759,6 +3817,7 @@ protected Processor(I iface, java.util.Map extends org.apache.thrift.ProcessFunction { + public getClusterId() { + super("getClusterId"); + } + + public getClusterId_args getEmptyArgsInstance() { + return new getClusterId_args(); + } + + protected boolean isOneway() { + return false; + } + + @Override + protected boolean rethrowUnhandledExceptions() { + return false; + } + + public getClusterId_result getResult(I iface, getClusterId_args args) throws org.apache.thrift.TException { + getClusterId_result result = new getClusterId_result(); + result.success = iface.getClusterId(); + return result; + } + } + } public static class AsyncProcessor extends org.apache.thrift.TBaseAsyncProcessor { @@ -5152,6 +5236,7 @@ protected AsyncProcessor(I iface, java.util.Map extends org.apache.thrift.AsyncProcessFunction { + public getClusterId() { + super("getClusterId"); + } + + public getClusterId_args getEmptyArgsInstance() { + return new getClusterId_args(); + } + + public org.apache.thrift.async.AsyncMethodCallback 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() { + public void onComplete(java.lang.String o) { + getClusterId_result result = new getClusterId_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); + } + } + public void onError(java.lang.Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TSerializable msg; + getClusterId_result result = new getClusterId_result(); + 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(); + } + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getClusterId_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + iface.getClusterId(resultHandler); + } + } + } public static class enableTable_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { @@ -60584,4 +60730,625 @@ private static S scheme(org.apache. } } + public static class getClusterId_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getClusterId_args"); + + + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getClusterId_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getClusterId_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 byName = new java.util.HashMap(); + + 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; + } + + public short getThriftFieldId() { + return _thriftId; + } + + 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(getClusterId_args.class, metaDataMap); + } + + public getClusterId_args() { + } + + /** + * Performs a deep copy on other. + */ + public getClusterId_args(getClusterId_args other) { + } + + public getClusterId_args deepCopy() { + return new getClusterId_args(this); + } + + @Override + public void clear() { + } + + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { + switch (field) { + } + } + + @org.apache.thrift.annotation.Nullable + 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 */ + 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 == null) + return false; + if (that instanceof getClusterId_args) + return this.equals((getClusterId_args)that); + return false; + } + + public boolean equals(getClusterId_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(getClusterId_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + return 0; + } + + @org.apache.thrift.annotation.Nullable + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + 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("getClusterId_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 getClusterId_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public getClusterId_argsStandardScheme getScheme() { + return new getClusterId_argsStandardScheme(); + } + } + + private static class getClusterId_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getClusterId_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(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getClusterId_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getClusterId_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public getClusterId_argsTupleScheme getScheme() { + return new getClusterId_argsTupleScheme(); + } + } + + private static class getClusterId_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getClusterId_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, getClusterId_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; + } + } + + private static 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(); + } + } + + public static class getClusterId_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getClusterId_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 getClusterId_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getClusterId_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 byName = new java.util.HashMap(); + + 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; + } + + public short getThriftFieldId() { + return _thriftId; + } + + 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(getClusterId_result.class, metaDataMap); + } + + public getClusterId_result() { + } + + public getClusterId_result( + java.lang.String success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public getClusterId_result(getClusterId_result other) { + if (other.isSetSuccess()) { + this.success = other.success; + } + } + + public getClusterId_result deepCopy() { + return new getClusterId_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + @org.apache.thrift.annotation.Nullable + public java.lang.String getSuccess() { + return this.success; + } + + public getClusterId_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; + } + } + + 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 + 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 */ + 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 == null) + return false; + if (that instanceof getClusterId_result) + return this.equals((getClusterId_result)that); + return false; + } + + public boolean equals(getClusterId_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(getClusterId_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = java.lang.Boolean.valueOf(isSetSuccess()).compareTo(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 + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + 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("getClusterId_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 getClusterId_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public getClusterId_resultStandardScheme getScheme() { + return new getClusterId_resultStandardScheme(); + } + } + + private static class getClusterId_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getClusterId_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(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getClusterId_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 getClusterId_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public getClusterId_resultTupleScheme getScheme() { + return new getClusterId_resultTupleScheme(); + } + } + + private static class getClusterId_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getClusterId_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, getClusterId_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 scheme(org.apache.thrift.protocol.TProtocol proto) { + return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme(); + } + } + } diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java index 59ef06c7c81c..ae4c706b33de 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java @@ -12,7 +12,7 @@ * to the Hbase master or an Hbase region server. Also used to return * more general Hbase error conditions. */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class IOError extends org.apache.thrift.TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("IOError"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java index 5b635aee7cab..01d445c9945d 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java @@ -11,7 +11,7 @@ * An IllegalArgument exception indicates an illegal or invalid * argument was passed into a procedure. */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class IllegalArgument extends org.apache.thrift.TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("IllegalArgument"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java index 68bbdd66c1dd..e26b8ce2568a 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java @@ -10,7 +10,7 @@ /** * A Mutation object is used to either update or delete a column-value. */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class Mutation implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Mutation"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TAppend.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TAppend.java index cd70b4a036ef..01dc0bc034fe 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TAppend.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TAppend.java @@ -10,7 +10,7 @@ /** * An Append object is used to specify the parameters for performing the append operation. */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TAppend implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAppend"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TCell.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TCell.java index 5f54dc0b850b..58c6f17947e4 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TCell.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TCell.java @@ -13,7 +13,7 @@ * the timestamp of a cell to a first-class value, making it easy to take * note of temporal data. Cell is used all the way from HStore up to HTable. */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TCell implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCell"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TColumn.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TColumn.java index 47d62d57391f..0bd0afec69a6 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TColumn.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TColumn.java @@ -10,7 +10,7 @@ /** * Holds column name and the cell. */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumn"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TIncrement.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TIncrement.java index 1ca6be3cce4e..8210a9f60429 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TIncrement.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TIncrement.java @@ -11,7 +11,7 @@ * For increments that are not incrementColumnValue * equivalents. */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TIncrement implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIncrement"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java index 8ae7e4024cbb..44c948dec4e2 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java @@ -10,7 +10,7 @@ /** * A TRegionInfo contains information about an HTable region. */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TRegionInfo implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRegionInfo"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java index bdae4fd85ad1..b08f722769ae 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java @@ -10,7 +10,7 @@ /** * Holds row name and then a map of columns to cells. */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TRowResult implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRowResult"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TScan.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TScan.java index c33d53893d73..81935ec765f9 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TScan.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TScan.java @@ -10,7 +10,7 @@ /** * A Scan object is used to specify scanner parameters when opening a scanner. */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TScan implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TScan"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TThriftServerType.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TThriftServerType.java index b376e8faae55..9824e75d028a 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TThriftServerType.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TThriftServerType.java @@ -10,7 +10,7 @@ /** * Specify type of thrift server: thrift and thrift2 */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public enum TThriftServerType implements org.apache.thrift.TEnum { ONE(1), TWO(2); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java index 778869814e83..62d55d0446f0 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java @@ -818,6 +818,11 @@ public TThriftServerType getThriftServerType() { return TThriftServerType.TWO; } + @Override + public String getClusterId() throws TException { + return connectionCache.getClusterId(); + } + @Override public List listNamespaceDescriptors() throws TIOError, TException { try { diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftConnection.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftConnection.java index 36e513c157f9..b453e6789bb3 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftConnection.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftConnection.java @@ -55,6 +55,7 @@ import org.apache.http.impl.client.DefaultHttpRequestRetryHandler; import org.apache.http.impl.client.HttpClientBuilder; import org.apache.http.protocol.HttpContext; +import org.apache.thrift.TException; import org.apache.thrift.protocol.TBinaryProtocol; import org.apache.thrift.protocol.TCompactProtocol; import org.apache.thrift.protocol.TProtocol; @@ -64,11 +65,14 @@ import org.apache.thrift.transport.TTransport; import org.apache.thrift.transport.TTransportException; import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; @InterfaceAudience.Private public class ThriftConnection implements Connection { + private static final Logger LOG = LoggerFactory.getLogger(ThriftConnection.class); private Configuration conf; private User user; // For HTTP protocol @@ -81,7 +85,8 @@ public class ThriftConnection implements Connection { private boolean isFramed = false; private boolean isCompact = false; - private ThriftClientBuilder clientBuilder; + // TODO: We can rip out the ThriftClient piece of it rather than creating a new client every time. + ThriftClientBuilder clientBuilder; private int operationTimeout; private int connectTimeout; @@ -146,10 +151,6 @@ public int getConnectTimeout() { return connectTimeout; } - public ThriftClientBuilder getClientBuilder() { - return clientBuilder; - } - /** * the default thrift client builder. * One can extend the ThriftClientBuilder to builder custom client, implement @@ -335,7 +336,6 @@ public Table build() { } catch (IOException ioE) { throw new RuntimeException(ioE); } - } }; } @@ -369,4 +369,15 @@ public RegionLocator getRegionLocator(TableName tableName) throws IOException { public void clearRegionLocationCache() { throw new NotImplementedException("clearRegionLocationCache not supported in ThriftTable"); } + + @Override + public String getClusterId() { + try { + Pair client = clientBuilder.getClient(); + return client.getFirst().getClusterId(); + } catch (TException | IOException e) { + LOG.error("Error fetching cluster ID: ", e); + } + return null; + } } diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAppend.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAppend.java index 51ee872d2063..51925575ab40 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAppend.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAppend.java @@ -7,7 +7,7 @@ package org.apache.hadoop.hbase.thrift2.generated; @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TAppend implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAppend"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAuthorization.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAuthorization.java index 09ced16950d3..f90854f1dd5f 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAuthorization.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAuthorization.java @@ -7,7 +7,7 @@ package org.apache.hadoop.hbase.thrift2.generated; @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TAuthorization implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAuthorization"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TBloomFilterType.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TBloomFilterType.java index 06b4e5b774ed..2766a47193a5 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TBloomFilterType.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TBloomFilterType.java @@ -11,7 +11,7 @@ * Thrift wrapper around * org.apache.hadoop.hbase.regionserver.BloomType */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public enum TBloomFilterType implements org.apache.thrift.TEnum { /** * Bloomfilters disabled diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCellVisibility.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCellVisibility.java index 2032c6d91393..c88e952ec81c 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCellVisibility.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCellVisibility.java @@ -7,7 +7,7 @@ package org.apache.hadoop.hbase.thrift2.generated; @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TCellVisibility implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCellVisibility"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java index eb3e0a60d5e4..46d32d6b848c 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java @@ -12,7 +12,7 @@ * in a HBase table by column family and optionally * a column qualifier and timestamp */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumn"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnFamilyDescriptor.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnFamilyDescriptor.java index 2a6a18451966..e0eaa13bcd15 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnFamilyDescriptor.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnFamilyDescriptor.java @@ -11,7 +11,7 @@ * Thrift wrapper around * org.apache.hadoop.hbase.client.ColumnFamilyDescriptor */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TColumnFamilyDescriptor implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnFamilyDescriptor"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnIncrement.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnIncrement.java index 47de646e176b..7d24ab052c7e 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnIncrement.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnIncrement.java @@ -10,7 +10,7 @@ /** * Represents a single cell and the amount to increment it by */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TColumnIncrement implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnIncrement"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnValue.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnValue.java index c519f28e93c2..3132b6d1c853 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnValue.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnValue.java @@ -10,7 +10,7 @@ /** * Represents a single cell and its value. */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TColumnValue implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnValue"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompareOp.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompareOp.java index ca4b40ee882d..f333df08e9c8 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompareOp.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompareOp.java @@ -11,7 +11,7 @@ * Thrift wrapper around * org.apache.hadoop.hbase.filter.CompareFilter$CompareOp. */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public enum TCompareOp implements org.apache.thrift.TEnum { LESS(0), LESS_OR_EQUAL(1), diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompressionAlgorithm.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompressionAlgorithm.java index 48c3b7013be2..721c6574113e 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompressionAlgorithm.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompressionAlgorithm.java @@ -11,7 +11,7 @@ * Thrift wrapper around * org.apache.hadoop.hbase.io.compress.Algorithm */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public enum TCompressionAlgorithm implements org.apache.thrift.TEnum { LZO(0), GZ(1), diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TConsistency.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TConsistency.java index a724db8f4cc9..d6f7e7fa5083 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TConsistency.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TConsistency.java @@ -12,7 +12,7 @@ * - STRONG means reads only from primary region * - TIMELINE means reads might return values from secondary region replicas */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public enum TConsistency implements org.apache.thrift.TEnum { STRONG(1), TIMELINE(2); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDataBlockEncoding.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDataBlockEncoding.java index 15fa8c8cf7b2..d5c241524331 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDataBlockEncoding.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDataBlockEncoding.java @@ -11,7 +11,7 @@ * Thrift wrapper around * org.apache.hadoop.hbase.io.encoding.DataBlockEncoding */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public enum TDataBlockEncoding implements org.apache.thrift.TEnum { /** * Disable data block encoding. diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDelete.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDelete.java index e865fc2f3c01..d9197119b1b9 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDelete.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDelete.java @@ -33,7 +33,7 @@ * by changing the durability. If you don't provide durability, it defaults to * column family's default setting for durability. */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TDelete implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TDelete"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDeleteType.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDeleteType.java index 8a9d2462b59b..616396ffcd08 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDeleteType.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDeleteType.java @@ -12,7 +12,7 @@ * - DELETE_COLUMN means exactly one version will be removed, * - DELETE_COLUMNS means previous versions will also be removed. */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public enum TDeleteType implements org.apache.thrift.TEnum { DELETE_COLUMN(0), DELETE_COLUMNS(1), diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDurability.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDurability.java index effcb855bd8e..b1da7a025629 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDurability.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDurability.java @@ -14,7 +14,7 @@ * - SYNC_WAL means write the Mutation to the WAL synchronously, * - FSYNC_WAL means Write the Mutation to the WAL synchronously and force the entries to disk. */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public enum TDurability implements org.apache.thrift.TEnum { USE_DEFAULT(0), SKIP_WAL(1), diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java index eafc26e99b36..89fb4fa150f8 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java @@ -20,7 +20,7 @@ * If you specify a time range and a timestamp the range is ignored. * Timestamps on TColumns are ignored. */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TGet implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGet"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java index 9c7f3fbae25e..fe35206ed00d 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java @@ -7,7 +7,7 @@ package org.apache.hadoop.hbase.thrift2.generated; @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class THBaseService { public interface Iface { @@ -507,6 +507,11 @@ public interface Iface { */ public TThriftServerType getThriftServerType() throws org.apache.thrift.TException; + /** + * Returns the cluster ID for this cluster. + */ + public java.lang.String getClusterId() throws org.apache.thrift.TException; + } public interface AsyncIface { @@ -605,6 +610,8 @@ public interface AsyncIface { public void getThriftServerType(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void getClusterId(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + } public static class Client extends org.apache.thrift.TServiceClient implements Iface { @@ -1839,6 +1846,28 @@ public TThriftServerType recv_getThriftServerType() throws org.apache.thrift.TEx throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getThriftServerType failed: unknown result"); } + public java.lang.String getClusterId() throws org.apache.thrift.TException + { + send_getClusterId(); + return recv_getClusterId(); + } + + public void send_getClusterId() throws org.apache.thrift.TException + { + getClusterId_args args = new getClusterId_args(); + sendBase("getClusterId", args); + } + + public java.lang.String recv_getClusterId() throws org.apache.thrift.TException + { + getClusterId_result result = new getClusterId_result(); + receiveBase(result, "getClusterId"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getClusterId failed: unknown result"); + } + } public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface { public static class Factory implements org.apache.thrift.async.TAsyncClientFactory { @@ -3475,6 +3504,35 @@ public TThriftServerType getResult() throws org.apache.thrift.TException { } } + public void getClusterId(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + getClusterId_call method_call = new getClusterId_call(resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class getClusterId_call extends org.apache.thrift.async.TAsyncMethodCall { + public getClusterId_call(org.apache.thrift.async.AsyncMethodCallback 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); + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getClusterId", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getClusterId_args args = new getClusterId_args(); + args.write(prot); + prot.writeMessageEnd(); + } + + 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_getClusterId(); + } + } + } public static class Processor extends org.apache.thrift.TBaseProcessor implements org.apache.thrift.TProcessor { @@ -3535,6 +3593,7 @@ protected Processor(I iface, java.util.Map extends org.apache.thrift.ProcessFunction { + public getClusterId() { + super("getClusterId"); + } + + public getClusterId_args getEmptyArgsInstance() { + return new getClusterId_args(); + } + + protected boolean isOneway() { + return false; + } + + @Override + protected boolean rethrowUnhandledExceptions() { + return false; + } + + public getClusterId_result getResult(I iface, getClusterId_args args) throws org.apache.thrift.TException { + getClusterId_result result = new getClusterId_result(); + result.success = iface.getClusterId(); + return result; + } + } + } public static class AsyncProcessor extends org.apache.thrift.TBaseAsyncProcessor { @@ -4971,6 +5055,7 @@ protected AsyncProcessor(I iface, java.util.Map extends org.apache.thrift.AsyncProcessFunction { + public getClusterId() { + super("getClusterId"); + } + + public getClusterId_args getEmptyArgsInstance() { + return new getClusterId_args(); + } + + public org.apache.thrift.async.AsyncMethodCallback 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() { + public void onComplete(java.lang.String o) { + getClusterId_result result = new getClusterId_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); + } + } + public void onError(java.lang.Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TSerializable msg; + getClusterId_result result = new getClusterId_result(); + 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(); + } + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getClusterId_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + iface.getClusterId(resultHandler); + } + } + } public static class exists_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { @@ -36537,13 +36683,759 @@ public java.lang.String getFieldName() { tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(createTable_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(createTable_result.class, metaDataMap); + } + + public createTable_result() { + } + + public createTable_result( + TIOError io) + { + this(); + this.io = io; + } + + /** + * Performs a deep copy on other. + */ + public createTable_result(createTable_result other) { + if (other.isSetIo()) { + this.io = new TIOError(other.io); + } + } + + public createTable_result deepCopy() { + return new createTable_result(this); + } + + @Override + public void clear() { + this.io = null; + } + + @org.apache.thrift.annotation.Nullable + public TIOError getIo() { + return this.io; + } + + public createTable_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { + this.io = io; + return this; + } + + public void unsetIo() { + this.io = null; + } + + /** Returns true if field io is set (has been assigned a value) and false otherwise */ + public boolean isSetIo() { + return this.io != null; + } + + public void setIoIsSet(boolean value) { + if (!value) { + this.io = null; + } + } + + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { + switch (field) { + case IO: + if (value == null) { + unsetIo(); + } else { + setIo((TIOError)value); + } + break; + + } + } + + @org.apache.thrift.annotation.Nullable + public java.lang.Object getFieldValue(_Fields field) { + switch (field) { + case IO: + return getIo(); + + } + throw new java.lang.IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new java.lang.IllegalArgumentException(); + } + + switch (field) { + case IO: + return isSetIo(); + } + throw new java.lang.IllegalStateException(); + } + + @Override + public boolean equals(java.lang.Object that) { + if (that == null) + return false; + if (that instanceof createTable_result) + return this.equals((createTable_result)that); + return false; + } + + public boolean equals(createTable_result that) { + if (that == null) + return false; + if (this == that) + return true; + + boolean this_present_io = true && this.isSetIo(); + boolean that_present_io = true && that.isSetIo(); + if (this_present_io || that_present_io) { + if (!(this_present_io && that_present_io)) + return false; + if (!this.io.equals(that.io)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + int hashCode = 1; + + hashCode = hashCode * 8191 + ((isSetIo()) ? 131071 : 524287); + if (isSetIo()) + hashCode = hashCode * 8191 + io.hashCode(); + + return hashCode; + } + + @Override + public int compareTo(createTable_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = java.lang.Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetIo()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + @org.apache.thrift.annotation.Nullable + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + 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("createTable_result("); + boolean first = true; + + sb.append("io:"); + if (this.io == null) { + sb.append("null"); + } else { + sb.append(this.io); + } + 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 createTable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public createTable_resultStandardScheme getScheme() { + return new createTable_resultStandardScheme(); + } + } + + private static class createTable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, createTable_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 1: // IO + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.io = new TIOError(); + struct.io.read(iprot); + struct.setIoIsSet(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(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, createTable_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.io != null) { + oprot.writeFieldBegin(IO_FIELD_DESC); + struct.io.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class createTable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public createTable_resultTupleScheme getScheme() { + return new createTable_resultTupleScheme(); + } + } + + private static class createTable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, createTable_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.isSetIo()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetIo()) { + struct.io.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, createTable_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.io = new TIOError(); + struct.io.read(iprot); + struct.setIoIsSet(true); + } + } + } + + private static 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(); + } + } + + public static class deleteTable_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteTable_args"); + + private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new deleteTable_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new deleteTable_argsTupleSchemeFactory(); + + /** + * the tablename to delete + */ + public @org.apache.thrift.annotation.Nullable TTableName tableName; // 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 { + /** + * the tablename to delete + */ + TABLE_NAME((short)1, "tableName"); + + private static final java.util.Map byName = new java.util.HashMap(); + + 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 1: // TABLE_NAME + return TABLE_NAME; + 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; + } + + public short getThriftFieldId() { + return _thriftId; + } + + 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.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableName.class))); + metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(deleteTable_args.class, metaDataMap); + } + + public deleteTable_args() { + } + + public deleteTable_args( + TTableName tableName) + { + this(); + this.tableName = tableName; + } + + /** + * Performs a deep copy on other. + */ + public deleteTable_args(deleteTable_args other) { + if (other.isSetTableName()) { + this.tableName = new TTableName(other.tableName); + } + } + + public deleteTable_args deepCopy() { + return new deleteTable_args(this); + } + + @Override + public void clear() { + this.tableName = null; + } + + /** + * the tablename to delete + */ + @org.apache.thrift.annotation.Nullable + public TTableName getTableName() { + return this.tableName; + } + + /** + * the tablename to delete + */ + public deleteTable_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) { + this.tableName = tableName; + return this; + } + + public void unsetTableName() { + this.tableName = null; + } + + /** Returns true if field tableName is set (has been assigned a value) and false otherwise */ + public boolean isSetTableName() { + return this.tableName != null; + } + + public void setTableNameIsSet(boolean value) { + if (!value) { + this.tableName = null; + } + } + + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { + switch (field) { + case TABLE_NAME: + if (value == null) { + unsetTableName(); + } else { + setTableName((TTableName)value); + } + break; + + } + } + + @org.apache.thrift.annotation.Nullable + public java.lang.Object getFieldValue(_Fields field) { + switch (field) { + case TABLE_NAME: + return getTableName(); + + } + throw new java.lang.IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new java.lang.IllegalArgumentException(); + } + + switch (field) { + case TABLE_NAME: + return isSetTableName(); + } + throw new java.lang.IllegalStateException(); + } + + @Override + public boolean equals(java.lang.Object that) { + if (that == null) + return false; + if (that instanceof deleteTable_args) + return this.equals((deleteTable_args)that); + return false; + } + + public boolean equals(deleteTable_args that) { + if (that == null) + return false; + if (this == that) + return true; + + boolean this_present_tableName = true && this.isSetTableName(); + boolean that_present_tableName = true && that.isSetTableName(); + if (this_present_tableName || that_present_tableName) { + if (!(this_present_tableName && that_present_tableName)) + return false; + if (!this.tableName.equals(that.tableName)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + int hashCode = 1; + + hashCode = hashCode * 8191 + ((isSetTableName()) ? 131071 : 524287); + if (isSetTableName()) + hashCode = hashCode * 8191 + tableName.hashCode(); + + return hashCode; + } + + @Override + public int compareTo(deleteTable_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = java.lang.Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTableName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + @org.apache.thrift.annotation.Nullable + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + 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("deleteTable_args("); + boolean first = true; + + sb.append("tableName:"); + if (this.tableName == null) { + sb.append("null"); + } else { + sb.append(this.tableName); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (tableName == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'tableName' was not present! Struct: " + toString()); + } + // check for sub-struct validity + if (tableName != null) { + tableName.validate(); + } + } + + 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 deleteTable_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public deleteTable_argsStandardScheme getScheme() { + return new deleteTable_argsStandardScheme(); + } + } + + private static class deleteTable_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, deleteTable_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) { + case 1: // TABLE_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.tableName = new TTableName(); + struct.tableName.read(iprot); + struct.setTableNameIsSet(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(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, deleteTable_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.tableName != null) { + oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC); + struct.tableName.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class deleteTable_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public deleteTable_argsTupleScheme getScheme() { + return new deleteTable_argsTupleScheme(); + } + } + + private static class deleteTable_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, deleteTable_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; + struct.tableName.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, deleteTable_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; + struct.tableName = new TTableName(); + struct.tableName.read(iprot); + struct.setTableNameIsSet(true); + } + } + + private static 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(); + } + } + + public static class deleteTable_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteTable_result"); + + private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new deleteTable_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new deleteTable_resultTupleSchemeFactory(); + + public @org.apache.thrift.annotation.Nullable TIOError io; // 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 { + IO((short)1, "io"); + + private static final java.util.Map byName = new java.util.HashMap(); + + 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 1: // IO + return IO; + 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; + } + + public short getThriftFieldId() { + return _thriftId; + } + + 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.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class))); + metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(deleteTable_result.class, metaDataMap); } - public createTable_result() { + public deleteTable_result() { } - public createTable_result( + public deleteTable_result( TIOError io) { this(); @@ -36553,14 +37445,14 @@ public createTable_result( /** * Performs a deep copy on other. */ - public createTable_result(createTable_result other) { + public deleteTable_result(deleteTable_result other) { if (other.isSetIo()) { this.io = new TIOError(other.io); } } - public createTable_result deepCopy() { - return new createTable_result(this); + public deleteTable_result deepCopy() { + return new deleteTable_result(this); } @Override @@ -36573,7 +37465,7 @@ public TIOError getIo() { return this.io; } - public createTable_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { + public deleteTable_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { this.io = io; return this; } @@ -36633,12 +37525,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof createTable_result) - return this.equals((createTable_result)that); + if (that instanceof deleteTable_result) + return this.equals((deleteTable_result)that); return false; } - public boolean equals(createTable_result that) { + public boolean equals(deleteTable_result that) { if (that == null) return false; if (this == that) @@ -36668,7 +37560,7 @@ public int hashCode() { } @Override - public int compareTo(createTable_result other) { + public int compareTo(deleteTable_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -36703,7 +37595,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("createTable_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("deleteTable_result("); boolean first = true; sb.append("io:"); @@ -36738,15 +37630,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class createTable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public createTable_resultStandardScheme getScheme() { - return new createTable_resultStandardScheme(); + private static class deleteTable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public deleteTable_resultStandardScheme getScheme() { + return new deleteTable_resultStandardScheme(); } } - private static class createTable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class deleteTable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, createTable_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, deleteTable_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -36776,7 +37668,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, createTable_result struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, createTable_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, deleteTable_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -36791,16 +37683,16 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, createTable_result } - private static class createTable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public createTable_resultTupleScheme getScheme() { - return new createTable_resultTupleScheme(); + private static class deleteTable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public deleteTable_resultTupleScheme getScheme() { + return new deleteTable_resultTupleScheme(); } } - private static class createTable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class deleteTable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, createTable_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, deleteTable_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.isSetIo()) { @@ -36813,7 +37705,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, createTable_result } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, createTable_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, deleteTable_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)) { @@ -36829,25 +37721,34 @@ private static S scheme(org.apache. } } - public static class deleteTable_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteTable_args"); + public static class truncateTable_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("truncateTable_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField PRESERVE_SPLITS_FIELD_DESC = new org.apache.thrift.protocol.TField("preserveSplits", org.apache.thrift.protocol.TType.BOOL, (short)2); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new deleteTable_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new deleteTable_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new truncateTable_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new truncateTable_argsTupleSchemeFactory(); /** - * the tablename to delete + * the tablename to truncate */ public @org.apache.thrift.annotation.Nullable TTableName tableName; // required + /** + * whether to preserve previous splits + */ + public boolean preserveSplits; // 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 { /** - * the tablename to delete + * the tablename to truncate */ - TABLE_NAME((short)1, "tableName"); + TABLE_NAME((short)1, "tableName"), + /** + * whether to preserve previous splits + */ + PRESERVE_SPLITS((short)2, "preserveSplits"); private static final java.util.Map byName = new java.util.HashMap(); @@ -36865,6 +37766,8 @@ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // TABLE_NAME return TABLE_NAME; + case 2: // PRESERVE_SPLITS + return PRESERVE_SPLITS; default: return null; } @@ -36906,45 +37809,56 @@ public java.lang.String getFieldName() { } // isset id assignments + private static final int __PRESERVESPLITS_ISSET_ID = 0; + private byte __isset_bitfield = 0; 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.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableName.class))); + tmpMap.put(_Fields.PRESERVE_SPLITS, new org.apache.thrift.meta_data.FieldMetaData("preserveSplits", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(deleteTable_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(truncateTable_args.class, metaDataMap); } - public deleteTable_args() { + public truncateTable_args() { } - public deleteTable_args( - TTableName tableName) + public truncateTable_args( + TTableName tableName, + boolean preserveSplits) { this(); this.tableName = tableName; + this.preserveSplits = preserveSplits; + setPreserveSplitsIsSet(true); } /** * Performs a deep copy on other. */ - public deleteTable_args(deleteTable_args other) { + public truncateTable_args(truncateTable_args other) { + __isset_bitfield = other.__isset_bitfield; if (other.isSetTableName()) { this.tableName = new TTableName(other.tableName); } + this.preserveSplits = other.preserveSplits; } - public deleteTable_args deepCopy() { - return new deleteTable_args(this); + public truncateTable_args deepCopy() { + return new truncateTable_args(this); } @Override public void clear() { this.tableName = null; + setPreserveSplitsIsSet(false); + this.preserveSplits = false; } /** - * the tablename to delete + * the tablename to truncate */ @org.apache.thrift.annotation.Nullable public TTableName getTableName() { @@ -36952,9 +37866,9 @@ public TTableName getTableName() { } /** - * the tablename to delete + * the tablename to truncate */ - public deleteTable_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) { + public truncateTable_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) { this.tableName = tableName; return this; } @@ -36974,6 +37888,35 @@ public void setTableNameIsSet(boolean value) { } } + /** + * whether to preserve previous splits + */ + public boolean isPreserveSplits() { + return this.preserveSplits; + } + + /** + * whether to preserve previous splits + */ + public truncateTable_args setPreserveSplits(boolean preserveSplits) { + this.preserveSplits = preserveSplits; + setPreserveSplitsIsSet(true); + return this; + } + + public void unsetPreserveSplits() { + __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __PRESERVESPLITS_ISSET_ID); + } + + /** Returns true if field preserveSplits is set (has been assigned a value) and false otherwise */ + public boolean isSetPreserveSplits() { + return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __PRESERVESPLITS_ISSET_ID); + } + + public void setPreserveSplitsIsSet(boolean value) { + __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __PRESERVESPLITS_ISSET_ID, value); + } + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { switch (field) { case TABLE_NAME: @@ -36984,6 +37927,14 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable } break; + case PRESERVE_SPLITS: + if (value == null) { + unsetPreserveSplits(); + } else { + setPreserveSplits((java.lang.Boolean)value); + } + break; + } } @@ -36993,6 +37944,9 @@ public java.lang.Object getFieldValue(_Fields field) { case TABLE_NAME: return getTableName(); + case PRESERVE_SPLITS: + return isPreserveSplits(); + } throw new java.lang.IllegalStateException(); } @@ -37006,6 +37960,8 @@ public boolean isSet(_Fields field) { switch (field) { case TABLE_NAME: return isSetTableName(); + case PRESERVE_SPLITS: + return isSetPreserveSplits(); } throw new java.lang.IllegalStateException(); } @@ -37014,12 +37970,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof deleteTable_args) - return this.equals((deleteTable_args)that); + if (that instanceof truncateTable_args) + return this.equals((truncateTable_args)that); return false; } - public boolean equals(deleteTable_args that) { + public boolean equals(truncateTable_args that) { if (that == null) return false; if (this == that) @@ -37034,6 +37990,15 @@ public boolean equals(deleteTable_args that) { return false; } + boolean this_present_preserveSplits = true; + boolean that_present_preserveSplits = true; + if (this_present_preserveSplits || that_present_preserveSplits) { + if (!(this_present_preserveSplits && that_present_preserveSplits)) + return false; + if (this.preserveSplits != that.preserveSplits) + return false; + } + return true; } @@ -37045,11 +38010,13 @@ public int hashCode() { if (isSetTableName()) hashCode = hashCode * 8191 + tableName.hashCode(); + hashCode = hashCode * 8191 + ((preserveSplits) ? 131071 : 524287); + return hashCode; } @Override - public int compareTo(deleteTable_args other) { + public int compareTo(truncateTable_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -37066,6 +38033,16 @@ public int compareTo(deleteTable_args other) { return lastComparison; } } + lastComparison = java.lang.Boolean.valueOf(isSetPreserveSplits()).compareTo(other.isSetPreserveSplits()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetPreserveSplits()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.preserveSplits, other.preserveSplits); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -37084,7 +38061,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("deleteTable_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("truncateTable_args("); boolean first = true; sb.append("tableName:"); @@ -37094,6 +38071,10 @@ public java.lang.String toString() { sb.append(this.tableName); } first = false; + if (!first) sb.append(", "); + sb.append("preserveSplits:"); + sb.append(this.preserveSplits); + first = false; sb.append(")"); return sb.toString(); } @@ -37103,6 +38084,7 @@ public void validate() throws org.apache.thrift.TException { if (tableName == null) { throw new org.apache.thrift.protocol.TProtocolException("Required field 'tableName' was not present! Struct: " + toString()); } + // alas, we cannot check 'preserveSplits' because it's a primitive and you chose the non-beans generator. // check for sub-struct validity if (tableName != null) { tableName.validate(); @@ -37119,21 +38101,23 @@ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOExcept private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException { try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; 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 deleteTable_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public deleteTable_argsStandardScheme getScheme() { - return new deleteTable_argsStandardScheme(); + private static class truncateTable_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public truncateTable_argsStandardScheme getScheme() { + return new truncateTable_argsStandardScheme(); } } - private static class deleteTable_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class truncateTable_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, deleteTable_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, truncateTable_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -37152,6 +38136,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, deleteTable_args st org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; + case 2: // PRESERVE_SPLITS + if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { + struct.preserveSplits = iprot.readBool(); + struct.setPreserveSplitsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -37160,10 +38152,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, deleteTable_args st iprot.readStructEnd(); // check for required fields of primitive type, which can't be checked in the validate method + if (!struct.isSetPreserveSplits()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'preserveSplits' was not found in serialized data! Struct: " + toString()); + } struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, deleteTable_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, truncateTable_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -37172,32 +38167,38 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, deleteTable_args s struct.tableName.write(oprot); oprot.writeFieldEnd(); } + oprot.writeFieldBegin(PRESERVE_SPLITS_FIELD_DESC); + oprot.writeBool(struct.preserveSplits); + oprot.writeFieldEnd(); oprot.writeFieldStop(); oprot.writeStructEnd(); } } - private static class deleteTable_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public deleteTable_argsTupleScheme getScheme() { - return new deleteTable_argsTupleScheme(); + private static class truncateTable_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public truncateTable_argsTupleScheme getScheme() { + return new truncateTable_argsTupleScheme(); } } - private static class deleteTable_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class truncateTable_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, deleteTable_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, truncateTable_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; struct.tableName.write(oprot); + oprot.writeBool(struct.preserveSplits); } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, deleteTable_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, truncateTable_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; struct.tableName = new TTableName(); struct.tableName.read(iprot); struct.setTableNameIsSet(true); + struct.preserveSplits = iprot.readBool(); + struct.setPreserveSplitsIsSet(true); } } @@ -37206,13 +38207,13 @@ private static S scheme(org.apache. } } - public static class deleteTable_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteTable_result"); + public static class truncateTable_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("truncateTable_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new deleteTable_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new deleteTable_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new truncateTable_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new truncateTable_resultTupleSchemeFactory(); public @org.apache.thrift.annotation.Nullable TIOError io; // required @@ -37283,13 +38284,13 @@ public java.lang.String getFieldName() { tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(deleteTable_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(truncateTable_result.class, metaDataMap); } - public deleteTable_result() { + public truncateTable_result() { } - public deleteTable_result( + public truncateTable_result( TIOError io) { this(); @@ -37299,14 +38300,14 @@ public deleteTable_result( /** * Performs a deep copy on other. */ - public deleteTable_result(deleteTable_result other) { + public truncateTable_result(truncateTable_result other) { if (other.isSetIo()) { this.io = new TIOError(other.io); } } - public deleteTable_result deepCopy() { - return new deleteTable_result(this); + public truncateTable_result deepCopy() { + return new truncateTable_result(this); } @Override @@ -37319,7 +38320,7 @@ public TIOError getIo() { return this.io; } - public deleteTable_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { + public truncateTable_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { this.io = io; return this; } @@ -37379,12 +38380,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof deleteTable_result) - return this.equals((deleteTable_result)that); + if (that instanceof truncateTable_result) + return this.equals((truncateTable_result)that); return false; } - public boolean equals(deleteTable_result that) { + public boolean equals(truncateTable_result that) { if (that == null) return false; if (this == that) @@ -37414,7 +38415,7 @@ public int hashCode() { } @Override - public int compareTo(deleteTable_result other) { + public int compareTo(truncateTable_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -37449,7 +38450,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("deleteTable_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("truncateTable_result("); boolean first = true; sb.append("io:"); @@ -37484,15 +38485,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class deleteTable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public deleteTable_resultStandardScheme getScheme() { - return new deleteTable_resultStandardScheme(); + private static class truncateTable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public truncateTable_resultStandardScheme getScheme() { + return new truncateTable_resultStandardScheme(); } } - private static class deleteTable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class truncateTable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, deleteTable_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, truncateTable_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -37522,7 +38523,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, deleteTable_result struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, deleteTable_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, truncateTable_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -37537,16 +38538,16 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, deleteTable_result } - private static class deleteTable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public deleteTable_resultTupleScheme getScheme() { - return new deleteTable_resultTupleScheme(); + private static class truncateTable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public truncateTable_resultTupleScheme getScheme() { + return new truncateTable_resultTupleScheme(); } } - private static class deleteTable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class truncateTable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, deleteTable_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, truncateTable_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.isSetIo()) { @@ -37559,7 +38560,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, deleteTable_result } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, deleteTable_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, truncateTable_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)) { @@ -37575,34 +38576,25 @@ private static S scheme(org.apache. } } - public static class truncateTable_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("truncateTable_args"); + public static class enableTable_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("enableTable_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField PRESERVE_SPLITS_FIELD_DESC = new org.apache.thrift.protocol.TField("preserveSplits", org.apache.thrift.protocol.TType.BOOL, (short)2); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new truncateTable_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new truncateTable_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new enableTable_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new enableTable_argsTupleSchemeFactory(); /** - * the tablename to truncate + * the tablename to enable */ public @org.apache.thrift.annotation.Nullable TTableName tableName; // required - /** - * whether to preserve previous splits - */ - public boolean preserveSplits; // 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 { /** - * the tablename to truncate - */ - TABLE_NAME((short)1, "tableName"), - /** - * whether to preserve previous splits + * the tablename to enable */ - PRESERVE_SPLITS((short)2, "preserveSplits"); + TABLE_NAME((short)1, "tableName"); private static final java.util.Map byName = new java.util.HashMap(); @@ -37620,8 +38612,6 @@ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // TABLE_NAME return TABLE_NAME; - case 2: // PRESERVE_SPLITS - return PRESERVE_SPLITS; default: return null; } @@ -37663,56 +38653,45 @@ public java.lang.String getFieldName() { } // isset id assignments - private static final int __PRESERVESPLITS_ISSET_ID = 0; - private byte __isset_bitfield = 0; 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.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableName.class))); - tmpMap.put(_Fields.PRESERVE_SPLITS, new org.apache.thrift.meta_data.FieldMetaData("preserveSplits", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(truncateTable_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(enableTable_args.class, metaDataMap); } - public truncateTable_args() { + public enableTable_args() { } - public truncateTable_args( - TTableName tableName, - boolean preserveSplits) + public enableTable_args( + TTableName tableName) { this(); this.tableName = tableName; - this.preserveSplits = preserveSplits; - setPreserveSplitsIsSet(true); } /** * Performs a deep copy on other. */ - public truncateTable_args(truncateTable_args other) { - __isset_bitfield = other.__isset_bitfield; + public enableTable_args(enableTable_args other) { if (other.isSetTableName()) { this.tableName = new TTableName(other.tableName); } - this.preserveSplits = other.preserveSplits; } - public truncateTable_args deepCopy() { - return new truncateTable_args(this); + public enableTable_args deepCopy() { + return new enableTable_args(this); } @Override public void clear() { this.tableName = null; - setPreserveSplitsIsSet(false); - this.preserveSplits = false; } /** - * the tablename to truncate + * the tablename to enable */ @org.apache.thrift.annotation.Nullable public TTableName getTableName() { @@ -37720,9 +38699,9 @@ public TTableName getTableName() { } /** - * the tablename to truncate + * the tablename to enable */ - public truncateTable_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) { + public enableTable_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) { this.tableName = tableName; return this; } @@ -37742,35 +38721,6 @@ public void setTableNameIsSet(boolean value) { } } - /** - * whether to preserve previous splits - */ - public boolean isPreserveSplits() { - return this.preserveSplits; - } - - /** - * whether to preserve previous splits - */ - public truncateTable_args setPreserveSplits(boolean preserveSplits) { - this.preserveSplits = preserveSplits; - setPreserveSplitsIsSet(true); - return this; - } - - public void unsetPreserveSplits() { - __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __PRESERVESPLITS_ISSET_ID); - } - - /** Returns true if field preserveSplits is set (has been assigned a value) and false otherwise */ - public boolean isSetPreserveSplits() { - return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __PRESERVESPLITS_ISSET_ID); - } - - public void setPreserveSplitsIsSet(boolean value) { - __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __PRESERVESPLITS_ISSET_ID, value); - } - public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { switch (field) { case TABLE_NAME: @@ -37781,14 +38731,6 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable } break; - case PRESERVE_SPLITS: - if (value == null) { - unsetPreserveSplits(); - } else { - setPreserveSplits((java.lang.Boolean)value); - } - break; - } } @@ -37798,9 +38740,6 @@ public java.lang.Object getFieldValue(_Fields field) { case TABLE_NAME: return getTableName(); - case PRESERVE_SPLITS: - return isPreserveSplits(); - } throw new java.lang.IllegalStateException(); } @@ -37814,8 +38753,6 @@ public boolean isSet(_Fields field) { switch (field) { case TABLE_NAME: return isSetTableName(); - case PRESERVE_SPLITS: - return isSetPreserveSplits(); } throw new java.lang.IllegalStateException(); } @@ -37824,12 +38761,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof truncateTable_args) - return this.equals((truncateTable_args)that); + if (that instanceof enableTable_args) + return this.equals((enableTable_args)that); return false; } - public boolean equals(truncateTable_args that) { + public boolean equals(enableTable_args that) { if (that == null) return false; if (this == that) @@ -37844,15 +38781,6 @@ public boolean equals(truncateTable_args that) { return false; } - boolean this_present_preserveSplits = true; - boolean that_present_preserveSplits = true; - if (this_present_preserveSplits || that_present_preserveSplits) { - if (!(this_present_preserveSplits && that_present_preserveSplits)) - return false; - if (this.preserveSplits != that.preserveSplits) - return false; - } - return true; } @@ -37864,13 +38792,11 @@ public int hashCode() { if (isSetTableName()) hashCode = hashCode * 8191 + tableName.hashCode(); - hashCode = hashCode * 8191 + ((preserveSplits) ? 131071 : 524287); - return hashCode; } @Override - public int compareTo(truncateTable_args other) { + public int compareTo(enableTable_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -37887,16 +38813,6 @@ public int compareTo(truncateTable_args other) { return lastComparison; } } - lastComparison = java.lang.Boolean.valueOf(isSetPreserveSplits()).compareTo(other.isSetPreserveSplits()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetPreserveSplits()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.preserveSplits, other.preserveSplits); - if (lastComparison != 0) { - return lastComparison; - } - } return 0; } @@ -37915,7 +38831,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("truncateTable_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("enableTable_args("); boolean first = true; sb.append("tableName:"); @@ -37925,10 +38841,6 @@ public java.lang.String toString() { sb.append(this.tableName); } first = false; - if (!first) sb.append(", "); - sb.append("preserveSplits:"); - sb.append(this.preserveSplits); - first = false; sb.append(")"); return sb.toString(); } @@ -37938,7 +38850,6 @@ public void validate() throws org.apache.thrift.TException { if (tableName == null) { throw new org.apache.thrift.protocol.TProtocolException("Required field 'tableName' was not present! Struct: " + toString()); } - // alas, we cannot check 'preserveSplits' because it's a primitive and you chose the non-beans generator. // check for sub-struct validity if (tableName != null) { tableName.validate(); @@ -37955,23 +38866,21 @@ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOExcept private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException { try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; 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 truncateTable_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public truncateTable_argsStandardScheme getScheme() { - return new truncateTable_argsStandardScheme(); + private static class enableTable_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public enableTable_argsStandardScheme getScheme() { + return new enableTable_argsStandardScheme(); } } - private static class truncateTable_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class enableTable_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, truncateTable_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, enableTable_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -37990,14 +38899,6 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, truncateTable_args org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 2: // PRESERVE_SPLITS - if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { - struct.preserveSplits = iprot.readBool(); - struct.setPreserveSplitsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -38006,13 +38907,10 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, truncateTable_args iprot.readStructEnd(); // check for required fields of primitive type, which can't be checked in the validate method - if (!struct.isSetPreserveSplits()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'preserveSplits' was not found in serialized data! Struct: " + toString()); - } struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, truncateTable_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, enableTable_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -38021,38 +38919,32 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, truncateTable_args struct.tableName.write(oprot); oprot.writeFieldEnd(); } - oprot.writeFieldBegin(PRESERVE_SPLITS_FIELD_DESC); - oprot.writeBool(struct.preserveSplits); - oprot.writeFieldEnd(); oprot.writeFieldStop(); oprot.writeStructEnd(); } } - private static class truncateTable_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public truncateTable_argsTupleScheme getScheme() { - return new truncateTable_argsTupleScheme(); + private static class enableTable_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public enableTable_argsTupleScheme getScheme() { + return new enableTable_argsTupleScheme(); } } - private static class truncateTable_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class enableTable_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, truncateTable_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, enableTable_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; struct.tableName.write(oprot); - oprot.writeBool(struct.preserveSplits); } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, truncateTable_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, enableTable_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; struct.tableName = new TTableName(); struct.tableName.read(iprot); struct.setTableNameIsSet(true); - struct.preserveSplits = iprot.readBool(); - struct.setPreserveSplitsIsSet(true); } } @@ -38061,13 +38953,13 @@ private static S scheme(org.apache. } } - public static class truncateTable_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("truncateTable_result"); + public static class enableTable_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("enableTable_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new truncateTable_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new truncateTable_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new enableTable_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new enableTable_resultTupleSchemeFactory(); public @org.apache.thrift.annotation.Nullable TIOError io; // required @@ -38138,13 +39030,13 @@ public java.lang.String getFieldName() { tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(truncateTable_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(enableTable_result.class, metaDataMap); } - public truncateTable_result() { + public enableTable_result() { } - public truncateTable_result( + public enableTable_result( TIOError io) { this(); @@ -38154,14 +39046,14 @@ public truncateTable_result( /** * Performs a deep copy on other. */ - public truncateTable_result(truncateTable_result other) { + public enableTable_result(enableTable_result other) { if (other.isSetIo()) { this.io = new TIOError(other.io); } } - public truncateTable_result deepCopy() { - return new truncateTable_result(this); + public enableTable_result deepCopy() { + return new enableTable_result(this); } @Override @@ -38174,7 +39066,7 @@ public TIOError getIo() { return this.io; } - public truncateTable_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { + public enableTable_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { this.io = io; return this; } @@ -38234,12 +39126,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof truncateTable_result) - return this.equals((truncateTable_result)that); + if (that instanceof enableTable_result) + return this.equals((enableTable_result)that); return false; } - public boolean equals(truncateTable_result that) { + public boolean equals(enableTable_result that) { if (that == null) return false; if (this == that) @@ -38269,7 +39161,7 @@ public int hashCode() { } @Override - public int compareTo(truncateTable_result other) { + public int compareTo(enableTable_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -38304,7 +39196,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("truncateTable_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("enableTable_result("); boolean first = true; sb.append("io:"); @@ -38339,15 +39231,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class truncateTable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public truncateTable_resultStandardScheme getScheme() { - return new truncateTable_resultStandardScheme(); + private static class enableTable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public enableTable_resultStandardScheme getScheme() { + return new enableTable_resultStandardScheme(); } } - private static class truncateTable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class enableTable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, truncateTable_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, enableTable_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -38377,7 +39269,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, truncateTable_resul struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, truncateTable_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, enableTable_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -38392,16 +39284,16 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, truncateTable_resu } - private static class truncateTable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public truncateTable_resultTupleScheme getScheme() { - return new truncateTable_resultTupleScheme(); + private static class enableTable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public enableTable_resultTupleScheme getScheme() { + return new enableTable_resultTupleScheme(); } } - private static class truncateTable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class enableTable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, truncateTable_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, enableTable_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.isSetIo()) { @@ -38414,7 +39306,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, truncateTable_resul } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, truncateTable_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, enableTable_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)) { @@ -38430,23 +39322,23 @@ private static S scheme(org.apache. } } - public static class enableTable_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("enableTable_args"); + public static class disableTable_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("disableTable_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new enableTable_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new enableTable_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new disableTable_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new disableTable_argsTupleSchemeFactory(); /** - * the tablename to enable + * the tablename to disable */ public @org.apache.thrift.annotation.Nullable TTableName tableName; // 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 { /** - * the tablename to enable + * the tablename to disable */ TABLE_NAME((short)1, "tableName"); @@ -38513,13 +39405,13 @@ public java.lang.String getFieldName() { tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableName.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(enableTable_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(disableTable_args.class, metaDataMap); } - public enableTable_args() { + public disableTable_args() { } - public enableTable_args( + public disableTable_args( TTableName tableName) { this(); @@ -38529,14 +39421,14 @@ public enableTable_args( /** * Performs a deep copy on other. */ - public enableTable_args(enableTable_args other) { + public disableTable_args(disableTable_args other) { if (other.isSetTableName()) { this.tableName = new TTableName(other.tableName); } } - public enableTable_args deepCopy() { - return new enableTable_args(this); + public disableTable_args deepCopy() { + return new disableTable_args(this); } @Override @@ -38545,7 +39437,7 @@ public void clear() { } /** - * the tablename to enable + * the tablename to disable */ @org.apache.thrift.annotation.Nullable public TTableName getTableName() { @@ -38553,9 +39445,9 @@ public TTableName getTableName() { } /** - * the tablename to enable + * the tablename to disable */ - public enableTable_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) { + public disableTable_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) { this.tableName = tableName; return this; } @@ -38615,12 +39507,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof enableTable_args) - return this.equals((enableTable_args)that); + if (that instanceof disableTable_args) + return this.equals((disableTable_args)that); return false; } - public boolean equals(enableTable_args that) { + public boolean equals(disableTable_args that) { if (that == null) return false; if (this == that) @@ -38650,7 +39542,7 @@ public int hashCode() { } @Override - public int compareTo(enableTable_args other) { + public int compareTo(disableTable_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -38685,7 +39577,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("enableTable_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("disableTable_args("); boolean first = true; sb.append("tableName:"); @@ -38726,15 +39618,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class enableTable_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public enableTable_argsStandardScheme getScheme() { - return new enableTable_argsStandardScheme(); + private static class disableTable_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public disableTable_argsStandardScheme getScheme() { + return new disableTable_argsStandardScheme(); } } - private static class enableTable_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class disableTable_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, enableTable_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, disableTable_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -38764,7 +39656,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, enableTable_args st struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, enableTable_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, disableTable_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -38779,22 +39671,22 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, enableTable_args s } - private static class enableTable_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public enableTable_argsTupleScheme getScheme() { - return new enableTable_argsTupleScheme(); + private static class disableTable_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public disableTable_argsTupleScheme getScheme() { + return new disableTable_argsTupleScheme(); } } - private static class enableTable_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class disableTable_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, enableTable_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, disableTable_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; struct.tableName.write(oprot); } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, enableTable_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, disableTable_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; struct.tableName = new TTableName(); struct.tableName.read(iprot); @@ -38807,13 +39699,13 @@ private static S scheme(org.apache. } } - public static class enableTable_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("enableTable_result"); + public static class disableTable_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("disableTable_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new enableTable_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new enableTable_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new disableTable_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new disableTable_resultTupleSchemeFactory(); public @org.apache.thrift.annotation.Nullable TIOError io; // required @@ -38884,13 +39776,13 @@ public java.lang.String getFieldName() { tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(enableTable_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(disableTable_result.class, metaDataMap); } - public enableTable_result() { + public disableTable_result() { } - public enableTable_result( + public disableTable_result( TIOError io) { this(); @@ -38900,14 +39792,14 @@ public enableTable_result( /** * Performs a deep copy on other. */ - public enableTable_result(enableTable_result other) { + public disableTable_result(disableTable_result other) { if (other.isSetIo()) { this.io = new TIOError(other.io); } } - public enableTable_result deepCopy() { - return new enableTable_result(this); + public disableTable_result deepCopy() { + return new disableTable_result(this); } @Override @@ -38920,7 +39812,7 @@ public TIOError getIo() { return this.io; } - public enableTable_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { + public disableTable_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { this.io = io; return this; } @@ -38980,12 +39872,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof enableTable_result) - return this.equals((enableTable_result)that); + if (that instanceof disableTable_result) + return this.equals((disableTable_result)that); return false; } - public boolean equals(enableTable_result that) { + public boolean equals(disableTable_result that) { if (that == null) return false; if (this == that) @@ -39015,7 +39907,7 @@ public int hashCode() { } @Override - public int compareTo(enableTable_result other) { + public int compareTo(disableTable_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -39050,7 +39942,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("enableTable_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("disableTable_result("); boolean first = true; sb.append("io:"); @@ -39085,15 +39977,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class enableTable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public enableTable_resultStandardScheme getScheme() { - return new enableTable_resultStandardScheme(); + private static class disableTable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public disableTable_resultStandardScheme getScheme() { + return new disableTable_resultStandardScheme(); } } - private static class enableTable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class disableTable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, enableTable_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, disableTable_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -39123,7 +40015,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, enableTable_result struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, enableTable_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, disableTable_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -39138,16 +40030,16 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, enableTable_result } - private static class enableTable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public enableTable_resultTupleScheme getScheme() { - return new enableTable_resultTupleScheme(); + private static class disableTable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public disableTable_resultTupleScheme getScheme() { + return new disableTable_resultTupleScheme(); } } - private static class enableTable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class disableTable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, enableTable_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, disableTable_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.isSetIo()) { @@ -39160,7 +40052,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, enableTable_result } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, enableTable_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, disableTable_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)) { @@ -39176,23 +40068,23 @@ private static S scheme(org.apache. } } - public static class disableTable_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("disableTable_args"); + public static class isTableEnabled_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableEnabled_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new disableTable_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new disableTable_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableEnabled_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableEnabled_argsTupleSchemeFactory(); /** - * the tablename to disable + * the tablename to check */ public @org.apache.thrift.annotation.Nullable TTableName tableName; // 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 { /** - * the tablename to disable + * the tablename to check */ TABLE_NAME((short)1, "tableName"); @@ -39259,13 +40151,13 @@ public java.lang.String getFieldName() { tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableName.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(disableTable_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableEnabled_args.class, metaDataMap); } - public disableTable_args() { + public isTableEnabled_args() { } - public disableTable_args( + public isTableEnabled_args( TTableName tableName) { this(); @@ -39275,14 +40167,14 @@ public disableTable_args( /** * Performs a deep copy on other. */ - public disableTable_args(disableTable_args other) { + public isTableEnabled_args(isTableEnabled_args other) { if (other.isSetTableName()) { this.tableName = new TTableName(other.tableName); } } - public disableTable_args deepCopy() { - return new disableTable_args(this); + public isTableEnabled_args deepCopy() { + return new isTableEnabled_args(this); } @Override @@ -39291,7 +40183,7 @@ public void clear() { } /** - * the tablename to disable + * the tablename to check */ @org.apache.thrift.annotation.Nullable public TTableName getTableName() { @@ -39299,9 +40191,9 @@ public TTableName getTableName() { } /** - * the tablename to disable + * the tablename to check */ - public disableTable_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) { + public isTableEnabled_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) { this.tableName = tableName; return this; } @@ -39361,12 +40253,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof disableTable_args) - return this.equals((disableTable_args)that); + if (that instanceof isTableEnabled_args) + return this.equals((isTableEnabled_args)that); return false; } - public boolean equals(disableTable_args that) { + public boolean equals(isTableEnabled_args that) { if (that == null) return false; if (this == that) @@ -39396,7 +40288,7 @@ public int hashCode() { } @Override - public int compareTo(disableTable_args other) { + public int compareTo(isTableEnabled_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -39431,7 +40323,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("disableTable_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableEnabled_args("); boolean first = true; sb.append("tableName:"); @@ -39472,15 +40364,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class disableTable_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public disableTable_argsStandardScheme getScheme() { - return new disableTable_argsStandardScheme(); + private static class isTableEnabled_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public isTableEnabled_argsStandardScheme getScheme() { + return new isTableEnabled_argsStandardScheme(); } } - private static class disableTable_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class isTableEnabled_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, disableTable_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, isTableEnabled_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -39510,7 +40402,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, disableTable_args s struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, disableTable_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, isTableEnabled_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -39525,22 +40417,22 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, disableTable_args } - private static class disableTable_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public disableTable_argsTupleScheme getScheme() { - return new disableTable_argsTupleScheme(); + private static class isTableEnabled_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public isTableEnabled_argsTupleScheme getScheme() { + return new isTableEnabled_argsTupleScheme(); } } - private static class disableTable_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class isTableEnabled_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, disableTable_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, isTableEnabled_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; struct.tableName.write(oprot); } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, disableTable_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, isTableEnabled_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; struct.tableName = new TTableName(); struct.tableName.read(iprot); @@ -39553,18 +40445,21 @@ private static S scheme(org.apache. } } - public static class disableTable_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("disableTable_result"); + public static class isTableEnabled_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableEnabled_result"); + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)0); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new disableTable_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new disableTable_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableEnabled_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableEnabled_resultTupleSchemeFactory(); + public boolean success; // required public @org.apache.thrift.annotation.Nullable TIOError io; // 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"), IO((short)1, "io"); private static final java.util.Map byName = new java.util.HashMap(); @@ -39581,6 +40476,8 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { @org.apache.thrift.annotation.Nullable public static _Fields findByThriftId(int fieldId) { switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; case 1: // IO return IO; default: @@ -39624,49 +40521,83 @@ public java.lang.String getFieldName() { } // isset id assignments + private static final int __SUCCESS_ISSET_ID = 0; + private byte __isset_bitfield = 0; 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.BOOL))); tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(disableTable_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableEnabled_result.class, metaDataMap); } - public disableTable_result() { + public isTableEnabled_result() { } - public disableTable_result( + public isTableEnabled_result( + boolean success, TIOError io) { this(); + this.success = success; + setSuccessIsSet(true); this.io = io; } /** * Performs a deep copy on other. */ - public disableTable_result(disableTable_result other) { + public isTableEnabled_result(isTableEnabled_result other) { + __isset_bitfield = other.__isset_bitfield; + this.success = other.success; if (other.isSetIo()) { this.io = new TIOError(other.io); } } - public disableTable_result deepCopy() { - return new disableTable_result(this); + public isTableEnabled_result deepCopy() { + return new isTableEnabled_result(this); } @Override public void clear() { + setSuccessIsSet(false); + this.success = false; this.io = null; } + public boolean isSuccess() { + return this.success; + } + + public isTableEnabled_result setSuccess(boolean success) { + this.success = success; + setSuccessIsSet(true); + return this; + } + + public void unsetSuccess() { + __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID); + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID); + } + + public void setSuccessIsSet(boolean value) { + __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value); + } + @org.apache.thrift.annotation.Nullable public TIOError getIo() { return this.io; } - public disableTable_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { + public isTableEnabled_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { this.io = io; return this; } @@ -39688,6 +40619,14 @@ public void setIoIsSet(boolean value) { 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.Boolean)value); + } + break; + case IO: if (value == null) { unsetIo(); @@ -39702,6 +40641,9 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable @org.apache.thrift.annotation.Nullable public java.lang.Object getFieldValue(_Fields field) { switch (field) { + case SUCCESS: + return isSuccess(); + case IO: return getIo(); @@ -39716,6 +40658,8 @@ public boolean isSet(_Fields field) { } switch (field) { + case SUCCESS: + return isSetSuccess(); case IO: return isSetIo(); } @@ -39726,17 +40670,26 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof disableTable_result) - return this.equals((disableTable_result)that); + if (that instanceof isTableEnabled_result) + return this.equals((isTableEnabled_result)that); return false; } - public boolean equals(disableTable_result that) { + public boolean equals(isTableEnabled_result that) { if (that == null) return false; if (this == that) return true; + boolean this_present_success = true; + boolean that_present_success = true; + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (this.success != that.success) + return false; + } + boolean this_present_io = true && this.isSetIo(); boolean that_present_io = true && that.isSetIo(); if (this_present_io || that_present_io) { @@ -39753,6 +40706,8 @@ public boolean equals(disableTable_result that) { public int hashCode() { int hashCode = 1; + hashCode = hashCode * 8191 + ((success) ? 131071 : 524287); + hashCode = hashCode * 8191 + ((isSetIo()) ? 131071 : 524287); if (isSetIo()) hashCode = hashCode * 8191 + io.hashCode(); @@ -39761,13 +40716,23 @@ public int hashCode() { } @Override - public int compareTo(disableTable_result other) { + public int compareTo(isTableEnabled_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; + lastComparison = java.lang.Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } lastComparison = java.lang.Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; @@ -39796,9 +40761,13 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("disableTable_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableEnabled_result("); boolean first = true; + sb.append("success:"); + sb.append(this.success); + first = false; + if (!first) sb.append(", "); sb.append("io:"); if (this.io == null) { sb.append("null"); @@ -39825,21 +40794,23 @@ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOExcept private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException { try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; 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 disableTable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public disableTable_resultStandardScheme getScheme() { - return new disableTable_resultStandardScheme(); + private static class isTableEnabled_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public isTableEnabled_resultStandardScheme getScheme() { + return new isTableEnabled_resultStandardScheme(); } } - private static class disableTable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class isTableEnabled_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, disableTable_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, isTableEnabled_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -39849,6 +40820,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, disableTable_result break; } switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { + struct.success = iprot.readBool(); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; case 1: // IO if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { struct.io = new TIOError(); @@ -39869,10 +40848,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, disableTable_result struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, disableTable_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, isTableEnabled_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); + if (struct.isSetSuccess()) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + oprot.writeBool(struct.success); + oprot.writeFieldEnd(); + } if (struct.io != null) { oprot.writeFieldBegin(IO_FIELD_DESC); struct.io.write(oprot); @@ -39884,32 +40868,42 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, disableTable_resul } - private static class disableTable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public disableTable_resultTupleScheme getScheme() { - return new disableTable_resultTupleScheme(); + private static class isTableEnabled_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public isTableEnabled_resultTupleScheme getScheme() { + return new isTableEnabled_resultTupleScheme(); } } - private static class disableTable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class isTableEnabled_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, disableTable_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, isTableEnabled_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.isSetIo()) { + if (struct.isSetSuccess()) { optionals.set(0); } - oprot.writeBitSet(optionals, 1); + if (struct.isSetIo()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetSuccess()) { + oprot.writeBool(struct.success); + } if (struct.isSetIo()) { struct.io.write(oprot); } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, disableTable_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, isTableEnabled_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); + java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { + struct.success = iprot.readBool(); + struct.setSuccessIsSet(true); + } + if (incoming.get(1)) { struct.io = new TIOError(); struct.io.read(iprot); struct.setIoIsSet(true); @@ -39922,13 +40916,13 @@ private static S scheme(org.apache. } } - public static class isTableEnabled_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableEnabled_args"); + public static class isTableDisabled_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableDisabled_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableEnabled_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableEnabled_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableDisabled_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableDisabled_argsTupleSchemeFactory(); /** * the tablename to check @@ -40005,13 +40999,13 @@ public java.lang.String getFieldName() { tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableName.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableEnabled_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableDisabled_args.class, metaDataMap); } - public isTableEnabled_args() { + public isTableDisabled_args() { } - public isTableEnabled_args( + public isTableDisabled_args( TTableName tableName) { this(); @@ -40021,14 +41015,14 @@ public isTableEnabled_args( /** * Performs a deep copy on other. */ - public isTableEnabled_args(isTableEnabled_args other) { + public isTableDisabled_args(isTableDisabled_args other) { if (other.isSetTableName()) { this.tableName = new TTableName(other.tableName); } } - public isTableEnabled_args deepCopy() { - return new isTableEnabled_args(this); + public isTableDisabled_args deepCopy() { + return new isTableDisabled_args(this); } @Override @@ -40047,7 +41041,7 @@ public TTableName getTableName() { /** * the tablename to check */ - public isTableEnabled_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) { + public isTableDisabled_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) { this.tableName = tableName; return this; } @@ -40107,12 +41101,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof isTableEnabled_args) - return this.equals((isTableEnabled_args)that); + if (that instanceof isTableDisabled_args) + return this.equals((isTableDisabled_args)that); return false; } - public boolean equals(isTableEnabled_args that) { + public boolean equals(isTableDisabled_args that) { if (that == null) return false; if (this == that) @@ -40142,7 +41136,7 @@ public int hashCode() { } @Override - public int compareTo(isTableEnabled_args other) { + public int compareTo(isTableDisabled_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -40177,7 +41171,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableEnabled_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableDisabled_args("); boolean first = true; sb.append("tableName:"); @@ -40218,15 +41212,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class isTableEnabled_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public isTableEnabled_argsStandardScheme getScheme() { - return new isTableEnabled_argsStandardScheme(); + private static class isTableDisabled_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public isTableDisabled_argsStandardScheme getScheme() { + return new isTableDisabled_argsStandardScheme(); } } - private static class isTableEnabled_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class isTableDisabled_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, isTableEnabled_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, isTableDisabled_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -40256,7 +41250,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, isTableEnabled_args struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, isTableEnabled_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, isTableDisabled_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -40271,22 +41265,22 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, isTableEnabled_arg } - private static class isTableEnabled_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public isTableEnabled_argsTupleScheme getScheme() { - return new isTableEnabled_argsTupleScheme(); + private static class isTableDisabled_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public isTableDisabled_argsTupleScheme getScheme() { + return new isTableDisabled_argsTupleScheme(); } } - private static class isTableEnabled_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class isTableDisabled_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, isTableEnabled_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, isTableDisabled_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; struct.tableName.write(oprot); } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, isTableEnabled_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, isTableDisabled_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; struct.tableName = new TTableName(); struct.tableName.read(iprot); @@ -40299,14 +41293,14 @@ private static S scheme(org.apache. } } - public static class isTableEnabled_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableEnabled_result"); + public static class isTableDisabled_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableDisabled_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)0); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableEnabled_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableEnabled_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableDisabled_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableDisabled_resultTupleSchemeFactory(); public boolean success; // required public @org.apache.thrift.annotation.Nullable TIOError io; // required @@ -40385,13 +41379,13 @@ public java.lang.String getFieldName() { tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableEnabled_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableDisabled_result.class, metaDataMap); } - public isTableEnabled_result() { + public isTableDisabled_result() { } - public isTableEnabled_result( + public isTableDisabled_result( boolean success, TIOError io) { @@ -40404,7 +41398,7 @@ public isTableEnabled_result( /** * Performs a deep copy on other. */ - public isTableEnabled_result(isTableEnabled_result other) { + public isTableDisabled_result(isTableDisabled_result other) { __isset_bitfield = other.__isset_bitfield; this.success = other.success; if (other.isSetIo()) { @@ -40412,8 +41406,8 @@ public isTableEnabled_result(isTableEnabled_result other) { } } - public isTableEnabled_result deepCopy() { - return new isTableEnabled_result(this); + public isTableDisabled_result deepCopy() { + return new isTableDisabled_result(this); } @Override @@ -40427,7 +41421,7 @@ public boolean isSuccess() { return this.success; } - public isTableEnabled_result setSuccess(boolean success) { + public isTableDisabled_result setSuccess(boolean success) { this.success = success; setSuccessIsSet(true); return this; @@ -40451,7 +41445,7 @@ public TIOError getIo() { return this.io; } - public isTableEnabled_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { + public isTableDisabled_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { this.io = io; return this; } @@ -40524,12 +41518,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof isTableEnabled_result) - return this.equals((isTableEnabled_result)that); + if (that instanceof isTableDisabled_result) + return this.equals((isTableDisabled_result)that); return false; } - public boolean equals(isTableEnabled_result that) { + public boolean equals(isTableDisabled_result that) { if (that == null) return false; if (this == that) @@ -40570,7 +41564,7 @@ public int hashCode() { } @Override - public int compareTo(isTableEnabled_result other) { + public int compareTo(isTableDisabled_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -40615,7 +41609,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableEnabled_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableDisabled_result("); boolean first = true; sb.append("success:"); @@ -40656,15 +41650,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class isTableEnabled_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public isTableEnabled_resultStandardScheme getScheme() { - return new isTableEnabled_resultStandardScheme(); + private static class isTableDisabled_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public isTableDisabled_resultStandardScheme getScheme() { + return new isTableDisabled_resultStandardScheme(); } } - private static class isTableEnabled_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class isTableDisabled_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, isTableEnabled_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, isTableDisabled_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -40702,7 +41696,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, isTableEnabled_resu struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, isTableEnabled_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, isTableDisabled_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -40722,16 +41716,16 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, isTableEnabled_res } - private static class isTableEnabled_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public isTableEnabled_resultTupleScheme getScheme() { - return new isTableEnabled_resultTupleScheme(); + private static class isTableDisabled_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public isTableDisabled_resultTupleScheme getScheme() { + return new isTableDisabled_resultTupleScheme(); } } - private static class isTableEnabled_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class isTableDisabled_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, isTableEnabled_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, isTableDisabled_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()) { @@ -40750,7 +41744,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, isTableEnabled_resu } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, isTableEnabled_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, isTableDisabled_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(2); if (incoming.get(0)) { @@ -40770,13 +41764,13 @@ private static S scheme(org.apache. } } - public static class isTableDisabled_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableDisabled_args"); + public static class isTableAvailable_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableAvailable_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableDisabled_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableDisabled_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableAvailable_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableAvailable_argsTupleSchemeFactory(); /** * the tablename to check @@ -40853,13 +41847,13 @@ public java.lang.String getFieldName() { tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableName.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableDisabled_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableAvailable_args.class, metaDataMap); } - public isTableDisabled_args() { + public isTableAvailable_args() { } - public isTableDisabled_args( + public isTableAvailable_args( TTableName tableName) { this(); @@ -40869,14 +41863,14 @@ public isTableDisabled_args( /** * Performs a deep copy on other. */ - public isTableDisabled_args(isTableDisabled_args other) { + public isTableAvailable_args(isTableAvailable_args other) { if (other.isSetTableName()) { this.tableName = new TTableName(other.tableName); } } - public isTableDisabled_args deepCopy() { - return new isTableDisabled_args(this); + public isTableAvailable_args deepCopy() { + return new isTableAvailable_args(this); } @Override @@ -40895,7 +41889,7 @@ public TTableName getTableName() { /** * the tablename to check */ - public isTableDisabled_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) { + public isTableAvailable_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) { this.tableName = tableName; return this; } @@ -40955,12 +41949,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof isTableDisabled_args) - return this.equals((isTableDisabled_args)that); + if (that instanceof isTableAvailable_args) + return this.equals((isTableAvailable_args)that); return false; } - public boolean equals(isTableDisabled_args that) { + public boolean equals(isTableAvailable_args that) { if (that == null) return false; if (this == that) @@ -40990,7 +41984,7 @@ public int hashCode() { } @Override - public int compareTo(isTableDisabled_args other) { + public int compareTo(isTableAvailable_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -41025,7 +42019,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableDisabled_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableAvailable_args("); boolean first = true; sb.append("tableName:"); @@ -41066,15 +42060,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class isTableDisabled_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public isTableDisabled_argsStandardScheme getScheme() { - return new isTableDisabled_argsStandardScheme(); + private static class isTableAvailable_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public isTableAvailable_argsStandardScheme getScheme() { + return new isTableAvailable_argsStandardScheme(); } } - private static class isTableDisabled_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class isTableAvailable_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, isTableDisabled_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, isTableAvailable_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -41104,7 +42098,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, isTableDisabled_arg struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, isTableDisabled_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, isTableAvailable_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -41119,22 +42113,22 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, isTableDisabled_ar } - private static class isTableDisabled_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public isTableDisabled_argsTupleScheme getScheme() { - return new isTableDisabled_argsTupleScheme(); + private static class isTableAvailable_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public isTableAvailable_argsTupleScheme getScheme() { + return new isTableAvailable_argsTupleScheme(); } } - private static class isTableDisabled_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class isTableAvailable_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, isTableDisabled_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, isTableAvailable_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; struct.tableName.write(oprot); } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, isTableDisabled_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, isTableAvailable_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; struct.tableName = new TTableName(); struct.tableName.read(iprot); @@ -41147,14 +42141,14 @@ private static S scheme(org.apache. } } - public static class isTableDisabled_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableDisabled_result"); + public static class isTableAvailable_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableAvailable_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)0); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableDisabled_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableDisabled_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableAvailable_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableAvailable_resultTupleSchemeFactory(); public boolean success; // required public @org.apache.thrift.annotation.Nullable TIOError io; // required @@ -41233,13 +42227,13 @@ public java.lang.String getFieldName() { tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableDisabled_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableAvailable_result.class, metaDataMap); } - public isTableDisabled_result() { + public isTableAvailable_result() { } - public isTableDisabled_result( + public isTableAvailable_result( boolean success, TIOError io) { @@ -41252,7 +42246,7 @@ public isTableDisabled_result( /** * Performs a deep copy on other. */ - public isTableDisabled_result(isTableDisabled_result other) { + public isTableAvailable_result(isTableAvailable_result other) { __isset_bitfield = other.__isset_bitfield; this.success = other.success; if (other.isSetIo()) { @@ -41260,8 +42254,8 @@ public isTableDisabled_result(isTableDisabled_result other) { } } - public isTableDisabled_result deepCopy() { - return new isTableDisabled_result(this); + public isTableAvailable_result deepCopy() { + return new isTableAvailable_result(this); } @Override @@ -41275,7 +42269,7 @@ public boolean isSuccess() { return this.success; } - public isTableDisabled_result setSuccess(boolean success) { + public isTableAvailable_result setSuccess(boolean success) { this.success = success; setSuccessIsSet(true); return this; @@ -41299,7 +42293,7 @@ public TIOError getIo() { return this.io; } - public isTableDisabled_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { + public isTableAvailable_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { this.io = io; return this; } @@ -41372,12 +42366,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof isTableDisabled_result) - return this.equals((isTableDisabled_result)that); + if (that instanceof isTableAvailable_result) + return this.equals((isTableAvailable_result)that); return false; } - public boolean equals(isTableDisabled_result that) { + public boolean equals(isTableAvailable_result that) { if (that == null) return false; if (this == that) @@ -41418,7 +42412,7 @@ public int hashCode() { } @Override - public int compareTo(isTableDisabled_result other) { + public int compareTo(isTableAvailable_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -41463,7 +42457,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableDisabled_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableAvailable_result("); boolean first = true; sb.append("success:"); @@ -41504,15 +42498,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class isTableDisabled_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public isTableDisabled_resultStandardScheme getScheme() { - return new isTableDisabled_resultStandardScheme(); + private static class isTableAvailable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public isTableAvailable_resultStandardScheme getScheme() { + return new isTableAvailable_resultStandardScheme(); } } - private static class isTableDisabled_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class isTableAvailable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, isTableDisabled_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, isTableAvailable_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -41550,7 +42544,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, isTableDisabled_res struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, isTableDisabled_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, isTableAvailable_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -41570,16 +42564,16 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, isTableDisabled_re } - private static class isTableDisabled_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public isTableDisabled_resultTupleScheme getScheme() { - return new isTableDisabled_resultTupleScheme(); + private static class isTableAvailable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public isTableAvailable_resultTupleScheme getScheme() { + return new isTableAvailable_resultTupleScheme(); } } - private static class isTableDisabled_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class isTableAvailable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, isTableDisabled_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, isTableAvailable_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()) { @@ -41598,7 +42592,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, isTableDisabled_res } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, isTableDisabled_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, isTableAvailable_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(2); if (incoming.get(0)) { @@ -41618,25 +42612,34 @@ private static S scheme(org.apache. } } - public static class isTableAvailable_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableAvailable_args"); + public static class isTableAvailableWithSplit_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableAvailableWithSplit_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField SPLIT_KEYS_FIELD_DESC = new org.apache.thrift.protocol.TField("splitKeys", org.apache.thrift.protocol.TType.LIST, (short)2); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableAvailable_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableAvailable_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableAvailableWithSplit_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableAvailableWithSplit_argsTupleSchemeFactory(); /** * the tablename to check */ public @org.apache.thrift.annotation.Nullable TTableName tableName; // required + /** + * keys to check if the table has been created with all split keys + */ + public @org.apache.thrift.annotation.Nullable java.util.List splitKeys; // 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 { /** * the tablename to check */ - TABLE_NAME((short)1, "tableName"); + TABLE_NAME((short)1, "tableName"), + /** + * keys to check if the table has been created with all split keys + */ + SPLIT_KEYS((short)2, "splitKeys"); private static final java.util.Map byName = new java.util.HashMap(); @@ -41654,6 +42657,8 @@ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // TABLE_NAME return TABLE_NAME; + case 2: // SPLIT_KEYS + return SPLIT_KEYS; default: return null; } @@ -41700,36 +42705,46 @@ public java.lang.String getFieldName() { 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.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableName.class))); + tmpMap.put(_Fields.SPLIT_KEYS, new org.apache.thrift.meta_data.FieldMetaData("splitKeys", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true)))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableAvailable_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableAvailableWithSplit_args.class, metaDataMap); } - public isTableAvailable_args() { + public isTableAvailableWithSplit_args() { } - public isTableAvailable_args( - TTableName tableName) + public isTableAvailableWithSplit_args( + TTableName tableName, + java.util.List splitKeys) { this(); this.tableName = tableName; + this.splitKeys = splitKeys; } /** * Performs a deep copy on other. */ - public isTableAvailable_args(isTableAvailable_args other) { + public isTableAvailableWithSplit_args(isTableAvailableWithSplit_args other) { if (other.isSetTableName()) { this.tableName = new TTableName(other.tableName); } + if (other.isSetSplitKeys()) { + java.util.List __this__splitKeys = new java.util.ArrayList(other.splitKeys); + this.splitKeys = __this__splitKeys; + } } - public isTableAvailable_args deepCopy() { - return new isTableAvailable_args(this); + public isTableAvailableWithSplit_args deepCopy() { + return new isTableAvailableWithSplit_args(this); } @Override public void clear() { this.tableName = null; + this.splitKeys = null; } /** @@ -41743,7 +42758,7 @@ public TTableName getTableName() { /** * the tablename to check */ - public isTableAvailable_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) { + public isTableAvailableWithSplit_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) { this.tableName = tableName; return this; } @@ -41763,6 +42778,53 @@ public void setTableNameIsSet(boolean value) { } } + public int getSplitKeysSize() { + return (this.splitKeys == null) ? 0 : this.splitKeys.size(); + } + + @org.apache.thrift.annotation.Nullable + public java.util.Iterator getSplitKeysIterator() { + return (this.splitKeys == null) ? null : this.splitKeys.iterator(); + } + + public void addToSplitKeys(java.nio.ByteBuffer elem) { + if (this.splitKeys == null) { + this.splitKeys = new java.util.ArrayList(); + } + this.splitKeys.add(elem); + } + + /** + * keys to check if the table has been created with all split keys + */ + @org.apache.thrift.annotation.Nullable + public java.util.List getSplitKeys() { + return this.splitKeys; + } + + /** + * keys to check if the table has been created with all split keys + */ + public isTableAvailableWithSplit_args setSplitKeys(@org.apache.thrift.annotation.Nullable java.util.List splitKeys) { + this.splitKeys = splitKeys; + return this; + } + + public void unsetSplitKeys() { + this.splitKeys = null; + } + + /** Returns true if field splitKeys is set (has been assigned a value) and false otherwise */ + public boolean isSetSplitKeys() { + return this.splitKeys != null; + } + + public void setSplitKeysIsSet(boolean value) { + if (!value) { + this.splitKeys = null; + } + } + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { switch (field) { case TABLE_NAME: @@ -41773,6 +42835,14 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable } break; + case SPLIT_KEYS: + if (value == null) { + unsetSplitKeys(); + } else { + setSplitKeys((java.util.List)value); + } + break; + } } @@ -41782,6 +42852,9 @@ public java.lang.Object getFieldValue(_Fields field) { case TABLE_NAME: return getTableName(); + case SPLIT_KEYS: + return getSplitKeys(); + } throw new java.lang.IllegalStateException(); } @@ -41795,6 +42868,8 @@ public boolean isSet(_Fields field) { switch (field) { case TABLE_NAME: return isSetTableName(); + case SPLIT_KEYS: + return isSetSplitKeys(); } throw new java.lang.IllegalStateException(); } @@ -41803,12 +42878,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof isTableAvailable_args) - return this.equals((isTableAvailable_args)that); + if (that instanceof isTableAvailableWithSplit_args) + return this.equals((isTableAvailableWithSplit_args)that); return false; } - public boolean equals(isTableAvailable_args that) { + public boolean equals(isTableAvailableWithSplit_args that) { if (that == null) return false; if (this == that) @@ -41823,6 +42898,15 @@ public boolean equals(isTableAvailable_args that) { return false; } + boolean this_present_splitKeys = true && this.isSetSplitKeys(); + boolean that_present_splitKeys = true && that.isSetSplitKeys(); + if (this_present_splitKeys || that_present_splitKeys) { + if (!(this_present_splitKeys && that_present_splitKeys)) + return false; + if (!this.splitKeys.equals(that.splitKeys)) + return false; + } + return true; } @@ -41834,11 +42918,15 @@ public int hashCode() { if (isSetTableName()) hashCode = hashCode * 8191 + tableName.hashCode(); + hashCode = hashCode * 8191 + ((isSetSplitKeys()) ? 131071 : 524287); + if (isSetSplitKeys()) + hashCode = hashCode * 8191 + splitKeys.hashCode(); + return hashCode; } @Override - public int compareTo(isTableAvailable_args other) { + public int compareTo(isTableAvailableWithSplit_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -41855,6 +42943,16 @@ public int compareTo(isTableAvailable_args other) { return lastComparison; } } + lastComparison = java.lang.Boolean.valueOf(isSetSplitKeys()).compareTo(other.isSetSplitKeys()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSplitKeys()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.splitKeys, other.splitKeys); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -41873,7 +42971,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableAvailable_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableAvailableWithSplit_args("); boolean first = true; sb.append("tableName:"); @@ -41883,6 +42981,14 @@ public java.lang.String toString() { sb.append(this.tableName); } first = false; + if (!first) sb.append(", "); + sb.append("splitKeys:"); + if (this.splitKeys == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.splitKeys, sb); + } + first = false; sb.append(")"); return sb.toString(); } @@ -41914,15 +43020,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class isTableAvailable_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public isTableAvailable_argsStandardScheme getScheme() { - return new isTableAvailable_argsStandardScheme(); + private static class isTableAvailableWithSplit_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public isTableAvailableWithSplit_argsStandardScheme getScheme() { + return new isTableAvailableWithSplit_argsStandardScheme(); } } - private static class isTableAvailable_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class isTableAvailableWithSplit_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, isTableAvailable_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, isTableAvailableWithSplit_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -41941,6 +43047,24 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, isTableAvailable_ar org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; + case 2: // SPLIT_KEYS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list326 = iprot.readListBegin(); + struct.splitKeys = new java.util.ArrayList(_list326.size); + @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem327; + for (int _i328 = 0; _i328 < _list326.size; ++_i328) + { + _elem327 = iprot.readBinary(); + struct.splitKeys.add(_elem327); + } + iprot.readListEnd(); + } + struct.setSplitKeysIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -41952,7 +43076,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, isTableAvailable_ar struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, isTableAvailable_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, isTableAvailableWithSplit_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -41961,32 +43085,72 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, isTableAvailable_a struct.tableName.write(oprot); oprot.writeFieldEnd(); } + if (struct.splitKeys != null) { + oprot.writeFieldBegin(SPLIT_KEYS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.splitKeys.size())); + for (java.nio.ByteBuffer _iter329 : struct.splitKeys) + { + oprot.writeBinary(_iter329); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } oprot.writeFieldStop(); oprot.writeStructEnd(); } } - private static class isTableAvailable_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public isTableAvailable_argsTupleScheme getScheme() { - return new isTableAvailable_argsTupleScheme(); + private static class isTableAvailableWithSplit_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public isTableAvailableWithSplit_argsTupleScheme getScheme() { + return new isTableAvailableWithSplit_argsTupleScheme(); } } - private static class isTableAvailable_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class isTableAvailableWithSplit_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, isTableAvailable_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, isTableAvailableWithSplit_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; struct.tableName.write(oprot); + java.util.BitSet optionals = new java.util.BitSet(); + if (struct.isSetSplitKeys()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSplitKeys()) { + { + oprot.writeI32(struct.splitKeys.size()); + for (java.nio.ByteBuffer _iter330 : struct.splitKeys) + { + oprot.writeBinary(_iter330); + } + } + } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, isTableAvailable_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, isTableAvailableWithSplit_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; struct.tableName = new TTableName(); struct.tableName.read(iprot); struct.setTableNameIsSet(true); + java.util.BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + { + org.apache.thrift.protocol.TList _list331 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.splitKeys = new java.util.ArrayList(_list331.size); + @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem332; + for (int _i333 = 0; _i333 < _list331.size; ++_i333) + { + _elem332 = iprot.readBinary(); + struct.splitKeys.add(_elem332); + } + } + struct.setSplitKeysIsSet(true); + } } } @@ -41995,14 +43159,14 @@ private static S scheme(org.apache. } } - public static class isTableAvailable_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableAvailable_result"); + public static class isTableAvailableWithSplit_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableAvailableWithSplit_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)0); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableAvailable_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableAvailable_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableAvailableWithSplit_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableAvailableWithSplit_resultTupleSchemeFactory(); public boolean success; // required public @org.apache.thrift.annotation.Nullable TIOError io; // required @@ -42081,13 +43245,13 @@ public java.lang.String getFieldName() { tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableAvailable_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableAvailableWithSplit_result.class, metaDataMap); } - public isTableAvailable_result() { + public isTableAvailableWithSplit_result() { } - public isTableAvailable_result( + public isTableAvailableWithSplit_result( boolean success, TIOError io) { @@ -42100,7 +43264,7 @@ public isTableAvailable_result( /** * Performs a deep copy on other. */ - public isTableAvailable_result(isTableAvailable_result other) { + public isTableAvailableWithSplit_result(isTableAvailableWithSplit_result other) { __isset_bitfield = other.__isset_bitfield; this.success = other.success; if (other.isSetIo()) { @@ -42108,8 +43272,8 @@ public isTableAvailable_result(isTableAvailable_result other) { } } - public isTableAvailable_result deepCopy() { - return new isTableAvailable_result(this); + public isTableAvailableWithSplit_result deepCopy() { + return new isTableAvailableWithSplit_result(this); } @Override @@ -42123,7 +43287,7 @@ public boolean isSuccess() { return this.success; } - public isTableAvailable_result setSuccess(boolean success) { + public isTableAvailableWithSplit_result setSuccess(boolean success) { this.success = success; setSuccessIsSet(true); return this; @@ -42147,7 +43311,7 @@ public TIOError getIo() { return this.io; } - public isTableAvailable_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { + public isTableAvailableWithSplit_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { this.io = io; return this; } @@ -42220,12 +43384,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof isTableAvailable_result) - return this.equals((isTableAvailable_result)that); + if (that instanceof isTableAvailableWithSplit_result) + return this.equals((isTableAvailableWithSplit_result)that); return false; } - public boolean equals(isTableAvailable_result that) { + public boolean equals(isTableAvailableWithSplit_result that) { if (that == null) return false; if (this == that) @@ -42266,7 +43430,7 @@ public int hashCode() { } @Override - public int compareTo(isTableAvailable_result other) { + public int compareTo(isTableAvailableWithSplit_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -42311,7 +43475,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableAvailable_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableAvailableWithSplit_result("); boolean first = true; sb.append("success:"); @@ -42352,15 +43516,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class isTableAvailable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public isTableAvailable_resultStandardScheme getScheme() { - return new isTableAvailable_resultStandardScheme(); + private static class isTableAvailableWithSplit_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public isTableAvailableWithSplit_resultStandardScheme getScheme() { + return new isTableAvailableWithSplit_resultStandardScheme(); } } - private static class isTableAvailable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class isTableAvailableWithSplit_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, isTableAvailable_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, isTableAvailableWithSplit_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -42398,7 +43562,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, isTableAvailable_re struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, isTableAvailable_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, isTableAvailableWithSplit_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -42418,16 +43582,16 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, isTableAvailable_r } - private static class isTableAvailable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public isTableAvailable_resultTupleScheme getScheme() { - return new isTableAvailable_resultTupleScheme(); + private static class isTableAvailableWithSplit_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public isTableAvailableWithSplit_resultTupleScheme getScheme() { + return new isTableAvailableWithSplit_resultTupleScheme(); } } - private static class isTableAvailable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class isTableAvailableWithSplit_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, isTableAvailable_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, isTableAvailableWithSplit_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()) { @@ -42446,7 +43610,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, isTableAvailable_re } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, isTableAvailable_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, isTableAvailableWithSplit_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(2); if (incoming.get(0)) { @@ -42466,34 +43630,34 @@ private static S scheme(org.apache. } } - public static class isTableAvailableWithSplit_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableAvailableWithSplit_args"); + public static class addColumnFamily_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("addColumnFamily_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField SPLIT_KEYS_FIELD_DESC = new org.apache.thrift.protocol.TField("splitKeys", org.apache.thrift.protocol.TType.LIST, (short)2); + private static final org.apache.thrift.protocol.TField COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("column", org.apache.thrift.protocol.TType.STRUCT, (short)2); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableAvailableWithSplit_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableAvailableWithSplit_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new addColumnFamily_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new addColumnFamily_argsTupleSchemeFactory(); /** - * the tablename to check + * the tablename to add column family to */ public @org.apache.thrift.annotation.Nullable TTableName tableName; // required /** - * keys to check if the table has been created with all split keys + * column family descriptor of column family to be added */ - public @org.apache.thrift.annotation.Nullable java.util.List splitKeys; // required + public @org.apache.thrift.annotation.Nullable TColumnFamilyDescriptor column; // 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 { /** - * the tablename to check + * the tablename to add column family to */ TABLE_NAME((short)1, "tableName"), /** - * keys to check if the table has been created with all split keys + * column family descriptor of column family to be added */ - SPLIT_KEYS((short)2, "splitKeys"); + COLUMN((short)2, "column"); private static final java.util.Map byName = new java.util.HashMap(); @@ -42511,8 +43675,8 @@ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // TABLE_NAME return TABLE_NAME; - case 2: // SPLIT_KEYS - return SPLIT_KEYS; + case 2: // COLUMN + return COLUMN; default: return null; } @@ -42559,50 +43723,48 @@ public java.lang.String getFieldName() { 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.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableName.class))); - tmpMap.put(_Fields.SPLIT_KEYS, new org.apache.thrift.meta_data.FieldMetaData("splitKeys", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true)))); + tmpMap.put(_Fields.COLUMN, new org.apache.thrift.meta_data.FieldMetaData("column", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TColumnFamilyDescriptor.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableAvailableWithSplit_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(addColumnFamily_args.class, metaDataMap); } - public isTableAvailableWithSplit_args() { + public addColumnFamily_args() { } - public isTableAvailableWithSplit_args( + public addColumnFamily_args( TTableName tableName, - java.util.List splitKeys) + TColumnFamilyDescriptor column) { this(); this.tableName = tableName; - this.splitKeys = splitKeys; + this.column = column; } /** * Performs a deep copy on other. */ - public isTableAvailableWithSplit_args(isTableAvailableWithSplit_args other) { + public addColumnFamily_args(addColumnFamily_args other) { if (other.isSetTableName()) { this.tableName = new TTableName(other.tableName); } - if (other.isSetSplitKeys()) { - java.util.List __this__splitKeys = new java.util.ArrayList(other.splitKeys); - this.splitKeys = __this__splitKeys; + if (other.isSetColumn()) { + this.column = new TColumnFamilyDescriptor(other.column); } } - public isTableAvailableWithSplit_args deepCopy() { - return new isTableAvailableWithSplit_args(this); + public addColumnFamily_args deepCopy() { + return new addColumnFamily_args(this); } @Override public void clear() { this.tableName = null; - this.splitKeys = null; + this.column = null; } /** - * the tablename to check + * the tablename to add column family to */ @org.apache.thrift.annotation.Nullable public TTableName getTableName() { @@ -42610,9 +43772,9 @@ public TTableName getTableName() { } /** - * the tablename to check + * the tablename to add column family to */ - public isTableAvailableWithSplit_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) { + public addColumnFamily_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) { this.tableName = tableName; return this; } @@ -42632,50 +43794,34 @@ public void setTableNameIsSet(boolean value) { } } - public int getSplitKeysSize() { - return (this.splitKeys == null) ? 0 : this.splitKeys.size(); - } - - @org.apache.thrift.annotation.Nullable - public java.util.Iterator getSplitKeysIterator() { - return (this.splitKeys == null) ? null : this.splitKeys.iterator(); - } - - public void addToSplitKeys(java.nio.ByteBuffer elem) { - if (this.splitKeys == null) { - this.splitKeys = new java.util.ArrayList(); - } - this.splitKeys.add(elem); - } - /** - * keys to check if the table has been created with all split keys + * column family descriptor of column family to be added */ @org.apache.thrift.annotation.Nullable - public java.util.List getSplitKeys() { - return this.splitKeys; + public TColumnFamilyDescriptor getColumn() { + return this.column; } /** - * keys to check if the table has been created with all split keys + * column family descriptor of column family to be added */ - public isTableAvailableWithSplit_args setSplitKeys(@org.apache.thrift.annotation.Nullable java.util.List splitKeys) { - this.splitKeys = splitKeys; + public addColumnFamily_args setColumn(@org.apache.thrift.annotation.Nullable TColumnFamilyDescriptor column) { + this.column = column; return this; } - public void unsetSplitKeys() { - this.splitKeys = null; + public void unsetColumn() { + this.column = null; } - /** Returns true if field splitKeys is set (has been assigned a value) and false otherwise */ - public boolean isSetSplitKeys() { - return this.splitKeys != null; + /** Returns true if field column is set (has been assigned a value) and false otherwise */ + public boolean isSetColumn() { + return this.column != null; } - public void setSplitKeysIsSet(boolean value) { + public void setColumnIsSet(boolean value) { if (!value) { - this.splitKeys = null; + this.column = null; } } @@ -42689,11 +43835,11 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable } break; - case SPLIT_KEYS: + case COLUMN: if (value == null) { - unsetSplitKeys(); + unsetColumn(); } else { - setSplitKeys((java.util.List)value); + setColumn((TColumnFamilyDescriptor)value); } break; @@ -42706,8 +43852,8 @@ public java.lang.Object getFieldValue(_Fields field) { case TABLE_NAME: return getTableName(); - case SPLIT_KEYS: - return getSplitKeys(); + case COLUMN: + return getColumn(); } throw new java.lang.IllegalStateException(); @@ -42722,8 +43868,8 @@ public boolean isSet(_Fields field) { switch (field) { case TABLE_NAME: return isSetTableName(); - case SPLIT_KEYS: - return isSetSplitKeys(); + case COLUMN: + return isSetColumn(); } throw new java.lang.IllegalStateException(); } @@ -42732,12 +43878,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof isTableAvailableWithSplit_args) - return this.equals((isTableAvailableWithSplit_args)that); + if (that instanceof addColumnFamily_args) + return this.equals((addColumnFamily_args)that); return false; } - public boolean equals(isTableAvailableWithSplit_args that) { + public boolean equals(addColumnFamily_args that) { if (that == null) return false; if (this == that) @@ -42752,12 +43898,12 @@ public boolean equals(isTableAvailableWithSplit_args that) { return false; } - boolean this_present_splitKeys = true && this.isSetSplitKeys(); - boolean that_present_splitKeys = true && that.isSetSplitKeys(); - if (this_present_splitKeys || that_present_splitKeys) { - if (!(this_present_splitKeys && that_present_splitKeys)) + boolean this_present_column = true && this.isSetColumn(); + boolean that_present_column = true && that.isSetColumn(); + if (this_present_column || that_present_column) { + if (!(this_present_column && that_present_column)) return false; - if (!this.splitKeys.equals(that.splitKeys)) + if (!this.column.equals(that.column)) return false; } @@ -42772,15 +43918,15 @@ public int hashCode() { if (isSetTableName()) hashCode = hashCode * 8191 + tableName.hashCode(); - hashCode = hashCode * 8191 + ((isSetSplitKeys()) ? 131071 : 524287); - if (isSetSplitKeys()) - hashCode = hashCode * 8191 + splitKeys.hashCode(); + hashCode = hashCode * 8191 + ((isSetColumn()) ? 131071 : 524287); + if (isSetColumn()) + hashCode = hashCode * 8191 + column.hashCode(); return hashCode; } @Override - public int compareTo(isTableAvailableWithSplit_args other) { + public int compareTo(addColumnFamily_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -42797,12 +43943,12 @@ public int compareTo(isTableAvailableWithSplit_args other) { return lastComparison; } } - lastComparison = java.lang.Boolean.valueOf(isSetSplitKeys()).compareTo(other.isSetSplitKeys()); + lastComparison = java.lang.Boolean.valueOf(isSetColumn()).compareTo(other.isSetColumn()); if (lastComparison != 0) { return lastComparison; } - if (isSetSplitKeys()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.splitKeys, other.splitKeys); + if (isSetColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.column, other.column); if (lastComparison != 0) { return lastComparison; } @@ -42825,7 +43971,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableAvailableWithSplit_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("addColumnFamily_args("); boolean first = true; sb.append("tableName:"); @@ -42836,11 +43982,11 @@ public java.lang.String toString() { } first = false; if (!first) sb.append(", "); - sb.append("splitKeys:"); - if (this.splitKeys == null) { + sb.append("column:"); + if (this.column == null) { sb.append("null"); } else { - org.apache.thrift.TBaseHelper.toString(this.splitKeys, sb); + sb.append(this.column); } first = false; sb.append(")"); @@ -42852,10 +43998,16 @@ public void validate() throws org.apache.thrift.TException { if (tableName == null) { throw new org.apache.thrift.protocol.TProtocolException("Required field 'tableName' was not present! Struct: " + toString()); } + if (column == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'column' was not present! Struct: " + toString()); + } // check for sub-struct validity if (tableName != null) { tableName.validate(); } + if (column != null) { + column.validate(); + } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -42874,15 +44026,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class isTableAvailableWithSplit_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public isTableAvailableWithSplit_argsStandardScheme getScheme() { - return new isTableAvailableWithSplit_argsStandardScheme(); + private static class addColumnFamily_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public addColumnFamily_argsStandardScheme getScheme() { + return new addColumnFamily_argsStandardScheme(); } } - private static class isTableAvailableWithSplit_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class addColumnFamily_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, isTableAvailableWithSplit_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, addColumnFamily_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -42901,20 +44053,11 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, isTableAvailableWit org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 2: // SPLIT_KEYS - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list326 = iprot.readListBegin(); - struct.splitKeys = new java.util.ArrayList(_list326.size); - @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem327; - for (int _i328 = 0; _i328 < _list326.size; ++_i328) - { - _elem327 = iprot.readBinary(); - struct.splitKeys.add(_elem327); - } - iprot.readListEnd(); - } - struct.setSplitKeysIsSet(true); + case 2: // COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.column = new TColumnFamilyDescriptor(); + struct.column.read(iprot); + struct.setColumnIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -42930,7 +44073,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, isTableAvailableWit struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, isTableAvailableWithSplit_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, addColumnFamily_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -42939,16 +44082,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, isTableAvailableWi struct.tableName.write(oprot); oprot.writeFieldEnd(); } - if (struct.splitKeys != null) { - oprot.writeFieldBegin(SPLIT_KEYS_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.splitKeys.size())); - for (java.nio.ByteBuffer _iter329 : struct.splitKeys) - { - oprot.writeBinary(_iter329); - } - oprot.writeListEnd(); - } + if (struct.column != null) { + oprot.writeFieldBegin(COLUMN_FIELD_DESC); + struct.column.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); @@ -42957,54 +44093,30 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, isTableAvailableWi } - private static class isTableAvailableWithSplit_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public isTableAvailableWithSplit_argsTupleScheme getScheme() { - return new isTableAvailableWithSplit_argsTupleScheme(); + private static class addColumnFamily_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public addColumnFamily_argsTupleScheme getScheme() { + return new addColumnFamily_argsTupleScheme(); } } - private static class isTableAvailableWithSplit_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class addColumnFamily_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, isTableAvailableWithSplit_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, addColumnFamily_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; struct.tableName.write(oprot); - java.util.BitSet optionals = new java.util.BitSet(); - if (struct.isSetSplitKeys()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSplitKeys()) { - { - oprot.writeI32(struct.splitKeys.size()); - for (java.nio.ByteBuffer _iter330 : struct.splitKeys) - { - oprot.writeBinary(_iter330); - } - } - } + struct.column.write(oprot); } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, isTableAvailableWithSplit_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, addColumnFamily_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; struct.tableName = new TTableName(); struct.tableName.read(iprot); struct.setTableNameIsSet(true); - java.util.BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - { - org.apache.thrift.protocol.TList _list331 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.splitKeys = new java.util.ArrayList(_list331.size); - @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem332; - for (int _i333 = 0; _i333 < _list331.size; ++_i333) - { - _elem332 = iprot.readBinary(); - struct.splitKeys.add(_elem332); - } - } - struct.setSplitKeysIsSet(true); - } + struct.column = new TColumnFamilyDescriptor(); + struct.column.read(iprot); + struct.setColumnIsSet(true); } } @@ -43013,21 +44125,18 @@ private static S scheme(org.apache. } } - public static class isTableAvailableWithSplit_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableAvailableWithSplit_result"); + public static class addColumnFamily_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("addColumnFamily_result"); - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)0); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableAvailableWithSplit_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableAvailableWithSplit_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new addColumnFamily_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new addColumnFamily_resultTupleSchemeFactory(); - public boolean success; // required public @org.apache.thrift.annotation.Nullable TIOError io; // 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"), IO((short)1, "io"); private static final java.util.Map byName = new java.util.HashMap(); @@ -43044,8 +44153,6 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { @org.apache.thrift.annotation.Nullable public static _Fields findByThriftId(int fieldId) { switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; case 1: // IO return IO; default: @@ -43089,83 +44196,49 @@ public java.lang.String getFieldName() { } // isset id assignments - private static final int __SUCCESS_ISSET_ID = 0; - private byte __isset_bitfield = 0; 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.BOOL))); tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableAvailableWithSplit_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(addColumnFamily_result.class, metaDataMap); } - public isTableAvailableWithSplit_result() { + public addColumnFamily_result() { } - public isTableAvailableWithSplit_result( - boolean success, + public addColumnFamily_result( TIOError io) { this(); - this.success = success; - setSuccessIsSet(true); this.io = io; } /** * Performs a deep copy on other. */ - public isTableAvailableWithSplit_result(isTableAvailableWithSplit_result other) { - __isset_bitfield = other.__isset_bitfield; - this.success = other.success; + public addColumnFamily_result(addColumnFamily_result other) { if (other.isSetIo()) { this.io = new TIOError(other.io); } } - public isTableAvailableWithSplit_result deepCopy() { - return new isTableAvailableWithSplit_result(this); + public addColumnFamily_result deepCopy() { + return new addColumnFamily_result(this); } @Override public void clear() { - setSuccessIsSet(false); - this.success = false; this.io = null; } - public boolean isSuccess() { - return this.success; - } - - public isTableAvailableWithSplit_result setSuccess(boolean success) { - this.success = success; - setSuccessIsSet(true); - return this; - } - - public void unsetSuccess() { - __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID); - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID); - } - - public void setSuccessIsSet(boolean value) { - __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value); - } - @org.apache.thrift.annotation.Nullable public TIOError getIo() { return this.io; } - public isTableAvailableWithSplit_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { + public addColumnFamily_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { this.io = io; return this; } @@ -43187,14 +44260,6 @@ public void setIoIsSet(boolean value) { 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.Boolean)value); - } - break; - case IO: if (value == null) { unsetIo(); @@ -43209,9 +44274,6 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable @org.apache.thrift.annotation.Nullable public java.lang.Object getFieldValue(_Fields field) { switch (field) { - case SUCCESS: - return isSuccess(); - case IO: return getIo(); @@ -43226,8 +44288,6 @@ public boolean isSet(_Fields field) { } switch (field) { - case SUCCESS: - return isSetSuccess(); case IO: return isSetIo(); } @@ -43238,26 +44298,17 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof isTableAvailableWithSplit_result) - return this.equals((isTableAvailableWithSplit_result)that); + if (that instanceof addColumnFamily_result) + return this.equals((addColumnFamily_result)that); return false; } - public boolean equals(isTableAvailableWithSplit_result that) { + public boolean equals(addColumnFamily_result that) { if (that == null) return false; if (this == that) return true; - boolean this_present_success = true; - boolean that_present_success = true; - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (this.success != that.success) - return false; - } - boolean this_present_io = true && this.isSetIo(); boolean that_present_io = true && that.isSetIo(); if (this_present_io || that_present_io) { @@ -43274,8 +44325,6 @@ public boolean equals(isTableAvailableWithSplit_result that) { public int hashCode() { int hashCode = 1; - hashCode = hashCode * 8191 + ((success) ? 131071 : 524287); - hashCode = hashCode * 8191 + ((isSetIo()) ? 131071 : 524287); if (isSetIo()) hashCode = hashCode * 8191 + io.hashCode(); @@ -43284,23 +44333,13 @@ public int hashCode() { } @Override - public int compareTo(isTableAvailableWithSplit_result other) { + public int compareTo(addColumnFamily_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - lastComparison = java.lang.Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); - if (lastComparison != 0) { - return lastComparison; - } - } lastComparison = java.lang.Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; @@ -43329,13 +44368,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableAvailableWithSplit_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("addColumnFamily_result("); boolean first = true; - sb.append("success:"); - sb.append(this.success); - first = false; - if (!first) sb.append(", "); sb.append("io:"); if (this.io == null) { sb.append("null"); @@ -43362,23 +44397,21 @@ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOExcept private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException { try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; 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 isTableAvailableWithSplit_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public isTableAvailableWithSplit_resultStandardScheme getScheme() { - return new isTableAvailableWithSplit_resultStandardScheme(); + private static class addColumnFamily_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public addColumnFamily_resultStandardScheme getScheme() { + return new addColumnFamily_resultStandardScheme(); } } - private static class isTableAvailableWithSplit_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class addColumnFamily_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, isTableAvailableWithSplit_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, addColumnFamily_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -43388,14 +44421,6 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, isTableAvailableWit break; } switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { - struct.success = iprot.readBool(); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; case 1: // IO if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { struct.io = new TIOError(); @@ -43416,15 +44441,10 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, isTableAvailableWit struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, isTableAvailableWithSplit_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, addColumnFamily_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - if (struct.isSetSuccess()) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - oprot.writeBool(struct.success); - oprot.writeFieldEnd(); - } if (struct.io != null) { oprot.writeFieldBegin(IO_FIELD_DESC); struct.io.write(oprot); @@ -43436,42 +44456,32 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, isTableAvailableWi } - private static class isTableAvailableWithSplit_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public isTableAvailableWithSplit_resultTupleScheme getScheme() { - return new isTableAvailableWithSplit_resultTupleScheme(); + private static class addColumnFamily_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public addColumnFamily_resultTupleScheme getScheme() { + return new addColumnFamily_resultTupleScheme(); } } - private static class isTableAvailableWithSplit_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class addColumnFamily_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, isTableAvailableWithSplit_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, addColumnFamily_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); - } if (struct.isSetIo()) { - optionals.set(1); - } - oprot.writeBitSet(optionals, 2); - if (struct.isSetSuccess()) { - oprot.writeBool(struct.success); + optionals.set(0); } + oprot.writeBitSet(optionals, 1); if (struct.isSetIo()) { struct.io.write(oprot); } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, isTableAvailableWithSplit_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, addColumnFamily_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(2); + java.util.BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { - struct.success = iprot.readBool(); - struct.setSuccessIsSet(true); - } - if (incoming.get(1)) { struct.io = new TIOError(); struct.io.read(iprot); struct.setIoIsSet(true); @@ -43484,32 +44494,32 @@ private static S scheme(org.apache. } } - public static class addColumnFamily_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("addColumnFamily_args"); + public static class deleteColumnFamily_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteColumnFamily_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("column", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("column", org.apache.thrift.protocol.TType.STRING, (short)2); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new addColumnFamily_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new addColumnFamily_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new deleteColumnFamily_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new deleteColumnFamily_argsTupleSchemeFactory(); /** - * the tablename to add column family to + * the tablename to delete column family from */ public @org.apache.thrift.annotation.Nullable TTableName tableName; // required /** - * column family descriptor of column family to be added + * name of column family to be deleted */ - public @org.apache.thrift.annotation.Nullable TColumnFamilyDescriptor column; // required + public @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer column; // 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 { /** - * the tablename to add column family to + * the tablename to delete column family from */ TABLE_NAME((short)1, "tableName"), /** - * column family descriptor of column family to be added + * name of column family to be deleted */ COLUMN((short)2, "column"); @@ -43578,37 +44588,37 @@ public java.lang.String getFieldName() { tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableName.class))); tmpMap.put(_Fields.COLUMN, new org.apache.thrift.meta_data.FieldMetaData("column", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TColumnFamilyDescriptor.class))); + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(addColumnFamily_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(deleteColumnFamily_args.class, metaDataMap); } - public addColumnFamily_args() { + public deleteColumnFamily_args() { } - public addColumnFamily_args( + public deleteColumnFamily_args( TTableName tableName, - TColumnFamilyDescriptor column) + java.nio.ByteBuffer column) { this(); this.tableName = tableName; - this.column = column; + this.column = org.apache.thrift.TBaseHelper.copyBinary(column); } /** * Performs a deep copy on other. */ - public addColumnFamily_args(addColumnFamily_args other) { + public deleteColumnFamily_args(deleteColumnFamily_args other) { if (other.isSetTableName()) { this.tableName = new TTableName(other.tableName); } if (other.isSetColumn()) { - this.column = new TColumnFamilyDescriptor(other.column); + this.column = org.apache.thrift.TBaseHelper.copyBinary(other.column); } } - public addColumnFamily_args deepCopy() { - return new addColumnFamily_args(this); + public deleteColumnFamily_args deepCopy() { + return new deleteColumnFamily_args(this); } @Override @@ -43618,7 +44628,7 @@ public void clear() { } /** - * the tablename to add column family to + * the tablename to delete column family from */ @org.apache.thrift.annotation.Nullable public TTableName getTableName() { @@ -43626,9 +44636,9 @@ public TTableName getTableName() { } /** - * the tablename to add column family to + * the tablename to delete column family from */ - public addColumnFamily_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) { + public deleteColumnFamily_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) { this.tableName = tableName; return this; } @@ -43649,18 +44659,27 @@ public void setTableNameIsSet(boolean value) { } /** - * column family descriptor of column family to be added + * name of column family to be deleted */ - @org.apache.thrift.annotation.Nullable - public TColumnFamilyDescriptor getColumn() { - return this.column; + public byte[] getColumn() { + setColumn(org.apache.thrift.TBaseHelper.rightSize(column)); + return column == null ? null : column.array(); + } + + public java.nio.ByteBuffer bufferForColumn() { + return org.apache.thrift.TBaseHelper.copyBinary(column); } /** - * column family descriptor of column family to be added + * name of column family to be deleted */ - public addColumnFamily_args setColumn(@org.apache.thrift.annotation.Nullable TColumnFamilyDescriptor column) { - this.column = column; + public deleteColumnFamily_args setColumn(byte[] column) { + this.column = column == null ? (java.nio.ByteBuffer)null : java.nio.ByteBuffer.wrap(column.clone()); + return this; + } + + public deleteColumnFamily_args setColumn(@org.apache.thrift.annotation.Nullable java.nio.ByteBuffer column) { + this.column = org.apache.thrift.TBaseHelper.copyBinary(column); return this; } @@ -43693,7 +44712,11 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable if (value == null) { unsetColumn(); } else { - setColumn((TColumnFamilyDescriptor)value); + if (value instanceof byte[]) { + setColumn((byte[])value); + } else { + setColumn((java.nio.ByteBuffer)value); + } } break; @@ -43732,12 +44755,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof addColumnFamily_args) - return this.equals((addColumnFamily_args)that); + if (that instanceof deleteColumnFamily_args) + return this.equals((deleteColumnFamily_args)that); return false; } - public boolean equals(addColumnFamily_args that) { + public boolean equals(deleteColumnFamily_args that) { if (that == null) return false; if (this == that) @@ -43780,7 +44803,7 @@ public int hashCode() { } @Override - public int compareTo(addColumnFamily_args other) { + public int compareTo(deleteColumnFamily_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -43825,7 +44848,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("addColumnFamily_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("deleteColumnFamily_args("); boolean first = true; sb.append("tableName:"); @@ -43840,7 +44863,7 @@ public java.lang.String toString() { if (this.column == null) { sb.append("null"); } else { - sb.append(this.column); + org.apache.thrift.TBaseHelper.toString(this.column, sb); } first = false; sb.append(")"); @@ -43859,9 +44882,6 @@ public void validate() throws org.apache.thrift.TException { if (tableName != null) { tableName.validate(); } - if (column != null) { - column.validate(); - } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -43880,15 +44900,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class addColumnFamily_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public addColumnFamily_argsStandardScheme getScheme() { - return new addColumnFamily_argsStandardScheme(); + private static class deleteColumnFamily_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public deleteColumnFamily_argsStandardScheme getScheme() { + return new deleteColumnFamily_argsStandardScheme(); } } - private static class addColumnFamily_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class deleteColumnFamily_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, addColumnFamily_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, deleteColumnFamily_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -43908,9 +44928,8 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, addColumnFamily_arg } break; case 2: // COLUMN - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.column = new TColumnFamilyDescriptor(); - struct.column.read(iprot); + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.column = iprot.readBinary(); struct.setColumnIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); @@ -43927,7 +44946,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, addColumnFamily_arg struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, addColumnFamily_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, deleteColumnFamily_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -43938,7 +44957,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, addColumnFamily_ar } if (struct.column != null) { oprot.writeFieldBegin(COLUMN_FIELD_DESC); - struct.column.write(oprot); + oprot.writeBinary(struct.column); oprot.writeFieldEnd(); } oprot.writeFieldStop(); @@ -43947,29 +44966,28 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, addColumnFamily_ar } - private static class addColumnFamily_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public addColumnFamily_argsTupleScheme getScheme() { - return new addColumnFamily_argsTupleScheme(); + private static class deleteColumnFamily_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public deleteColumnFamily_argsTupleScheme getScheme() { + return new deleteColumnFamily_argsTupleScheme(); } } - private static class addColumnFamily_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class deleteColumnFamily_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, addColumnFamily_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, deleteColumnFamily_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; struct.tableName.write(oprot); - struct.column.write(oprot); + oprot.writeBinary(struct.column); } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, addColumnFamily_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, deleteColumnFamily_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; struct.tableName = new TTableName(); struct.tableName.read(iprot); struct.setTableNameIsSet(true); - struct.column = new TColumnFamilyDescriptor(); - struct.column.read(iprot); + struct.column = iprot.readBinary(); struct.setColumnIsSet(true); } } @@ -43979,13 +44997,13 @@ private static S scheme(org.apache. } } - public static class addColumnFamily_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("addColumnFamily_result"); + public static class deleteColumnFamily_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteColumnFamily_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new addColumnFamily_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new addColumnFamily_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new deleteColumnFamily_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new deleteColumnFamily_resultTupleSchemeFactory(); public @org.apache.thrift.annotation.Nullable TIOError io; // required @@ -44056,13 +45074,13 @@ public java.lang.String getFieldName() { tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(addColumnFamily_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(deleteColumnFamily_result.class, metaDataMap); } - public addColumnFamily_result() { + public deleteColumnFamily_result() { } - public addColumnFamily_result( + public deleteColumnFamily_result( TIOError io) { this(); @@ -44072,14 +45090,14 @@ public addColumnFamily_result( /** * Performs a deep copy on other. */ - public addColumnFamily_result(addColumnFamily_result other) { + public deleteColumnFamily_result(deleteColumnFamily_result other) { if (other.isSetIo()) { this.io = new TIOError(other.io); } } - public addColumnFamily_result deepCopy() { - return new addColumnFamily_result(this); + public deleteColumnFamily_result deepCopy() { + return new deleteColumnFamily_result(this); } @Override @@ -44092,7 +45110,7 @@ public TIOError getIo() { return this.io; } - public addColumnFamily_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { + public deleteColumnFamily_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { this.io = io; return this; } @@ -44152,12 +45170,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof addColumnFamily_result) - return this.equals((addColumnFamily_result)that); + if (that instanceof deleteColumnFamily_result) + return this.equals((deleteColumnFamily_result)that); return false; } - public boolean equals(addColumnFamily_result that) { + public boolean equals(deleteColumnFamily_result that) { if (that == null) return false; if (this == that) @@ -44187,7 +45205,7 @@ public int hashCode() { } @Override - public int compareTo(addColumnFamily_result other) { + public int compareTo(deleteColumnFamily_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -44222,7 +45240,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("addColumnFamily_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("deleteColumnFamily_result("); boolean first = true; sb.append("io:"); @@ -44257,15 +45275,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class addColumnFamily_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public addColumnFamily_resultStandardScheme getScheme() { - return new addColumnFamily_resultStandardScheme(); + private static class deleteColumnFamily_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public deleteColumnFamily_resultStandardScheme getScheme() { + return new deleteColumnFamily_resultStandardScheme(); } } - private static class addColumnFamily_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class deleteColumnFamily_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, addColumnFamily_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, deleteColumnFamily_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -44295,7 +45313,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, addColumnFamily_res struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, addColumnFamily_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, deleteColumnFamily_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -44310,16 +45328,16 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, addColumnFamily_re } - private static class addColumnFamily_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public addColumnFamily_resultTupleScheme getScheme() { - return new addColumnFamily_resultTupleScheme(); + private static class deleteColumnFamily_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public deleteColumnFamily_resultTupleScheme getScheme() { + return new deleteColumnFamily_resultTupleScheme(); } } - private static class addColumnFamily_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class deleteColumnFamily_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, addColumnFamily_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, deleteColumnFamily_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.isSetIo()) { @@ -44332,7 +45350,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, addColumnFamily_res } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, addColumnFamily_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, deleteColumnFamily_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)) { @@ -44348,32 +45366,32 @@ private static S scheme(org.apache. } } - public static class deleteColumnFamily_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteColumnFamily_args"); + public static class modifyColumnFamily_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("modifyColumnFamily_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("column", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("column", org.apache.thrift.protocol.TType.STRUCT, (short)2); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new deleteColumnFamily_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new deleteColumnFamily_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifyColumnFamily_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifyColumnFamily_argsTupleSchemeFactory(); /** - * the tablename to delete column family from + * the tablename to modify column family */ public @org.apache.thrift.annotation.Nullable TTableName tableName; // required /** - * name of column family to be deleted + * column family descriptor of column family to be modified */ - public @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer column; // required + public @org.apache.thrift.annotation.Nullable TColumnFamilyDescriptor column; // 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 { /** - * the tablename to delete column family from + * the tablename to modify column family */ TABLE_NAME((short)1, "tableName"), /** - * name of column family to be deleted + * column family descriptor of column family to be modified */ COLUMN((short)2, "column"); @@ -44442,37 +45460,37 @@ public java.lang.String getFieldName() { tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableName.class))); tmpMap.put(_Fields.COLUMN, new org.apache.thrift.meta_data.FieldMetaData("column", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TColumnFamilyDescriptor.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(deleteColumnFamily_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(modifyColumnFamily_args.class, metaDataMap); } - public deleteColumnFamily_args() { + public modifyColumnFamily_args() { } - public deleteColumnFamily_args( + public modifyColumnFamily_args( TTableName tableName, - java.nio.ByteBuffer column) + TColumnFamilyDescriptor column) { this(); this.tableName = tableName; - this.column = org.apache.thrift.TBaseHelper.copyBinary(column); + this.column = column; } /** * Performs a deep copy on other. */ - public deleteColumnFamily_args(deleteColumnFamily_args other) { + public modifyColumnFamily_args(modifyColumnFamily_args other) { if (other.isSetTableName()) { this.tableName = new TTableName(other.tableName); } if (other.isSetColumn()) { - this.column = org.apache.thrift.TBaseHelper.copyBinary(other.column); + this.column = new TColumnFamilyDescriptor(other.column); } } - public deleteColumnFamily_args deepCopy() { - return new deleteColumnFamily_args(this); + public modifyColumnFamily_args deepCopy() { + return new modifyColumnFamily_args(this); } @Override @@ -44482,7 +45500,7 @@ public void clear() { } /** - * the tablename to delete column family from + * the tablename to modify column family */ @org.apache.thrift.annotation.Nullable public TTableName getTableName() { @@ -44490,9 +45508,9 @@ public TTableName getTableName() { } /** - * the tablename to delete column family from + * the tablename to modify column family */ - public deleteColumnFamily_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) { + public modifyColumnFamily_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) { this.tableName = tableName; return this; } @@ -44513,27 +45531,18 @@ public void setTableNameIsSet(boolean value) { } /** - * name of column family to be deleted + * column family descriptor of column family to be modified */ - public byte[] getColumn() { - setColumn(org.apache.thrift.TBaseHelper.rightSize(column)); - return column == null ? null : column.array(); - } - - public java.nio.ByteBuffer bufferForColumn() { - return org.apache.thrift.TBaseHelper.copyBinary(column); + @org.apache.thrift.annotation.Nullable + public TColumnFamilyDescriptor getColumn() { + return this.column; } /** - * name of column family to be deleted + * column family descriptor of column family to be modified */ - public deleteColumnFamily_args setColumn(byte[] column) { - this.column = column == null ? (java.nio.ByteBuffer)null : java.nio.ByteBuffer.wrap(column.clone()); - return this; - } - - public deleteColumnFamily_args setColumn(@org.apache.thrift.annotation.Nullable java.nio.ByteBuffer column) { - this.column = org.apache.thrift.TBaseHelper.copyBinary(column); + public modifyColumnFamily_args setColumn(@org.apache.thrift.annotation.Nullable TColumnFamilyDescriptor column) { + this.column = column; return this; } @@ -44566,11 +45575,7 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable if (value == null) { unsetColumn(); } else { - if (value instanceof byte[]) { - setColumn((byte[])value); - } else { - setColumn((java.nio.ByteBuffer)value); - } + setColumn((TColumnFamilyDescriptor)value); } break; @@ -44609,12 +45614,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof deleteColumnFamily_args) - return this.equals((deleteColumnFamily_args)that); + if (that instanceof modifyColumnFamily_args) + return this.equals((modifyColumnFamily_args)that); return false; } - public boolean equals(deleteColumnFamily_args that) { + public boolean equals(modifyColumnFamily_args that) { if (that == null) return false; if (this == that) @@ -44657,7 +45662,7 @@ public int hashCode() { } @Override - public int compareTo(deleteColumnFamily_args other) { + public int compareTo(modifyColumnFamily_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -44702,7 +45707,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("deleteColumnFamily_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("modifyColumnFamily_args("); boolean first = true; sb.append("tableName:"); @@ -44717,7 +45722,7 @@ public java.lang.String toString() { if (this.column == null) { sb.append("null"); } else { - org.apache.thrift.TBaseHelper.toString(this.column, sb); + sb.append(this.column); } first = false; sb.append(")"); @@ -44736,6 +45741,9 @@ public void validate() throws org.apache.thrift.TException { if (tableName != null) { tableName.validate(); } + if (column != null) { + column.validate(); + } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -44754,15 +45762,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class deleteColumnFamily_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public deleteColumnFamily_argsStandardScheme getScheme() { - return new deleteColumnFamily_argsStandardScheme(); + private static class modifyColumnFamily_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public modifyColumnFamily_argsStandardScheme getScheme() { + return new modifyColumnFamily_argsStandardScheme(); } } - private static class deleteColumnFamily_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class modifyColumnFamily_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, deleteColumnFamily_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, modifyColumnFamily_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -44782,8 +45790,9 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, deleteColumnFamily_ } break; case 2: // COLUMN - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.column = iprot.readBinary(); + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.column = new TColumnFamilyDescriptor(); + struct.column.read(iprot); struct.setColumnIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); @@ -44800,7 +45809,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, deleteColumnFamily_ struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, deleteColumnFamily_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, modifyColumnFamily_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -44811,7 +45820,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, deleteColumnFamily } if (struct.column != null) { oprot.writeFieldBegin(COLUMN_FIELD_DESC); - oprot.writeBinary(struct.column); + struct.column.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); @@ -44820,28 +45829,29 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, deleteColumnFamily } - private static class deleteColumnFamily_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public deleteColumnFamily_argsTupleScheme getScheme() { - return new deleteColumnFamily_argsTupleScheme(); + private static class modifyColumnFamily_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public modifyColumnFamily_argsTupleScheme getScheme() { + return new modifyColumnFamily_argsTupleScheme(); } } - private static class deleteColumnFamily_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class modifyColumnFamily_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, deleteColumnFamily_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, modifyColumnFamily_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; struct.tableName.write(oprot); - oprot.writeBinary(struct.column); + struct.column.write(oprot); } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, deleteColumnFamily_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, modifyColumnFamily_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; struct.tableName = new TTableName(); struct.tableName.read(iprot); struct.setTableNameIsSet(true); - struct.column = iprot.readBinary(); + struct.column = new TColumnFamilyDescriptor(); + struct.column.read(iprot); struct.setColumnIsSet(true); } } @@ -44851,13 +45861,13 @@ private static S scheme(org.apache. } } - public static class deleteColumnFamily_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteColumnFamily_result"); + public static class modifyColumnFamily_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("modifyColumnFamily_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new deleteColumnFamily_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new deleteColumnFamily_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifyColumnFamily_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifyColumnFamily_resultTupleSchemeFactory(); public @org.apache.thrift.annotation.Nullable TIOError io; // required @@ -44928,13 +45938,13 @@ public java.lang.String getFieldName() { tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(deleteColumnFamily_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(modifyColumnFamily_result.class, metaDataMap); } - public deleteColumnFamily_result() { + public modifyColumnFamily_result() { } - public deleteColumnFamily_result( + public modifyColumnFamily_result( TIOError io) { this(); @@ -44944,14 +45954,14 @@ public deleteColumnFamily_result( /** * Performs a deep copy on other. */ - public deleteColumnFamily_result(deleteColumnFamily_result other) { + public modifyColumnFamily_result(modifyColumnFamily_result other) { if (other.isSetIo()) { this.io = new TIOError(other.io); } } - public deleteColumnFamily_result deepCopy() { - return new deleteColumnFamily_result(this); + public modifyColumnFamily_result deepCopy() { + return new modifyColumnFamily_result(this); } @Override @@ -44964,7 +45974,7 @@ public TIOError getIo() { return this.io; } - public deleteColumnFamily_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { + public modifyColumnFamily_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { this.io = io; return this; } @@ -45024,12 +46034,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof deleteColumnFamily_result) - return this.equals((deleteColumnFamily_result)that); + if (that instanceof modifyColumnFamily_result) + return this.equals((modifyColumnFamily_result)that); return false; } - public boolean equals(deleteColumnFamily_result that) { + public boolean equals(modifyColumnFamily_result that) { if (that == null) return false; if (this == that) @@ -45059,7 +46069,7 @@ public int hashCode() { } @Override - public int compareTo(deleteColumnFamily_result other) { + public int compareTo(modifyColumnFamily_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -45094,7 +46104,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("deleteColumnFamily_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("modifyColumnFamily_result("); boolean first = true; sb.append("io:"); @@ -45129,15 +46139,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class deleteColumnFamily_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public deleteColumnFamily_resultStandardScheme getScheme() { - return new deleteColumnFamily_resultStandardScheme(); + private static class modifyColumnFamily_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public modifyColumnFamily_resultStandardScheme getScheme() { + return new modifyColumnFamily_resultStandardScheme(); } } - private static class deleteColumnFamily_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class modifyColumnFamily_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, deleteColumnFamily_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, modifyColumnFamily_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -45167,7 +46177,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, deleteColumnFamily_ struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, deleteColumnFamily_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, modifyColumnFamily_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -45182,16 +46192,16 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, deleteColumnFamily } - private static class deleteColumnFamily_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public deleteColumnFamily_resultTupleScheme getScheme() { - return new deleteColumnFamily_resultTupleScheme(); + private static class modifyColumnFamily_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public modifyColumnFamily_resultTupleScheme getScheme() { + return new modifyColumnFamily_resultTupleScheme(); } } - private static class deleteColumnFamily_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class modifyColumnFamily_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, deleteColumnFamily_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, modifyColumnFamily_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.isSetIo()) { @@ -45204,7 +46214,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, deleteColumnFamily_ } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, deleteColumnFamily_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, modifyColumnFamily_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)) { @@ -45220,34 +46230,25 @@ private static S scheme(org.apache. } } - public static class modifyColumnFamily_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("modifyColumnFamily_args"); + public static class modifyTable_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("modifyTable_args"); - private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("column", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField DESC_FIELD_DESC = new org.apache.thrift.protocol.TField("desc", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifyColumnFamily_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifyColumnFamily_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifyTable_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifyTable_argsTupleSchemeFactory(); /** - * the tablename to modify column family - */ - public @org.apache.thrift.annotation.Nullable TTableName tableName; // required - /** - * column family descriptor of column family to be modified + * the descriptor of the table to modify */ - public @org.apache.thrift.annotation.Nullable TColumnFamilyDescriptor column; // required + public @org.apache.thrift.annotation.Nullable TTableDescriptor desc; // 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 { /** - * the tablename to modify column family - */ - TABLE_NAME((short)1, "tableName"), - /** - * column family descriptor of column family to be modified + * the descriptor of the table to modify */ - COLUMN((short)2, "column"); + DESC((short)1, "desc"); private static final java.util.Map byName = new java.util.HashMap(); @@ -45263,10 +46264,8 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { @org.apache.thrift.annotation.Nullable public static _Fields findByThriftId(int fieldId) { switch(fieldId) { - case 1: // TABLE_NAME - return TABLE_NAME; - case 2: // COLUMN - return COLUMN; + case 1: // DESC + return DESC; default: return null; } @@ -45311,125 +46310,78 @@ public java.lang.String getFieldName() { 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.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableName.class))); - tmpMap.put(_Fields.COLUMN, new org.apache.thrift.meta_data.FieldMetaData("column", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TColumnFamilyDescriptor.class))); + tmpMap.put(_Fields.DESC, new org.apache.thrift.meta_data.FieldMetaData("desc", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableDescriptor.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(modifyColumnFamily_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(modifyTable_args.class, metaDataMap); } - public modifyColumnFamily_args() { + public modifyTable_args() { } - public modifyColumnFamily_args( - TTableName tableName, - TColumnFamilyDescriptor column) + public modifyTable_args( + TTableDescriptor desc) { this(); - this.tableName = tableName; - this.column = column; + this.desc = desc; } /** * Performs a deep copy on other. */ - public modifyColumnFamily_args(modifyColumnFamily_args other) { - if (other.isSetTableName()) { - this.tableName = new TTableName(other.tableName); - } - if (other.isSetColumn()) { - this.column = new TColumnFamilyDescriptor(other.column); + public modifyTable_args(modifyTable_args other) { + if (other.isSetDesc()) { + this.desc = new TTableDescriptor(other.desc); } } - public modifyColumnFamily_args deepCopy() { - return new modifyColumnFamily_args(this); + public modifyTable_args deepCopy() { + return new modifyTable_args(this); } @Override public void clear() { - this.tableName = null; - this.column = null; - } - - /** - * the tablename to modify column family - */ - @org.apache.thrift.annotation.Nullable - public TTableName getTableName() { - return this.tableName; - } - - /** - * the tablename to modify column family - */ - public modifyColumnFamily_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) { - this.tableName = tableName; - return this; - } - - public void unsetTableName() { - this.tableName = null; - } - - /** Returns true if field tableName is set (has been assigned a value) and false otherwise */ - public boolean isSetTableName() { - return this.tableName != null; - } - - public void setTableNameIsSet(boolean value) { - if (!value) { - this.tableName = null; - } + this.desc = null; } /** - * column family descriptor of column family to be modified + * the descriptor of the table to modify */ @org.apache.thrift.annotation.Nullable - public TColumnFamilyDescriptor getColumn() { - return this.column; + public TTableDescriptor getDesc() { + return this.desc; } /** - * column family descriptor of column family to be modified + * the descriptor of the table to modify */ - public modifyColumnFamily_args setColumn(@org.apache.thrift.annotation.Nullable TColumnFamilyDescriptor column) { - this.column = column; + public modifyTable_args setDesc(@org.apache.thrift.annotation.Nullable TTableDescriptor desc) { + this.desc = desc; return this; } - public void unsetColumn() { - this.column = null; + public void unsetDesc() { + this.desc = null; } - /** Returns true if field column is set (has been assigned a value) and false otherwise */ - public boolean isSetColumn() { - return this.column != null; + /** Returns true if field desc is set (has been assigned a value) and false otherwise */ + public boolean isSetDesc() { + return this.desc != null; } - public void setColumnIsSet(boolean value) { + public void setDescIsSet(boolean value) { if (!value) { - this.column = null; + this.desc = null; } } public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { switch (field) { - case TABLE_NAME: - if (value == null) { - unsetTableName(); - } else { - setTableName((TTableName)value); - } - break; - - case COLUMN: + case DESC: if (value == null) { - unsetColumn(); + unsetDesc(); } else { - setColumn((TColumnFamilyDescriptor)value); + setDesc((TTableDescriptor)value); } break; @@ -45439,11 +46391,8 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable @org.apache.thrift.annotation.Nullable public java.lang.Object getFieldValue(_Fields field) { switch (field) { - case TABLE_NAME: - return getTableName(); - - case COLUMN: - return getColumn(); + case DESC: + return getDesc(); } throw new java.lang.IllegalStateException(); @@ -45456,10 +46405,8 @@ public boolean isSet(_Fields field) { } switch (field) { - case TABLE_NAME: - return isSetTableName(); - case COLUMN: - return isSetColumn(); + case DESC: + return isSetDesc(); } throw new java.lang.IllegalStateException(); } @@ -45468,32 +46415,23 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof modifyColumnFamily_args) - return this.equals((modifyColumnFamily_args)that); + if (that instanceof modifyTable_args) + return this.equals((modifyTable_args)that); return false; } - public boolean equals(modifyColumnFamily_args that) { + public boolean equals(modifyTable_args that) { if (that == null) return false; if (this == that) return true; - boolean this_present_tableName = true && this.isSetTableName(); - boolean that_present_tableName = true && that.isSetTableName(); - if (this_present_tableName || that_present_tableName) { - if (!(this_present_tableName && that_present_tableName)) - return false; - if (!this.tableName.equals(that.tableName)) - return false; - } - - boolean this_present_column = true && this.isSetColumn(); - boolean that_present_column = true && that.isSetColumn(); - if (this_present_column || that_present_column) { - if (!(this_present_column && that_present_column)) + boolean this_present_desc = true && this.isSetDesc(); + boolean that_present_desc = true && that.isSetDesc(); + if (this_present_desc || that_present_desc) { + if (!(this_present_desc && that_present_desc)) return false; - if (!this.column.equals(that.column)) + if (!this.desc.equals(that.desc)) return false; } @@ -45504,41 +46442,27 @@ public boolean equals(modifyColumnFamily_args that) { public int hashCode() { int hashCode = 1; - hashCode = hashCode * 8191 + ((isSetTableName()) ? 131071 : 524287); - if (isSetTableName()) - hashCode = hashCode * 8191 + tableName.hashCode(); - - hashCode = hashCode * 8191 + ((isSetColumn()) ? 131071 : 524287); - if (isSetColumn()) - hashCode = hashCode * 8191 + column.hashCode(); + hashCode = hashCode * 8191 + ((isSetDesc()) ? 131071 : 524287); + if (isSetDesc()) + hashCode = hashCode * 8191 + desc.hashCode(); return hashCode; } @Override - public int compareTo(modifyColumnFamily_args other) { + public int compareTo(modifyTable_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - lastComparison = java.lang.Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = java.lang.Boolean.valueOf(isSetColumn()).compareTo(other.isSetColumn()); + lastComparison = java.lang.Boolean.valueOf(isSetDesc()).compareTo(other.isSetDesc()); if (lastComparison != 0) { return lastComparison; } - if (isSetColumn()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.column, other.column); + if (isSetDesc()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.desc, other.desc); if (lastComparison != 0) { return lastComparison; } @@ -45561,22 +46485,14 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("modifyColumnFamily_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("modifyTable_args("); boolean first = true; - sb.append("tableName:"); - if (this.tableName == null) { - sb.append("null"); - } else { - sb.append(this.tableName); - } - first = false; - if (!first) sb.append(", "); - sb.append("column:"); - if (this.column == null) { + sb.append("desc:"); + if (this.desc == null) { sb.append("null"); } else { - sb.append(this.column); + sb.append(this.desc); } first = false; sb.append(")"); @@ -45585,18 +46501,12 @@ public java.lang.String toString() { public void validate() throws org.apache.thrift.TException { // check for required fields - if (tableName == null) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'tableName' was not present! Struct: " + toString()); - } - if (column == null) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'column' was not present! Struct: " + toString()); + if (desc == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'desc' was not present! Struct: " + toString()); } // check for sub-struct validity - if (tableName != null) { - tableName.validate(); - } - if (column != null) { - column.validate(); + if (desc != null) { + desc.validate(); } } @@ -45616,15 +46526,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class modifyColumnFamily_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public modifyColumnFamily_argsStandardScheme getScheme() { - return new modifyColumnFamily_argsStandardScheme(); + private static class modifyTable_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public modifyTable_argsStandardScheme getScheme() { + return new modifyTable_argsStandardScheme(); } } - private static class modifyColumnFamily_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class modifyTable_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, modifyColumnFamily_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, modifyTable_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -45634,20 +46544,11 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, modifyColumnFamily_ break; } switch (schemeField.id) { - case 1: // TABLE_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.tableName = new TTableName(); - struct.tableName.read(iprot); - struct.setTableNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // COLUMN + case 1: // DESC if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.column = new TColumnFamilyDescriptor(); - struct.column.read(iprot); - struct.setColumnIsSet(true); + struct.desc = new TTableDescriptor(); + struct.desc.read(iprot); + struct.setDescIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -45663,18 +46564,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, modifyColumnFamily_ struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, modifyColumnFamily_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, modifyTable_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - if (struct.tableName != null) { - oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC); - struct.tableName.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.column != null) { - oprot.writeFieldBegin(COLUMN_FIELD_DESC); - struct.column.write(oprot); + if (struct.desc != null) { + oprot.writeFieldBegin(DESC_FIELD_DESC); + struct.desc.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); @@ -45683,30 +46579,26 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, modifyColumnFamily } - private static class modifyColumnFamily_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public modifyColumnFamily_argsTupleScheme getScheme() { - return new modifyColumnFamily_argsTupleScheme(); + private static class modifyTable_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public modifyTable_argsTupleScheme getScheme() { + return new modifyTable_argsTupleScheme(); } } - private static class modifyColumnFamily_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class modifyTable_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, modifyColumnFamily_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, modifyTable_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; - struct.tableName.write(oprot); - struct.column.write(oprot); + struct.desc.write(oprot); } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, modifyColumnFamily_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, modifyTable_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; - struct.tableName = new TTableName(); - struct.tableName.read(iprot); - struct.setTableNameIsSet(true); - struct.column = new TColumnFamilyDescriptor(); - struct.column.read(iprot); - struct.setColumnIsSet(true); + struct.desc = new TTableDescriptor(); + struct.desc.read(iprot); + struct.setDescIsSet(true); } } @@ -45715,13 +46607,13 @@ private static S scheme(org.apache. } } - public static class modifyColumnFamily_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("modifyColumnFamily_result"); + public static class modifyTable_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("modifyTable_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifyColumnFamily_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifyColumnFamily_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifyTable_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifyTable_resultTupleSchemeFactory(); public @org.apache.thrift.annotation.Nullable TIOError io; // required @@ -45792,13 +46684,13 @@ public java.lang.String getFieldName() { tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(modifyColumnFamily_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(modifyTable_result.class, metaDataMap); } - public modifyColumnFamily_result() { + public modifyTable_result() { } - public modifyColumnFamily_result( + public modifyTable_result( TIOError io) { this(); @@ -45808,14 +46700,14 @@ public modifyColumnFamily_result( /** * Performs a deep copy on other. */ - public modifyColumnFamily_result(modifyColumnFamily_result other) { + public modifyTable_result(modifyTable_result other) { if (other.isSetIo()) { this.io = new TIOError(other.io); } } - public modifyColumnFamily_result deepCopy() { - return new modifyColumnFamily_result(this); + public modifyTable_result deepCopy() { + return new modifyTable_result(this); } @Override @@ -45828,7 +46720,7 @@ public TIOError getIo() { return this.io; } - public modifyColumnFamily_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { + public modifyTable_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { this.io = io; return this; } @@ -45888,12 +46780,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof modifyColumnFamily_result) - return this.equals((modifyColumnFamily_result)that); + if (that instanceof modifyTable_result) + return this.equals((modifyTable_result)that); return false; } - public boolean equals(modifyColumnFamily_result that) { + public boolean equals(modifyTable_result that) { if (that == null) return false; if (this == that) @@ -45923,7 +46815,7 @@ public int hashCode() { } @Override - public int compareTo(modifyColumnFamily_result other) { + public int compareTo(modifyTable_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -45958,7 +46850,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("modifyColumnFamily_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("modifyTable_result("); boolean first = true; sb.append("io:"); @@ -45993,15 +46885,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class modifyColumnFamily_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public modifyColumnFamily_resultStandardScheme getScheme() { - return new modifyColumnFamily_resultStandardScheme(); + private static class modifyTable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public modifyTable_resultStandardScheme getScheme() { + return new modifyTable_resultStandardScheme(); } } - private static class modifyColumnFamily_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class modifyTable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, modifyColumnFamily_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, modifyTable_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -46031,7 +46923,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, modifyColumnFamily_ struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, modifyColumnFamily_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, modifyTable_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -46046,16 +46938,16 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, modifyColumnFamily } - private static class modifyColumnFamily_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public modifyColumnFamily_resultTupleScheme getScheme() { - return new modifyColumnFamily_resultTupleScheme(); + private static class modifyTable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public modifyTable_resultTupleScheme getScheme() { + return new modifyTable_resultTupleScheme(); } } - private static class modifyColumnFamily_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class modifyTable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, modifyColumnFamily_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, modifyTable_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.isSetIo()) { @@ -46068,7 +46960,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, modifyColumnFamily_ } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, modifyColumnFamily_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, modifyTable_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)) { @@ -46084,25 +46976,25 @@ private static S scheme(org.apache. } } - public static class modifyTable_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("modifyTable_args"); + public static class createNamespace_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("createNamespace_args"); - private static final org.apache.thrift.protocol.TField DESC_FIELD_DESC = new org.apache.thrift.protocol.TField("desc", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField NAMESPACE_DESC_FIELD_DESC = new org.apache.thrift.protocol.TField("namespaceDesc", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifyTable_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifyTable_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new createNamespace_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new createNamespace_argsTupleSchemeFactory(); /** - * the descriptor of the table to modify + * descriptor which describes the new namespace */ - public @org.apache.thrift.annotation.Nullable TTableDescriptor desc; // required + public @org.apache.thrift.annotation.Nullable TNamespaceDescriptor namespaceDesc; // 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 { /** - * the descriptor of the table to modify + * descriptor which describes the new namespace */ - DESC((short)1, "desc"); + NAMESPACE_DESC((short)1, "namespaceDesc"); private static final java.util.Map byName = new java.util.HashMap(); @@ -46118,8 +47010,8 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { @org.apache.thrift.annotation.Nullable public static _Fields findByThriftId(int fieldId) { switch(fieldId) { - case 1: // DESC - return DESC; + case 1: // NAMESPACE_DESC + return NAMESPACE_DESC; default: return null; } @@ -46164,78 +47056,78 @@ public java.lang.String getFieldName() { 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.DESC, new org.apache.thrift.meta_data.FieldMetaData("desc", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableDescriptor.class))); + tmpMap.put(_Fields.NAMESPACE_DESC, new org.apache.thrift.meta_data.FieldMetaData("namespaceDesc", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TNamespaceDescriptor.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(modifyTable_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(createNamespace_args.class, metaDataMap); } - public modifyTable_args() { + public createNamespace_args() { } - public modifyTable_args( - TTableDescriptor desc) + public createNamespace_args( + TNamespaceDescriptor namespaceDesc) { this(); - this.desc = desc; + this.namespaceDesc = namespaceDesc; } /** * Performs a deep copy on other. */ - public modifyTable_args(modifyTable_args other) { - if (other.isSetDesc()) { - this.desc = new TTableDescriptor(other.desc); + public createNamespace_args(createNamespace_args other) { + if (other.isSetNamespaceDesc()) { + this.namespaceDesc = new TNamespaceDescriptor(other.namespaceDesc); } } - public modifyTable_args deepCopy() { - return new modifyTable_args(this); + public createNamespace_args deepCopy() { + return new createNamespace_args(this); } @Override public void clear() { - this.desc = null; + this.namespaceDesc = null; } /** - * the descriptor of the table to modify + * descriptor which describes the new namespace */ @org.apache.thrift.annotation.Nullable - public TTableDescriptor getDesc() { - return this.desc; + public TNamespaceDescriptor getNamespaceDesc() { + return this.namespaceDesc; } /** - * the descriptor of the table to modify + * descriptor which describes the new namespace */ - public modifyTable_args setDesc(@org.apache.thrift.annotation.Nullable TTableDescriptor desc) { - this.desc = desc; + public createNamespace_args setNamespaceDesc(@org.apache.thrift.annotation.Nullable TNamespaceDescriptor namespaceDesc) { + this.namespaceDesc = namespaceDesc; return this; } - public void unsetDesc() { - this.desc = null; + public void unsetNamespaceDesc() { + this.namespaceDesc = null; } - /** Returns true if field desc is set (has been assigned a value) and false otherwise */ - public boolean isSetDesc() { - return this.desc != null; + /** Returns true if field namespaceDesc is set (has been assigned a value) and false otherwise */ + public boolean isSetNamespaceDesc() { + return this.namespaceDesc != null; } - public void setDescIsSet(boolean value) { + public void setNamespaceDescIsSet(boolean value) { if (!value) { - this.desc = null; + this.namespaceDesc = null; } } public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { switch (field) { - case DESC: + case NAMESPACE_DESC: if (value == null) { - unsetDesc(); + unsetNamespaceDesc(); } else { - setDesc((TTableDescriptor)value); + setNamespaceDesc((TNamespaceDescriptor)value); } break; @@ -46245,8 +47137,8 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable @org.apache.thrift.annotation.Nullable public java.lang.Object getFieldValue(_Fields field) { switch (field) { - case DESC: - return getDesc(); + case NAMESPACE_DESC: + return getNamespaceDesc(); } throw new java.lang.IllegalStateException(); @@ -46259,8 +47151,8 @@ public boolean isSet(_Fields field) { } switch (field) { - case DESC: - return isSetDesc(); + case NAMESPACE_DESC: + return isSetNamespaceDesc(); } throw new java.lang.IllegalStateException(); } @@ -46269,23 +47161,23 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof modifyTable_args) - return this.equals((modifyTable_args)that); + if (that instanceof createNamespace_args) + return this.equals((createNamespace_args)that); return false; } - public boolean equals(modifyTable_args that) { + public boolean equals(createNamespace_args that) { if (that == null) return false; if (this == that) return true; - boolean this_present_desc = true && this.isSetDesc(); - boolean that_present_desc = true && that.isSetDesc(); - if (this_present_desc || that_present_desc) { - if (!(this_present_desc && that_present_desc)) + boolean this_present_namespaceDesc = true && this.isSetNamespaceDesc(); + boolean that_present_namespaceDesc = true && that.isSetNamespaceDesc(); + if (this_present_namespaceDesc || that_present_namespaceDesc) { + if (!(this_present_namespaceDesc && that_present_namespaceDesc)) return false; - if (!this.desc.equals(that.desc)) + if (!this.namespaceDesc.equals(that.namespaceDesc)) return false; } @@ -46296,27 +47188,27 @@ public boolean equals(modifyTable_args that) { public int hashCode() { int hashCode = 1; - hashCode = hashCode * 8191 + ((isSetDesc()) ? 131071 : 524287); - if (isSetDesc()) - hashCode = hashCode * 8191 + desc.hashCode(); + hashCode = hashCode * 8191 + ((isSetNamespaceDesc()) ? 131071 : 524287); + if (isSetNamespaceDesc()) + hashCode = hashCode * 8191 + namespaceDesc.hashCode(); return hashCode; } @Override - public int compareTo(modifyTable_args other) { + public int compareTo(createNamespace_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - lastComparison = java.lang.Boolean.valueOf(isSetDesc()).compareTo(other.isSetDesc()); + lastComparison = java.lang.Boolean.valueOf(isSetNamespaceDesc()).compareTo(other.isSetNamespaceDesc()); if (lastComparison != 0) { return lastComparison; } - if (isSetDesc()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.desc, other.desc); + if (isSetNamespaceDesc()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.namespaceDesc, other.namespaceDesc); if (lastComparison != 0) { return lastComparison; } @@ -46339,14 +47231,14 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("modifyTable_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("createNamespace_args("); boolean first = true; - sb.append("desc:"); - if (this.desc == null) { + sb.append("namespaceDesc:"); + if (this.namespaceDesc == null) { sb.append("null"); } else { - sb.append(this.desc); + sb.append(this.namespaceDesc); } first = false; sb.append(")"); @@ -46355,12 +47247,12 @@ public java.lang.String toString() { public void validate() throws org.apache.thrift.TException { // check for required fields - if (desc == null) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'desc' was not present! Struct: " + toString()); + if (namespaceDesc == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'namespaceDesc' was not present! Struct: " + toString()); } // check for sub-struct validity - if (desc != null) { - desc.validate(); + if (namespaceDesc != null) { + namespaceDesc.validate(); } } @@ -46380,15 +47272,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class modifyTable_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public modifyTable_argsStandardScheme getScheme() { - return new modifyTable_argsStandardScheme(); + private static class createNamespace_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public createNamespace_argsStandardScheme getScheme() { + return new createNamespace_argsStandardScheme(); } } - private static class modifyTable_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class createNamespace_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, modifyTable_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, createNamespace_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -46398,11 +47290,11 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, modifyTable_args st break; } switch (schemeField.id) { - case 1: // DESC + case 1: // NAMESPACE_DESC if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.desc = new TTableDescriptor(); - struct.desc.read(iprot); - struct.setDescIsSet(true); + struct.namespaceDesc = new TNamespaceDescriptor(); + struct.namespaceDesc.read(iprot); + struct.setNamespaceDescIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -46418,13 +47310,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, modifyTable_args st struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, modifyTable_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, createNamespace_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - if (struct.desc != null) { - oprot.writeFieldBegin(DESC_FIELD_DESC); - struct.desc.write(oprot); + if (struct.namespaceDesc != null) { + oprot.writeFieldBegin(NAMESPACE_DESC_FIELD_DESC); + struct.namespaceDesc.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); @@ -46433,26 +47325,26 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, modifyTable_args s } - private static class modifyTable_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public modifyTable_argsTupleScheme getScheme() { - return new modifyTable_argsTupleScheme(); + private static class createNamespace_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public createNamespace_argsTupleScheme getScheme() { + return new createNamespace_argsTupleScheme(); } } - private static class modifyTable_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class createNamespace_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, modifyTable_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, createNamespace_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; - struct.desc.write(oprot); + struct.namespaceDesc.write(oprot); } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, modifyTable_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, createNamespace_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; - struct.desc = new TTableDescriptor(); - struct.desc.read(iprot); - struct.setDescIsSet(true); + struct.namespaceDesc = new TNamespaceDescriptor(); + struct.namespaceDesc.read(iprot); + struct.setNamespaceDescIsSet(true); } } @@ -46461,13 +47353,13 @@ private static S scheme(org.apache. } } - public static class modifyTable_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("modifyTable_result"); + public static class createNamespace_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("createNamespace_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifyTable_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifyTable_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new createNamespace_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new createNamespace_resultTupleSchemeFactory(); public @org.apache.thrift.annotation.Nullable TIOError io; // required @@ -46538,13 +47430,13 @@ public java.lang.String getFieldName() { tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(modifyTable_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(createNamespace_result.class, metaDataMap); } - public modifyTable_result() { + public createNamespace_result() { } - public modifyTable_result( + public createNamespace_result( TIOError io) { this(); @@ -46554,14 +47446,14 @@ public modifyTable_result( /** * Performs a deep copy on other. */ - public modifyTable_result(modifyTable_result other) { + public createNamespace_result(createNamespace_result other) { if (other.isSetIo()) { this.io = new TIOError(other.io); } } - public modifyTable_result deepCopy() { - return new modifyTable_result(this); + public createNamespace_result deepCopy() { + return new createNamespace_result(this); } @Override @@ -46574,7 +47466,7 @@ public TIOError getIo() { return this.io; } - public modifyTable_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { + public createNamespace_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { this.io = io; return this; } @@ -46634,12 +47526,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof modifyTable_result) - return this.equals((modifyTable_result)that); + if (that instanceof createNamespace_result) + return this.equals((createNamespace_result)that); return false; } - public boolean equals(modifyTable_result that) { + public boolean equals(createNamespace_result that) { if (that == null) return false; if (this == that) @@ -46669,7 +47561,7 @@ public int hashCode() { } @Override - public int compareTo(modifyTable_result other) { + public int compareTo(createNamespace_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -46704,7 +47596,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("modifyTable_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("createNamespace_result("); boolean first = true; sb.append("io:"); @@ -46739,15 +47631,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class modifyTable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public modifyTable_resultStandardScheme getScheme() { - return new modifyTable_resultStandardScheme(); + private static class createNamespace_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public createNamespace_resultStandardScheme getScheme() { + return new createNamespace_resultStandardScheme(); } } - private static class modifyTable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class createNamespace_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, modifyTable_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, createNamespace_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -46777,7 +47669,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, modifyTable_result struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, modifyTable_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, createNamespace_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -46792,16 +47684,16 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, modifyTable_result } - private static class modifyTable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public modifyTable_resultTupleScheme getScheme() { - return new modifyTable_resultTupleScheme(); + private static class createNamespace_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public createNamespace_resultTupleScheme getScheme() { + return new createNamespace_resultTupleScheme(); } } - private static class modifyTable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class createNamespace_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, modifyTable_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, createNamespace_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.isSetIo()) { @@ -46814,7 +47706,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, modifyTable_result } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, modifyTable_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, createNamespace_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)) { @@ -46830,13 +47722,13 @@ private static S scheme(org.apache. } } - public static class createNamespace_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("createNamespace_args"); + public static class modifyNamespace_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("modifyNamespace_args"); private static final org.apache.thrift.protocol.TField NAMESPACE_DESC_FIELD_DESC = new org.apache.thrift.protocol.TField("namespaceDesc", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new createNamespace_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new createNamespace_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifyNamespace_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifyNamespace_argsTupleSchemeFactory(); /** * descriptor which describes the new namespace @@ -46913,13 +47805,13 @@ public java.lang.String getFieldName() { tmpMap.put(_Fields.NAMESPACE_DESC, new org.apache.thrift.meta_data.FieldMetaData("namespaceDesc", org.apache.thrift.TFieldRequirementType.REQUIRED, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TNamespaceDescriptor.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(createNamespace_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(modifyNamespace_args.class, metaDataMap); } - public createNamespace_args() { + public modifyNamespace_args() { } - public createNamespace_args( + public modifyNamespace_args( TNamespaceDescriptor namespaceDesc) { this(); @@ -46929,14 +47821,14 @@ public createNamespace_args( /** * Performs a deep copy on other. */ - public createNamespace_args(createNamespace_args other) { + public modifyNamespace_args(modifyNamespace_args other) { if (other.isSetNamespaceDesc()) { this.namespaceDesc = new TNamespaceDescriptor(other.namespaceDesc); } } - public createNamespace_args deepCopy() { - return new createNamespace_args(this); + public modifyNamespace_args deepCopy() { + return new modifyNamespace_args(this); } @Override @@ -46955,7 +47847,7 @@ public TNamespaceDescriptor getNamespaceDesc() { /** * descriptor which describes the new namespace */ - public createNamespace_args setNamespaceDesc(@org.apache.thrift.annotation.Nullable TNamespaceDescriptor namespaceDesc) { + public modifyNamespace_args setNamespaceDesc(@org.apache.thrift.annotation.Nullable TNamespaceDescriptor namespaceDesc) { this.namespaceDesc = namespaceDesc; return this; } @@ -47015,12 +47907,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof createNamespace_args) - return this.equals((createNamespace_args)that); + if (that instanceof modifyNamespace_args) + return this.equals((modifyNamespace_args)that); return false; } - public boolean equals(createNamespace_args that) { + public boolean equals(modifyNamespace_args that) { if (that == null) return false; if (this == that) @@ -47050,7 +47942,7 @@ public int hashCode() { } @Override - public int compareTo(createNamespace_args other) { + public int compareTo(modifyNamespace_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -47085,7 +47977,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("createNamespace_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("modifyNamespace_args("); boolean first = true; sb.append("namespaceDesc:"); @@ -47126,15 +48018,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class createNamespace_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public createNamespace_argsStandardScheme getScheme() { - return new createNamespace_argsStandardScheme(); + private static class modifyNamespace_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public modifyNamespace_argsStandardScheme getScheme() { + return new modifyNamespace_argsStandardScheme(); } } - private static class createNamespace_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class modifyNamespace_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, createNamespace_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, modifyNamespace_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -47164,7 +48056,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, createNamespace_arg struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, createNamespace_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, modifyNamespace_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -47179,22 +48071,22 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, createNamespace_ar } - private static class createNamespace_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public createNamespace_argsTupleScheme getScheme() { - return new createNamespace_argsTupleScheme(); + private static class modifyNamespace_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public modifyNamespace_argsTupleScheme getScheme() { + return new modifyNamespace_argsTupleScheme(); } } - private static class createNamespace_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class modifyNamespace_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, createNamespace_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, modifyNamespace_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; struct.namespaceDesc.write(oprot); } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, createNamespace_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, modifyNamespace_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; struct.namespaceDesc = new TNamespaceDescriptor(); struct.namespaceDesc.read(iprot); @@ -47207,13 +48099,13 @@ private static S scheme(org.apache. } } - public static class createNamespace_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("createNamespace_result"); + public static class modifyNamespace_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("modifyNamespace_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new createNamespace_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new createNamespace_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifyNamespace_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifyNamespace_resultTupleSchemeFactory(); public @org.apache.thrift.annotation.Nullable TIOError io; // required @@ -47284,13 +48176,13 @@ public java.lang.String getFieldName() { tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(createNamespace_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(modifyNamespace_result.class, metaDataMap); } - public createNamespace_result() { + public modifyNamespace_result() { } - public createNamespace_result( + public modifyNamespace_result( TIOError io) { this(); @@ -47300,14 +48192,14 @@ public createNamespace_result( /** * Performs a deep copy on other. */ - public createNamespace_result(createNamespace_result other) { + public modifyNamespace_result(modifyNamespace_result other) { if (other.isSetIo()) { this.io = new TIOError(other.io); } } - public createNamespace_result deepCopy() { - return new createNamespace_result(this); + public modifyNamespace_result deepCopy() { + return new modifyNamespace_result(this); } @Override @@ -47320,7 +48212,7 @@ public TIOError getIo() { return this.io; } - public createNamespace_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { + public modifyNamespace_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { this.io = io; return this; } @@ -47380,12 +48272,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof createNamespace_result) - return this.equals((createNamespace_result)that); + if (that instanceof modifyNamespace_result) + return this.equals((modifyNamespace_result)that); return false; } - public boolean equals(createNamespace_result that) { + public boolean equals(modifyNamespace_result that) { if (that == null) return false; if (this == that) @@ -47415,7 +48307,7 @@ public int hashCode() { } @Override - public int compareTo(createNamespace_result other) { + public int compareTo(modifyNamespace_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -47450,7 +48342,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("createNamespace_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("modifyNamespace_result("); boolean first = true; sb.append("io:"); @@ -47485,15 +48377,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class createNamespace_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public createNamespace_resultStandardScheme getScheme() { - return new createNamespace_resultStandardScheme(); + private static class modifyNamespace_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public modifyNamespace_resultStandardScheme getScheme() { + return new modifyNamespace_resultStandardScheme(); } } - private static class createNamespace_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class modifyNamespace_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, createNamespace_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, modifyNamespace_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -47523,7 +48415,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, createNamespace_res struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, createNamespace_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, modifyNamespace_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -47538,16 +48430,16 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, createNamespace_re } - private static class createNamespace_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public createNamespace_resultTupleScheme getScheme() { - return new createNamespace_resultTupleScheme(); + private static class modifyNamespace_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public modifyNamespace_resultTupleScheme getScheme() { + return new modifyNamespace_resultTupleScheme(); } } - private static class createNamespace_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class modifyNamespace_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, createNamespace_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, modifyNamespace_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.isSetIo()) { @@ -47560,7 +48452,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, createNamespace_res } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, createNamespace_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, modifyNamespace_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)) { @@ -47576,25 +48468,25 @@ private static S scheme(org.apache. } } - public static class modifyNamespace_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("modifyNamespace_args"); + public static class deleteNamespace_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteNamespace_args"); - private static final org.apache.thrift.protocol.TField NAMESPACE_DESC_FIELD_DESC = new org.apache.thrift.protocol.TField("namespaceDesc", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifyNamespace_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifyNamespace_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new deleteNamespace_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new deleteNamespace_argsTupleSchemeFactory(); /** - * descriptor which describes the new namespace + * namespace name */ - public @org.apache.thrift.annotation.Nullable TNamespaceDescriptor namespaceDesc; // required + public @org.apache.thrift.annotation.Nullable java.lang.String name; // 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 { /** - * descriptor which describes the new namespace + * namespace name */ - NAMESPACE_DESC((short)1, "namespaceDesc"); + NAME((short)1, "name"); private static final java.util.Map byName = new java.util.HashMap(); @@ -47610,8 +48502,8 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { @org.apache.thrift.annotation.Nullable public static _Fields findByThriftId(int fieldId) { switch(fieldId) { - case 1: // NAMESPACE_DESC - return NAMESPACE_DESC; + case 1: // NAME + return NAME; default: return null; } @@ -47656,78 +48548,78 @@ public java.lang.String getFieldName() { 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.NAMESPACE_DESC, new org.apache.thrift.meta_data.FieldMetaData("namespaceDesc", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TNamespaceDescriptor.class))); + tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.REQUIRED, + 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(modifyNamespace_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(deleteNamespace_args.class, metaDataMap); } - public modifyNamespace_args() { + public deleteNamespace_args() { } - public modifyNamespace_args( - TNamespaceDescriptor namespaceDesc) + public deleteNamespace_args( + java.lang.String name) { this(); - this.namespaceDesc = namespaceDesc; + this.name = name; } /** * Performs a deep copy on other. */ - public modifyNamespace_args(modifyNamespace_args other) { - if (other.isSetNamespaceDesc()) { - this.namespaceDesc = new TNamespaceDescriptor(other.namespaceDesc); + public deleteNamespace_args(deleteNamespace_args other) { + if (other.isSetName()) { + this.name = other.name; } } - public modifyNamespace_args deepCopy() { - return new modifyNamespace_args(this); + public deleteNamespace_args deepCopy() { + return new deleteNamespace_args(this); } @Override public void clear() { - this.namespaceDesc = null; + this.name = null; } /** - * descriptor which describes the new namespace + * namespace name */ @org.apache.thrift.annotation.Nullable - public TNamespaceDescriptor getNamespaceDesc() { - return this.namespaceDesc; + public java.lang.String getName() { + return this.name; } /** - * descriptor which describes the new namespace + * namespace name */ - public modifyNamespace_args setNamespaceDesc(@org.apache.thrift.annotation.Nullable TNamespaceDescriptor namespaceDesc) { - this.namespaceDesc = namespaceDesc; + public deleteNamespace_args setName(@org.apache.thrift.annotation.Nullable java.lang.String name) { + this.name = name; return this; } - public void unsetNamespaceDesc() { - this.namespaceDesc = null; + public void unsetName() { + this.name = null; } - /** Returns true if field namespaceDesc is set (has been assigned a value) and false otherwise */ - public boolean isSetNamespaceDesc() { - return this.namespaceDesc != null; + /** Returns true if field name is set (has been assigned a value) and false otherwise */ + public boolean isSetName() { + return this.name != null; } - public void setNamespaceDescIsSet(boolean value) { + public void setNameIsSet(boolean value) { if (!value) { - this.namespaceDesc = null; + this.name = null; } } public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { switch (field) { - case NAMESPACE_DESC: + case NAME: if (value == null) { - unsetNamespaceDesc(); + unsetName(); } else { - setNamespaceDesc((TNamespaceDescriptor)value); + setName((java.lang.String)value); } break; @@ -47737,8 +48629,8 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable @org.apache.thrift.annotation.Nullable public java.lang.Object getFieldValue(_Fields field) { switch (field) { - case NAMESPACE_DESC: - return getNamespaceDesc(); + case NAME: + return getName(); } throw new java.lang.IllegalStateException(); @@ -47751,8 +48643,8 @@ public boolean isSet(_Fields field) { } switch (field) { - case NAMESPACE_DESC: - return isSetNamespaceDesc(); + case NAME: + return isSetName(); } throw new java.lang.IllegalStateException(); } @@ -47761,23 +48653,23 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof modifyNamespace_args) - return this.equals((modifyNamespace_args)that); + if (that instanceof deleteNamespace_args) + return this.equals((deleteNamespace_args)that); return false; } - public boolean equals(modifyNamespace_args that) { + public boolean equals(deleteNamespace_args that) { if (that == null) return false; if (this == that) return true; - boolean this_present_namespaceDesc = true && this.isSetNamespaceDesc(); - boolean that_present_namespaceDesc = true && that.isSetNamespaceDesc(); - if (this_present_namespaceDesc || that_present_namespaceDesc) { - if (!(this_present_namespaceDesc && that_present_namespaceDesc)) + boolean this_present_name = true && this.isSetName(); + boolean that_present_name = true && that.isSetName(); + if (this_present_name || that_present_name) { + if (!(this_present_name && that_present_name)) return false; - if (!this.namespaceDesc.equals(that.namespaceDesc)) + if (!this.name.equals(that.name)) return false; } @@ -47788,27 +48680,27 @@ public boolean equals(modifyNamespace_args that) { public int hashCode() { int hashCode = 1; - hashCode = hashCode * 8191 + ((isSetNamespaceDesc()) ? 131071 : 524287); - if (isSetNamespaceDesc()) - hashCode = hashCode * 8191 + namespaceDesc.hashCode(); + hashCode = hashCode * 8191 + ((isSetName()) ? 131071 : 524287); + if (isSetName()) + hashCode = hashCode * 8191 + name.hashCode(); return hashCode; } @Override - public int compareTo(modifyNamespace_args other) { + public int compareTo(deleteNamespace_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - lastComparison = java.lang.Boolean.valueOf(isSetNamespaceDesc()).compareTo(other.isSetNamespaceDesc()); + lastComparison = java.lang.Boolean.valueOf(isSetName()).compareTo(other.isSetName()); if (lastComparison != 0) { return lastComparison; } - if (isSetNamespaceDesc()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.namespaceDesc, other.namespaceDesc); + if (isSetName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name); if (lastComparison != 0) { return lastComparison; } @@ -47831,14 +48723,14 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("modifyNamespace_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("deleteNamespace_args("); boolean first = true; - sb.append("namespaceDesc:"); - if (this.namespaceDesc == null) { + sb.append("name:"); + if (this.name == null) { sb.append("null"); } else { - sb.append(this.namespaceDesc); + sb.append(this.name); } first = false; sb.append(")"); @@ -47847,13 +48739,10 @@ public java.lang.String toString() { public void validate() throws org.apache.thrift.TException { // check for required fields - if (namespaceDesc == null) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'namespaceDesc' was not present! Struct: " + toString()); + if (name == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'name' was not present! Struct: " + toString()); } // check for sub-struct validity - if (namespaceDesc != null) { - namespaceDesc.validate(); - } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -47872,15 +48761,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class modifyNamespace_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public modifyNamespace_argsStandardScheme getScheme() { - return new modifyNamespace_argsStandardScheme(); + private static class deleteNamespace_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public deleteNamespace_argsStandardScheme getScheme() { + return new deleteNamespace_argsStandardScheme(); } } - private static class modifyNamespace_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class deleteNamespace_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, modifyNamespace_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, deleteNamespace_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -47890,11 +48779,10 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, modifyNamespace_arg break; } switch (schemeField.id) { - case 1: // NAMESPACE_DESC - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.namespaceDesc = new TNamespaceDescriptor(); - struct.namespaceDesc.read(iprot); - struct.setNamespaceDescIsSet(true); + case 1: // NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.name = iprot.readString(); + struct.setNameIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -47910,13 +48798,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, modifyNamespace_arg struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, modifyNamespace_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, deleteNamespace_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - if (struct.namespaceDesc != null) { - oprot.writeFieldBegin(NAMESPACE_DESC_FIELD_DESC); - struct.namespaceDesc.write(oprot); + if (struct.name != null) { + oprot.writeFieldBegin(NAME_FIELD_DESC); + oprot.writeString(struct.name); oprot.writeFieldEnd(); } oprot.writeFieldStop(); @@ -47925,26 +48813,25 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, modifyNamespace_ar } - private static class modifyNamespace_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public modifyNamespace_argsTupleScheme getScheme() { - return new modifyNamespace_argsTupleScheme(); + private static class deleteNamespace_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public deleteNamespace_argsTupleScheme getScheme() { + return new deleteNamespace_argsTupleScheme(); } } - private static class modifyNamespace_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class deleteNamespace_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, modifyNamespace_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, deleteNamespace_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; - struct.namespaceDesc.write(oprot); + oprot.writeString(struct.name); } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, modifyNamespace_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, deleteNamespace_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; - struct.namespaceDesc = new TNamespaceDescriptor(); - struct.namespaceDesc.read(iprot); - struct.setNamespaceDescIsSet(true); + struct.name = iprot.readString(); + struct.setNameIsSet(true); } } @@ -47953,13 +48840,13 @@ private static S scheme(org.apache. } } - public static class modifyNamespace_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("modifyNamespace_result"); + public static class deleteNamespace_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteNamespace_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifyNamespace_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifyNamespace_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new deleteNamespace_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new deleteNamespace_resultTupleSchemeFactory(); public @org.apache.thrift.annotation.Nullable TIOError io; // required @@ -48030,13 +48917,13 @@ public java.lang.String getFieldName() { tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(modifyNamespace_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(deleteNamespace_result.class, metaDataMap); } - public modifyNamespace_result() { + public deleteNamespace_result() { } - public modifyNamespace_result( + public deleteNamespace_result( TIOError io) { this(); @@ -48046,14 +48933,14 @@ public modifyNamespace_result( /** * Performs a deep copy on other. */ - public modifyNamespace_result(modifyNamespace_result other) { + public deleteNamespace_result(deleteNamespace_result other) { if (other.isSetIo()) { this.io = new TIOError(other.io); } } - public modifyNamespace_result deepCopy() { - return new modifyNamespace_result(this); + public deleteNamespace_result deepCopy() { + return new deleteNamespace_result(this); } @Override @@ -48066,7 +48953,7 @@ public TIOError getIo() { return this.io; } - public modifyNamespace_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { + public deleteNamespace_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { this.io = io; return this; } @@ -48126,12 +49013,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof modifyNamespace_result) - return this.equals((modifyNamespace_result)that); + if (that instanceof deleteNamespace_result) + return this.equals((deleteNamespace_result)that); return false; } - public boolean equals(modifyNamespace_result that) { + public boolean equals(deleteNamespace_result that) { if (that == null) return false; if (this == that) @@ -48161,7 +49048,7 @@ public int hashCode() { } @Override - public int compareTo(modifyNamespace_result other) { + public int compareTo(deleteNamespace_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -48196,7 +49083,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("modifyNamespace_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("deleteNamespace_result("); boolean first = true; sb.append("io:"); @@ -48231,15 +49118,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class modifyNamespace_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public modifyNamespace_resultStandardScheme getScheme() { - return new modifyNamespace_resultStandardScheme(); + private static class deleteNamespace_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public deleteNamespace_resultStandardScheme getScheme() { + return new deleteNamespace_resultStandardScheme(); } } - private static class modifyNamespace_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class deleteNamespace_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, modifyNamespace_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, deleteNamespace_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -48269,7 +49156,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, modifyNamespace_res struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, modifyNamespace_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, deleteNamespace_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -48284,16 +49171,16 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, modifyNamespace_re } - private static class modifyNamespace_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public modifyNamespace_resultTupleScheme getScheme() { - return new modifyNamespace_resultTupleScheme(); + private static class deleteNamespace_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public deleteNamespace_resultTupleScheme getScheme() { + return new deleteNamespace_resultTupleScheme(); } } - private static class modifyNamespace_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class deleteNamespace_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, modifyNamespace_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, deleteNamespace_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.isSetIo()) { @@ -48306,7 +49193,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, modifyNamespace_res } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, modifyNamespace_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, deleteNamespace_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)) { @@ -48322,23 +49209,23 @@ private static S scheme(org.apache. } } - public static class deleteNamespace_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteNamespace_args"); + public static class getNamespaceDescriptor_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getNamespaceDescriptor_args"); private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new deleteNamespace_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new deleteNamespace_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getNamespaceDescriptor_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getNamespaceDescriptor_argsTupleSchemeFactory(); /** - * namespace name + * name of namespace descriptor */ public @org.apache.thrift.annotation.Nullable java.lang.String name; // 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 { /** - * namespace name + * name of namespace descriptor */ NAME((short)1, "name"); @@ -48405,13 +49292,13 @@ public java.lang.String getFieldName() { tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.REQUIRED, 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(deleteNamespace_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getNamespaceDescriptor_args.class, metaDataMap); } - public deleteNamespace_args() { + public getNamespaceDescriptor_args() { } - public deleteNamespace_args( + public getNamespaceDescriptor_args( java.lang.String name) { this(); @@ -48421,14 +49308,14 @@ public deleteNamespace_args( /** * Performs a deep copy on other. */ - public deleteNamespace_args(deleteNamespace_args other) { + public getNamespaceDescriptor_args(getNamespaceDescriptor_args other) { if (other.isSetName()) { this.name = other.name; } } - public deleteNamespace_args deepCopy() { - return new deleteNamespace_args(this); + public getNamespaceDescriptor_args deepCopy() { + return new getNamespaceDescriptor_args(this); } @Override @@ -48437,7 +49324,7 @@ public void clear() { } /** - * namespace name + * name of namespace descriptor */ @org.apache.thrift.annotation.Nullable public java.lang.String getName() { @@ -48445,9 +49332,9 @@ public java.lang.String getName() { } /** - * namespace name + * name of namespace descriptor */ - public deleteNamespace_args setName(@org.apache.thrift.annotation.Nullable java.lang.String name) { + public getNamespaceDescriptor_args setName(@org.apache.thrift.annotation.Nullable java.lang.String name) { this.name = name; return this; } @@ -48507,12 +49394,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof deleteNamespace_args) - return this.equals((deleteNamespace_args)that); + if (that instanceof getNamespaceDescriptor_args) + return this.equals((getNamespaceDescriptor_args)that); return false; } - public boolean equals(deleteNamespace_args that) { + public boolean equals(getNamespaceDescriptor_args that) { if (that == null) return false; if (this == that) @@ -48542,7 +49429,7 @@ public int hashCode() { } @Override - public int compareTo(deleteNamespace_args other) { + public int compareTo(getNamespaceDescriptor_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -48577,7 +49464,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("deleteNamespace_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("getNamespaceDescriptor_args("); boolean first = true; sb.append("name:"); @@ -48615,15 +49502,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class deleteNamespace_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public deleteNamespace_argsStandardScheme getScheme() { - return new deleteNamespace_argsStandardScheme(); + private static class getNamespaceDescriptor_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public getNamespaceDescriptor_argsStandardScheme getScheme() { + return new getNamespaceDescriptor_argsStandardScheme(); } } - private static class deleteNamespace_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class getNamespaceDescriptor_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, deleteNamespace_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getNamespaceDescriptor_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -48652,7 +49539,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, deleteNamespace_arg struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, deleteNamespace_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getNamespaceDescriptor_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -48667,22 +49554,22 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, deleteNamespace_ar } - private static class deleteNamespace_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public deleteNamespace_argsTupleScheme getScheme() { - return new deleteNamespace_argsTupleScheme(); + private static class getNamespaceDescriptor_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public getNamespaceDescriptor_argsTupleScheme getScheme() { + return new getNamespaceDescriptor_argsTupleScheme(); } } - private static class deleteNamespace_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class getNamespaceDescriptor_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, deleteNamespace_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getNamespaceDescriptor_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; oprot.writeString(struct.name); } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, deleteNamespace_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getNamespaceDescriptor_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; struct.name = iprot.readString(); struct.setNameIsSet(true); @@ -48694,18 +49581,21 @@ private static S scheme(org.apache. } } - public static class deleteNamespace_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteNamespace_result"); + public static class getNamespaceDescriptor_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getNamespaceDescriptor_result"); + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new deleteNamespace_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new deleteNamespace_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getNamespaceDescriptor_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getNamespaceDescriptor_resultTupleSchemeFactory(); + public @org.apache.thrift.annotation.Nullable TNamespaceDescriptor success; // required public @org.apache.thrift.annotation.Nullable TIOError io; // 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"), IO((short)1, "io"); private static final java.util.Map byName = new java.util.HashMap(); @@ -48722,6 +49612,8 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { @org.apache.thrift.annotation.Nullable public static _Fields findByThriftId(int fieldId) { switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; case 1: // IO return IO; default: @@ -48768,46 +49660,79 @@ public java.lang.String getFieldName() { 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.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TNamespaceDescriptor.class))); tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(deleteNamespace_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getNamespaceDescriptor_result.class, metaDataMap); } - public deleteNamespace_result() { + public getNamespaceDescriptor_result() { } - public deleteNamespace_result( + public getNamespaceDescriptor_result( + TNamespaceDescriptor success, TIOError io) { this(); + this.success = success; this.io = io; } /** * Performs a deep copy on other. */ - public deleteNamespace_result(deleteNamespace_result other) { + public getNamespaceDescriptor_result(getNamespaceDescriptor_result other) { + if (other.isSetSuccess()) { + this.success = new TNamespaceDescriptor(other.success); + } if (other.isSetIo()) { this.io = new TIOError(other.io); } } - public deleteNamespace_result deepCopy() { - return new deleteNamespace_result(this); + public getNamespaceDescriptor_result deepCopy() { + return new getNamespaceDescriptor_result(this); } @Override public void clear() { + this.success = null; this.io = null; } + @org.apache.thrift.annotation.Nullable + public TNamespaceDescriptor getSuccess() { + return this.success; + } + + public getNamespaceDescriptor_result setSuccess(@org.apache.thrift.annotation.Nullable TNamespaceDescriptor 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; + } + } + @org.apache.thrift.annotation.Nullable public TIOError getIo() { return this.io; } - public deleteNamespace_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { + public getNamespaceDescriptor_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { this.io = io; return this; } @@ -48829,6 +49754,14 @@ public void setIoIsSet(boolean value) { public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TNamespaceDescriptor)value); + } + break; + case IO: if (value == null) { unsetIo(); @@ -48843,6 +49776,9 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable @org.apache.thrift.annotation.Nullable public java.lang.Object getFieldValue(_Fields field) { switch (field) { + case SUCCESS: + return getSuccess(); + case IO: return getIo(); @@ -48857,6 +49793,8 @@ public boolean isSet(_Fields field) { } switch (field) { + case SUCCESS: + return isSetSuccess(); case IO: return isSetIo(); } @@ -48867,17 +49805,26 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof deleteNamespace_result) - return this.equals((deleteNamespace_result)that); + if (that instanceof getNamespaceDescriptor_result) + return this.equals((getNamespaceDescriptor_result)that); return false; } - public boolean equals(deleteNamespace_result that) { + public boolean equals(getNamespaceDescriptor_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; + } + boolean this_present_io = true && this.isSetIo(); boolean that_present_io = true && that.isSetIo(); if (this_present_io || that_present_io) { @@ -48894,6 +49841,10 @@ public boolean equals(deleteNamespace_result that) { public int hashCode() { int hashCode = 1; + hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287); + if (isSetSuccess()) + hashCode = hashCode * 8191 + success.hashCode(); + hashCode = hashCode * 8191 + ((isSetIo()) ? 131071 : 524287); if (isSetIo()) hashCode = hashCode * 8191 + io.hashCode(); @@ -48902,13 +49853,23 @@ public int hashCode() { } @Override - public int compareTo(deleteNamespace_result other) { + public int compareTo(getNamespaceDescriptor_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; + lastComparison = java.lang.Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } lastComparison = java.lang.Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; @@ -48937,9 +49898,17 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("deleteNamespace_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("getNamespaceDescriptor_result("); boolean first = true; + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + if (!first) sb.append(", "); sb.append("io:"); if (this.io == null) { sb.append("null"); @@ -48954,6 +49923,9 @@ public java.lang.String toString() { public void validate() throws org.apache.thrift.TException { // check for required fields // check for sub-struct validity + if (success != null) { + success.validate(); + } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -48972,15 +49944,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class deleteNamespace_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public deleteNamespace_resultStandardScheme getScheme() { - return new deleteNamespace_resultStandardScheme(); + private static class getNamespaceDescriptor_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public getNamespaceDescriptor_resultStandardScheme getScheme() { + return new getNamespaceDescriptor_resultStandardScheme(); } } - private static class deleteNamespace_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class getNamespaceDescriptor_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, deleteNamespace_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getNamespaceDescriptor_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -48990,6 +49962,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, deleteNamespace_res break; } switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TNamespaceDescriptor(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; case 1: // IO if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { struct.io = new TIOError(); @@ -49010,10 +49991,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, deleteNamespace_res struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, deleteNamespace_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getNamespaceDescriptor_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } if (struct.io != null) { oprot.writeFieldBegin(IO_FIELD_DESC); struct.io.write(oprot); @@ -49025,32 +50011,43 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, deleteNamespace_re } - private static class deleteNamespace_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public deleteNamespace_resultTupleScheme getScheme() { - return new deleteNamespace_resultTupleScheme(); + private static class getNamespaceDescriptor_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public getNamespaceDescriptor_resultTupleScheme getScheme() { + return new getNamespaceDescriptor_resultTupleScheme(); } } - private static class deleteNamespace_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class getNamespaceDescriptor_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, deleteNamespace_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getNamespaceDescriptor_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.isSetIo()) { + if (struct.isSetSuccess()) { optionals.set(0); } - oprot.writeBitSet(optionals, 1); + if (struct.isSetIo()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } if (struct.isSetIo()) { struct.io.write(oprot); } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, deleteNamespace_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getNamespaceDescriptor_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); + java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { + struct.success = new TNamespaceDescriptor(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + if (incoming.get(1)) { struct.io = new TIOError(); struct.io.read(iprot); struct.setIoIsSet(true); @@ -49063,25 +50060,17 @@ private static S scheme(org.apache. } } - public static class getNamespaceDescriptor_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getNamespaceDescriptor_args"); + public static class listNamespaceDescriptors_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("listNamespaceDescriptors_args"); - private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getNamespaceDescriptor_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getNamespaceDescriptor_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new listNamespaceDescriptors_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new listNamespaceDescriptors_argsTupleSchemeFactory(); - /** - * name of namespace descriptor - */ - public @org.apache.thrift.annotation.Nullable java.lang.String name; // 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 { - /** - * name of namespace descriptor - */ - NAME((short)1, "name"); +; private static final java.util.Map byName = new java.util.HashMap(); @@ -49097,8 +50086,6 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { @org.apache.thrift.annotation.Nullable public static _Fields findByThriftId(int fieldId) { switch(fieldId) { - case 1: // NAME - return NAME; default: return null; } @@ -49138,95 +50125,38 @@ 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.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.REQUIRED, - 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(getNamespaceDescriptor_args.class, metaDataMap); - } - - public getNamespaceDescriptor_args() { + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(listNamespaceDescriptors_args.class, metaDataMap); } - public getNamespaceDescriptor_args( - java.lang.String name) - { - this(); - this.name = name; + public listNamespaceDescriptors_args() { } /** * Performs a deep copy on other. */ - public getNamespaceDescriptor_args(getNamespaceDescriptor_args other) { - if (other.isSetName()) { - this.name = other.name; - } + public listNamespaceDescriptors_args(listNamespaceDescriptors_args other) { } - public getNamespaceDescriptor_args deepCopy() { - return new getNamespaceDescriptor_args(this); + public listNamespaceDescriptors_args deepCopy() { + return new listNamespaceDescriptors_args(this); } @Override public void clear() { - this.name = null; - } - - /** - * name of namespace descriptor - */ - @org.apache.thrift.annotation.Nullable - public java.lang.String getName() { - return this.name; - } - - /** - * name of namespace descriptor - */ - public getNamespaceDescriptor_args setName(@org.apache.thrift.annotation.Nullable java.lang.String name) { - this.name = name; - return this; - } - - public void unsetName() { - this.name = null; - } - - /** Returns true if field name is set (has been assigned a value) and false otherwise */ - public boolean isSetName() { - return this.name != null; - } - - public void setNameIsSet(boolean value) { - if (!value) { - this.name = null; - } } public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { switch (field) { - case NAME: - if (value == null) { - unsetName(); - } else { - setName((java.lang.String)value); - } - break; - } } @org.apache.thrift.annotation.Nullable public java.lang.Object getFieldValue(_Fields field) { switch (field) { - case NAME: - return getName(); - } throw new java.lang.IllegalStateException(); } @@ -49238,8 +50168,6 @@ public boolean isSet(_Fields field) { } switch (field) { - case NAME: - return isSetName(); } throw new java.lang.IllegalStateException(); } @@ -49248,26 +50176,17 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof getNamespaceDescriptor_args) - return this.equals((getNamespaceDescriptor_args)that); + if (that instanceof listNamespaceDescriptors_args) + return this.equals((listNamespaceDescriptors_args)that); return false; } - public boolean equals(getNamespaceDescriptor_args that) { + public boolean equals(listNamespaceDescriptors_args that) { if (that == null) return false; if (this == that) return true; - boolean this_present_name = true && this.isSetName(); - boolean that_present_name = true && that.isSetName(); - if (this_present_name || that_present_name) { - if (!(this_present_name && that_present_name)) - return false; - if (!this.name.equals(that.name)) - return false; - } - return true; } @@ -49275,31 +50194,17 @@ public boolean equals(getNamespaceDescriptor_args that) { public int hashCode() { int hashCode = 1; - hashCode = hashCode * 8191 + ((isSetName()) ? 131071 : 524287); - if (isSetName()) - hashCode = hashCode * 8191 + name.hashCode(); - return hashCode; } @Override - public int compareTo(getNamespaceDescriptor_args other) { + public int compareTo(listNamespaceDescriptors_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - lastComparison = java.lang.Boolean.valueOf(isSetName()).compareTo(other.isSetName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name); - if (lastComparison != 0) { - return lastComparison; - } - } return 0; } @@ -49318,25 +50223,15 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("getNamespaceDescriptor_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("listNamespaceDescriptors_args("); boolean first = true; - sb.append("name:"); - if (this.name == null) { - sb.append("null"); - } else { - sb.append(this.name); - } - first = false; sb.append(")"); return sb.toString(); } public void validate() throws org.apache.thrift.TException { // check for required fields - if (name == null) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'name' was not present! Struct: " + toString()); - } // check for sub-struct validity } @@ -49356,15 +50251,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class getNamespaceDescriptor_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public getNamespaceDescriptor_argsStandardScheme getScheme() { - return new getNamespaceDescriptor_argsStandardScheme(); + private static class listNamespaceDescriptors_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public listNamespaceDescriptors_argsStandardScheme getScheme() { + return new listNamespaceDescriptors_argsStandardScheme(); } } - private static class getNamespaceDescriptor_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class listNamespaceDescriptors_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, getNamespaceDescriptor_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, listNamespaceDescriptors_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -49374,14 +50269,6 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getNamespaceDescrip break; } switch (schemeField.id) { - case 1: // NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.name = iprot.readString(); - struct.setNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -49393,40 +50280,32 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getNamespaceDescrip struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, getNamespaceDescriptor_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, listNamespaceDescriptors_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - if (struct.name != null) { - oprot.writeFieldBegin(NAME_FIELD_DESC); - oprot.writeString(struct.name); - oprot.writeFieldEnd(); - } oprot.writeFieldStop(); oprot.writeStructEnd(); } } - private static class getNamespaceDescriptor_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public getNamespaceDescriptor_argsTupleScheme getScheme() { - return new getNamespaceDescriptor_argsTupleScheme(); + private static class listNamespaceDescriptors_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public listNamespaceDescriptors_argsTupleScheme getScheme() { + return new listNamespaceDescriptors_argsTupleScheme(); } } - private static class getNamespaceDescriptor_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class listNamespaceDescriptors_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getNamespaceDescriptor_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, listNamespaceDescriptors_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; - oprot.writeString(struct.name); } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getNamespaceDescriptor_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, listNamespaceDescriptors_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; - struct.name = iprot.readString(); - struct.setNameIsSet(true); } } @@ -49435,16 +50314,16 @@ private static S scheme(org.apache. } } - public static class getNamespaceDescriptor_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getNamespaceDescriptor_result"); + public static class listNamespaceDescriptors_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("listNamespaceDescriptors_result"); - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getNamespaceDescriptor_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getNamespaceDescriptor_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new listNamespaceDescriptors_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new listNamespaceDescriptors_resultTupleSchemeFactory(); - public @org.apache.thrift.annotation.Nullable TNamespaceDescriptor success; // required + public @org.apache.thrift.annotation.Nullable java.util.List success; // required public @org.apache.thrift.annotation.Nullable TIOError io; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ @@ -49515,18 +50394,19 @@ public java.lang.String getFieldName() { 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.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TNamespaceDescriptor.class))); + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TNamespaceDescriptor.class)))); tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getNamespaceDescriptor_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(listNamespaceDescriptors_result.class, metaDataMap); } - public getNamespaceDescriptor_result() { + public listNamespaceDescriptors_result() { } - public getNamespaceDescriptor_result( - TNamespaceDescriptor success, + public listNamespaceDescriptors_result( + java.util.List success, TIOError io) { this(); @@ -49537,17 +50417,21 @@ public getNamespaceDescriptor_result( /** * Performs a deep copy on other. */ - public getNamespaceDescriptor_result(getNamespaceDescriptor_result other) { + public listNamespaceDescriptors_result(listNamespaceDescriptors_result other) { if (other.isSetSuccess()) { - this.success = new TNamespaceDescriptor(other.success); + java.util.List __this__success = new java.util.ArrayList(other.success.size()); + for (TNamespaceDescriptor other_element : other.success) { + __this__success.add(new TNamespaceDescriptor(other_element)); + } + this.success = __this__success; } if (other.isSetIo()) { this.io = new TIOError(other.io); } } - public getNamespaceDescriptor_result deepCopy() { - return new getNamespaceDescriptor_result(this); + public listNamespaceDescriptors_result deepCopy() { + return new listNamespaceDescriptors_result(this); } @Override @@ -49556,12 +50440,28 @@ public void clear() { this.io = null; } + public int getSuccessSize() { + return (this.success == null) ? 0 : this.success.size(); + } + @org.apache.thrift.annotation.Nullable - public TNamespaceDescriptor getSuccess() { + public java.util.Iterator getSuccessIterator() { + return (this.success == null) ? null : this.success.iterator(); + } + + public void addToSuccess(TNamespaceDescriptor elem) { + if (this.success == null) { + this.success = new java.util.ArrayList(); + } + this.success.add(elem); + } + + @org.apache.thrift.annotation.Nullable + public java.util.List getSuccess() { return this.success; } - public getNamespaceDescriptor_result setSuccess(@org.apache.thrift.annotation.Nullable TNamespaceDescriptor success) { + public listNamespaceDescriptors_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.List success) { this.success = success; return this; } @@ -49586,7 +50486,7 @@ public TIOError getIo() { return this.io; } - public getNamespaceDescriptor_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { + public listNamespaceDescriptors_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { this.io = io; return this; } @@ -49612,7 +50512,7 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable if (value == null) { unsetSuccess(); } else { - setSuccess((TNamespaceDescriptor)value); + setSuccess((java.util.List)value); } break; @@ -49659,12 +50559,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof getNamespaceDescriptor_result) - return this.equals((getNamespaceDescriptor_result)that); + if (that instanceof listNamespaceDescriptors_result) + return this.equals((listNamespaceDescriptors_result)that); return false; } - public boolean equals(getNamespaceDescriptor_result that) { + public boolean equals(listNamespaceDescriptors_result that) { if (that == null) return false; if (this == that) @@ -49707,7 +50607,7 @@ public int hashCode() { } @Override - public int compareTo(getNamespaceDescriptor_result other) { + public int compareTo(listNamespaceDescriptors_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -49752,7 +50652,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("getNamespaceDescriptor_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("listNamespaceDescriptors_result("); boolean first = true; sb.append("success:"); @@ -49777,9 +50677,6 @@ public java.lang.String toString() { public void validate() throws org.apache.thrift.TException { // check for required fields // check for sub-struct validity - if (success != null) { - success.validate(); - } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -49798,15 +50695,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class getNamespaceDescriptor_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public getNamespaceDescriptor_resultStandardScheme getScheme() { - return new getNamespaceDescriptor_resultStandardScheme(); + private static class listNamespaceDescriptors_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public listNamespaceDescriptors_resultStandardScheme getScheme() { + return new listNamespaceDescriptors_resultStandardScheme(); } } - private static class getNamespaceDescriptor_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class listNamespaceDescriptors_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, getNamespaceDescriptor_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, listNamespaceDescriptors_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -49817,9 +50714,19 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getNamespaceDescrip } switch (schemeField.id) { case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TNamespaceDescriptor(); - struct.success.read(iprot); + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list334 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list334.size); + @org.apache.thrift.annotation.Nullable TNamespaceDescriptor _elem335; + for (int _i336 = 0; _i336 < _list334.size; ++_i336) + { + _elem335 = new TNamespaceDescriptor(); + _elem335.read(iprot); + struct.success.add(_elem335); + } + iprot.readListEnd(); + } struct.setSuccessIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); @@ -49845,13 +50752,20 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getNamespaceDescrip struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, getNamespaceDescriptor_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, listNamespaceDescriptors_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); if (struct.success != null) { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); + for (TNamespaceDescriptor _iter337 : struct.success) + { + _iter337.write(oprot); + } + oprot.writeListEnd(); + } oprot.writeFieldEnd(); } if (struct.io != null) { @@ -49865,16 +50779,16 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getNamespaceDescri } - private static class getNamespaceDescriptor_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public getNamespaceDescriptor_resultTupleScheme getScheme() { - return new getNamespaceDescriptor_resultTupleScheme(); + private static class listNamespaceDescriptors_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public listNamespaceDescriptors_resultTupleScheme getScheme() { + return new listNamespaceDescriptors_resultTupleScheme(); } } - private static class getNamespaceDescriptor_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class listNamespaceDescriptors_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getNamespaceDescriptor_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, listNamespaceDescriptors_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()) { @@ -49885,7 +50799,13 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getNamespaceDescrip } oprot.writeBitSet(optionals, 2); if (struct.isSetSuccess()) { - struct.success.write(oprot); + { + oprot.writeI32(struct.success.size()); + for (TNamespaceDescriptor _iter338 : struct.success) + { + _iter338.write(oprot); + } + } } if (struct.isSetIo()) { struct.io.write(oprot); @@ -49893,12 +50813,21 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getNamespaceDescrip } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getNamespaceDescriptor_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, listNamespaceDescriptors_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(2); if (incoming.get(0)) { - struct.success = new TNamespaceDescriptor(); - struct.success.read(iprot); + { + org.apache.thrift.protocol.TList _list339 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new java.util.ArrayList(_list339.size); + @org.apache.thrift.annotation.Nullable TNamespaceDescriptor _elem340; + for (int _i341 = 0; _i341 < _list339.size; ++_i341) + { + _elem340 = new TNamespaceDescriptor(); + _elem340.read(iprot); + struct.success.add(_elem340); + } + } struct.setSuccessIsSet(true); } if (incoming.get(1)) { @@ -49914,12 +50843,12 @@ private static S scheme(org.apache. } } - public static class listNamespaceDescriptors_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("listNamespaceDescriptors_args"); + public static class listNamespaces_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("listNamespaces_args"); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new listNamespaceDescriptors_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new listNamespaceDescriptors_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new listNamespaces_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new listNamespaces_argsTupleSchemeFactory(); /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ @@ -49983,20 +50912,20 @@ public java.lang.String getFieldName() { 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(listNamespaceDescriptors_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(listNamespaces_args.class, metaDataMap); } - public listNamespaceDescriptors_args() { + public listNamespaces_args() { } /** * Performs a deep copy on other. */ - public listNamespaceDescriptors_args(listNamespaceDescriptors_args other) { + public listNamespaces_args(listNamespaces_args other) { } - public listNamespaceDescriptors_args deepCopy() { - return new listNamespaceDescriptors_args(this); + public listNamespaces_args deepCopy() { + return new listNamespaces_args(this); } @Override @@ -50030,12 +50959,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof listNamespaceDescriptors_args) - return this.equals((listNamespaceDescriptors_args)that); + if (that instanceof listNamespaces_args) + return this.equals((listNamespaces_args)that); return false; } - public boolean equals(listNamespaceDescriptors_args that) { + public boolean equals(listNamespaces_args that) { if (that == null) return false; if (this == that) @@ -50052,7 +50981,7 @@ public int hashCode() { } @Override - public int compareTo(listNamespaceDescriptors_args other) { + public int compareTo(listNamespaces_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -50077,7 +51006,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("listNamespaceDescriptors_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("listNamespaces_args("); boolean first = true; sb.append(")"); @@ -50105,15 +51034,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class listNamespaceDescriptors_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public listNamespaceDescriptors_argsStandardScheme getScheme() { - return new listNamespaceDescriptors_argsStandardScheme(); + private static class listNamespaces_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public listNamespaces_argsStandardScheme getScheme() { + return new listNamespaces_argsStandardScheme(); } } - private static class listNamespaceDescriptors_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class listNamespaces_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, listNamespaceDescriptors_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, listNamespaces_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -50134,7 +51063,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, listNamespaceDescri struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, listNamespaceDescriptors_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, listNamespaces_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -50144,21 +51073,21 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, listNamespaceDescr } - private static class listNamespaceDescriptors_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public listNamespaceDescriptors_argsTupleScheme getScheme() { - return new listNamespaceDescriptors_argsTupleScheme(); + private static class listNamespaces_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public listNamespaces_argsTupleScheme getScheme() { + return new listNamespaces_argsTupleScheme(); } } - private static class listNamespaceDescriptors_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class listNamespaces_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, listNamespaceDescriptors_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, listNamespaces_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, listNamespaceDescriptors_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, listNamespaces_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; } } @@ -50168,16 +51097,16 @@ private static S scheme(org.apache. } } - public static class listNamespaceDescriptors_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("listNamespaceDescriptors_result"); + public static class listNamespaces_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("listNamespaces_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new listNamespaceDescriptors_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new listNamespaceDescriptors_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new listNamespaces_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new listNamespaces_resultTupleSchemeFactory(); - public @org.apache.thrift.annotation.Nullable java.util.List success; // required + public @org.apache.thrift.annotation.Nullable java.util.List success; // required public @org.apache.thrift.annotation.Nullable TIOError io; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ @@ -50249,18 +51178,18 @@ public java.lang.String getFieldName() { 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.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TNamespaceDescriptor.class)))); + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(listNamespaceDescriptors_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(listNamespaces_result.class, metaDataMap); } - public listNamespaceDescriptors_result() { + public listNamespaces_result() { } - public listNamespaceDescriptors_result( - java.util.List success, + public listNamespaces_result( + java.util.List success, TIOError io) { this(); @@ -50271,12 +51200,9 @@ public listNamespaceDescriptors_result( /** * Performs a deep copy on other. */ - public listNamespaceDescriptors_result(listNamespaceDescriptors_result other) { + public listNamespaces_result(listNamespaces_result other) { if (other.isSetSuccess()) { - java.util.List __this__success = new java.util.ArrayList(other.success.size()); - for (TNamespaceDescriptor other_element : other.success) { - __this__success.add(new TNamespaceDescriptor(other_element)); - } + java.util.List __this__success = new java.util.ArrayList(other.success); this.success = __this__success; } if (other.isSetIo()) { @@ -50284,8 +51210,8 @@ public listNamespaceDescriptors_result(listNamespaceDescriptors_result other) { } } - public listNamespaceDescriptors_result deepCopy() { - return new listNamespaceDescriptors_result(this); + public listNamespaces_result deepCopy() { + return new listNamespaces_result(this); } @Override @@ -50299,23 +51225,23 @@ public int getSuccessSize() { } @org.apache.thrift.annotation.Nullable - public java.util.Iterator getSuccessIterator() { + public java.util.Iterator getSuccessIterator() { return (this.success == null) ? null : this.success.iterator(); } - public void addToSuccess(TNamespaceDescriptor elem) { + public void addToSuccess(java.lang.String elem) { if (this.success == null) { - this.success = new java.util.ArrayList(); + this.success = new java.util.ArrayList(); } this.success.add(elem); } @org.apache.thrift.annotation.Nullable - public java.util.List getSuccess() { + public java.util.List getSuccess() { return this.success; } - public listNamespaceDescriptors_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.List success) { + public listNamespaces_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.List success) { this.success = success; return this; } @@ -50340,7 +51266,7 @@ public TIOError getIo() { return this.io; } - public listNamespaceDescriptors_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { + public listNamespaces_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { this.io = io; return this; } @@ -50366,7 +51292,7 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable if (value == null) { unsetSuccess(); } else { - setSuccess((java.util.List)value); + setSuccess((java.util.List)value); } break; @@ -50413,12 +51339,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof listNamespaceDescriptors_result) - return this.equals((listNamespaceDescriptors_result)that); + if (that instanceof listNamespaces_result) + return this.equals((listNamespaces_result)that); return false; } - public boolean equals(listNamespaceDescriptors_result that) { + public boolean equals(listNamespaces_result that) { if (that == null) return false; if (this == that) @@ -50461,7 +51387,7 @@ public int hashCode() { } @Override - public int compareTo(listNamespaceDescriptors_result other) { + public int compareTo(listNamespaces_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -50506,7 +51432,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("listNamespaceDescriptors_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("listNamespaces_result("); boolean first = true; sb.append("success:"); @@ -50549,15 +51475,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class listNamespaceDescriptors_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public listNamespaceDescriptors_resultStandardScheme getScheme() { - return new listNamespaceDescriptors_resultStandardScheme(); + private static class listNamespaces_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public listNamespaces_resultStandardScheme getScheme() { + return new listNamespaces_resultStandardScheme(); } } - private static class listNamespaceDescriptors_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class listNamespaces_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, listNamespaceDescriptors_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, listNamespaces_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -50570,14 +51496,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, listNamespaceDescri case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list334 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list334.size); - @org.apache.thrift.annotation.Nullable TNamespaceDescriptor _elem335; - for (int _i336 = 0; _i336 < _list334.size; ++_i336) + org.apache.thrift.protocol.TList _list342 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list342.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem343; + for (int _i344 = 0; _i344 < _list342.size; ++_i344) { - _elem335 = new TNamespaceDescriptor(); - _elem335.read(iprot); - struct.success.add(_elem335); + _elem343 = iprot.readString(); + struct.success.add(_elem343); } iprot.readListEnd(); } @@ -50606,17 +51531,17 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, listNamespaceDescri struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, listNamespaceDescriptors_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, listNamespaces_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); if (struct.success != null) { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (TNamespaceDescriptor _iter337 : struct.success) + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); + for (java.lang.String _iter345 : struct.success) { - _iter337.write(oprot); + oprot.writeString(_iter345); } oprot.writeListEnd(); } @@ -50633,16 +51558,16 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, listNamespaceDescr } - private static class listNamespaceDescriptors_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public listNamespaceDescriptors_resultTupleScheme getScheme() { - return new listNamespaceDescriptors_resultTupleScheme(); + private static class listNamespaces_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public listNamespaces_resultTupleScheme getScheme() { + return new listNamespaces_resultTupleScheme(); } } - private static class listNamespaceDescriptors_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class listNamespaces_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, listNamespaceDescriptors_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, listNamespaces_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()) { @@ -50655,9 +51580,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, listNamespaceDescri if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (TNamespaceDescriptor _iter338 : struct.success) + for (java.lang.String _iter346 : struct.success) { - _iter338.write(oprot); + oprot.writeString(_iter346); } } } @@ -50667,19 +51592,18 @@ public void write(org.apache.thrift.protocol.TProtocol prot, listNamespaceDescri } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, listNamespaceDescriptors_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, listNamespaces_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(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list339 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new java.util.ArrayList(_list339.size); - @org.apache.thrift.annotation.Nullable TNamespaceDescriptor _elem340; - for (int _i341 = 0; _i341 < _list339.size; ++_i341) + org.apache.thrift.protocol.TList _list347 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.success = new java.util.ArrayList(_list347.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem348; + for (int _i349 = 0; _i349 < _list347.size; ++_i349) { - _elem340 = new TNamespaceDescriptor(); - _elem340.read(iprot); - struct.success.add(_elem340); + _elem348 = iprot.readString(); + struct.success.add(_elem348); } } struct.setSuccessIsSet(true); @@ -50697,12 +51621,12 @@ private static S scheme(org.apache. } } - public static class listNamespaces_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("listNamespaces_args"); + public static class getThriftServerType_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getThriftServerType_args"); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new listNamespaces_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new listNamespaces_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getThriftServerType_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getThriftServerType_argsTupleSchemeFactory(); /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ @@ -50766,20 +51690,20 @@ public java.lang.String getFieldName() { 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(listNamespaces_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getThriftServerType_args.class, metaDataMap); } - public listNamespaces_args() { + public getThriftServerType_args() { } /** * Performs a deep copy on other. */ - public listNamespaces_args(listNamespaces_args other) { + public getThriftServerType_args(getThriftServerType_args other) { } - public listNamespaces_args deepCopy() { - return new listNamespaces_args(this); + public getThriftServerType_args deepCopy() { + return new getThriftServerType_args(this); } @Override @@ -50813,12 +51737,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof listNamespaces_args) - return this.equals((listNamespaces_args)that); + if (that instanceof getThriftServerType_args) + return this.equals((getThriftServerType_args)that); return false; } - public boolean equals(listNamespaces_args that) { + public boolean equals(getThriftServerType_args that) { if (that == null) return false; if (this == that) @@ -50835,7 +51759,7 @@ public int hashCode() { } @Override - public int compareTo(listNamespaces_args other) { + public int compareTo(getThriftServerType_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -50860,7 +51784,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("listNamespaces_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("getThriftServerType_args("); boolean first = true; sb.append(")"); @@ -50888,15 +51812,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class listNamespaces_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public listNamespaces_argsStandardScheme getScheme() { - return new listNamespaces_argsStandardScheme(); + private static class getThriftServerType_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public getThriftServerType_argsStandardScheme getScheme() { + return new getThriftServerType_argsStandardScheme(); } } - private static class listNamespaces_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class getThriftServerType_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, listNamespaces_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getThriftServerType_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -50917,7 +51841,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, listNamespaces_args struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, listNamespaces_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getThriftServerType_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -50927,21 +51851,21 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, listNamespaces_arg } - private static class listNamespaces_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public listNamespaces_argsTupleScheme getScheme() { - return new listNamespaces_argsTupleScheme(); + private static class getThriftServerType_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public getThriftServerType_argsTupleScheme getScheme() { + return new getThriftServerType_argsTupleScheme(); } } - private static class listNamespaces_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class getThriftServerType_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, listNamespaces_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getThriftServerType_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, listNamespaces_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getThriftServerType_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; } } @@ -50951,22 +51875,27 @@ private static S scheme(org.apache. } } - public static class listNamespaces_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("listNamespaces_result"); + public static class getThriftServerType_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getThriftServerType_result"); - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0); - private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.I32, (short)0); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new listNamespaces_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new listNamespaces_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getThriftServerType_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getThriftServerType_resultTupleSchemeFactory(); - public @org.apache.thrift.annotation.Nullable java.util.List success; // required - public @org.apache.thrift.annotation.Nullable TIOError io; // required + /** + * + * @see TThriftServerType + */ + public @org.apache.thrift.annotation.Nullable TThriftServerType 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"), - IO((short)1, "io"); + /** + * + * @see TThriftServerType + */ + SUCCESS((short)0, "success"); private static final java.util.Map byName = new java.util.HashMap(); @@ -50984,8 +51913,6 @@ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 0: // SUCCESS return SUCCESS; - case 1: // IO - return IO; default: return null; } @@ -51031,71 +51958,53 @@ public java.lang.String getFieldName() { 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.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); - tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class))); + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TThriftServerType.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(listNamespaces_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getThriftServerType_result.class, metaDataMap); } - public listNamespaces_result() { + public getThriftServerType_result() { } - public listNamespaces_result( - java.util.List success, - TIOError io) + public getThriftServerType_result( + TThriftServerType success) { this(); this.success = success; - this.io = io; } /** * Performs a deep copy on other. */ - public listNamespaces_result(listNamespaces_result other) { + public getThriftServerType_result(getThriftServerType_result other) { if (other.isSetSuccess()) { - java.util.List __this__success = new java.util.ArrayList(other.success); - this.success = __this__success; - } - if (other.isSetIo()) { - this.io = new TIOError(other.io); + this.success = other.success; } } - public listNamespaces_result deepCopy() { - return new listNamespaces_result(this); + public getThriftServerType_result deepCopy() { + return new getThriftServerType_result(this); } @Override public void clear() { this.success = null; - this.io = null; - } - - public int getSuccessSize() { - return (this.success == null) ? 0 : this.success.size(); - } - - @org.apache.thrift.annotation.Nullable - public java.util.Iterator getSuccessIterator() { - return (this.success == null) ? null : this.success.iterator(); - } - - public void addToSuccess(java.lang.String elem) { - if (this.success == null) { - this.success = new java.util.ArrayList(); - } - this.success.add(elem); } + /** + * + * @see TThriftServerType + */ @org.apache.thrift.annotation.Nullable - public java.util.List getSuccess() { + public TThriftServerType getSuccess() { return this.success; } - public listNamespaces_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.List success) { + /** + * + * @see TThriftServerType + */ + public getThriftServerType_result setSuccess(@org.apache.thrift.annotation.Nullable TThriftServerType success) { this.success = success; return this; } @@ -51115,46 +52024,13 @@ public void setSuccessIsSet(boolean value) { } } - @org.apache.thrift.annotation.Nullable - public TIOError getIo() { - return this.io; - } - - public listNamespaces_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) { - this.io = io; - return this; - } - - public void unsetIo() { - this.io = null; - } - - /** Returns true if field io is set (has been assigned a value) and false otherwise */ - public boolean isSetIo() { - return this.io != null; - } - - public void setIoIsSet(boolean value) { - if (!value) { - this.io = null; - } - } - 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.util.List)value); - } - break; - - case IO: - if (value == null) { - unsetIo(); - } else { - setIo((TIOError)value); + setSuccess((TThriftServerType)value); } break; @@ -51167,9 +52043,6 @@ public java.lang.Object getFieldValue(_Fields field) { case SUCCESS: return getSuccess(); - case IO: - return getIo(); - } throw new java.lang.IllegalStateException(); } @@ -51183,8 +52056,6 @@ public boolean isSet(_Fields field) { switch (field) { case SUCCESS: return isSetSuccess(); - case IO: - return isSetIo(); } throw new java.lang.IllegalStateException(); } @@ -51193,12 +52064,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof listNamespaces_result) - return this.equals((listNamespaces_result)that); + if (that instanceof getThriftServerType_result) + return this.equals((getThriftServerType_result)that); return false; } - public boolean equals(listNamespaces_result that) { + public boolean equals(getThriftServerType_result that) { if (that == null) return false; if (this == that) @@ -51213,15 +52084,6 @@ public boolean equals(listNamespaces_result that) { return false; } - boolean this_present_io = true && this.isSetIo(); - boolean that_present_io = true && that.isSetIo(); - if (this_present_io || that_present_io) { - if (!(this_present_io && that_present_io)) - return false; - if (!this.io.equals(that.io)) - return false; - } - return true; } @@ -51231,17 +52093,13 @@ public int hashCode() { hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287); if (isSetSuccess()) - hashCode = hashCode * 8191 + success.hashCode(); - - hashCode = hashCode * 8191 + ((isSetIo()) ? 131071 : 524287); - if (isSetIo()) - hashCode = hashCode * 8191 + io.hashCode(); + hashCode = hashCode * 8191 + success.getValue(); return hashCode; } @Override - public int compareTo(listNamespaces_result other) { + public int compareTo(getThriftServerType_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -51258,16 +52116,6 @@ public int compareTo(listNamespaces_result other) { return lastComparison; } } - lastComparison = java.lang.Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); - if (lastComparison != 0) { - return lastComparison; - } - } return 0; } @@ -51286,7 +52134,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("listNamespaces_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("getThriftServerType_result("); boolean first = true; sb.append("success:"); @@ -51296,14 +52144,6 @@ public java.lang.String toString() { sb.append(this.success); } first = false; - if (!first) sb.append(", "); - sb.append("io:"); - if (this.io == null) { - sb.append("null"); - } else { - sb.append(this.io); - } - first = false; sb.append(")"); return sb.toString(); } @@ -51329,15 +52169,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class listNamespaces_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public listNamespaces_resultStandardScheme getScheme() { - return new listNamespaces_resultStandardScheme(); + private static class getThriftServerType_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public getThriftServerType_resultStandardScheme getScheme() { + return new getThriftServerType_resultStandardScheme(); } } - private static class listNamespaces_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class getThriftServerType_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, listNamespaces_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getThriftServerType_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -51348,32 +52188,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, listNamespaces_resu } switch (schemeField.id) { case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list342 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list342.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem343; - for (int _i344 = 0; _i344 < _list342.size; ++_i344) - { - _elem343 = iprot.readString(); - struct.success.add(_elem343); - } - iprot.readListEnd(); - } + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.success = org.apache.hadoop.hbase.thrift2.generated.TThriftServerType.findByValue(iprot.readI32()); struct.setSuccessIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 1: // IO - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.io = new TIOError(); - struct.io.read(iprot); - struct.setIoIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -51385,25 +52206,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, listNamespaces_resu struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, listNamespaces_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getThriftServerType_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); if (struct.success != null) { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter345 : struct.success) - { - oprot.writeString(_iter345); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - if (struct.io != null) { - oprot.writeFieldBegin(IO_FIELD_DESC); - struct.io.write(oprot); + oprot.writeI32(struct.success.getValue()); oprot.writeFieldEnd(); } oprot.writeFieldStop(); @@ -51412,61 +52221,35 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, listNamespaces_res } - private static class listNamespaces_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public listNamespaces_resultTupleScheme getScheme() { - return new listNamespaces_resultTupleScheme(); + private static class getThriftServerType_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public getThriftServerType_resultTupleScheme getScheme() { + return new getThriftServerType_resultTupleScheme(); } } - private static class listNamespaces_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class getThriftServerType_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, listNamespaces_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getThriftServerType_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); } - if (struct.isSetIo()) { - optionals.set(1); - } - oprot.writeBitSet(optionals, 2); + oprot.writeBitSet(optionals, 1); if (struct.isSetSuccess()) { - { - oprot.writeI32(struct.success.size()); - for (java.lang.String _iter346 : struct.success) - { - oprot.writeString(_iter346); - } - } - } - if (struct.isSetIo()) { - struct.io.write(oprot); + oprot.writeI32(struct.success.getValue()); } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, listNamespaces_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getThriftServerType_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(2); + java.util.BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { - { - org.apache.thrift.protocol.TList _list347 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.success = new java.util.ArrayList(_list347.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem348; - for (int _i349 = 0; _i349 < _list347.size; ++_i349) - { - _elem348 = iprot.readString(); - struct.success.add(_elem348); - } - } + struct.success = org.apache.hadoop.hbase.thrift2.generated.TThriftServerType.findByValue(iprot.readI32()); struct.setSuccessIsSet(true); } - if (incoming.get(1)) { - struct.io = new TIOError(); - struct.io.read(iprot); - struct.setIoIsSet(true); - } } } @@ -51475,12 +52258,12 @@ private static S scheme(org.apache. } } - public static class getThriftServerType_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getThriftServerType_args"); + public static class getClusterId_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getClusterId_args"); - private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getThriftServerType_argsStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getThriftServerType_argsTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getClusterId_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getClusterId_argsTupleSchemeFactory(); /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ @@ -51544,20 +52327,20 @@ public java.lang.String getFieldName() { 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(getThriftServerType_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getClusterId_args.class, metaDataMap); } - public getThriftServerType_args() { + public getClusterId_args() { } /** * Performs a deep copy on other. */ - public getThriftServerType_args(getThriftServerType_args other) { + public getClusterId_args(getClusterId_args other) { } - public getThriftServerType_args deepCopy() { - return new getThriftServerType_args(this); + public getClusterId_args deepCopy() { + return new getClusterId_args(this); } @Override @@ -51591,12 +52374,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof getThriftServerType_args) - return this.equals((getThriftServerType_args)that); + if (that instanceof getClusterId_args) + return this.equals((getClusterId_args)that); return false; } - public boolean equals(getThriftServerType_args that) { + public boolean equals(getClusterId_args that) { if (that == null) return false; if (this == that) @@ -51613,7 +52396,7 @@ public int hashCode() { } @Override - public int compareTo(getThriftServerType_args other) { + public int compareTo(getClusterId_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -51638,7 +52421,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("getThriftServerType_args("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("getClusterId_args("); boolean first = true; sb.append(")"); @@ -51666,15 +52449,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class getThriftServerType_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public getThriftServerType_argsStandardScheme getScheme() { - return new getThriftServerType_argsStandardScheme(); + private static class getClusterId_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public getClusterId_argsStandardScheme getScheme() { + return new getClusterId_argsStandardScheme(); } } - private static class getThriftServerType_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class getClusterId_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, getThriftServerType_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getClusterId_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -51695,7 +52478,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getThriftServerType struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, getThriftServerType_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getClusterId_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -51705,21 +52488,21 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getThriftServerTyp } - private static class getThriftServerType_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public getThriftServerType_argsTupleScheme getScheme() { - return new getThriftServerType_argsTupleScheme(); + private static class getClusterId_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public getClusterId_argsTupleScheme getScheme() { + return new getClusterId_argsTupleScheme(); } } - private static class getThriftServerType_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class getClusterId_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getThriftServerType_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getClusterId_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, getThriftServerType_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getClusterId_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; } } @@ -51729,26 +52512,18 @@ private static S scheme(org.apache. } } - public static class getThriftServerType_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getThriftServerType_result"); + public static class getClusterId_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getClusterId_result"); - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.I32, (short)0); + 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 getThriftServerType_resultStandardSchemeFactory(); - private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getThriftServerType_resultTupleSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getClusterId_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getClusterId_resultTupleSchemeFactory(); - /** - * - * @see TThriftServerType - */ - public @org.apache.thrift.annotation.Nullable TThriftServerType success; // required + 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 { - /** - * - * @see TThriftServerType - */ SUCCESS((short)0, "success"); private static final java.util.Map byName = new java.util.HashMap(); @@ -51812,16 +52587,16 @@ public java.lang.String getFieldName() { 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.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TThriftServerType.class))); + 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(getThriftServerType_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getClusterId_result.class, metaDataMap); } - public getThriftServerType_result() { + public getClusterId_result() { } - public getThriftServerType_result( - TThriftServerType success) + public getClusterId_result( + java.lang.String success) { this(); this.success = success; @@ -51830,14 +52605,14 @@ public getThriftServerType_result( /** * Performs a deep copy on other. */ - public getThriftServerType_result(getThriftServerType_result other) { + public getClusterId_result(getClusterId_result other) { if (other.isSetSuccess()) { this.success = other.success; } } - public getThriftServerType_result deepCopy() { - return new getThriftServerType_result(this); + public getClusterId_result deepCopy() { + return new getClusterId_result(this); } @Override @@ -51845,20 +52620,12 @@ public void clear() { this.success = null; } - /** - * - * @see TThriftServerType - */ @org.apache.thrift.annotation.Nullable - public TThriftServerType getSuccess() { + public java.lang.String getSuccess() { return this.success; } - /** - * - * @see TThriftServerType - */ - public getThriftServerType_result setSuccess(@org.apache.thrift.annotation.Nullable TThriftServerType success) { + public getClusterId_result setSuccess(@org.apache.thrift.annotation.Nullable java.lang.String success) { this.success = success; return this; } @@ -51884,7 +52651,7 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable if (value == null) { unsetSuccess(); } else { - setSuccess((TThriftServerType)value); + setSuccess((java.lang.String)value); } break; @@ -51918,12 +52685,12 @@ public boolean isSet(_Fields field) { public boolean equals(java.lang.Object that) { if (that == null) return false; - if (that instanceof getThriftServerType_result) - return this.equals((getThriftServerType_result)that); + if (that instanceof getClusterId_result) + return this.equals((getClusterId_result)that); return false; } - public boolean equals(getThriftServerType_result that) { + public boolean equals(getClusterId_result that) { if (that == null) return false; if (this == that) @@ -51947,13 +52714,13 @@ public int hashCode() { hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287); if (isSetSuccess()) - hashCode = hashCode * 8191 + success.getValue(); + hashCode = hashCode * 8191 + success.hashCode(); return hashCode; } @Override - public int compareTo(getThriftServerType_result other) { + public int compareTo(getClusterId_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -51988,7 +52755,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public java.lang.String toString() { - java.lang.StringBuilder sb = new java.lang.StringBuilder("getThriftServerType_result("); + java.lang.StringBuilder sb = new java.lang.StringBuilder("getClusterId_result("); boolean first = true; sb.append("success:"); @@ -52023,15 +52790,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class getThriftServerType_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public getThriftServerType_resultStandardScheme getScheme() { - return new getThriftServerType_resultStandardScheme(); + private static class getClusterId_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public getClusterId_resultStandardScheme getScheme() { + return new getClusterId_resultStandardScheme(); } } - private static class getThriftServerType_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + private static class getClusterId_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, getThriftServerType_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getClusterId_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -52042,8 +52809,8 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getThriftServerType } switch (schemeField.id) { case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.success = org.apache.hadoop.hbase.thrift2.generated.TThriftServerType.findByValue(iprot.readI32()); + 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); @@ -52060,13 +52827,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getThriftServerType struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, getThriftServerType_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getClusterId_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); if (struct.success != null) { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - oprot.writeI32(struct.success.getValue()); + oprot.writeString(struct.success); oprot.writeFieldEnd(); } oprot.writeFieldStop(); @@ -52075,16 +52842,16 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getThriftServerTyp } - private static class getThriftServerType_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { - public getThriftServerType_resultTupleScheme getScheme() { - return new getThriftServerType_resultTupleScheme(); + private static class getClusterId_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + public getClusterId_resultTupleScheme getScheme() { + return new getClusterId_resultTupleScheme(); } } - private static class getThriftServerType_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + private static class getClusterId_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getThriftServerType_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getClusterId_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()) { @@ -52092,16 +52859,16 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getThriftServerType } oprot.writeBitSet(optionals, 1); if (struct.isSetSuccess()) { - oprot.writeI32(struct.success.getValue()); + oprot.writeString(struct.success); } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getThriftServerType_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getClusterId_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 = org.apache.hadoop.hbase.thrift2.generated.TThriftServerType.findByValue(iprot.readI32()); + struct.success = iprot.readString(); struct.setSuccessIsSet(true); } } diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionInfo.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionInfo.java index 7fbea3c0b23e..4e2757cfe5a3 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionInfo.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionInfo.java @@ -7,7 +7,7 @@ package org.apache.hadoop.hbase.thrift2.generated; @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class THRegionInfo implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("THRegionInfo"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionLocation.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionLocation.java index f16e2d723ad0..82d30c28af44 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionLocation.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionLocation.java @@ -7,7 +7,7 @@ package org.apache.hadoop.hbase.thrift2.generated; @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class THRegionLocation implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("THRegionLocation"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIOError.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIOError.java index 38f5558c5b59..394ceb7d5294 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIOError.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIOError.java @@ -12,7 +12,7 @@ * to the HBase master or a HBase region server. Also used to return * more general HBase error conditions. */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TIOError extends org.apache.thrift.TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIOError"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIllegalArgument.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIllegalArgument.java index 14d9d3fe13e4..03a851afb6e1 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIllegalArgument.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIllegalArgument.java @@ -11,7 +11,7 @@ * A TIllegalArgument exception indicates an illegal or invalid * argument was passed into a procedure. */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TIllegalArgument extends org.apache.thrift.TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIllegalArgument"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIncrement.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIncrement.java index 64b13258a772..da42ac2bd8e5 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIncrement.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIncrement.java @@ -14,7 +14,7 @@ * by changing the durability. If you don't provide durability, it defaults to * column family's default setting for durability. */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TIncrement implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIncrement"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TKeepDeletedCells.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TKeepDeletedCells.java index aad2d249973d..3fe89c7b1559 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TKeepDeletedCells.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TKeepDeletedCells.java @@ -11,7 +11,7 @@ * Thrift wrapper around * org.apache.hadoop.hbase.KeepDeletedCells */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public enum TKeepDeletedCells implements org.apache.thrift.TEnum { /** * Deleted Cells are not retained. diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TMutation.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TMutation.java index 9b0b8d812aae..fe6bf737a4e5 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TMutation.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TMutation.java @@ -10,7 +10,7 @@ /** * Atomic mutation for the specified row. It can be either Put or Delete. */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TMutation extends org.apache.thrift.TUnion { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TMutation"); private static final org.apache.thrift.protocol.TField PUT_FIELD_DESC = new org.apache.thrift.protocol.TField("put", org.apache.thrift.protocol.TType.STRUCT, (short)1); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TNamespaceDescriptor.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TNamespaceDescriptor.java index 6c2f6349364b..16205bed8e35 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TNamespaceDescriptor.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TNamespaceDescriptor.java @@ -11,7 +11,7 @@ * Thrift wrapper around * org.apache.hadoop.hbase.NamespaceDescriptor */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TNamespaceDescriptor implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TNamespaceDescriptor"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TPut.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TPut.java index d3632a64b3a1..8d8086147718 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TPut.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TPut.java @@ -19,7 +19,7 @@ * by changing the durability. If you don't provide durability, it defaults to * column family's default setting for durability. */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TPut implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TPut"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TReadType.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TReadType.java index d1dd929bfe3c..e078f64f7bdb 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TReadType.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TReadType.java @@ -7,7 +7,7 @@ package org.apache.hadoop.hbase.thrift2.generated; -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public enum TReadType implements org.apache.thrift.TEnum { DEFAULT(1), STREAM(2), diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TResult.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TResult.java index bd402b2fcad7..38c1b740e799 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TResult.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TResult.java @@ -10,7 +10,7 @@ /** * if no Result is found, row and columnValues will not be set. */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TResult implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TResult"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TRowMutations.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TRowMutations.java index b16d5baa6a5c..87fc91eb12e0 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TRowMutations.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TRowMutations.java @@ -10,7 +10,7 @@ /** * A TRowMutations object is used to apply a number of Mutations to a single row. */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TRowMutations implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRowMutations"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TScan.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TScan.java index 5f07106e20d1..d1f016d83eea 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TScan.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TScan.java @@ -11,7 +11,7 @@ * Any timestamps in the columns are ignored but the colFamTimeRangeMap included, use timeRange to select by timestamp. * Max versions defaults to 1. */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TScan implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TScan"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TServerName.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TServerName.java index c4cc687a5a2e..0a5ba11066e8 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TServerName.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TServerName.java @@ -7,7 +7,7 @@ package org.apache.hadoop.hbase.thrift2.generated; @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TServerName implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TServerName"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableDescriptor.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableDescriptor.java index 0f8f916562cc..1f681b0d6216 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableDescriptor.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableDescriptor.java @@ -11,7 +11,7 @@ * Thrift wrapper around * org.apache.hadoop.hbase.client.TableDescriptor */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TTableDescriptor implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTableDescriptor"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableName.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableName.java index 55b4dc86a7cd..f6003ef3b7a9 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableName.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableName.java @@ -11,7 +11,7 @@ * Thrift wrapper around * org.apache.hadoop.hbase.TableName */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TTableName implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTableName"); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TThriftServerType.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TThriftServerType.java index 0f392c7c5c31..610d0e6dcccd 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TThriftServerType.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TThriftServerType.java @@ -10,7 +10,7 @@ /** * Specify type of thrift server: thrift and thrift2 */ -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public enum TThriftServerType implements org.apache.thrift.TEnum { ONE(1), TWO(2); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTimeRange.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTimeRange.java index 60631e4bf99c..9b69bf256069 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTimeRange.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTimeRange.java @@ -7,7 +7,7 @@ package org.apache.hadoop.hbase.thrift2.generated; @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-26") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-02-01") public class TTimeRange implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTimeRange"); diff --git a/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift/Hbase.thrift b/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift/Hbase.thrift index 91ed9dae0e48..b20cd11fe856 100644 --- a/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift/Hbase.thrift +++ b/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift/Hbase.thrift @@ -972,4 +972,9 @@ service Hbase { * @return the type of this thrift server */ TThriftServerType getThriftServerType() + + /** + * Returns the cluster ID for this cluster. + */ + string getClusterId() } diff --git a/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift b/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift index e2601bf81a4f..c4542f7d6efb 100644 --- a/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift +++ b/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift @@ -1053,4 +1053,9 @@ service THBaseService { * @return the type of this thrift server */ TThriftServerType getThriftServerType() + + /** + * Returns the cluster ID for this cluster. + */ + string getClusterId() } diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftConnection.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftConnection.java index 5015718abfff..57cb9d30cb92 100644 --- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftConnection.java +++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftConnection.java @@ -197,10 +197,19 @@ public static void shutdown() throws Exception { } @Test - public void testThrfitAdmin() throws Exception { - testThriftAdmin(thriftConnection, "testThrfitAdminNamesapce", "testThrfitAdminTable"); - testThriftAdmin(thriftHttpConnection, "testThrfitHttpAdminNamesapce", - "testThrfitHttpAdminTable"); + public void testGetClusterId() { + String actualClusterId = TEST_UTIL.getMiniHBaseCluster().getMaster().getClusterId(); + for (Connection conn: new Connection[] {thriftConnection, thriftHttpConnection}) { + String thriftClusterId = conn.getClusterId(); + assertEquals(actualClusterId, thriftClusterId); + } + } + + @Test + public void testThriftAdmin() throws Exception { + testThriftAdmin(thriftConnection, "testThriftAdminNamespace", "testThriftAdminTable"); + testThriftAdmin(thriftHttpConnection, "testThriftHttpAdminNamespace", + "testThriftHttpAdminTable"); } @Test @@ -210,7 +219,7 @@ public void testGet() throws Exception { } - public void testGet(Connection connection, String tableName) throws IOException { + private void testGet(Connection connection, String tableName) throws IOException { createTable(thriftAdmin, tableName); try (Table table = connection.getTable(TableName.valueOf(tableName))){ Get get = new Get(ROW_1);