diff --git common/src/java/org/apache/hadoop/hive/conf/HiveConf.java common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index 1af68a6..0c03194 100644 --- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -262,6 +262,8 @@ // Number of times to retry a Thrift metastore call upon failure METASTORETHRIFTFAILURERETRIES("hive.metastore.failure.retries", 1), + STREAMING_TMPDIR("hive.streaming.tempdir", "/user/hive/streaming/tmp"), + // Number of seconds the client should wait between connection attempts METASTORE_CLIENT_CONNECT_RETRY_DELAY("hive.metastore.client.connect.retry.delay", 1), // Socket timeout for the client connection (in seconds) diff --git hcatalog/webhcat/svr/pom.xml hcatalog/webhcat/svr/pom.xml index 16a870e..3479d2d 100644 --- hcatalog/webhcat/svr/pom.xml +++ hcatalog/webhcat/svr/pom.xml @@ -106,5 +106,11 @@ ${slf4j.version} compile + + org.apache.hive + hive-jdbc + ${hive.version} + test + diff --git hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/HcatStreamingDelegator.java hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/HcatStreamingDelegator.java new file mode 100644 index 0000000..7c4681d --- /dev/null +++ hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/HcatStreamingDelegator.java @@ -0,0 +1,377 @@ +/** + * 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.hive.hcatalog.templeton; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.Warehouse; +import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; +import org.apache.hadoop.hive.metastore.api.InvalidInputException; +import org.apache.hadoop.hive.metastore.api.InvalidObjectException; +import org.apache.hadoop.hive.metastore.api.InvalidOperationException; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.thrift.TException; +import org.eclipse.jetty.http.HttpStatus; + +import java.io.File; +import java.io.IOException; +import java.util.LinkedHashMap; +import java.util.Map; + + +import javax.ws.rs.core.Response; + + +/** + * Entry point for all hive streaming operations + * + * Concurrency Notes: + * Brief summary of why each operation should be 'concurrency safe': + * streamingStatus : Readonly metastore operation + * chunkGet : This is an atomic metastore operation + * chunkAbort : Just deletes a file. It is atomic. So no concurrency issues here. + * chunkCommit : Just renames a file. So only one of concurrent operations will succeed. + * disableStreaming : This is an atomic metastore operation + * enableStreaming : Does a couple of mkdirs (for setup) followed by an atomic metastore operation. + * mkdirs() is idempotent, so all concurrent calls succeed. All concurrent + * invocations enter a transaction to do the metastore update atomically...only + * one should update metastore. + * partitionRoll : Creates empty dir for the new current partition & then atomically updates + * metastore as follows: + * - Make note of this new current partition dir + * - Do an addPartition() on the previous current partition. + * If concurrent partitionRoll() invocations use same arguments, the addPartition() + * step will fail on all but one. If arguments are not same in concurrent + * invocations, they all succeed and updates made by the last invocation to exit + * the metastore transaction would override the others. + */ +public class HcatStreamingDelegator extends LauncherDelegator { + + public HcatStreamingDelegator(AppConfig appConf) { + super(appConf); + } + + /** + * Return a json containing the webhdfs link to the chunk file. + */ + public Response chunkGet(String user, String database, String table, String schema, + String format, String record_sperator, String field_separator) + throws IOException, MetaException, TException { + String streamingBaseUrl = appConf.get("hive.webhcat.streaming.partition.baseurl"); + HiveMetaStoreClient client = new HiveMetaStoreClient(new HiveConf()); + // TODO : checks + try { + long chunkId = client.getNextChunkID(database, table); + String streamingTmpDir = client.getStreamingTmpDir(database, table); // "19303940203"; + String chunkPath = getChunkPath(streamingTmpDir, chunkId); + return JsonBuilder.create() + .put("status", "OK") + .put("chunkid", chunkId) + .put("chunkfileurl", streamingBaseUrl + File.separator + chunkPath) + .put("chunkfile", chunkPath) + .build(); + } catch (InvalidInputException e) { + return JsonBuilder.create() + .put("status", "FAILED") + .put("message",e.getMessage()) + .buildResponse(HttpStatus.NOT_FOUND_404); + } catch (InvalidObjectException e) { + return JsonBuilder.create() + .put("status", "FAILED") + .put("message",e.getMessage()) + .buildResponse(HttpStatus.NOT_FOUND_404); + } catch (InvalidOperationException e) { + return JsonBuilder.create() + .put("status", "FAILED") + .put("message",e.getMessage()) + .buildResponse(HttpStatus.METHOD_NOT_ALLOWED_405); + } + } + + + private static String getChunkPath(String baseDir, long chunkId) { + return baseDir + File.separatorChar + chunkId; + } + + /** + * Deletes the chunkfile on HDFS. + */ + public Response chunkAbort(String user, String chunkfile) + throws IOException { + Path chunkFilePath = new Path(chunkfile); + FileSystem hdfs = FileSystem.get(new HiveConf()); + if( hdfs.delete(chunkFilePath, false) ) { + return JsonBuilder.create() + .put("status", "OK") + .put("chunkfile", chunkfile) + .build(); + } else { + // to avoid name node calls, additional checks for error reporting done if commit fails + if(!hdfs.exists(chunkFilePath)) { + return JsonBuilder.create() + .put("status", "FAILED") + .put("message", "Specified chunk does not exist " + chunkFilePath ) + .buildResponse(HttpStatus.NOT_FOUND_404); + } else { + return JsonBuilder.create() + .put("status", "FAILED") + .put("message", "Delete failed: " + chunkFilePath ) + .buildResponse(HttpStatus.INTERNAL_SERVER_ERROR_500); + } + } + + } + + + /** + * Move the chunk file to current warehouse. + */ + public Response chunkCommit(String user, String database, String table, String chunkfile) + throws IOException, MetaException, TException { + HiveConf hiveConf = new HiveConf(); + FileSystem hdfs = FileSystem.get(hiveConf); + HiveMetaStoreClient client = new HiveMetaStoreClient(hiveConf); + String currentStreamingPartition = client.getCurrentStreamingPartitionPath(database, table); + String chunkFileNameOnly = getFileName(chunkfile); + Path destinationFilePath = new Path(currentStreamingPartition + File.separator + + chunkFileNameOnly); + Path chunkFilePath = new Path(chunkfile); + if(hdfs.rename(chunkFilePath, destinationFilePath) ) { + return JsonBuilder.create() + .put("status", "OK") + .put("chunkfile", destinationFilePath.toString()) + .build(); + } else { + // to avoid name node calls, additional checks for error reporting done if commit fails + if(!hdfs.exists(chunkFilePath)) { + return JsonBuilder.create() + .put("status", "FAILED") + .put("message", "This chunk does not exist " + chunkFilePath.toString()) + .buildResponse(HttpStatus.NOT_FOUND_404); + } else { + return JsonBuilder.create() + .put("status", "FAILED") + .put("message", "Move failed: " + chunkFilePath.toString() + " to " + + destinationFilePath ) + .buildResponse(HttpStatus.INTERNAL_SERVER_ERROR_500); + } + } + } + + /** + * Given a full path, returns just the file name part + */ + private static String getFileName(String filePath) { + String[] pathComponents = filePath.split(File.separator); + return pathComponents[pathComponents.length - 1]; + } + + /** + * Close current streaming partition and open a new one atomically + * + * @return Status result on the commit of the previous streaming partition and a URL for the + * directory of the new streaming partition. The commit status contains three fields, an + * integer status code returned by the "alter table add partition" command, any message + * returned by that command, and the directory that was used for the old streaming + * partition. The message and old streaming partition are set only if the commit fails. + */ + public Response partitionRoll(String user, String database, String table, String part_col + , String part_val) throws IOException, MetaException, TException { + + HiveConf hiveConf = new HiveConf(); + HiveMetaStoreClient client = new HiveMetaStoreClient(hiveConf); + + // 1- Create a directory for the new streaming partition. + Path newPartitionPath = getStreamingPartitionPath(database, table, part_col, part_val + , hiveConf, client); + FileSystem fs = FileSystem.get(hiveConf); + boolean created = false; + try { + if(fs.exists(newPartitionPath)) { + throw new AlreadyExistsException("Partition directory already exists: " + newPartitionPath); + } + created = fs.mkdirs(newPartitionPath); + + // 2- Atomically, set new streaming partition & commit previous streaming partition + String oldPartitionPathStr = client.rollStreamingPartition(database, table + , newPartitionPath.toString(), part_val); + + // 3- Return the new & old streaming partition info + return JsonBuilder.create() + .put("status", "OK") + .put("old_streaming_partition", oldPartitionPathStr) + .put("new_streaming_partition", newPartitionPath.toString()) + .build(); + } catch (InvalidInputException e) { + if(created) { + fs.delete(newPartitionPath,true); + } + return JsonBuilder.create() + .put("status", "FAILED") + .put("message",e.getMessage()) + .buildResponse(HttpStatus.NOT_FOUND_404); + } catch (InvalidObjectException e) { + if(created) { + fs.delete(newPartitionPath,true); + } + return JsonBuilder.create() + .put("status", "FAILED") + .put("message",e.getMessage()) + .buildResponse(HttpStatus.NOT_FOUND_404); + } catch (InvalidOperationException e) { + if(created) { + fs.delete(newPartitionPath,true); + } + return JsonBuilder.create() + .put("status", "FAILED") + .put("message",e.getMessage()) + .buildResponse(HttpStatus.METHOD_NOT_ALLOWED_405); + } catch (AlreadyExistsException e) { + if(created) { + fs.delete(newPartitionPath,true); + } + return JsonBuilder.create() + .put("status", "FAILED") + .put("message",e.getMessage()) + .buildResponse(HttpStatus.BAD_REQUEST_400); + } + } + + // get Location where the committed chunks for a streaming partition would go + private Path getStreamingPartitionPath(String database, String table + , String part_col, String part_val + , HiveConf hiveConf, HiveMetaStoreClient client) throws TException { + Table tableObj = client.getTable(database,table); + Path tablePath = new Path( tableObj.getSd().getLocation() ); + LinkedHashMap partition = new LinkedHashMap(); + partition.put(part_col,part_val); + return new Warehouse(hiveConf).getPartitionPath(tablePath, partition); + } + + public Response enableStreaming(String database, String table, String partitionCol + , String partitionVal) throws IOException, TException { + HiveConf hiveConf = new HiveConf(); + HiveMetaStoreClient client = new HiveMetaStoreClient(hiveConf); + + try { + // 1 Create directories for the streaming partition's committed files + Path partitionPath = getStreamingPartitionPath(database, table, partitionCol + , partitionVal, hiveConf ,client); + FileSystem fs = FileSystem.get(hiveConf); + if (!fs.exists(partitionPath)) + fs.mkdirs(partitionPath); + + // 2 Create directories for the streaming partition's uncommitted files + Path streamingTmpTblPath = getStreamingTempPath(database, table, hiveConf); + + if (!fs.exists(streamingTmpTblPath)) + fs.mkdirs(streamingTmpTblPath); + // 3 Update the metastore + client.enableStreaming(database, table, streamingTmpTblPath.toString() + , partitionPath.toString(), partitionVal); + return JsonBuilder.create() + .put("status", "OK") + .build(); + } catch (InvalidInputException e) { + return JsonBuilder.create() + .put("status", "FAILED") + .put("message",e.getMessage()) + .buildResponse(HttpStatus.NOT_FOUND_404); + } catch (InvalidObjectException e) { + return JsonBuilder.create() + .put("status", "FAILED") + .put("message",e.getMessage()) + .buildResponse(HttpStatus.NOT_FOUND_404); + } catch (NoSuchObjectException e) { + return JsonBuilder.create() + .put("status", "FAILED") + .put("message",e.getMessage()) + .buildResponse(HttpStatus.NOT_FOUND_404); + } catch (InvalidOperationException e) { + return JsonBuilder.create() + .put("status", "FAILED") + .put("message",e.getMessage()) + .buildResponse(HttpStatus.METHOD_NOT_ALLOWED_405); + } + } + + private Path getStreamingTempPath(String database, String table, HiveConf hiveConf) { + String streaminTmpBase = hiveConf.get(HiveConf.ConfVars.STREAMING_TMPDIR.toString()); + Path streamingTmpDbPath = new Path(streaminTmpBase, database.toLowerCase()); + return new Path(streamingTmpDbPath,table.toLowerCase()); + } + + public Response disableStreaming(String database, String table) throws IOException + , MetaException, TException { + HiveConf hiveConf = new HiveConf(); + HiveMetaStoreClient client = new HiveMetaStoreClient(hiveConf); + try { + // 1 Update the metastore + client.disableStreaming(database, table); + return JsonBuilder.create() + .put("status", "OK") + .build(); + } catch (InvalidInputException e) { + return JsonBuilder.create() + .put("status", "FAILED") + .put("message",e.getMessage()) + .buildResponse(HttpStatus.NOT_FOUND_404); + } catch (InvalidObjectException e) { + return JsonBuilder.create() + .put("status", "FAILED") + .put("message",e.getMessage()) + .buildResponse(HttpStatus.NOT_FOUND_404); + } catch (InvalidOperationException e) { + return JsonBuilder.create() + .put("status", "FAILED") + .put("message",e.getMessage()) + .buildResponse(HttpStatus.METHOD_NOT_ALLOWED_405); + } + } + + public Response streamingStatus(String database, String table) + throws IOException, MetaException, TException { + HiveMetaStoreClient client = new HiveMetaStoreClient(new HiveConf()); + try { + Map status = client.streamingStatus(database, table); + return JsonBuilder.create() + .put("streaming_enabled", status.get("streaming_enabled")) + .put("tempDir", status.get("tempDir")) + .put("location", status.get("location")) + .put("value", status.get("value")) + .put("chunkid", status.get("chunkid")) + .build(); + } catch (InvalidInputException e) { + return JsonBuilder.create() + .put("status", "FAILED") + .put("message",e.getMessage()) + .buildResponse(HttpStatus.NOT_FOUND_404); + } catch (InvalidObjectException e) { + return JsonBuilder.create() + .put("status", "FAILED") + .put("message",e.getMessage()) + .buildResponse(HttpStatus.NOT_FOUND_404); + } + } + +} diff --git hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/Main.java hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/Main.java index 0f37278..89fd726 100644 --- hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/Main.java +++ hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/Main.java @@ -180,6 +180,8 @@ public Server runServer(int port) FilterMapping.REQUEST); root.addFilter(fHolder, "/" + SERVLET_PATH + "/v1/mapreduce/*", FilterMapping.REQUEST); + root.addFilter(fHolder, "/" + SERVLET_PATH + "/v1/streaming/*", + FilterMapping.REQUEST); root.addFilter(fHolder, "/" + SERVLET_PATH + "/v1/status/*", FilterMapping.REQUEST); root.addFilter(fHolder, "/" + SERVLET_PATH + "/v1/version/*", diff --git hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/Server.java hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/Server.java index d8b6ce2..de99a72 100644 --- hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/Server.java +++ hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/Server.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.net.InetAddress; import java.net.UnknownHostException; +import java.security.PrivilegedExceptionAction; import java.text.MessageFormat; import java.util.ArrayList; import java.util.Collections; @@ -51,6 +52,8 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authentication.client.PseudoAuthenticator; import org.apache.hive.hcatalog.templeton.tool.TempletonUtils; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.thrift.TException; /** * The Templeton Web API server. @@ -815,6 +818,199 @@ public CompleteBean completeJob(@PathParam("jobid") String jobid) return d.run(jobid); } + /** + * Sets up a streaming partition for a table + */ + @GET + @Path("streaming/enable") + @Produces({MediaType.APPLICATION_JSON}) + public Response streamingEnable( @QueryParam("database") String database, + @QueryParam("table") final String table, + @QueryParam("col") final String partitionCol, + @QueryParam("value") final String partitionVal) + throws NotAuthorizedException, BadParam, TException, IOException, InterruptedException { + verifyUser(); + verifyParam(table, "table"); + final String db = (database==null) ? "default" : database; + UserGroupInformation ugi = UgiFactory.getUgi(getDoAsUser()); + return ugi.doAs( + new PrivilegedExceptionAction() { + @Override + public Response run() throws Exception { + HcatStreamingDelegator streamingDelegator = new HcatStreamingDelegator(appConf); + return streamingDelegator.enableStreaming(db, table, partitionCol, partitionVal); + } + } + ); + } + + /** + * Sets up a streaming partition for a table + */ + @GET + @Path("streaming/disable") + @Produces({MediaType.APPLICATION_JSON}) + public Response streamingDisable( @QueryParam("database") final String database, + @QueryParam("table") final String table) + throws NotAuthorizedException, BadParam, IOException, TException, MetaException, InterruptedException { + verifyUser(); + verifyParam(table, "table"); + final String db = (database==null) ? "default" : database; + + UserGroupInformation ugi = UgiFactory.getUgi(getDoAsUser()); + return ugi.doAs( + new PrivilegedExceptionAction() { + @Override + public Response run() throws Exception { + HcatStreamingDelegator streamingDelegator = new HcatStreamingDelegator(appConf); + return streamingDelegator.disableStreaming(db, table); + } + } + ); + + } + + /** + * Closes the current streaming partition and opens a new one + */ + @GET + @Path("streaming/status") + @Produces({MediaType.APPLICATION_JSON}) + public Response streamingStatus( @QueryParam("database") final String database, + @QueryParam("table") final String table) + throws NotAuthorizedException, BadParam, IOException, TException, MetaException, InterruptedException { + verifyUser(); + + verifyParam(table, "table"); + final String db = (database==null) ? "default" : database; + + UserGroupInformation ugi = UgiFactory.getUgi(getDoAsUser()); + return ugi.doAs( + new PrivilegedExceptionAction() { + @Override + public Response run() throws Exception { + HcatStreamingDelegator streamingDelegator = new HcatStreamingDelegator(appConf); + return streamingDelegator.streamingStatus(db, table); + } + } + ); + + } + + /** + * Returns a file name to which a chunk of recrods can be written to by streaming client + */ + @GET + @Path("streaming/chunkget") + @Produces({MediaType.APPLICATION_JSON}) + public Response streamingGetChunk(@QueryParam("database") String database, + @QueryParam("table") final String table, + @QueryParam("schema") final String schema, + @QueryParam("format") final String format, + @QueryParam("record_separator") final String record_sperator, + @QueryParam("field_separator") final String field_separator) + throws NotAuthorizedException, BusyException, BadParam, IOException, TException, InterruptedException { + verifyUser(); + verifyParam(table, "table"); + + verifyParam(schema, "schema"); + verifyParam(format, "format"); + verifyParam(record_sperator, "record_separator"); + verifyParam(field_separator, "field_separator"); + final String db = (database==null) ? "default" : database; + + UserGroupInformation ugi = UgiFactory.getUgi(getDoAsUser()); + return ugi.doAs( + new PrivilegedExceptionAction() { + @Override + public Response run() throws Exception { + HcatStreamingDelegator streamingDelegator = new HcatStreamingDelegator(appConf); + return streamingDelegator.chunkGet(getRequestingUser(), db, table, schema, format, record_sperator, field_separator); + } + } + ); + } + + /** + * Commits the chunk to the currently open streaming partition + */ + @GET + @Path("streaming/chunkcommit") + @Produces({MediaType.APPLICATION_JSON}) + public Response streamingChunkCommit( @QueryParam("database") String database, + @QueryParam("table") final String table, + @QueryParam("chunkfile") final String chunkfile) + throws NotAuthorizedException, BadParam, IOException, TException, InterruptedException { + verifyUser(); + verifyParam(table, "table"); + verifyParam(chunkfile, "chunkfile"); + final String db = (database==null) ? "default" : database; + + UserGroupInformation ugi = UgiFactory.getUgi(getDoAsUser()); + return ugi.doAs( + new PrivilegedExceptionAction() { + @Override + public Response run() throws Exception { + HcatStreamingDelegator streamingDelegator = new HcatStreamingDelegator(appConf); + return streamingDelegator.chunkCommit(getRequestingUser(), db, table, chunkfile); + } + } + ); + + } + + /** + * Commits the chunk to the currently open streaming partition + */ + @GET + @Path("streaming/chunkabort") + @Produces({MediaType.APPLICATION_JSON}) + public Response streamingChunkAbort( @QueryParam("chunkfile") final String chunkfile) + throws NotAuthorizedException, BadParam, IOException, InterruptedException { + verifyUser(); + verifyParam(chunkfile, "chunkfile"); + UserGroupInformation ugi = UgiFactory.getUgi(getDoAsUser()); + return ugi.doAs( + new PrivilegedExceptionAction() { + @Override + public Response run() throws Exception { + HcatStreamingDelegator streamingDelegator = new HcatStreamingDelegator(appConf); + return streamingDelegator.chunkAbort(getRequestingUser(), chunkfile); + } + } + ); + } + + /** + * Closes the current streaming partition and opens a new one + */ + @GET + @Path("streaming/partitionroll") + @Produces({MediaType.APPLICATION_JSON}) + public Response streamingPartitionRoll( @QueryParam("database") final String database, + @QueryParam("table") final String table, + @QueryParam("partition_column") final String partCol, + @QueryParam("partition_value") final String partVal) + throws NotAuthorizedException, BadParam, IOException, TException, MetaException, InterruptedException { + verifyUser(); + verifyParam(table, "table"); + verifyParam(partCol, "part_column"); + verifyParam(partVal, "part_value"); + final String db = (database==null) ? "default" : database; + + UserGroupInformation ugi = UgiFactory.getUgi(getDoAsUser()); + return ugi.doAs( + new PrivilegedExceptionAction() { + @Override + public Response run() throws Exception { + HcatStreamingDelegator streamingDelegator = new HcatStreamingDelegator(appConf); + return streamingDelegator.partitionRoll(getRequestingUser(), db, table, partCol, partVal); + } + } + ); + + } + /** * Verify that we have a valid user. Throw an exception if invalid. */ diff --git hcatalog/webhcat/svr/src/test/java/org/apache/hive/hcatalog/templeton/TestStreaming.java hcatalog/webhcat/svr/src/test/java/org/apache/hive/hcatalog/templeton/TestStreaming.java new file mode 100644 index 0000000..679268d --- /dev/null +++ hcatalog/webhcat/svr/src/test/java/org/apache/hive/hcatalog/templeton/TestStreaming.java @@ -0,0 +1,610 @@ +/** + * 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.hive.hcatalog.templeton; + +import junit.framework.Assert; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hive.hcatalog.templeton.tool.HiveJdbcClient; +import org.apache.hive.hcatalog.templeton.tool.WebHCatClient; +import org.apache.log4j.Level; +import org.apache.log4j.LogManager; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.eclipse.jetty.http.HttpStatus; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Map; + +/** + * A set of tests exercising streaming WebHCat APIs. + */ +public class TestStreaming { + private static final Logger LOG = + LoggerFactory.getLogger(TestStreaming.class); + private static final String templetonBaseUrl = + "http://localhost:50111/templeton/v1/streaming"; + private static final String username= "johndoe"; + private static final String ERROR_CODE = "errorCode"; + private static Main templetonServer; + private static final String charSet = "UTF-8"; + private static HiveJdbcClient hclient = null; + Configuration conf = null; + FileSystem hdfs = null; + Path commitPath = null; + Path tmpPath = null; + + public TestStreaming() throws SQLException, ClassNotFoundException, IOException { + LogManager.getRootLogger().setLevel(Level.WARN); + conf = new HiveConf(); + hdfs = FileSystem.get(conf); + tmpPath = new Path( conf.get(HiveConf.ConfVars.STREAMING_TMPDIR.toString()) + + "/testing/weblogs" ); + } + + @BeforeClass + public static void beforeClass() throws SQLException, ClassNotFoundException { + templetonServer = new Main(new String[] {"-D" + AppConfig.UNIT_TEST_MODE + "=true"}); + templetonServer.run(); + LOG.info("WebHcat started"); + + hclient = new HiveJdbcClient(); + hclient.exec("drop table if exists testing.weblogs"); + hclient.exec("drop database if exists testing"); + } + + @AfterClass + public static void afterClass() throws SQLException { + hclient.close(); + if(templetonServer != null) { + LOG.info("Stopping WebHcat"); + templetonServer.stop(); + LOG.info("WebHcat stopped"); + } + } + + @Before + public void beforeTest() throws SQLException, IOException { + dropDatabase(); + hclient.exec("create database testing"); + hclient.exec("use testing"); + hclient.exec("create table weblogs (log String) PARTITIONED BY (date String)"); + } + + private void dropDatabase() throws SQLException, IOException { + hclient.exec("ALTER TABLE weblogs DROP IF EXISTS PARTITION (date='1000')"); + hclient.exec("ALTER TABLE weblogs DROP IF EXISTS PARTITION (date='2000')"); + hclient.exec("ALTER TABLE weblogs DROP IF EXISTS PARTITION (date='3000')"); + hclient.exec("drop table if exists testing.weblogs"); + hclient.exec("drop database if exists testing"); + if(commitPath!=null) hdfs.delete(commitPath, true); + hdfs.delete(tmpPath, true); + } + + @Test + public void testGetStatus() throws IOException, SQLException, ClassNotFoundException { + WebHCatClient.Response r = WebHCatClient.doGet(templetonBaseUrl + + "/status?user.name=jdoe&database=testing&table=weblogs"); + Assert.assertEquals(HttpStatus.OK_200, r.status); + Assert.assertTrue(r.body.contains("\"streaming_enabled\":\"false\"")); + Assert.assertTrue(r.body.contains("\"chunkid\":\"null\"")); + + // Try once more. ChunkId should remain unchanged + r = WebHCatClient.doGet(templetonBaseUrl + + "/status?user.name=jdoe&database=testing&table=weblogs"); + Assert.assertEquals(HttpStatus.OK_200, r.status); + Assert.assertTrue(r.body.contains("\"streaming_enabled\":\"false\"")); + Assert.assertTrue(r.body.contains("\"chunkid\":\"null\"")); + } + + @Test + public void testEnableDisable() throws IOException, SQLException, ClassNotFoundException { + // check disabled status + WebHCatClient.Response r = WebHCatClient.doGet(templetonBaseUrl + + "/status?user.name=jdoe&database=testing&table=weblogs"); + Assert.assertEquals(HttpStatus.OK_200, r.status); + Assert.assertTrue(r.body.contains("\"streaming_enabled\":\"false\"")); + Assert.assertTrue(r.body.contains("\"chunkid\":\"null\"")); + + // enable streaming + r = WebHCatClient.doGet(templetonBaseUrl + + "/enable?user.name=jdoe&database=testing&table=weblogs&col=date&value=1000"); + Assert.assertTrue(r.body.contains("\"status\":\"OK\"")); + + // check enabled status + r = WebHCatClient.doGet(templetonBaseUrl + + "/status?user.name=jdoe&database=testing&table=weblogs"); + Assert.assertEquals(HttpStatus.OK_200, r.status); + Assert.assertTrue(r.body.contains("\"streaming_enabled\":\"true\"")); + Assert.assertTrue(r.body.contains("\"chunkid\":\"0\"")); + + // disable streaming + r = WebHCatClient.doGet(templetonBaseUrl + + "/disable?user.name=jdoe&database=testing&table=weblogs"); + Assert.assertTrue(r.body.contains("\"status\":\"OK\"")); + + // check enabled status + r = WebHCatClient.doGet(templetonBaseUrl + + "/status?user.name=jdoe&database=testing&table=weblogs"); + Assert.assertEquals(HttpStatus.OK_200, r.status); + Assert.assertTrue(r.body.contains("\"streaming_enabled\":\"false\"")); + Assert.assertTrue(r.body.contains("\"chunkid\":\"0\"")); + + // enable streaming on bogus table + r = WebHCatClient.doGet(templetonBaseUrl + + "/enable?user.name=jdoe&database=testing&table=bogus&col=date&value=1000"); + Assert.assertEquals(HttpStatus.NOT_FOUND_404, r.status); + } + + @Test + /** + * Tests the abort chunk functionality + */ + public void testAbortChunks() throws IOException, SQLException, ClassNotFoundException { + String chunkDir = conf.get("hive.streaming.tempdir") + "/testing/weblogs"; + Path chunkFile1 = new Path(chunkDir + "/1"); + String chunkCommitDir = conf.get("hive.streaming.basedir") + "/testing/weblogs/date=1000"; + Path chunkCommitFile1 = new Path(chunkCommitDir + "/1"); + + hdfs.delete(chunkFile1,false); + hdfs.delete(chunkCommitFile1,false); + + + // check disabled status + WebHCatClient.Response r = WebHCatClient.doGet(templetonBaseUrl + + "/status?user.name=jdoe&database=testing&table=weblogs"); + Assert.assertEquals(HttpStatus.OK_200, r.status); + Assert.assertTrue(r.body.contains("\"streaming_enabled\":\"false\"")); + Assert.assertTrue(r.body.contains("\"chunkid\":\"null\"")); + + // 1 - enable streaming + r = WebHCatClient.doGet(templetonBaseUrl + + "/enable?user.name=jdoe&database=testing&table=weblogs&col=date&value=1000"); + Assert.assertTrue(r.body.contains("\"status\":\"OK\"")); + + // 2 - check enabled status + r = WebHCatClient.doGet(templetonBaseUrl + + "/status?user.name=jdoe&database=testing&table=weblogs"); + + Assert.assertEquals(HttpStatus.OK_200, r.status); + Assert.assertTrue(r.body.contains("\"streaming_enabled\":\"true\"")); + Assert.assertTrue(r.body.contains("\"chunkid\":\"0\"")); + + // 3 - get chunk + r = WebHCatClient.doGet(templetonBaseUrl + + "/chunkget?user.name=jdoe&database=testing&table=weblogs&schema=blah&format=blah" + + "&record_separator=blah&field_separator=blah"); + + Map response = JsonBuilder.jsonToMap(r.body); + Assert.assertEquals(response.get("status"),"OK"); + Assert.assertEquals(response.get("chunkid"),1); + Assert.assertEquals(response.get("chunkfile"),chunkFile1.toString()); + + // 4 - write to chunk + FSDataOutputStream output = hdfs.create(chunkFile1,true); // enable overwriting file + InputStream input = + new ByteArrayInputStream(("What time is it when an elephant sits on your fence? " + + "A: Time for a new fence.").getBytes()); + IOUtils.copyBytes(input, output, conf); + output.close(); + + // 5 - abort chunk + r = WebHCatClient.doGet(templetonBaseUrl + + "/chunkabort?user.name=jdoe&database=testing&table=weblogs&chunkfile=" + + chunkFile1); + response = JsonBuilder.jsonToMap(r.body); + Assert.assertEquals(response.get("status"), "OK"); + Assert.assertEquals(response.get("chunkfile"),chunkFile1.toString()); + + // 6 - verify that file has been deleted from tmp directory + Assert.assertFalse("Aborted chunk file should not found in commit location" + , hdfs.exists(chunkCommitFile1)); + Assert.assertFalse("Aborted chink file should be removed from tmp location on abort" + , hdfs.exists(chunkFile1) ); + } + + @Test + /** + * Tests the commit chunk functionality + */ + public void testCommitChunks() throws IOException, SQLException, ClassNotFoundException { + String chunkDir = conf.get(HiveConf.ConfVars.STREAMING_TMPDIR.toString()) + "/testing/weblogs"; + Path chunkFile1 = new Path(chunkDir, "1"); + Path chunkFile2 = new Path(chunkDir, "2"); + String chunkCommitDir = "file:/user/hive/warehouse/testing.db/weblogs/date=1000"; + Path chunkCommitFile1 = new Path(chunkCommitDir, "1"); + Path chunkCommitFile2 = new Path(chunkCommitDir, "2"); + + hdfs.delete(chunkFile1,false); + hdfs.delete(chunkFile2,false); + hdfs.delete(chunkCommitFile1,false); + hdfs.delete(chunkCommitFile2,false); + + + // check disabled status + WebHCatClient.Response r = WebHCatClient.doGet(templetonBaseUrl + + "/status?user.name=jdoe&database=testing&table=weblogs"); + Assert.assertEquals(HttpStatus.OK_200, r.status); + Assert.assertTrue(r.body.contains("\"streaming_enabled\":\"false\"")); + Assert.assertTrue(r.body.contains("\"chunkid\":\"null\"")); + + // 1 - enable streaming + r = WebHCatClient.doGet(templetonBaseUrl + + "/enable?user.name=jdoe&database=testing&table=weblogs&col=date&value=1000"); + Assert.assertTrue(r.body.contains("\"status\":\"OK\"")); + + // 2 - check enabled status + r = WebHCatClient.doGet(templetonBaseUrl + + "/status?user.name=jdoe&database=testing&table=weblogs"); + + Assert.assertEquals(HttpStatus.OK_200, r.status); + Assert.assertTrue(r.body.contains("\"streaming_enabled\":\"true\"")); + Assert.assertTrue(r.body.contains("\"chunkid\":\"0\"")); + + // 3 - get chunk + r = WebHCatClient.doGet(templetonBaseUrl + + "/chunkget?user.name=jdoe&database=testing&table=weblogs&schema=blah&format=blah" + + "&record_separator=blah&field_separator=blah"); + + Map response = JsonBuilder.jsonToMap(r.body); + Assert.assertEquals(response.get("status"),"OK"); + Assert.assertEquals(response.get("chunkid"),1); + Assert.assertEquals(response.get("chunkfile"),chunkFile1.toString()); + + // 4 - write to chunk + FSDataOutputStream output = hdfs.create(chunkFile1,true); // enable overwriting file + InputStream input = new ByteArrayInputStream(("How do you get four elephants into a Mini?" + + "A: Two in the front, two in the back.").getBytes()); + IOUtils.copyBytes(input, output, conf); + output.close(); + + // 5 - commit chunk + r = WebHCatClient.doGet(templetonBaseUrl + + "/chunkcommit?user.name=jdoe&database=testing&table=weblogs&chunkfile=" + + chunkFile1); + + response = JsonBuilder.jsonToMap(r.body); + Assert.assertEquals("OK", response.get("status")); + Assert.assertEquals(chunkCommitFile1.toString(), response.get("chunkfile")); + + // 6 - verify that file has been moved to commit directory + Assert.assertTrue( "Committed chunk file not found in commit location" + , hdfs.exists(chunkCommitFile1) ); + Assert.assertFalse("Chunk file was not removed from tmp location on commit" + , hdfs.exists(chunkFile1)); + + // 7 - TODO: once HIVE-5142 is completed, query & verify the committed chunk is in result set + + // 8 - verify bogus commits do not work + String bogusChunk = chunkDir + "/bogus"; + String bogusChunkCommitted = chunkCommitDir + "/bogus"; + r = WebHCatClient.doGet(templetonBaseUrl + + "/chunkcommit?user.name=jdoe&database=testing&table=weblogs&chunkfile="+ bogusChunk); + + Assert.assertFalse("Bogus chunk file should not be found in commit location." + , hdfs.exists(new Path(bogusChunk)) ); + Assert.assertFalse("Bogus chunk file should not be found in tmp location." + , hdfs.exists(new Path(bogusChunkCommitted)) ); + + // 9 - try another chunkget + r = WebHCatClient.doGet(templetonBaseUrl + + "/chunkget?user.name=jdoe&database=testing&table=weblogs&schema=blah&format=blah" + + "&record_separator=blah&field_separator=blah"); + + response = JsonBuilder.jsonToMap(r.body); + Assert.assertEquals("OK", response.get("status")); + Assert.assertEquals(2,response.get("chunkid")); + Assert.assertEquals(chunkFile2.toString(), response.get("chunkfile")); + + // 10 - write + output = hdfs.create(chunkFile2,true); // enable overwriting file + input = new ByteArrayInputStream(("How do you know there are two elephants in your fridge? " + + "A: The door won't close.").getBytes()); + IOUtils.copyBytes(input, output, conf); + output.close(); + + // 11 - commit + r = WebHCatClient.doGet(templetonBaseUrl + + "/chunkcommit?user.name=jdoe&database=testing&table=weblogs&chunkfile=" + + chunkFile2); + response = JsonBuilder.jsonToMap(r.body); + Assert.assertEquals("OK", response.get("status")); + Assert.assertEquals(chunkCommitFile2.toString(), response.get("chunkfile")); + + // 12 - verify that file has been moved to commit directory + Assert.assertTrue( "Committed chunk file not found in commit location" + , hdfs.exists(chunkCommitFile2) ); + Assert.assertFalse("Chunk file was not removed from tmp location on commit" + , hdfs.exists(chunkFile2)); + } + + @Test + /** + * Tests the roll partition functionality + */ + public void testRollPartition() throws IOException, SQLException, ClassNotFoundException { + String chunkDir = conf.get("hive.streaming.tempdir") + "/testing/weblogs"; + Path chunkFile1 = new Path(chunkDir, "1"); + Path chunkFile2 = new Path(chunkDir, "2"); + String chunkCommitDir1 = "file:/user/hive/warehouse/testing.db/weblogs/date=1000"; + Path chunkCommitFile1 = new Path(chunkCommitDir1, "1"); + String chunkCommitDir2 = "file:/user/hive/warehouse/testing.db/weblogs/date=2000"; + Path chunkCommitFile2 = new Path(chunkCommitDir2, "2"); + + + hdfs.delete(chunkFile1,false); + hdfs.delete(chunkFile2,false); + hdfs.delete(chunkCommitFile1, false); + hdfs.delete(chunkCommitFile2, false); + + // check disabled status + WebHCatClient.Response r = WebHCatClient.doGet(templetonBaseUrl + + "/status?user.name=jdoe&database=testing&table=weblogs"); + Assert.assertEquals(HttpStatus.OK_200, r.status); + Assert.assertTrue(r.body.contains("\"streaming_enabled\":\"false\"")); + Assert.assertTrue(r.body.contains("\"chunkid\":\"null\"")); + + // 1 - enable streaming + r = WebHCatClient.doGet(templetonBaseUrl + + "/enable?user.name=jdoe&database=testing&table=weblogs&col=date&value=1000"); + Assert.assertTrue(r.body.contains("\"status\":\"OK\"")); + + // 2 - check enabled status + r = WebHCatClient.doGet(templetonBaseUrl + + "/status?user.name=jdoe&database=testing&table=weblogs"); + + Assert.assertEquals(HttpStatus.OK_200, r.status); + Assert.assertTrue(r.body.contains("\"streaming_enabled\":\"true\"")); + Assert.assertTrue(r.body.contains("\"chunkid\":\"0\"")); + + // 3 - get chunk + r = WebHCatClient.doGet(templetonBaseUrl + + "/chunkget?user.name=jdoe&database=testing&table=weblogs&schema=blah&format=blah" + + "&record_separator=blah&field_separator=blah"); + + Map response = JsonBuilder.jsonToMap(r.body); + Assert.assertEquals(response.get("status"),"OK"); + Assert.assertEquals(response.get("chunkid"),1); + Assert.assertEquals(response.get("chunkfile"),chunkFile1.toString()); + + // 4 - write two records (i.e. lines) to chunk + FSDataOutputStream output = hdfs.create(chunkFile1,true); // enable overwriting file + String line1 = "What is the difference between en elephant and a plum?"; + String line2 = "A: An elephant is grey."; + InputStream input = new ByteArrayInputStream((line1 + "\n" + line2).getBytes()); + IOUtils.copyBytes(input, output, conf); + output.close(); + + // 5 - commit chunk + r = WebHCatClient.doGet(templetonBaseUrl + + "/chunkcommit?user.name=jdoe&database=testing&table=weblogs&chunkfile=" + + chunkFile1); + + response = JsonBuilder.jsonToMap(r.body); + Assert.assertEquals("OK",response.get("status")); + Assert.assertEquals(chunkCommitFile1.toString(),response.get("chunkfile")); + + // 6 - verify that file has been moved to commit directory + Assert.assertTrue("Committed chunk file not found in commit location" + , hdfs.exists(chunkCommitFile1)); + Assert.assertFalse("Chunk file was not removed from tmp location on commit" + , hdfs.exists(chunkFile1)); + + // 7 - roll partition + r = WebHCatClient.doGet(templetonBaseUrl + + "/partitionroll?user.name=jdoe&" + + "database=testing&table=weblogs&partition_column=date&partition_value=2000"); + response = JsonBuilder.jsonToMap(r.body); + Assert.assertEquals(response.get("status"),"OK"); + Assert.assertEquals(chunkCommitDir1 + , response.get("old_streaming_partition")); + Assert.assertEquals(chunkCommitDir2 + ,response.get("new_streaming_partition")); + + // 8 check table to see if the partition was added + hclient.exec("use testing"); + ArrayList> parts = hclient.execQuery("SHOW PARTITIONS weblogs"); + Assert.assertEquals("Expecting only 1 partition", parts.size(),1); + Assert.assertEquals("Partition name should be date=1000", parts.get(0).get(0),"date=1000"); + + // 9 - Query it to verify data + ArrayList> data = hclient.execQuery("select * from weblogs"); + Assert.assertEquals("Expecting 2 records", 2, data.size()); + Assert.assertEquals("1st record has wrong data: ",line1, data.get(0).get(0)); + Assert.assertEquals("2nd record has wrong data:",line2, data.get(1).get(0)); + + ///----- Add another partition and verify --- + + // 10 - get chunk + r = WebHCatClient.doGet(templetonBaseUrl + + "/chunkget?user.name=jdoe&database=testing&table=weblogs&" + + "schema=blah&format=blah&record_separator=blah&field_separator=blah"); + + response = JsonBuilder.jsonToMap(r.body); + Assert.assertEquals(response.get("status"),"OK"); + Assert.assertEquals(response.get("chunkid"),2); + Assert.assertEquals(response.get("chunkfile"), chunkFile2.toString()); + + // 11 - write two records (i.e. lines) to chunk + output = hdfs.create(chunkFile2,true); // enable overwriting file + String line3 = "What is as big as an elephant but weighs nothing ?"; + String line4 = "A: Elephant's shadow."; + input = new ByteArrayInputStream((line3 + "\n" + line4).getBytes()); + IOUtils.copyBytes(input, output, conf); + output.close(); + + // 12 - commit chunk + r = WebHCatClient.doGet(templetonBaseUrl + + "/chunkcommit?user.name=jdoe&database=testing&table=weblogs&chunkfile=" + + chunkFile2); + + response = JsonBuilder.jsonToMap(r.body); + Assert.assertEquals(response.get("status"),"OK"); + Assert.assertEquals(response.get("chunkfile"),chunkCommitFile2.toString()); + + // 13 - verify that file has been moved to commit directory + Assert.assertTrue( "Committed chunk file not found in commit location" + , hdfs.exists(chunkCommitFile2) ); + Assert.assertFalse("Chunk file was not removed from tmp location on commit" + , hdfs.exists(chunkFile2)); + + // 14 - roll partition + r = WebHCatClient.doGet(templetonBaseUrl + + "/partitionroll?user.name=jdoe&database=testing&table=weblogs&" + + "partition_column=date&partition_value=3000"); + + // 15 check table to see if the partition was added + parts = hclient.execQuery("SHOW PARTITIONS weblogs"); + Assert.assertEquals("Expecting 2 partitions", 2, parts.size()); + Assert.assertEquals("Partition name should be date=1000", "date=1000", parts.get(0).get(0)); + Assert.assertEquals("Partition name should be date=2000", "date=2000", parts.get(1).get(0)); + + // 16 - Query it to verify data + data = hclient.execQuery("select * from weblogs"); + Assert.assertEquals("Expecting 4 records", data.size(),4); + Assert.assertEquals("1st record has wrong data:", data.get(0).get(0),line1); + Assert.assertEquals("2nd record has wrong data:", data.get(1).get(0),line2); + Assert.assertEquals("3rd record has wrong data:", data.get(2).get(0),line3); + Assert.assertEquals("4th record has wrong data:", data.get(3).get(0),line4); + } + + @Test + /** + * Tests the rollPartition feature on a table where streaming is disabled + */ + public void testRollWhenDisabled() throws IOException, SQLException, ClassNotFoundException { + String chunkDir = conf.get("hive.streaming.tempdir") + "/testing/weblogs"; + Path chunkFile1 = new Path(chunkDir, "1"); + Path chunkFile2 = new Path(chunkDir, "2"); + String chunkCommitDir1 = "file:/user/hive/warehouse/testing.db/weblogs/date=1000"; + Path chunkCommitFile1 = new Path(chunkCommitDir1, "1"); + String chunkCommitDir2 = "file:/user/hive/warehouse/testing.db/weblogs/date=2000"; + Path chunkCommitFile2 = new Path(chunkCommitDir2, "2"); + + + hdfs.delete(chunkFile1,false); + hdfs.delete(chunkFile2,false); + hdfs.delete(chunkCommitFile1, false); + hdfs.delete(chunkCommitFile2, false); + + // check disabled status + WebHCatClient.Response r = WebHCatClient.doGet(templetonBaseUrl + + "/status?user.name=jdoe&database=testing&table=weblogs"); + Assert.assertEquals(HttpStatus.OK_200, r.status); + Assert.assertTrue(r.body.contains("\"streaming_enabled\":\"false\"")); + Assert.assertTrue(r.body.contains("\"chunkid\":\"null\"")); + + // 1 - enable streaming + r = WebHCatClient.doGet(templetonBaseUrl + + "/enable?user.name=jdoe&database=testing&table=weblogs&col=date&value=1000"); + Assert.assertTrue(r.body.contains("\"status\":\"OK\"")); + + // 2 - check enabled status + r = WebHCatClient.doGet(templetonBaseUrl + + "/status?user.name=jdoe&database=testing&table=weblogs"); + + Assert.assertEquals(HttpStatus.OK_200, r.status); + Assert.assertTrue(r.body.contains("\"streaming_enabled\":\"true\"")); + Assert.assertTrue(r.body.contains("\"chunkid\":\"0\"")); + + // 3 - get chunk + r = WebHCatClient.doGet(templetonBaseUrl + + "/chunkget?user.name=jdoe&database=testing&table=weblogs&schema=blah&format=blah" + + "&record_separator=blah&field_separator=blah"); + + Map response = JsonBuilder.jsonToMap(r.body); + Assert.assertEquals(response.get("status"),"OK"); + Assert.assertEquals(response.get("chunkid"),1); + Assert.assertEquals(response.get("chunkfile"),chunkFile1.toString()); + + // 4 - write two records (i.e. lines) to chunk + FSDataOutputStream output = hdfs.create(chunkFile1,true); // enable overwriting file + String line1 = "What is the difference between en elephant and a plum?"; + String line2 = "A: An elephant is grey."; + InputStream input = new ByteArrayInputStream((line1 + "\n" + line2).getBytes()); + IOUtils.copyBytes(input, output, conf); + output.close(); + + // 5 - commit chunk + r = WebHCatClient.doGet(templetonBaseUrl + + "/chunkcommit?user.name=jdoe&database=testing&table=weblogs&chunkfile=" + + chunkFile1); + + response = JsonBuilder.jsonToMap(r.body); + Assert.assertEquals("OK",response.get("status")); + Assert.assertEquals(chunkCommitFile1.toString(),response.get("chunkfile")); + + // 6 - verify that file has been moved to commit directory + Assert.assertTrue("Committed chunk file not found in commit location" + , hdfs.exists(chunkCommitFile1)); + Assert.assertFalse("Chunk file was not removed from tmp location on commit" + , hdfs.exists(chunkFile1)); + + // 7 - disable streaming + r = WebHCatClient.doGet(templetonBaseUrl + + "/disable?user.name=jdoe&database=testing&table=weblogs"); + response = JsonBuilder.jsonToMap(r.body); + Assert.assertEquals("OK", response.get("status")); + + + // 8 - roll partition + r = WebHCatClient.doGet(templetonBaseUrl + "/partitionroll?user.name=jdoe&" + + "database=testing&table=weblogs&partition_column=date&partition_value=2000"); + response = JsonBuilder.jsonToMap(r.body); + Assert.assertEquals(HttpStatus.METHOD_NOT_ALLOWED_405, r.status); + Assert.assertEquals("FAILED", response.get("status")); + + // 9 check table to verify that the partition was NOT added + hclient.exec("use testing"); + ArrayList> parts = hclient.execQuery("SHOW PARTITIONS weblogs"); + Assert.assertEquals("Expecting 0 partitions", parts.size(),0); + } + + /** + * Encapsulates information from HTTP method call + */ + private static class MethodCallRetVal { + private final int httpStatusCode; + private final String responseBody; + private final String submittedURL; + private final String methodName; + private MethodCallRetVal(int httpStatusCode, String responseBody, String submittedURL + , String methodName) { + this.httpStatusCode = httpStatusCode; + this.responseBody = responseBody; + this.submittedURL = submittedURL; + this.methodName = methodName; + } + String getAssertMsg() { + return methodName + " " + submittedURL + " " + responseBody; + } + } +} diff --git hcatalog/webhcat/svr/src/test/java/org/apache/hive/hcatalog/templeton/tool/HiveJdbcClient.java hcatalog/webhcat/svr/src/test/java/org/apache/hive/hcatalog/templeton/tool/HiveJdbcClient.java new file mode 100644 index 0000000..75d60bc --- /dev/null +++ hcatalog/webhcat/svr/src/test/java/org/apache/hive/hcatalog/templeton/tool/HiveJdbcClient.java @@ -0,0 +1,100 @@ +/** + * 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.hive.hcatalog.templeton.tool; + + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; +import java.sql.ResultSet; +import java.util.ArrayList; + + +public class HiveJdbcClient { + + private static String driverName = "org.apache.hive.jdbc.HiveDriver"; + private Connection con ; + + public HiveJdbcClient() throws SQLException, ClassNotFoundException { + Class.forName(driverName); + con = DriverManager.getConnection("jdbc:hive2://", "", ""); + } + + public ArrayList> execQuery(String query) throws SQLException { + ArrayList> result = new ArrayList>(); + Statement stmt = con.createStatement(); + ResultSet rs = null; + try { + rs = stmt.executeQuery(query); + int nCols = rs.getMetaData().getColumnCount(); + while(rs.next()) { + ArrayList row = new ArrayList(nCols); + for(int i=1; i<=nCols; ++i) { + row.add(rs.getString(i)); + } + result.add(row); + } + return result; + } catch (SQLException e) { + System.out.println("Error Executing:" + query); + System.out.println("Reason :" + e.getMessage()); + throw e; + } finally { + if(rs!=null) + rs.close(); + stmt.close(); + } + } + + public void exec(String hql) throws SQLException { + Statement stmt = con.createStatement(); + try { + stmt.execute(hql); + } catch (SQLException e) { + System.out.println("Error Executing:" + hql); + System.out.println("Reason :" + e.getMessage()); + return; + } finally { + stmt.close(); + } + } + + public void close() throws SQLException { + try { + con.close(); + } catch (Exception e) { + // supress + } + } + + public static void dispResults(ArrayList> rows) throws SQLException { + for(ArrayList row : rows) { + int i=1; + for (String cell : row ) { + if (i++ > 1) { + System.err.print("\t"); + } + System.err.print(cell); + } + System.err.println(); + } + } + +} diff --git hcatalog/webhcat/svr/src/test/java/org/apache/hive/hcatalog/templeton/tool/WebHCatClient.java hcatalog/webhcat/svr/src/test/java/org/apache/hive/hcatalog/templeton/tool/WebHCatClient.java new file mode 100644 index 0000000..95910dc --- /dev/null +++ hcatalog/webhcat/svr/src/test/java/org/apache/hive/hcatalog/templeton/tool/WebHCatClient.java @@ -0,0 +1,57 @@ +/** + * 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.hive.hcatalog.templeton.tool; + +import org.apache.commons.httpclient.HttpClient; +import org.apache.commons.httpclient.HttpMethod; +import org.apache.commons.httpclient.methods.GetMethod; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import java.io.IOException; + +public class WebHCatClient { + private static final Log LOG = LogFactory.getLog(WebHCatClient.class); + /** + * Encapsulates information from HTTP method call + */ + public static class Response { + public final int status; + public final String body; + public final String requestURL; + + private Response(int httpStatus, String responseBody, String requestURL) { + this.status = httpStatus; + this.body = responseBody; + this.requestURL = requestURL; + } + } + + public static Response doGet(String uri) throws IOException { + HttpMethod method = new GetMethod(uri); + String actualUri = method.getURI().toString(); //should this be escaped string? + LOG.debug("GET : " + method.getURI().getEscapedURI()); + + HttpClient client = new HttpClient(); + int httpStatus = client.executeMethod(method); + LOG.debug("Http Status Code=" + httpStatus); + String resp = method.getResponseBodyAsString(); + LOG.debug("response: " + resp); + return new Response(httpStatus, resp, actualUri); + } +} diff --git metastore/if/hive_metastore.thrift metastore/if/hive_metastore.thrift index 3fea057..a263469 100755 --- metastore/if/hive_metastore.thrift +++ metastore/if/hive_metastore.thrift @@ -593,7 +593,37 @@ service ThriftHiveMetastore extends fb303.FacebookService (1:NoSuchObjectException o1, 2:MetaException o2, 3:InvalidObjectException o3, 4:InvalidInputException o4) - //authorization privileges + // Streaming partition APIs + + map streamingStatus(1:string db_name, 2:string table_name) throws + (1:InvalidInputException o1, 2:InvalidObjectException o2, 3:MetaException o3) + + void enableStreaming(1:string db_name, 2:string table_name, 3:string streaming_tmp_dir, 4:string partition_path, 5:string partition_val) throws + (1:InvalidInputException o1, 2:InvalidObjectException o2, 3:InvalidOperationException o3, 4:MetaException o4) + + void disableStreaming(1:string db_name, 2:string table_name) throws + (1:InvalidInputException o1, 2:InvalidObjectException o2, 3:InvalidOperationException o3, 4:MetaException o4) + + string getCurrentStreamingPartitionPath(1:string dbName, 2:string tableName) throws + (1:InvalidInputException o1, 2:InvalidObjectException o2, + 3:InvalidOperationException o3, 4:MetaException o4) + + string rollStreamingPartition(1:string dbName, 2:string tableName, 3:string newPartitionPath 4:string newPartitionVal) throws + (1:InvalidInputException o1, 2:InvalidObjectException o2, + 3:InvalidOperationException o3, 4:AlreadyExistsException o4, 5:MetaException o5) + + void updateStreamingTempLocation(1:string dbName, 2:string tableName, 3:string streamingTempDir) throws + (1:InvalidInputException o1, 2:InvalidObjectException o2, 3:MetaException o3) + + i64 getNextChunkID(1:string db_name, 2:string table_name) throws + (1:InvalidInputException o1, 2:InvalidObjectException o2, + 3:InvalidOperationException o3, 4:MetaException o4) + + string getStreamingTmpDir(1:string dbName, 2:string tableName) throws + (1:InvalidInputException o1, 2:InvalidObjectException o2, + 3:InvalidOperationException o3, 4:MetaException o4) + + ////authorization privileges bool create_role(1:Role role) throws(1:MetaException o1) bool drop_role(1:string role_name) throws(1:MetaException o1) diff --git metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp index 3d69472..d4da72f 100644 --- metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp +++ metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp @@ -17284,7 +17284,7 @@ uint32_t ThriftHiveMetastore_delete_table_column_statistics_presult::read(::apac return xfer; } -uint32_t ThriftHiveMetastore_create_role_args::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_streamingStatus_args::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -17305,9 +17305,17 @@ uint32_t ThriftHiveMetastore_create_role_args::read(::apache::thrift::protocol:: switch (fid) { case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->role.read(iprot); - this->__isset.role = true; + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->table_name); + this->__isset.table_name = true; } else { xfer += iprot->skip(ftype); } @@ -17324,12 +17332,16 @@ uint32_t ThriftHiveMetastore_create_role_args::read(::apache::thrift::protocol:: return xfer; } -uint32_t ThriftHiveMetastore_create_role_args::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_streamingStatus_args::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_role_args"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_streamingStatus_args"); - xfer += oprot->writeFieldBegin("role", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->role.write(oprot); + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->table_name); xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); @@ -17337,12 +17349,16 @@ uint32_t ThriftHiveMetastore_create_role_args::write(::apache::thrift::protocol: return xfer; } -uint32_t ThriftHiveMetastore_create_role_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_streamingStatus_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_role_pargs"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_streamingStatus_pargs"); - xfer += oprot->writeFieldBegin("role", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->role)).write(oprot); + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->table_name))); xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); @@ -17350,7 +17366,7 @@ uint32_t ThriftHiveMetastore_create_role_pargs::write(::apache::thrift::protocol return xfer; } -uint32_t ThriftHiveMetastore_create_role_result::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_streamingStatus_result::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -17371,8 +17387,23 @@ uint32_t ThriftHiveMetastore_create_role_result::read(::apache::thrift::protocol switch (fid) { case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); + if (ftype == ::apache::thrift::protocol::T_MAP) { + { + this->success.clear(); + uint32_t _size595; + ::apache::thrift::protocol::TType _ktype596; + ::apache::thrift::protocol::TType _vtype597; + xfer += iprot->readMapBegin(_ktype596, _vtype597, _size595); + uint32_t _i599; + for (_i599 = 0; _i599 < _size595; ++_i599) + { + std::string _key600; + xfer += iprot->readString(_key600); + std::string& _val601 = this->success[_key600]; + xfer += iprot->readString(_val601); + } + xfer += iprot->readMapEnd(); + } this->__isset.success = true; } else { xfer += iprot->skip(ftype); @@ -17386,6 +17417,22 @@ uint32_t ThriftHiveMetastore_create_role_result::read(::apache::thrift::protocol xfer += iprot->skip(ftype); } break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; default: xfer += iprot->skip(ftype); break; @@ -17398,27 +17445,44 @@ uint32_t ThriftHiveMetastore_create_role_result::read(::apache::thrift::protocol return xfer; } -uint32_t ThriftHiveMetastore_create_role_result::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_streamingStatus_result::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_role_result"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_streamingStatus_result"); if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); - xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0); + { + xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); + std::map ::const_iterator _iter602; + for (_iter602 = this->success.begin(); _iter602 != this->success.end(); ++_iter602) + { + xfer += oprot->writeString(_iter602->first); + xfer += oprot->writeString(_iter602->second); + } + xfer += oprot->writeMapEnd(); + } xfer += oprot->writeFieldEnd(); } else if (this->__isset.o1) { xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); xfer += this->o1.write(oprot); xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); } xfer += oprot->writeFieldStop(); xfer += oprot->writeStructEnd(); return xfer; } -uint32_t ThriftHiveMetastore_create_role_presult::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_streamingStatus_presult::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -17439,8 +17503,23 @@ uint32_t ThriftHiveMetastore_create_role_presult::read(::apache::thrift::protoco switch (fid) { case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool((*(this->success))); + if (ftype == ::apache::thrift::protocol::T_MAP) { + { + (*(this->success)).clear(); + uint32_t _size603; + ::apache::thrift::protocol::TType _ktype604; + ::apache::thrift::protocol::TType _vtype605; + xfer += iprot->readMapBegin(_ktype604, _vtype605, _size603); + uint32_t _i607; + for (_i607 = 0; _i607 < _size603; ++_i607) + { + std::string _key608; + xfer += iprot->readString(_key608); + std::string& _val609 = (*(this->success))[_key608]; + xfer += iprot->readString(_val609); + } + xfer += iprot->readMapEnd(); + } this->__isset.success = true; } else { xfer += iprot->skip(ftype); @@ -17454,6 +17533,22 @@ uint32_t ThriftHiveMetastore_create_role_presult::read(::apache::thrift::protoco xfer += iprot->skip(ftype); } break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; default: xfer += iprot->skip(ftype); break; @@ -17466,7 +17561,7 @@ uint32_t ThriftHiveMetastore_create_role_presult::read(::apache::thrift::protoco return xfer; } -uint32_t ThriftHiveMetastore_drop_role_args::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_enableStreaming_args::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -17488,8 +17583,40 @@ uint32_t ThriftHiveMetastore_drop_role_args::read(::apache::thrift::protocol::TP { case 1: if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->role_name); - this->__isset.role_name = true; + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->table_name); + this->__isset.table_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->streaming_tmp_dir); + this->__isset.streaming_tmp_dir = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->partition_path); + this->__isset.partition_path = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->partition_val); + this->__isset.partition_val = true; } else { xfer += iprot->skip(ftype); } @@ -17506,12 +17633,28 @@ uint32_t ThriftHiveMetastore_drop_role_args::read(::apache::thrift::protocol::TP return xfer; } -uint32_t ThriftHiveMetastore_drop_role_args::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_enableStreaming_args::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_role_args"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_enableStreaming_args"); - xfer += oprot->writeFieldBegin("role_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->role_name); + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->table_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("streaming_tmp_dir", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->streaming_tmp_dir); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("partition_path", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->partition_path); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("partition_val", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeString(this->partition_val); xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); @@ -17519,12 +17662,28 @@ uint32_t ThriftHiveMetastore_drop_role_args::write(::apache::thrift::protocol::T return xfer; } -uint32_t ThriftHiveMetastore_drop_role_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_enableStreaming_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_role_pargs"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_enableStreaming_pargs"); - xfer += oprot->writeFieldBegin("role_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->role_name))); + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->table_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("streaming_tmp_dir", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString((*(this->streaming_tmp_dir))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("partition_path", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString((*(this->partition_path))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("partition_val", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeString((*(this->partition_val))); xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); @@ -17532,7 +17691,7 @@ uint32_t ThriftHiveMetastore_drop_role_pargs::write(::apache::thrift::protocol:: return xfer; } -uint32_t ThriftHiveMetastore_drop_role_result::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_enableStreaming_result::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -17552,18 +17711,34 @@ uint32_t ThriftHiveMetastore_drop_role_result::read(::apache::thrift::protocol:: } switch (fid) { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); - this->__isset.success = true; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; } else { xfer += iprot->skip(ftype); } break; - case 1: + case 2: if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; } else { xfer += iprot->skip(ftype); } @@ -17580,27 +17755,35 @@ uint32_t ThriftHiveMetastore_drop_role_result::read(::apache::thrift::protocol:: return xfer; } -uint32_t ThriftHiveMetastore_drop_role_result::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_enableStreaming_result::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_role_result"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_enableStreaming_result"); - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); - xfer += oprot->writeBool(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { + if (this->__isset.o1) { xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); xfer += this->o1.write(oprot); xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o4) { + xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->o4.write(oprot); + xfer += oprot->writeFieldEnd(); } xfer += oprot->writeFieldStop(); xfer += oprot->writeStructEnd(); return xfer; } -uint32_t ThriftHiveMetastore_drop_role_presult::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_enableStreaming_presult::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -17620,18 +17803,34 @@ uint32_t ThriftHiveMetastore_drop_role_presult::read(::apache::thrift::protocol: } switch (fid) { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool((*(this->success))); - this->__isset.success = true; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; } else { xfer += iprot->skip(ftype); } break; - case 1: + case 2: if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; } else { xfer += iprot->skip(ftype); } @@ -17648,7 +17847,7 @@ uint32_t ThriftHiveMetastore_drop_role_presult::read(::apache::thrift::protocol: return xfer; } -uint32_t ThriftHiveMetastore_get_role_names_args::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_disableStreaming_args::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -17666,7 +17865,28 @@ uint32_t ThriftHiveMetastore_get_role_names_args::read(::apache::thrift::protoco if (ftype == ::apache::thrift::protocol::T_STOP) { break; } - xfer += iprot->skip(ftype); + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->table_name); + this->__isset.table_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } xfer += iprot->readFieldEnd(); } @@ -17675,25 +17895,41 @@ uint32_t ThriftHiveMetastore_get_role_names_args::read(::apache::thrift::protoco return xfer; } -uint32_t ThriftHiveMetastore_get_role_names_args::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_disableStreaming_args::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_role_names_args"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_disableStreaming_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->table_name); + xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); xfer += oprot->writeStructEnd(); return xfer; } -uint32_t ThriftHiveMetastore_get_role_names_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_disableStreaming_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_role_names_pargs"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_disableStreaming_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->table_name))); + xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); xfer += oprot->writeStructEnd(); return xfer; } -uint32_t ThriftHiveMetastore_get_role_names_result::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_disableStreaming_result::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -17713,30 +17949,34 @@ uint32_t ThriftHiveMetastore_get_role_names_result::read(::apache::thrift::proto } switch (fid) { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size595; - ::apache::thrift::protocol::TType _etype598; - xfer += iprot->readListBegin(_etype598, _size595); - this->success.resize(_size595); - uint32_t _i599; - for (_i599 = 0; _i599 < _size595; ++_i599) - { - xfer += iprot->readString(this->success[_i599]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; } else { xfer += iprot->skip(ftype); } break; - case 1: + case 2: if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; } else { xfer += iprot->skip(ftype); } @@ -17753,35 +17993,35 @@ uint32_t ThriftHiveMetastore_get_role_names_result::read(::apache::thrift::proto return xfer; } -uint32_t ThriftHiveMetastore_get_role_names_result::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_disableStreaming_result::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_role_names_result"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_disableStreaming_result"); - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter600; - for (_iter600 = this->success.begin(); _iter600 != this->success.end(); ++_iter600) - { - xfer += oprot->writeString((*_iter600)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { + if (this->__isset.o1) { xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); xfer += this->o1.write(oprot); xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o4) { + xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->o4.write(oprot); + xfer += oprot->writeFieldEnd(); } xfer += oprot->writeFieldStop(); xfer += oprot->writeStructEnd(); return xfer; } -uint32_t ThriftHiveMetastore_get_role_names_presult::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_disableStreaming_presult::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -17801,30 +18041,34 @@ uint32_t ThriftHiveMetastore_get_role_names_presult::read(::apache::thrift::prot } switch (fid) { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size601; - ::apache::thrift::protocol::TType _etype604; - xfer += iprot->readListBegin(_etype604, _size601); - (*(this->success)).resize(_size601); - uint32_t _i605; - for (_i605 = 0; _i605 < _size601; ++_i605) - { - xfer += iprot->readString((*(this->success))[_i605]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; } else { xfer += iprot->skip(ftype); } break; - case 1: + case 2: if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; } else { xfer += iprot->skip(ftype); } @@ -17841,7 +18085,7 @@ uint32_t ThriftHiveMetastore_get_role_names_presult::read(::apache::thrift::prot return xfer; } -uint32_t ThriftHiveMetastore_grant_role_args::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_getCurrentStreamingPartitionPath_args::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -17863,52 +18107,16 @@ uint32_t ThriftHiveMetastore_grant_role_args::read(::apache::thrift::protocol::T { case 1: if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->role_name); - this->__isset.role_name = true; + xfer += iprot->readString(this->dbName); + this->__isset.dbName = true; } else { xfer += iprot->skip(ftype); } break; case 2: if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->principal_name); - this->__isset.principal_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast606; - xfer += iprot->readI32(ecast606); - this->principal_type = (PrincipalType::type)ecast606; - this->__isset.principal_type = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->grantor); - this->__isset.grantor = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast607; - xfer += iprot->readI32(ecast607); - this->grantorType = (PrincipalType::type)ecast607; - this->__isset.grantorType = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->grant_option); - this->__isset.grant_option = true; + xfer += iprot->readString(this->tableName); + this->__isset.tableName = true; } else { xfer += iprot->skip(ftype); } @@ -17925,32 +18133,16 @@ uint32_t ThriftHiveMetastore_grant_role_args::read(::apache::thrift::protocol::T return xfer; } -uint32_t ThriftHiveMetastore_grant_role_args::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_getCurrentStreamingPartitionPath_args::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_role_args"); - - xfer += oprot->writeFieldBegin("role_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->role_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->principal_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 3); - xfer += oprot->writeI32((int32_t)this->principal_type); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("grantor", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->grantor); - xfer += oprot->writeFieldEnd(); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_getCurrentStreamingPartitionPath_args"); - xfer += oprot->writeFieldBegin("grantorType", ::apache::thrift::protocol::T_I32, 5); - xfer += oprot->writeI32((int32_t)this->grantorType); + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbName); xfer += oprot->writeFieldEnd(); - xfer += oprot->writeFieldBegin("grant_option", ::apache::thrift::protocol::T_BOOL, 6); - xfer += oprot->writeBool(this->grant_option); + xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tableName); xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); @@ -17958,32 +18150,16 @@ uint32_t ThriftHiveMetastore_grant_role_args::write(::apache::thrift::protocol:: return xfer; } -uint32_t ThriftHiveMetastore_grant_role_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_getCurrentStreamingPartitionPath_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_role_pargs"); - - xfer += oprot->writeFieldBegin("role_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->role_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->principal_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 3); - xfer += oprot->writeI32((int32_t)(*(this->principal_type))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("grantor", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString((*(this->grantor))); - xfer += oprot->writeFieldEnd(); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_getCurrentStreamingPartitionPath_pargs"); - xfer += oprot->writeFieldBegin("grantorType", ::apache::thrift::protocol::T_I32, 5); - xfer += oprot->writeI32((int32_t)(*(this->grantorType))); + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->dbName))); xfer += oprot->writeFieldEnd(); - xfer += oprot->writeFieldBegin("grant_option", ::apache::thrift::protocol::T_BOOL, 6); - xfer += oprot->writeBool((*(this->grant_option))); + xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tableName))); xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); @@ -17991,7 +18167,7 @@ uint32_t ThriftHiveMetastore_grant_role_pargs::write(::apache::thrift::protocol: return xfer; } -uint32_t ThriftHiveMetastore_grant_role_result::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_getCurrentStreamingPartitionPath_result::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -18012,8 +18188,8 @@ uint32_t ThriftHiveMetastore_grant_role_result::read(::apache::thrift::protocol: switch (fid) { case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->success); this->__isset.success = true; } else { xfer += iprot->skip(ftype); @@ -18027,6 +18203,30 @@ uint32_t ThriftHiveMetastore_grant_role_result::read(::apache::thrift::protocol: xfer += iprot->skip(ftype); } break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; default: xfer += iprot->skip(ftype); break; @@ -18039,27 +18239,39 @@ uint32_t ThriftHiveMetastore_grant_role_result::read(::apache::thrift::protocol: return xfer; } -uint32_t ThriftHiveMetastore_grant_role_result::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_getCurrentStreamingPartitionPath_result::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_role_result"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_getCurrentStreamingPartitionPath_result"); if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); - xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRING, 0); + xfer += oprot->writeString(this->success); xfer += oprot->writeFieldEnd(); } else if (this->__isset.o1) { xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); xfer += this->o1.write(oprot); xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o4) { + xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->o4.write(oprot); + xfer += oprot->writeFieldEnd(); } xfer += oprot->writeFieldStop(); xfer += oprot->writeStructEnd(); return xfer; } -uint32_t ThriftHiveMetastore_grant_role_presult::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_getCurrentStreamingPartitionPath_presult::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -18080,8 +18292,8 @@ uint32_t ThriftHiveMetastore_grant_role_presult::read(::apache::thrift::protocol switch (fid) { case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool((*(this->success))); + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString((*(this->success))); this->__isset.success = true; } else { xfer += iprot->skip(ftype); @@ -18095,6 +18307,30 @@ uint32_t ThriftHiveMetastore_grant_role_presult::read(::apache::thrift::protocol xfer += iprot->skip(ftype); } break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; default: xfer += iprot->skip(ftype); break; @@ -18107,7 +18343,7 @@ uint32_t ThriftHiveMetastore_grant_role_presult::read(::apache::thrift::protocol return xfer; } -uint32_t ThriftHiveMetastore_revoke_role_args::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_rollStreamingPartition_args::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -18129,26 +18365,32 @@ uint32_t ThriftHiveMetastore_revoke_role_args::read(::apache::thrift::protocol:: { case 1: if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->role_name); - this->__isset.role_name = true; + xfer += iprot->readString(this->dbName); + this->__isset.dbName = true; } else { xfer += iprot->skip(ftype); } break; case 2: if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->principal_name); - this->__isset.principal_name = true; + xfer += iprot->readString(this->tableName); + this->__isset.tableName = true; } else { xfer += iprot->skip(ftype); } break; case 3: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast608; - xfer += iprot->readI32(ecast608); - this->principal_type = (PrincipalType::type)ecast608; - this->__isset.principal_type = true; + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->newPartitionPath); + this->__isset.newPartitionPath = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->newPartitionVal); + this->__isset.newPartitionVal = true; } else { xfer += iprot->skip(ftype); } @@ -18165,20 +18407,24 @@ uint32_t ThriftHiveMetastore_revoke_role_args::read(::apache::thrift::protocol:: return xfer; } -uint32_t ThriftHiveMetastore_revoke_role_args::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_rollStreamingPartition_args::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_revoke_role_args"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_rollStreamingPartition_args"); - xfer += oprot->writeFieldBegin("role_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->role_name); + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbName); xfer += oprot->writeFieldEnd(); - xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->principal_name); + xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tableName); xfer += oprot->writeFieldEnd(); - xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 3); - xfer += oprot->writeI32((int32_t)this->principal_type); + xfer += oprot->writeFieldBegin("newPartitionPath", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->newPartitionPath); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("newPartitionVal", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->newPartitionVal); xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); @@ -18186,20 +18432,24 @@ uint32_t ThriftHiveMetastore_revoke_role_args::write(::apache::thrift::protocol: return xfer; } -uint32_t ThriftHiveMetastore_revoke_role_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_rollStreamingPartition_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_revoke_role_pargs"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_rollStreamingPartition_pargs"); - xfer += oprot->writeFieldBegin("role_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->role_name))); + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->dbName))); xfer += oprot->writeFieldEnd(); - xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->principal_name))); + xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tableName))); xfer += oprot->writeFieldEnd(); - xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 3); - xfer += oprot->writeI32((int32_t)(*(this->principal_type))); + xfer += oprot->writeFieldBegin("newPartitionPath", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString((*(this->newPartitionPath))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("newPartitionVal", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString((*(this->newPartitionVal))); xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); @@ -18207,7 +18457,7 @@ uint32_t ThriftHiveMetastore_revoke_role_pargs::write(::apache::thrift::protocol return xfer; } -uint32_t ThriftHiveMetastore_revoke_role_result::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_rollStreamingPartition_result::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -18228,8 +18478,8 @@ uint32_t ThriftHiveMetastore_revoke_role_result::read(::apache::thrift::protocol switch (fid) { case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->success); this->__isset.success = true; } else { xfer += iprot->skip(ftype); @@ -18243,6 +18493,38 @@ uint32_t ThriftHiveMetastore_revoke_role_result::read(::apache::thrift::protocol xfer += iprot->skip(ftype); } break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o5.read(iprot); + this->__isset.o5 = true; + } else { + xfer += iprot->skip(ftype); + } + break; default: xfer += iprot->skip(ftype); break; @@ -18255,27 +18537,43 @@ uint32_t ThriftHiveMetastore_revoke_role_result::read(::apache::thrift::protocol return xfer; } -uint32_t ThriftHiveMetastore_revoke_role_result::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_rollStreamingPartition_result::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_revoke_role_result"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_rollStreamingPartition_result"); if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); - xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRING, 0); + xfer += oprot->writeString(this->success); xfer += oprot->writeFieldEnd(); } else if (this->__isset.o1) { xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); xfer += this->o1.write(oprot); xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o4) { + xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->o4.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o5) { + xfer += oprot->writeFieldBegin("o5", ::apache::thrift::protocol::T_STRUCT, 5); + xfer += this->o5.write(oprot); + xfer += oprot->writeFieldEnd(); } xfer += oprot->writeFieldStop(); xfer += oprot->writeStructEnd(); return xfer; } -uint32_t ThriftHiveMetastore_revoke_role_presult::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_rollStreamingPartition_presult::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -18296,8 +18594,8 @@ uint32_t ThriftHiveMetastore_revoke_role_presult::read(::apache::thrift::protoco switch (fid) { case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool((*(this->success))); + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString((*(this->success))); this->__isset.success = true; } else { xfer += iprot->skip(ftype); @@ -18311,6 +18609,38 @@ uint32_t ThriftHiveMetastore_revoke_role_presult::read(::apache::thrift::protoco xfer += iprot->skip(ftype); } break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o5.read(iprot); + this->__isset.o5 = true; + } else { + xfer += iprot->skip(ftype); + } + break; default: xfer += iprot->skip(ftype); break; @@ -18323,7 +18653,7 @@ uint32_t ThriftHiveMetastore_revoke_role_presult::read(::apache::thrift::protoco return xfer; } -uint32_t ThriftHiveMetastore_list_roles_args::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_updateStreamingTempLocation_args::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -18345,18 +18675,24 @@ uint32_t ThriftHiveMetastore_list_roles_args::read(::apache::thrift::protocol::T { case 1: if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->principal_name); - this->__isset.principal_name = true; + xfer += iprot->readString(this->dbName); + this->__isset.dbName = true; } else { xfer += iprot->skip(ftype); } break; case 2: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast609; - xfer += iprot->readI32(ecast609); - this->principal_type = (PrincipalType::type)ecast609; - this->__isset.principal_type = true; + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tableName); + this->__isset.tableName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->streamingTempDir); + this->__isset.streamingTempDir = true; } else { xfer += iprot->skip(ftype); } @@ -18373,16 +18709,20 @@ uint32_t ThriftHiveMetastore_list_roles_args::read(::apache::thrift::protocol::T return xfer; } -uint32_t ThriftHiveMetastore_list_roles_args::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_updateStreamingTempLocation_args::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_list_roles_args"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_updateStreamingTempLocation_args"); - xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->principal_name); + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbName); xfer += oprot->writeFieldEnd(); - xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 2); - xfer += oprot->writeI32((int32_t)this->principal_type); + xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tableName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("streamingTempDir", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->streamingTempDir); xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); @@ -18390,16 +18730,20 @@ uint32_t ThriftHiveMetastore_list_roles_args::write(::apache::thrift::protocol:: return xfer; } -uint32_t ThriftHiveMetastore_list_roles_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_updateStreamingTempLocation_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_list_roles_pargs"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_updateStreamingTempLocation_pargs"); - xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->principal_name))); + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->dbName))); xfer += oprot->writeFieldEnd(); - xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 2); - xfer += oprot->writeI32((int32_t)(*(this->principal_type))); + xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tableName))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("streamingTempDir", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString((*(this->streamingTempDir))); xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); @@ -18407,7 +18751,7 @@ uint32_t ThriftHiveMetastore_list_roles_pargs::write(::apache::thrift::protocol: return xfer; } -uint32_t ThriftHiveMetastore_list_roles_result::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_updateStreamingTempLocation_result::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -18427,30 +18771,26 @@ uint32_t ThriftHiveMetastore_list_roles_result::read(::apache::thrift::protocol: } switch (fid) { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size610; - ::apache::thrift::protocol::TType _etype613; - xfer += iprot->readListBegin(_etype613, _size610); - this->success.resize(_size610); - uint32_t _i614; - for (_i614 = 0; _i614 < _size610; ++_i614) - { - xfer += this->success[_i614].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; } else { xfer += iprot->skip(ftype); } break; - case 1: + case 2: if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; } else { xfer += iprot->skip(ftype); } @@ -18467,35 +18807,31 @@ uint32_t ThriftHiveMetastore_list_roles_result::read(::apache::thrift::protocol: return xfer; } -uint32_t ThriftHiveMetastore_list_roles_result::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_updateStreamingTempLocation_result::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_list_roles_result"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_updateStreamingTempLocation_result"); - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter615; - for (_iter615 = this->success.begin(); _iter615 != this->success.end(); ++_iter615) - { - xfer += (*_iter615).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { + if (this->__isset.o1) { xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); xfer += this->o1.write(oprot); xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); } xfer += oprot->writeFieldStop(); xfer += oprot->writeStructEnd(); return xfer; } -uint32_t ThriftHiveMetastore_list_roles_presult::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_updateStreamingTempLocation_presult::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -18515,30 +18851,26 @@ uint32_t ThriftHiveMetastore_list_roles_presult::read(::apache::thrift::protocol } switch (fid) { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size616; - ::apache::thrift::protocol::TType _etype619; - xfer += iprot->readListBegin(_etype619, _size616); - (*(this->success)).resize(_size616); - uint32_t _i620; - for (_i620 = 0; _i620 < _size616; ++_i620) - { - xfer += (*(this->success))[_i620].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; } else { xfer += iprot->skip(ftype); } break; - case 1: + case 2: if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; } else { xfer += iprot->skip(ftype); } @@ -18555,7 +18887,7 @@ uint32_t ThriftHiveMetastore_list_roles_presult::read(::apache::thrift::protocol return xfer; } -uint32_t ThriftHiveMetastore_get_privilege_set_args::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_getNextChunkID_args::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -18576,37 +18908,17 @@ uint32_t ThriftHiveMetastore_get_privilege_set_args::read(::apache::thrift::prot switch (fid) { case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->hiveObject.read(iprot); - this->__isset.hiveObject = true; + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; } else { xfer += iprot->skip(ftype); } break; case 2: if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->user_name); - this->__isset.user_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->group_names.clear(); - uint32_t _size621; - ::apache::thrift::protocol::TType _etype624; - xfer += iprot->readListBegin(_etype624, _size621); - this->group_names.resize(_size621); - uint32_t _i625; - for (_i625 = 0; _i625 < _size621; ++_i625) - { - xfer += iprot->readString(this->group_names[_i625]); - } - xfer += iprot->readListEnd(); - } - this->__isset.group_names = true; + xfer += iprot->readString(this->table_name); + this->__isset.table_name = true; } else { xfer += iprot->skip(ftype); } @@ -18623,28 +18935,16 @@ uint32_t ThriftHiveMetastore_get_privilege_set_args::read(::apache::thrift::prot return xfer; } -uint32_t ThriftHiveMetastore_get_privilege_set_args::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_getNextChunkID_args::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_privilege_set_args"); - - xfer += oprot->writeFieldBegin("hiveObject", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->hiveObject.write(oprot); - xfer += oprot->writeFieldEnd(); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_getNextChunkID_args"); - xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->user_name); + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); xfer += oprot->writeFieldEnd(); - xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->group_names.size())); - std::vector ::const_iterator _iter626; - for (_iter626 = this->group_names.begin(); _iter626 != this->group_names.end(); ++_iter626) - { - xfer += oprot->writeString((*_iter626)); - } - xfer += oprot->writeListEnd(); - } + xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->table_name); xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); @@ -18652,28 +18952,16 @@ uint32_t ThriftHiveMetastore_get_privilege_set_args::write(::apache::thrift::pro return xfer; } -uint32_t ThriftHiveMetastore_get_privilege_set_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_getNextChunkID_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_privilege_set_pargs"); - - xfer += oprot->writeFieldBegin("hiveObject", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->hiveObject)).write(oprot); - xfer += oprot->writeFieldEnd(); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_getNextChunkID_pargs"); - xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->user_name))); + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); xfer += oprot->writeFieldEnd(); - xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->group_names)).size())); - std::vector ::const_iterator _iter627; - for (_iter627 = (*(this->group_names)).begin(); _iter627 != (*(this->group_names)).end(); ++_iter627) - { - xfer += oprot->writeString((*_iter627)); - } - xfer += oprot->writeListEnd(); - } + xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->table_name))); xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); @@ -18681,7 +18969,7 @@ uint32_t ThriftHiveMetastore_get_privilege_set_pargs::write(::apache::thrift::pr return xfer; } -uint32_t ThriftHiveMetastore_get_privilege_set_result::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_getNextChunkID_result::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -18702,8 +18990,8 @@ uint32_t ThriftHiveMetastore_get_privilege_set_result::read(::apache::thrift::pr switch (fid) { case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->success); this->__isset.success = true; } else { xfer += iprot->skip(ftype); @@ -18717,6 +19005,30 @@ uint32_t ThriftHiveMetastore_get_privilege_set_result::read(::apache::thrift::pr xfer += iprot->skip(ftype); } break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; default: xfer += iprot->skip(ftype); break; @@ -18729,27 +19041,39 @@ uint32_t ThriftHiveMetastore_get_privilege_set_result::read(::apache::thrift::pr return xfer; } -uint32_t ThriftHiveMetastore_get_privilege_set_result::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_getNextChunkID_result::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_privilege_set_result"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_getNextChunkID_result"); if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_I64, 0); + xfer += oprot->writeI64(this->success); xfer += oprot->writeFieldEnd(); } else if (this->__isset.o1) { xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); xfer += this->o1.write(oprot); xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o4) { + xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->o4.write(oprot); + xfer += oprot->writeFieldEnd(); } xfer += oprot->writeFieldStop(); xfer += oprot->writeStructEnd(); return xfer; } -uint32_t ThriftHiveMetastore_get_privilege_set_presult::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_getNextChunkID_presult::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -18770,8 +19094,8 @@ uint32_t ThriftHiveMetastore_get_privilege_set_presult::read(::apache::thrift::p switch (fid) { case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64((*(this->success))); this->__isset.success = true; } else { xfer += iprot->skip(ftype); @@ -18785,6 +19109,30 @@ uint32_t ThriftHiveMetastore_get_privilege_set_presult::read(::apache::thrift::p xfer += iprot->skip(ftype); } break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; default: xfer += iprot->skip(ftype); break; @@ -18797,7 +19145,7 @@ uint32_t ThriftHiveMetastore_get_privilege_set_presult::read(::apache::thrift::p return xfer; } -uint32_t ThriftHiveMetastore_list_privileges_args::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_getStreamingTmpDir_args::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -18819,26 +19167,16 @@ uint32_t ThriftHiveMetastore_list_privileges_args::read(::apache::thrift::protoc { case 1: if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->principal_name); - this->__isset.principal_name = true; + xfer += iprot->readString(this->dbName); + this->__isset.dbName = true; } else { xfer += iprot->skip(ftype); } break; case 2: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast628; - xfer += iprot->readI32(ecast628); - this->principal_type = (PrincipalType::type)ecast628; - this->__isset.principal_type = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->hiveObject.read(iprot); - this->__isset.hiveObject = true; + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tableName); + this->__isset.tableName = true; } else { xfer += iprot->skip(ftype); } @@ -18855,20 +19193,16 @@ uint32_t ThriftHiveMetastore_list_privileges_args::read(::apache::thrift::protoc return xfer; } -uint32_t ThriftHiveMetastore_list_privileges_args::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_getStreamingTmpDir_args::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_list_privileges_args"); - - xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->principal_name); - xfer += oprot->writeFieldEnd(); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_getStreamingTmpDir_args"); - xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 2); - xfer += oprot->writeI32((int32_t)this->principal_type); + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbName); xfer += oprot->writeFieldEnd(); - xfer += oprot->writeFieldBegin("hiveObject", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->hiveObject.write(oprot); + xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tableName); xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); @@ -18876,20 +19210,16 @@ uint32_t ThriftHiveMetastore_list_privileges_args::write(::apache::thrift::proto return xfer; } -uint32_t ThriftHiveMetastore_list_privileges_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_getStreamingTmpDir_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_list_privileges_pargs"); - - xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->principal_name))); - xfer += oprot->writeFieldEnd(); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_getStreamingTmpDir_pargs"); - xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 2); - xfer += oprot->writeI32((int32_t)(*(this->principal_type))); + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->dbName))); xfer += oprot->writeFieldEnd(); - xfer += oprot->writeFieldBegin("hiveObject", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += (*(this->hiveObject)).write(oprot); + xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tableName))); xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); @@ -18897,7 +19227,7 @@ uint32_t ThriftHiveMetastore_list_privileges_pargs::write(::apache::thrift::prot return xfer; } -uint32_t ThriftHiveMetastore_list_privileges_result::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_getStreamingTmpDir_result::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -18918,20 +19248,8 @@ uint32_t ThriftHiveMetastore_list_privileges_result::read(::apache::thrift::prot switch (fid) { case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size629; - ::apache::thrift::protocol::TType _etype632; - xfer += iprot->readListBegin(_etype632, _size629); - this->success.resize(_size629); - uint32_t _i633; - for (_i633 = 0; _i633 < _size629; ++_i633) - { - xfer += this->success[_i633].read(iprot); - } - xfer += iprot->readListEnd(); - } + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->success); this->__isset.success = true; } else { xfer += iprot->skip(ftype); @@ -18945,6 +19263,30 @@ uint32_t ThriftHiveMetastore_list_privileges_result::read(::apache::thrift::prot xfer += iprot->skip(ftype); } break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; default: xfer += iprot->skip(ftype); break; @@ -18957,35 +19299,39 @@ uint32_t ThriftHiveMetastore_list_privileges_result::read(::apache::thrift::prot return xfer; } -uint32_t ThriftHiveMetastore_list_privileges_result::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_getStreamingTmpDir_result::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_list_privileges_result"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_getStreamingTmpDir_result"); if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter634; - for (_iter634 = this->success.begin(); _iter634 != this->success.end(); ++_iter634) - { - xfer += (*_iter634).write(oprot); - } - xfer += oprot->writeListEnd(); - } + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRING, 0); + xfer += oprot->writeString(this->success); xfer += oprot->writeFieldEnd(); } else if (this->__isset.o1) { xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); xfer += this->o1.write(oprot); xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -uint32_t ThriftHiveMetastore_list_privileges_presult::read(::apache::thrift::protocol::TProtocol* iprot) { + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o4) { + xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->o4.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +uint32_t ThriftHiveMetastore_getStreamingTmpDir_presult::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -19006,20 +19352,8 @@ uint32_t ThriftHiveMetastore_list_privileges_presult::read(::apache::thrift::pro switch (fid) { case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size635; - ::apache::thrift::protocol::TType _etype638; - xfer += iprot->readListBegin(_etype638, _size635); - (*(this->success)).resize(_size635); - uint32_t _i639; - for (_i639 = 0; _i639 < _size635; ++_i639) - { - xfer += (*(this->success))[_i639].read(iprot); - } - xfer += iprot->readListEnd(); - } + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString((*(this->success))); this->__isset.success = true; } else { xfer += iprot->skip(ftype); @@ -19033,6 +19367,30 @@ uint32_t ThriftHiveMetastore_list_privileges_presult::read(::apache::thrift::pro xfer += iprot->skip(ftype); } break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; default: xfer += iprot->skip(ftype); break; @@ -19045,7 +19403,7 @@ uint32_t ThriftHiveMetastore_list_privileges_presult::read(::apache::thrift::pro return xfer; } -uint32_t ThriftHiveMetastore_grant_privileges_args::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_create_role_args::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -19067,8 +19425,8 @@ uint32_t ThriftHiveMetastore_grant_privileges_args::read(::apache::thrift::proto { case 1: if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->privileges.read(iprot); - this->__isset.privileges = true; + xfer += this->role.read(iprot); + this->__isset.role = true; } else { xfer += iprot->skip(ftype); } @@ -19085,12 +19443,12 @@ uint32_t ThriftHiveMetastore_grant_privileges_args::read(::apache::thrift::proto return xfer; } -uint32_t ThriftHiveMetastore_grant_privileges_args::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_create_role_args::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_privileges_args"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_role_args"); - xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->privileges.write(oprot); + xfer += oprot->writeFieldBegin("role", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->role.write(oprot); xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); @@ -19098,12 +19456,12 @@ uint32_t ThriftHiveMetastore_grant_privileges_args::write(::apache::thrift::prot return xfer; } -uint32_t ThriftHiveMetastore_grant_privileges_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_create_role_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_privileges_pargs"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_role_pargs"); - xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->privileges)).write(oprot); + xfer += oprot->writeFieldBegin("role", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->role)).write(oprot); xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); @@ -19111,7 +19469,7 @@ uint32_t ThriftHiveMetastore_grant_privileges_pargs::write(::apache::thrift::pro return xfer; } -uint32_t ThriftHiveMetastore_grant_privileges_result::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_create_role_result::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -19159,11 +19517,11 @@ uint32_t ThriftHiveMetastore_grant_privileges_result::read(::apache::thrift::pro return xfer; } -uint32_t ThriftHiveMetastore_grant_privileges_result::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_create_role_result::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_privileges_result"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_role_result"); if (this->__isset.success) { xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); @@ -19179,7 +19537,7 @@ uint32_t ThriftHiveMetastore_grant_privileges_result::write(::apache::thrift::pr return xfer; } -uint32_t ThriftHiveMetastore_grant_privileges_presult::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_create_role_presult::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -19227,7 +19585,7 @@ uint32_t ThriftHiveMetastore_grant_privileges_presult::read(::apache::thrift::pr return xfer; } -uint32_t ThriftHiveMetastore_revoke_privileges_args::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_drop_role_args::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -19248,9 +19606,9 @@ uint32_t ThriftHiveMetastore_revoke_privileges_args::read(::apache::thrift::prot switch (fid) { case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->privileges.read(iprot); - this->__isset.privileges = true; + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->role_name); + this->__isset.role_name = true; } else { xfer += iprot->skip(ftype); } @@ -19267,12 +19625,12 @@ uint32_t ThriftHiveMetastore_revoke_privileges_args::read(::apache::thrift::prot return xfer; } -uint32_t ThriftHiveMetastore_revoke_privileges_args::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_drop_role_args::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_revoke_privileges_args"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_role_args"); - xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->privileges.write(oprot); + xfer += oprot->writeFieldBegin("role_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->role_name); xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); @@ -19280,12 +19638,12 @@ uint32_t ThriftHiveMetastore_revoke_privileges_args::write(::apache::thrift::pro return xfer; } -uint32_t ThriftHiveMetastore_revoke_privileges_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_drop_role_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_revoke_privileges_pargs"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_role_pargs"); - xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->privileges)).write(oprot); + xfer += oprot->writeFieldBegin("role_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->role_name))); xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); @@ -19293,7 +19651,7 @@ uint32_t ThriftHiveMetastore_revoke_privileges_pargs::write(::apache::thrift::pr return xfer; } -uint32_t ThriftHiveMetastore_revoke_privileges_result::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_drop_role_result::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -19341,11 +19699,11 @@ uint32_t ThriftHiveMetastore_revoke_privileges_result::read(::apache::thrift::pr return xfer; } -uint32_t ThriftHiveMetastore_revoke_privileges_result::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_drop_role_result::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_revoke_privileges_result"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_role_result"); if (this->__isset.success) { xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); @@ -19361,7 +19719,7 @@ uint32_t ThriftHiveMetastore_revoke_privileges_result::write(::apache::thrift::p return xfer; } -uint32_t ThriftHiveMetastore_revoke_privileges_presult::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_drop_role_presult::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -19409,7 +19767,7 @@ uint32_t ThriftHiveMetastore_revoke_privileges_presult::read(::apache::thrift::p return xfer; } -uint32_t ThriftHiveMetastore_set_ugi_args::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_get_role_names_args::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -19427,40 +19785,7 @@ uint32_t ThriftHiveMetastore_set_ugi_args::read(::apache::thrift::protocol::TPro if (ftype == ::apache::thrift::protocol::T_STOP) { break; } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->user_name); - this->__isset.user_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->group_names.clear(); - uint32_t _size640; - ::apache::thrift::protocol::TType _etype643; - xfer += iprot->readListBegin(_etype643, _size640); - this->group_names.resize(_size640); - uint32_t _i644; - for (_i644 = 0; _i644 < _size640; ++_i644) - { - xfer += iprot->readString(this->group_names[_i644]); - } - xfer += iprot->readListEnd(); - } - this->__isset.group_names = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } + xfer += iprot->skip(ftype); xfer += iprot->readFieldEnd(); } @@ -19469,57 +19794,25 @@ uint32_t ThriftHiveMetastore_set_ugi_args::read(::apache::thrift::protocol::TPro return xfer; } -uint32_t ThriftHiveMetastore_set_ugi_args::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_get_role_names_args::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_set_ugi_args"); - - xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->user_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 2); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->group_names.size())); - std::vector ::const_iterator _iter645; - for (_iter645 = this->group_names.begin(); _iter645 != this->group_names.end(); ++_iter645) - { - xfer += oprot->writeString((*_iter645)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_role_names_args"); xfer += oprot->writeFieldStop(); xfer += oprot->writeStructEnd(); return xfer; } -uint32_t ThriftHiveMetastore_set_ugi_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_get_role_names_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_set_ugi_pargs"); - - xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->user_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 2); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->group_names)).size())); - std::vector ::const_iterator _iter646; - for (_iter646 = (*(this->group_names)).begin(); _iter646 != (*(this->group_names)).end(); ++_iter646) - { - xfer += oprot->writeString((*_iter646)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_role_names_pargs"); xfer += oprot->writeFieldStop(); xfer += oprot->writeStructEnd(); return xfer; } -uint32_t ThriftHiveMetastore_set_ugi_result::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_get_role_names_result::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -19543,14 +19836,14 @@ uint32_t ThriftHiveMetastore_set_ugi_result::read(::apache::thrift::protocol::TP if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size647; - ::apache::thrift::protocol::TType _etype650; - xfer += iprot->readListBegin(_etype650, _size647); - this->success.resize(_size647); - uint32_t _i651; - for (_i651 = 0; _i651 < _size647; ++_i651) + uint32_t _size610; + ::apache::thrift::protocol::TType _etype613; + xfer += iprot->readListBegin(_etype613, _size610); + this->success.resize(_size610); + uint32_t _i614; + for (_i614 = 0; _i614 < _size610; ++_i614) { - xfer += iprot->readString(this->success[_i651]); + xfer += iprot->readString(this->success[_i614]); } xfer += iprot->readListEnd(); } @@ -19579,20 +19872,20 @@ uint32_t ThriftHiveMetastore_set_ugi_result::read(::apache::thrift::protocol::TP return xfer; } -uint32_t ThriftHiveMetastore_set_ugi_result::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_get_role_names_result::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_set_ugi_result"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_role_names_result"); if (this->__isset.success) { xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter652; - for (_iter652 = this->success.begin(); _iter652 != this->success.end(); ++_iter652) + std::vector ::const_iterator _iter615; + for (_iter615 = this->success.begin(); _iter615 != this->success.end(); ++_iter615) { - xfer += oprot->writeString((*_iter652)); + xfer += oprot->writeString((*_iter615)); } xfer += oprot->writeListEnd(); } @@ -19607,7 +19900,7 @@ uint32_t ThriftHiveMetastore_set_ugi_result::write(::apache::thrift::protocol::T return xfer; } -uint32_t ThriftHiveMetastore_set_ugi_presult::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_get_role_names_presult::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -19631,14 +19924,14 @@ uint32_t ThriftHiveMetastore_set_ugi_presult::read(::apache::thrift::protocol::T if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size653; - ::apache::thrift::protocol::TType _etype656; - xfer += iprot->readListBegin(_etype656, _size653); - (*(this->success)).resize(_size653); - uint32_t _i657; - for (_i657 = 0; _i657 < _size653; ++_i657) + uint32_t _size616; + ::apache::thrift::protocol::TType _etype619; + xfer += iprot->readListBegin(_etype619, _size616); + (*(this->success)).resize(_size616); + uint32_t _i620; + for (_i620 = 0; _i620 < _size616; ++_i620) { - xfer += iprot->readString((*(this->success))[_i657]); + xfer += iprot->readString((*(this->success))[_i620]); } xfer += iprot->readListEnd(); } @@ -19667,7 +19960,7 @@ uint32_t ThriftHiveMetastore_set_ugi_presult::read(::apache::thrift::protocol::T return xfer; } -uint32_t ThriftHiveMetastore_get_delegation_token_args::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_grant_role_args::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -19689,16 +19982,52 @@ uint32_t ThriftHiveMetastore_get_delegation_token_args::read(::apache::thrift::p { case 1: if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->token_owner); - this->__isset.token_owner = true; + xfer += iprot->readString(this->role_name); + this->__isset.role_name = true; } else { xfer += iprot->skip(ftype); } break; case 2: if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->renewer_kerberos_principal_name); - this->__isset.renewer_kerberos_principal_name = true; + xfer += iprot->readString(this->principal_name); + this->__isset.principal_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast621; + xfer += iprot->readI32(ecast621); + this->principal_type = (PrincipalType::type)ecast621; + this->__isset.principal_type = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->grantor); + this->__isset.grantor = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast622; + xfer += iprot->readI32(ecast622); + this->grantorType = (PrincipalType::type)ecast622; + this->__isset.grantorType = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->grant_option); + this->__isset.grant_option = true; } else { xfer += iprot->skip(ftype); } @@ -19715,33 +20044,65 @@ uint32_t ThriftHiveMetastore_get_delegation_token_args::read(::apache::thrift::p return xfer; } -uint32_t ThriftHiveMetastore_get_delegation_token_args::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_grant_role_args::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_delegation_token_args"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_role_args"); - xfer += oprot->writeFieldBegin("token_owner", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->token_owner); + xfer += oprot->writeFieldBegin("role_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->role_name); xfer += oprot->writeFieldEnd(); - xfer += oprot->writeFieldBegin("renewer_kerberos_principal_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->renewer_kerberos_principal_name); + xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->principal_name); xfer += oprot->writeFieldEnd(); - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -uint32_t ThriftHiveMetastore_get_delegation_token_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 3); + xfer += oprot->writeI32((int32_t)this->principal_type); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("grantor", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->grantor); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("grantorType", ::apache::thrift::protocol::T_I32, 5); + xfer += oprot->writeI32((int32_t)this->grantorType); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("grant_option", ::apache::thrift::protocol::T_BOOL, 6); + xfer += oprot->writeBool(this->grant_option); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +uint32_t ThriftHiveMetastore_grant_role_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_delegation_token_pargs"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_role_pargs"); - xfer += oprot->writeFieldBegin("token_owner", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->token_owner))); + xfer += oprot->writeFieldBegin("role_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->role_name))); xfer += oprot->writeFieldEnd(); - xfer += oprot->writeFieldBegin("renewer_kerberos_principal_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->renewer_kerberos_principal_name))); + xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->principal_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 3); + xfer += oprot->writeI32((int32_t)(*(this->principal_type))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("grantor", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString((*(this->grantor))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("grantorType", ::apache::thrift::protocol::T_I32, 5); + xfer += oprot->writeI32((int32_t)(*(this->grantorType))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("grant_option", ::apache::thrift::protocol::T_BOOL, 6); + xfer += oprot->writeBool((*(this->grant_option))); xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); @@ -19749,7 +20110,7 @@ uint32_t ThriftHiveMetastore_get_delegation_token_pargs::write(::apache::thrift: return xfer; } -uint32_t ThriftHiveMetastore_get_delegation_token_result::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_grant_role_result::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -19770,8 +20131,8 @@ uint32_t ThriftHiveMetastore_get_delegation_token_result::read(::apache::thrift: switch (fid) { case 0: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->success); + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); this->__isset.success = true; } else { xfer += iprot->skip(ftype); @@ -19797,15 +20158,15 @@ uint32_t ThriftHiveMetastore_get_delegation_token_result::read(::apache::thrift: return xfer; } -uint32_t ThriftHiveMetastore_get_delegation_token_result::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_grant_role_result::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_delegation_token_result"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_role_result"); if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRING, 0); - xfer += oprot->writeString(this->success); + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); + xfer += oprot->writeBool(this->success); xfer += oprot->writeFieldEnd(); } else if (this->__isset.o1) { xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); @@ -19817,7 +20178,7 @@ uint32_t ThriftHiveMetastore_get_delegation_token_result::write(::apache::thrift return xfer; } -uint32_t ThriftHiveMetastore_get_delegation_token_presult::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_grant_role_presult::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -19838,8 +20199,8 @@ uint32_t ThriftHiveMetastore_get_delegation_token_presult::read(::apache::thrift switch (fid) { case 0: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString((*(this->success))); + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool((*(this->success))); this->__isset.success = true; } else { xfer += iprot->skip(ftype); @@ -19865,7 +20226,7 @@ uint32_t ThriftHiveMetastore_get_delegation_token_presult::read(::apache::thrift return xfer; } -uint32_t ThriftHiveMetastore_renew_delegation_token_args::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_revoke_role_args::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -19887,8 +20248,26 @@ uint32_t ThriftHiveMetastore_renew_delegation_token_args::read(::apache::thrift: { case 1: if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->token_str_form); - this->__isset.token_str_form = true; + xfer += iprot->readString(this->role_name); + this->__isset.role_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->principal_name); + this->__isset.principal_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast623; + xfer += iprot->readI32(ecast623); + this->principal_type = (PrincipalType::type)ecast623; + this->__isset.principal_type = true; } else { xfer += iprot->skip(ftype); } @@ -19905,12 +20284,20 @@ uint32_t ThriftHiveMetastore_renew_delegation_token_args::read(::apache::thrift: return xfer; } -uint32_t ThriftHiveMetastore_renew_delegation_token_args::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_revoke_role_args::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_renew_delegation_token_args"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_revoke_role_args"); - xfer += oprot->writeFieldBegin("token_str_form", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->token_str_form); + xfer += oprot->writeFieldBegin("role_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->role_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->principal_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 3); + xfer += oprot->writeI32((int32_t)this->principal_type); xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); @@ -19918,12 +20305,20 @@ uint32_t ThriftHiveMetastore_renew_delegation_token_args::write(::apache::thrift return xfer; } -uint32_t ThriftHiveMetastore_renew_delegation_token_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_revoke_role_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_renew_delegation_token_pargs"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_revoke_role_pargs"); - xfer += oprot->writeFieldBegin("token_str_form", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->token_str_form))); + xfer += oprot->writeFieldBegin("role_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->role_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->principal_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 3); + xfer += oprot->writeI32((int32_t)(*(this->principal_type))); xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); @@ -19931,7 +20326,7 @@ uint32_t ThriftHiveMetastore_renew_delegation_token_pargs::write(::apache::thrif return xfer; } -uint32_t ThriftHiveMetastore_renew_delegation_token_result::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_revoke_role_result::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -19952,8 +20347,8 @@ uint32_t ThriftHiveMetastore_renew_delegation_token_result::read(::apache::thrif switch (fid) { case 0: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->success); + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); this->__isset.success = true; } else { xfer += iprot->skip(ftype); @@ -19979,15 +20374,15 @@ uint32_t ThriftHiveMetastore_renew_delegation_token_result::read(::apache::thrif return xfer; } -uint32_t ThriftHiveMetastore_renew_delegation_token_result::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_revoke_role_result::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_renew_delegation_token_result"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_revoke_role_result"); if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_I64, 0); - xfer += oprot->writeI64(this->success); + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); + xfer += oprot->writeBool(this->success); xfer += oprot->writeFieldEnd(); } else if (this->__isset.o1) { xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); @@ -19999,7 +20394,7 @@ uint32_t ThriftHiveMetastore_renew_delegation_token_result::write(::apache::thri return xfer; } -uint32_t ThriftHiveMetastore_renew_delegation_token_presult::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_revoke_role_presult::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -20020,8 +20415,8 @@ uint32_t ThriftHiveMetastore_renew_delegation_token_presult::read(::apache::thri switch (fid) { case 0: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64((*(this->success))); + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool((*(this->success))); this->__isset.success = true; } else { xfer += iprot->skip(ftype); @@ -20047,7 +20442,7 @@ uint32_t ThriftHiveMetastore_renew_delegation_token_presult::read(::apache::thri return xfer; } -uint32_t ThriftHiveMetastore_cancel_delegation_token_args::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_list_roles_args::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -20069,8 +20464,18 @@ uint32_t ThriftHiveMetastore_cancel_delegation_token_args::read(::apache::thrift { case 1: if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->token_str_form); - this->__isset.token_str_form = true; + xfer += iprot->readString(this->principal_name); + this->__isset.principal_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast624; + xfer += iprot->readI32(ecast624); + this->principal_type = (PrincipalType::type)ecast624; + this->__isset.principal_type = true; } else { xfer += iprot->skip(ftype); } @@ -20087,12 +20492,16 @@ uint32_t ThriftHiveMetastore_cancel_delegation_token_args::read(::apache::thrift return xfer; } -uint32_t ThriftHiveMetastore_cancel_delegation_token_args::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_list_roles_args::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_cancel_delegation_token_args"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_list_roles_args"); - xfer += oprot->writeFieldBegin("token_str_form", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->token_str_form); + xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->principal_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 2); + xfer += oprot->writeI32((int32_t)this->principal_type); xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); @@ -20100,12 +20509,16 @@ uint32_t ThriftHiveMetastore_cancel_delegation_token_args::write(::apache::thrif return xfer; } -uint32_t ThriftHiveMetastore_cancel_delegation_token_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_list_roles_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_cancel_delegation_token_pargs"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_list_roles_pargs"); - xfer += oprot->writeFieldBegin("token_str_form", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->token_str_form))); + xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->principal_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 2); + xfer += oprot->writeI32((int32_t)(*(this->principal_type))); xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); @@ -20113,7 +20526,7 @@ uint32_t ThriftHiveMetastore_cancel_delegation_token_pargs::write(::apache::thri return xfer; } -uint32_t ThriftHiveMetastore_cancel_delegation_token_result::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_list_roles_result::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -20133,6 +20546,26 @@ uint32_t ThriftHiveMetastore_cancel_delegation_token_result::read(::apache::thri } switch (fid) { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size625; + ::apache::thrift::protocol::TType _etype628; + xfer += iprot->readListBegin(_etype628, _size625); + this->success.resize(_size625); + uint32_t _i629; + for (_i629 = 0; _i629 < _size625; ++_i629) + { + xfer += this->success[_i629].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; case 1: if (ftype == ::apache::thrift::protocol::T_STRUCT) { xfer += this->o1.read(iprot); @@ -20153,13 +20586,25 @@ uint32_t ThriftHiveMetastore_cancel_delegation_token_result::read(::apache::thri return xfer; } -uint32_t ThriftHiveMetastore_cancel_delegation_token_result::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t ThriftHiveMetastore_list_roles_result::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("ThriftHiveMetastore_cancel_delegation_token_result"); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_list_roles_result"); - if (this->__isset.o1) { + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); + std::vector ::const_iterator _iter630; + for (_iter630 = this->success.begin(); _iter630 != this->success.end(); ++_iter630) + { + xfer += (*_iter630).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); xfer += this->o1.write(oprot); xfer += oprot->writeFieldEnd(); @@ -20169,7 +20614,7 @@ uint32_t ThriftHiveMetastore_cancel_delegation_token_result::write(::apache::thr return xfer; } -uint32_t ThriftHiveMetastore_cancel_delegation_token_presult::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t ThriftHiveMetastore_list_roles_presult::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -20189,6 +20634,26 @@ uint32_t ThriftHiveMetastore_cancel_delegation_token_presult::read(::apache::thr } switch (fid) { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size631; + ::apache::thrift::protocol::TType _etype634; + xfer += iprot->readListBegin(_etype634, _size631); + (*(this->success)).resize(_size631); + uint32_t _i635; + for (_i635 = 0; _i635 < _size631; ++_i635) + { + xfer += (*(this->success))[_i635].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; case 1: if (ftype == ::apache::thrift::protocol::T_STRUCT) { xfer += this->o1.read(iprot); @@ -20209,36 +20674,2061 @@ uint32_t ThriftHiveMetastore_cancel_delegation_token_presult::read(::apache::thr return xfer; } -void ThriftHiveMetastoreClient::create_database(const Database& database) -{ - send_create_database(database); - recv_create_database(); -} - -void ThriftHiveMetastoreClient::send_create_database(const Database& database) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("create_database", ::apache::thrift::protocol::T_CALL, cseqid); +uint32_t ThriftHiveMetastore_get_privilege_set_args::read(::apache::thrift::protocol::TProtocol* iprot) { - ThriftHiveMetastore_create_database_pargs args; - args.database = &database; - args.write(oprot_); + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} + xfer += iprot->readStructBegin(fname); -void ThriftHiveMetastoreClient::recv_create_database() -{ + using ::apache::thrift::protocol::TProtocolException; - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->hiveObject.read(iprot); + this->__isset.hiveObject = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->user_name); + this->__isset.user_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->group_names.clear(); + uint32_t _size636; + ::apache::thrift::protocol::TType _etype639; + xfer += iprot->readListBegin(_etype639, _size636); + this->group_names.resize(_size636); + uint32_t _i640; + for (_i640 = 0; _i640 < _size636; ++_i640) + { + xfer += iprot->readString(this->group_names[_i640]); + } + xfer += iprot->readListEnd(); + } + this->__isset.group_names = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_privilege_set_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_privilege_set_args"); + + xfer += oprot->writeFieldBegin("hiveObject", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->hiveObject.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->user_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->group_names.size())); + std::vector ::const_iterator _iter641; + for (_iter641 = this->group_names.begin(); _iter641 != this->group_names.end(); ++_iter641) + { + xfer += oprot->writeString((*_iter641)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +uint32_t ThriftHiveMetastore_get_privilege_set_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_privilege_set_pargs"); + + xfer += oprot->writeFieldBegin("hiveObject", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->hiveObject)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->user_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->group_names)).size())); + std::vector ::const_iterator _iter642; + for (_iter642 = (*(this->group_names)).begin(); _iter642 != (*(this->group_names)).end(); ++_iter642) + { + xfer += oprot->writeString((*_iter642)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +uint32_t ThriftHiveMetastore_get_privilege_set_result::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_privilege_set_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_privilege_set_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +uint32_t ThriftHiveMetastore_get_privilege_set_presult::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_list_privileges_args::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->principal_name); + this->__isset.principal_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast643; + xfer += iprot->readI32(ecast643); + this->principal_type = (PrincipalType::type)ecast643; + this->__isset.principal_type = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->hiveObject.read(iprot); + this->__isset.hiveObject = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_list_privileges_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + xfer += oprot->writeStructBegin("ThriftHiveMetastore_list_privileges_args"); + + xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->principal_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 2); + xfer += oprot->writeI32((int32_t)this->principal_type); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("hiveObject", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->hiveObject.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +uint32_t ThriftHiveMetastore_list_privileges_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + xfer += oprot->writeStructBegin("ThriftHiveMetastore_list_privileges_pargs"); + + xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->principal_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 2); + xfer += oprot->writeI32((int32_t)(*(this->principal_type))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("hiveObject", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += (*(this->hiveObject)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +uint32_t ThriftHiveMetastore_list_privileges_result::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size644; + ::apache::thrift::protocol::TType _etype647; + xfer += iprot->readListBegin(_etype647, _size644); + this->success.resize(_size644); + uint32_t _i648; + for (_i648 = 0; _i648 < _size644; ++_i648) + { + xfer += this->success[_i648].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_list_privileges_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_list_privileges_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); + std::vector ::const_iterator _iter649; + for (_iter649 = this->success.begin(); _iter649 != this->success.end(); ++_iter649) + { + xfer += (*_iter649).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +uint32_t ThriftHiveMetastore_list_privileges_presult::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size650; + ::apache::thrift::protocol::TType _etype653; + xfer += iprot->readListBegin(_etype653, _size650); + (*(this->success)).resize(_size650); + uint32_t _i654; + for (_i654 = 0; _i654 < _size650; ++_i654) + { + xfer += (*(this->success))[_i654].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_grant_privileges_args::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->privileges.read(iprot); + this->__isset.privileges = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_grant_privileges_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_privileges_args"); + + xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->privileges.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +uint32_t ThriftHiveMetastore_grant_privileges_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_privileges_pargs"); + + xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->privileges)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +uint32_t ThriftHiveMetastore_grant_privileges_result::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_grant_privileges_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_privileges_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); + xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +uint32_t ThriftHiveMetastore_grant_privileges_presult::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_revoke_privileges_args::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->privileges.read(iprot); + this->__isset.privileges = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_revoke_privileges_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + xfer += oprot->writeStructBegin("ThriftHiveMetastore_revoke_privileges_args"); + + xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->privileges.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +uint32_t ThriftHiveMetastore_revoke_privileges_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + xfer += oprot->writeStructBegin("ThriftHiveMetastore_revoke_privileges_pargs"); + + xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->privileges)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +uint32_t ThriftHiveMetastore_revoke_privileges_result::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_revoke_privileges_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_revoke_privileges_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); + xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +uint32_t ThriftHiveMetastore_revoke_privileges_presult::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_set_ugi_args::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->user_name); + this->__isset.user_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->group_names.clear(); + uint32_t _size655; + ::apache::thrift::protocol::TType _etype658; + xfer += iprot->readListBegin(_etype658, _size655); + this->group_names.resize(_size655); + uint32_t _i659; + for (_i659 = 0; _i659 < _size655; ++_i659) + { + xfer += iprot->readString(this->group_names[_i659]); + } + xfer += iprot->readListEnd(); + } + this->__isset.group_names = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_set_ugi_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + xfer += oprot->writeStructBegin("ThriftHiveMetastore_set_ugi_args"); + + xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->user_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 2); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->group_names.size())); + std::vector ::const_iterator _iter660; + for (_iter660 = this->group_names.begin(); _iter660 != this->group_names.end(); ++_iter660) + { + xfer += oprot->writeString((*_iter660)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +uint32_t ThriftHiveMetastore_set_ugi_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + xfer += oprot->writeStructBegin("ThriftHiveMetastore_set_ugi_pargs"); + + xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->user_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 2); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->group_names)).size())); + std::vector ::const_iterator _iter661; + for (_iter661 = (*(this->group_names)).begin(); _iter661 != (*(this->group_names)).end(); ++_iter661) + { + xfer += oprot->writeString((*_iter661)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +uint32_t ThriftHiveMetastore_set_ugi_result::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size662; + ::apache::thrift::protocol::TType _etype665; + xfer += iprot->readListBegin(_etype665, _size662); + this->success.resize(_size662); + uint32_t _i666; + for (_i666 = 0; _i666 < _size662; ++_i666) + { + xfer += iprot->readString(this->success[_i666]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_set_ugi_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_set_ugi_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); + std::vector ::const_iterator _iter667; + for (_iter667 = this->success.begin(); _iter667 != this->success.end(); ++_iter667) + { + xfer += oprot->writeString((*_iter667)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +uint32_t ThriftHiveMetastore_set_ugi_presult::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size668; + ::apache::thrift::protocol::TType _etype671; + xfer += iprot->readListBegin(_etype671, _size668); + (*(this->success)).resize(_size668); + uint32_t _i672; + for (_i672 = 0; _i672 < _size668; ++_i672) + { + xfer += iprot->readString((*(this->success))[_i672]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_delegation_token_args::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->token_owner); + this->__isset.token_owner = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->renewer_kerberos_principal_name); + this->__isset.renewer_kerberos_principal_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_delegation_token_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_delegation_token_args"); + + xfer += oprot->writeFieldBegin("token_owner", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->token_owner); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("renewer_kerberos_principal_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->renewer_kerberos_principal_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +uint32_t ThriftHiveMetastore_get_delegation_token_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_delegation_token_pargs"); + + xfer += oprot->writeFieldBegin("token_owner", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->token_owner))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("renewer_kerberos_principal_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->renewer_kerberos_principal_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +uint32_t ThriftHiveMetastore_get_delegation_token_result::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_delegation_token_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_delegation_token_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRING, 0); + xfer += oprot->writeString(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +uint32_t ThriftHiveMetastore_get_delegation_token_presult::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_renew_delegation_token_args::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->token_str_form); + this->__isset.token_str_form = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_renew_delegation_token_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + xfer += oprot->writeStructBegin("ThriftHiveMetastore_renew_delegation_token_args"); + + xfer += oprot->writeFieldBegin("token_str_form", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->token_str_form); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +uint32_t ThriftHiveMetastore_renew_delegation_token_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + xfer += oprot->writeStructBegin("ThriftHiveMetastore_renew_delegation_token_pargs"); + + xfer += oprot->writeFieldBegin("token_str_form", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->token_str_form))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +uint32_t ThriftHiveMetastore_renew_delegation_token_result::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_renew_delegation_token_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_renew_delegation_token_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_I64, 0); + xfer += oprot->writeI64(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +uint32_t ThriftHiveMetastore_renew_delegation_token_presult::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_cancel_delegation_token_args::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->token_str_form); + this->__isset.token_str_form = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_cancel_delegation_token_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + xfer += oprot->writeStructBegin("ThriftHiveMetastore_cancel_delegation_token_args"); + + xfer += oprot->writeFieldBegin("token_str_form", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->token_str_form); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +uint32_t ThriftHiveMetastore_cancel_delegation_token_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + xfer += oprot->writeStructBegin("ThriftHiveMetastore_cancel_delegation_token_pargs"); + + xfer += oprot->writeFieldBegin("token_str_form", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->token_str_form))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +uint32_t ThriftHiveMetastore_cancel_delegation_token_result::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_cancel_delegation_token_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_cancel_delegation_token_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +uint32_t ThriftHiveMetastore_cancel_delegation_token_presult::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +void ThriftHiveMetastoreClient::create_database(const Database& database) +{ + send_create_database(database); + recv_create_database(); +} + +void ThriftHiveMetastoreClient::send_create_database(const Database& database) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("create_database", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_create_database_pargs args; + args.database = &database; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_create_database() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("create_database") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_create_database_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + return; +} + +void ThriftHiveMetastoreClient::get_database(Database& _return, const std::string& name) +{ + send_get_database(name); + recv_get_database(_return); +} + +void ThriftHiveMetastoreClient::send_get_database(const std::string& name) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_database", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_database_pargs args; + args.name = &name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_database(Database& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_database") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_database_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_database failed: unknown result"); +} + +void ThriftHiveMetastoreClient::drop_database(const std::string& name, const bool deleteData, const bool cascade) +{ + send_drop_database(name, deleteData, cascade); + recv_drop_database(); +} + +void ThriftHiveMetastoreClient::send_drop_database(const std::string& name, const bool deleteData, const bool cascade) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("drop_database", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_database_pargs args; + args.name = &name; + args.deleteData = &deleteData; + args.cascade = &cascade; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_drop_database() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_database") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_drop_database_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + return; +} + +void ThriftHiveMetastoreClient::get_databases(std::vector & _return, const std::string& pattern) +{ + send_get_databases(pattern); + recv_get_databases(_return); +} + +void ThriftHiveMetastoreClient::send_get_databases(const std::string& pattern) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_databases", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_databases_pargs args; + args.pattern = &pattern; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_databases(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_databases") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_databases_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_databases failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_all_databases(std::vector & _return) +{ + send_get_all_databases(); + recv_get_all_databases(_return); +} + +void ThriftHiveMetastoreClient::send_get_all_databases() +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_all_databases", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_all_databases_pargs args; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_all_databases(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_all_databases") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_all_databases_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_all_databases failed: unknown result"); +} + +void ThriftHiveMetastoreClient::alter_database(const std::string& dbname, const Database& db) +{ + send_alter_database(dbname, db); + recv_alter_database(); +} + +void ThriftHiveMetastoreClient::send_alter_database(const std::string& dbname, const Database& db) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("alter_database", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_alter_database_pargs args; + args.dbname = &dbname; + args.db = &db; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_alter_database() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("alter_database") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_alter_database_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + return; +} + +void ThriftHiveMetastoreClient::get_type(Type& _return, const std::string& name) +{ + send_get_type(name); + recv_get_type(_return); +} + +void ThriftHiveMetastoreClient::send_get_type(const std::string& name) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_type", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_type_pargs args; + args.name = &name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_type(Type& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; x.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); @@ -20249,16 +22739,85 @@ void ThriftHiveMetastoreClient::recv_create_database() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("create_database") != 0) { + if (fname.compare("get_type") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_create_database_presult result; + ThriftHiveMetastore_get_type_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_type failed: unknown result"); +} + +bool ThriftHiveMetastoreClient::create_type(const Type& type) +{ + send_create_type(type); + return recv_create_type(); +} + +void ThriftHiveMetastoreClient::send_create_type(const Type& type) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("create_type", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_create_type_pargs args; + args.type = &type; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +bool ThriftHiveMetastoreClient::recv_create_type() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("create_type") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + bool _return; + ThriftHiveMetastore_create_type_presult result; + result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); + if (result.__isset.success) { + return _return; + } if (result.__isset.o1) { throw result.o1; } @@ -20268,22 +22827,22 @@ void ThriftHiveMetastoreClient::recv_create_database() if (result.__isset.o3) { throw result.o3; } - return; + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "create_type failed: unknown result"); } -void ThriftHiveMetastoreClient::get_database(Database& _return, const std::string& name) +bool ThriftHiveMetastoreClient::drop_type(const std::string& type) { - send_get_database(name); - recv_get_database(_return); + send_drop_type(type); + return recv_drop_type(); } -void ThriftHiveMetastoreClient::send_get_database(const std::string& name) +void ThriftHiveMetastoreClient::send_drop_type(const std::string& type) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_database", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("drop_type", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_database_pargs args; - args.name = &name; + ThriftHiveMetastore_drop_type_pargs args; + args.type = &type; args.write(oprot_); oprot_->writeMessageEnd(); @@ -20291,7 +22850,7 @@ void ThriftHiveMetastoreClient::send_get_database(const std::string& name) oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_database(Database& _return) +bool ThriftHiveMetastoreClient::recv_drop_type() { int32_t rseqid = 0; @@ -20311,20 +22870,20 @@ void ThriftHiveMetastoreClient::recv_get_database(Database& _return) iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_database") != 0) { + if (fname.compare("drop_type") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_database_presult result; + bool _return; + ThriftHiveMetastore_drop_type_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); if (result.__isset.success) { - // _return pointer has now been filled - return; + return _return; } if (result.__isset.o1) { throw result.o1; @@ -20332,24 +22891,22 @@ void ThriftHiveMetastoreClient::recv_get_database(Database& _return) if (result.__isset.o2) { throw result.o2; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_database failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_type failed: unknown result"); } -void ThriftHiveMetastoreClient::drop_database(const std::string& name, const bool deleteData, const bool cascade) +void ThriftHiveMetastoreClient::get_type_all(std::map & _return, const std::string& name) { - send_drop_database(name, deleteData, cascade); - recv_drop_database(); + send_get_type_all(name); + recv_get_type_all(_return); } -void ThriftHiveMetastoreClient::send_drop_database(const std::string& name, const bool deleteData, const bool cascade) +void ThriftHiveMetastoreClient::send_get_type_all(const std::string& name) { int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_database", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_type_all", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_drop_database_pargs args; + ThriftHiveMetastore_get_type_all_pargs args; args.name = &name; - args.deleteData = &deleteData; - args.cascade = &cascade; args.write(oprot_); oprot_->writeMessageEnd(); @@ -20357,7 +22914,7 @@ void ThriftHiveMetastoreClient::send_drop_database(const std::string& name, cons oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_drop_database() +void ThriftHiveMetastoreClient::recv_get_type_all(std::map & _return) { int32_t rseqid = 0; @@ -20377,16 +22934,83 @@ void ThriftHiveMetastoreClient::recv_drop_database() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("drop_database") != 0) { + if (fname.compare("get_type_all") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_drop_database_presult result; + ThriftHiveMetastore_get_type_all_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_type_all failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_fields(std::vector & _return, const std::string& db_name, const std::string& table_name) +{ + send_get_fields(db_name, table_name); + recv_get_fields(_return); +} + +void ThriftHiveMetastoreClient::send_get_fields(const std::string& db_name, const std::string& table_name) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_fields", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_fields_pargs args; + args.db_name = &db_name; + args.table_name = &table_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_fields(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_fields") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_fields_presult result; + result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); + if (result.__isset.success) { + // _return pointer has now been filled + return; + } if (result.__isset.o1) { throw result.o1; } @@ -20396,22 +23020,23 @@ void ThriftHiveMetastoreClient::recv_drop_database() if (result.__isset.o3) { throw result.o3; } - return; + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_fields failed: unknown result"); } -void ThriftHiveMetastoreClient::get_databases(std::vector & _return, const std::string& pattern) +void ThriftHiveMetastoreClient::get_schema(std::vector & _return, const std::string& db_name, const std::string& table_name) { - send_get_databases(pattern); - recv_get_databases(_return); + send_get_schema(db_name, table_name); + recv_get_schema(_return); } -void ThriftHiveMetastoreClient::send_get_databases(const std::string& pattern) +void ThriftHiveMetastoreClient::send_get_schema(const std::string& db_name, const std::string& table_name) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_databases", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_schema", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_databases_pargs args; - args.pattern = &pattern; + ThriftHiveMetastore_get_schema_pargs args; + args.db_name = &db_name; + args.table_name = &table_name; args.write(oprot_); oprot_->writeMessageEnd(); @@ -20419,7 +23044,7 @@ void ThriftHiveMetastoreClient::send_get_databases(const std::string& pattern) oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_databases(std::vector & _return) +void ThriftHiveMetastoreClient::recv_get_schema(std::vector & _return) { int32_t rseqid = 0; @@ -20439,12 +23064,12 @@ void ThriftHiveMetastoreClient::recv_get_databases(std::vector & _r iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_databases") != 0) { + if (fname.compare("get_schema") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_databases_presult result; + ThriftHiveMetastore_get_schema_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -20457,21 +23082,28 @@ void ThriftHiveMetastoreClient::recv_get_databases(std::vector & _r if (result.__isset.o1) { throw result.o1; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_databases failed: unknown result"); + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_schema failed: unknown result"); } -void ThriftHiveMetastoreClient::get_all_databases(std::vector & _return) +void ThriftHiveMetastoreClient::create_table(const Table& tbl) { - send_get_all_databases(); - recv_get_all_databases(_return); + send_create_table(tbl); + recv_create_table(); } -void ThriftHiveMetastoreClient::send_get_all_databases() +void ThriftHiveMetastoreClient::send_create_table(const Table& tbl) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_all_databases", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("create_table", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_all_databases_pargs args; + ThriftHiveMetastore_create_table_pargs args; + args.tbl = &tbl; args.write(oprot_); oprot_->writeMessageEnd(); @@ -20479,7 +23111,7 @@ void ThriftHiveMetastoreClient::send_get_all_databases() oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_all_databases(std::vector & _return) +void ThriftHiveMetastoreClient::recv_create_table() { int32_t rseqid = 0; @@ -20499,41 +23131,45 @@ void ThriftHiveMetastoreClient::recv_get_all_databases(std::vector iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_all_databases") != 0) { + if (fname.compare("create_table") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_all_databases_presult result; - result.success = &_return; + ThriftHiveMetastore_create_table_presult result; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); - if (result.__isset.success) { - // _return pointer has now been filled - return; - } if (result.__isset.o1) { throw result.o1; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_all_databases failed: unknown result"); + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + if (result.__isset.o4) { + throw result.o4; + } + return; } -void ThriftHiveMetastoreClient::alter_database(const std::string& dbname, const Database& db) +void ThriftHiveMetastoreClient::create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context) { - send_alter_database(dbname, db); - recv_alter_database(); + send_create_table_with_environment_context(tbl, environment_context); + recv_create_table_with_environment_context(); } -void ThriftHiveMetastoreClient::send_alter_database(const std::string& dbname, const Database& db) +void ThriftHiveMetastoreClient::send_create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context) { int32_t cseqid = 0; - oprot_->writeMessageBegin("alter_database", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("create_table_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_alter_database_pargs args; - args.dbname = &dbname; - args.db = &db; + ThriftHiveMetastore_create_table_with_environment_context_pargs args; + args.tbl = &tbl; + args.environment_context = &environment_context; args.write(oprot_); oprot_->writeMessageEnd(); @@ -20541,7 +23177,7 @@ void ThriftHiveMetastoreClient::send_alter_database(const std::string& dbname, c oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_alter_database() +void ThriftHiveMetastoreClient::recv_create_table_with_environment_context() { int32_t rseqid = 0; @@ -20561,12 +23197,12 @@ void ThriftHiveMetastoreClient::recv_alter_database() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("alter_database") != 0) { + if (fname.compare("create_table_with_environment_context") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_alter_database_presult result; + ThriftHiveMetastore_create_table_with_environment_context_presult result; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); @@ -20577,22 +23213,30 @@ void ThriftHiveMetastoreClient::recv_alter_database() if (result.__isset.o2) { throw result.o2; } + if (result.__isset.o3) { + throw result.o3; + } + if (result.__isset.o4) { + throw result.o4; + } return; } -void ThriftHiveMetastoreClient::get_type(Type& _return, const std::string& name) +void ThriftHiveMetastoreClient::drop_table(const std::string& dbname, const std::string& name, const bool deleteData) { - send_get_type(name); - recv_get_type(_return); + send_drop_table(dbname, name, deleteData); + recv_drop_table(); } -void ThriftHiveMetastoreClient::send_get_type(const std::string& name) +void ThriftHiveMetastoreClient::send_drop_table(const std::string& dbname, const std::string& name, const bool deleteData) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_type", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("drop_table", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_type_pargs args; + ThriftHiveMetastore_drop_table_pargs args; + args.dbname = &dbname; args.name = &name; + args.deleteData = &deleteData; args.write(oprot_); oprot_->writeMessageEnd(); @@ -20600,7 +23244,7 @@ void ThriftHiveMetastoreClient::send_get_type(const std::string& name) oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_type(Type& _return) +void ThriftHiveMetastoreClient::recv_drop_table() { int32_t rseqid = 0; @@ -20620,43 +23264,41 @@ void ThriftHiveMetastoreClient::recv_get_type(Type& _return) iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_type") != 0) { + if (fname.compare("drop_table") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_type_presult result; - result.success = &_return; + ThriftHiveMetastore_drop_table_presult result; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); - if (result.__isset.success) { - // _return pointer has now been filled - return; - } if (result.__isset.o1) { throw result.o1; } - if (result.__isset.o2) { - throw result.o2; + if (result.__isset.o3) { + throw result.o3; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_type failed: unknown result"); + return; } -bool ThriftHiveMetastoreClient::create_type(const Type& type) +void ThriftHiveMetastoreClient::drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context) { - send_create_type(type); - return recv_create_type(); + send_drop_table_with_environment_context(dbname, name, deleteData, environment_context); + recv_drop_table_with_environment_context(); } -void ThriftHiveMetastoreClient::send_create_type(const Type& type) +void ThriftHiveMetastoreClient::send_drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context) { int32_t cseqid = 0; - oprot_->writeMessageBegin("create_type", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("drop_table_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_create_type_pargs args; - args.type = &type; + ThriftHiveMetastore_drop_table_with_environment_context_pargs args; + args.dbname = &dbname; + args.name = &name; + args.deleteData = &deleteData; + args.environment_context = &environment_context; args.write(oprot_); oprot_->writeMessageEnd(); @@ -20664,7 +23306,7 @@ void ThriftHiveMetastoreClient::send_create_type(const Type& type) oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_create_type() +void ThriftHiveMetastoreClient::recv_drop_table_with_environment_context() { int32_t rseqid = 0; @@ -20684,46 +23326,39 @@ bool ThriftHiveMetastoreClient::recv_create_type() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("create_type") != 0) { + if (fname.compare("drop_table_with_environment_context") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - bool _return; - ThriftHiveMetastore_create_type_presult result; - result.success = &_return; + ThriftHiveMetastore_drop_table_with_environment_context_presult result; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); - if (result.__isset.success) { - return _return; - } if (result.__isset.o1) { throw result.o1; } - if (result.__isset.o2) { - throw result.o2; - } if (result.__isset.o3) { throw result.o3; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "create_type failed: unknown result"); + return; } -bool ThriftHiveMetastoreClient::drop_type(const std::string& type) +void ThriftHiveMetastoreClient::get_tables(std::vector & _return, const std::string& db_name, const std::string& pattern) { - send_drop_type(type); - return recv_drop_type(); + send_get_tables(db_name, pattern); + recv_get_tables(_return); } -void ThriftHiveMetastoreClient::send_drop_type(const std::string& type) +void ThriftHiveMetastoreClient::send_get_tables(const std::string& db_name, const std::string& pattern) { int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_type", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_tables", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_drop_type_pargs args; - args.type = &type; + ThriftHiveMetastore_get_tables_pargs args; + args.db_name = &db_name; + args.pattern = &pattern; args.write(oprot_); oprot_->writeMessageEnd(); @@ -20731,7 +23366,7 @@ void ThriftHiveMetastoreClient::send_drop_type(const std::string& type) oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_drop_type() +void ThriftHiveMetastoreClient::recv_get_tables(std::vector & _return) { int32_t rseqid = 0; @@ -20751,43 +23386,40 @@ bool ThriftHiveMetastoreClient::recv_drop_type() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("drop_type") != 0) { + if (fname.compare("get_tables") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - bool _return; - ThriftHiveMetastore_drop_type_presult result; + ThriftHiveMetastore_get_tables_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); if (result.__isset.success) { - return _return; + // _return pointer has now been filled + return; } if (result.__isset.o1) { throw result.o1; } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_type failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_tables failed: unknown result"); } -void ThriftHiveMetastoreClient::get_type_all(std::map & _return, const std::string& name) +void ThriftHiveMetastoreClient::get_all_tables(std::vector & _return, const std::string& db_name) { - send_get_type_all(name); - recv_get_type_all(_return); + send_get_all_tables(db_name); + recv_get_all_tables(_return); } -void ThriftHiveMetastoreClient::send_get_type_all(const std::string& name) +void ThriftHiveMetastoreClient::send_get_all_tables(const std::string& db_name) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_type_all", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_all_tables", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_type_all_pargs args; - args.name = &name; + ThriftHiveMetastore_get_all_tables_pargs args; + args.db_name = &db_name; args.write(oprot_); oprot_->writeMessageEnd(); @@ -20795,7 +23427,7 @@ void ThriftHiveMetastoreClient::send_get_type_all(const std::string& name) oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_type_all(std::map & _return) +void ThriftHiveMetastoreClient::recv_get_all_tables(std::vector & _return) { int32_t rseqid = 0; @@ -20815,12 +23447,12 @@ void ThriftHiveMetastoreClient::recv_get_type_all(std::map & iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_type_all") != 0) { + if (fname.compare("get_all_tables") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_type_all_presult result; + ThriftHiveMetastore_get_all_tables_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -20830,26 +23462,26 @@ void ThriftHiveMetastoreClient::recv_get_type_all(std::map & // _return pointer has now been filled return; } - if (result.__isset.o2) { - throw result.o2; + if (result.__isset.o1) { + throw result.o1; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_type_all failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_all_tables failed: unknown result"); } -void ThriftHiveMetastoreClient::get_fields(std::vector & _return, const std::string& db_name, const std::string& table_name) +void ThriftHiveMetastoreClient::get_table(Table& _return, const std::string& dbname, const std::string& tbl_name) { - send_get_fields(db_name, table_name); - recv_get_fields(_return); + send_get_table(dbname, tbl_name); + recv_get_table(_return); } -void ThriftHiveMetastoreClient::send_get_fields(const std::string& db_name, const std::string& table_name) +void ThriftHiveMetastoreClient::send_get_table(const std::string& dbname, const std::string& tbl_name) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_fields", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_table", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_fields_pargs args; - args.db_name = &db_name; - args.table_name = &table_name; + ThriftHiveMetastore_get_table_pargs args; + args.dbname = &dbname; + args.tbl_name = &tbl_name; args.write(oprot_); oprot_->writeMessageEnd(); @@ -20857,7 +23489,7 @@ void ThriftHiveMetastoreClient::send_get_fields(const std::string& db_name, cons oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_fields(std::vector & _return) +void ThriftHiveMetastoreClient::recv_get_table(Table& _return) { int32_t rseqid = 0; @@ -20877,12 +23509,12 @@ void ThriftHiveMetastoreClient::recv_get_fields(std::vector & _retu iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_fields") != 0) { + if (fname.compare("get_table") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_fields_presult result; + ThriftHiveMetastore_get_table_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -20898,26 +23530,23 @@ void ThriftHiveMetastoreClient::recv_get_fields(std::vector & _retu if (result.__isset.o2) { throw result.o2; } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_fields failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table failed: unknown result"); } -void ThriftHiveMetastoreClient::get_schema(std::vector & _return, const std::string& db_name, const std::string& table_name) +void ThriftHiveMetastoreClient::get_table_objects_by_name(std::vector & _return, const std::string& dbname, const std::vector & tbl_names) { - send_get_schema(db_name, table_name); - recv_get_schema(_return); + send_get_table_objects_by_name(dbname, tbl_names); + recv_get_table_objects_by_name(_return); } -void ThriftHiveMetastoreClient::send_get_schema(const std::string& db_name, const std::string& table_name) +void ThriftHiveMetastoreClient::send_get_table_objects_by_name(const std::string& dbname, const std::vector & tbl_names) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_schema", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_table_objects_by_name", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_schema_pargs args; - args.db_name = &db_name; - args.table_name = &table_name; + ThriftHiveMetastore_get_table_objects_by_name_pargs args; + args.dbname = &dbname; + args.tbl_names = &tbl_names; args.write(oprot_); oprot_->writeMessageEnd(); @@ -20925,7 +23554,7 @@ void ThriftHiveMetastoreClient::send_get_schema(const std::string& db_name, cons oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_schema(std::vector & _return) +void ThriftHiveMetastoreClient::recv_get_table_objects_by_name(std::vector
& _return) { int32_t rseqid = 0; @@ -20945,12 +23574,12 @@ void ThriftHiveMetastoreClient::recv_get_schema(std::vector & _retu iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_schema") != 0) { + if (fname.compare("get_table_objects_by_name") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_schema_presult result; + ThriftHiveMetastore_get_table_objects_by_name_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -20969,22 +23598,24 @@ void ThriftHiveMetastoreClient::recv_get_schema(std::vector & _retu if (result.__isset.o3) { throw result.o3; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_schema failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_objects_by_name failed: unknown result"); } -void ThriftHiveMetastoreClient::create_table(const Table& tbl) +void ThriftHiveMetastoreClient::get_table_names_by_filter(std::vector & _return, const std::string& dbname, const std::string& filter, const int16_t max_tables) { - send_create_table(tbl); - recv_create_table(); + send_get_table_names_by_filter(dbname, filter, max_tables); + recv_get_table_names_by_filter(_return); } -void ThriftHiveMetastoreClient::send_create_table(const Table& tbl) +void ThriftHiveMetastoreClient::send_get_table_names_by_filter(const std::string& dbname, const std::string& filter, const int16_t max_tables) { int32_t cseqid = 0; - oprot_->writeMessageBegin("create_table", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_table_names_by_filter", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_create_table_pargs args; - args.tbl = &tbl; + ThriftHiveMetastore_get_table_names_by_filter_pargs args; + args.dbname = &dbname; + args.filter = &filter; + args.max_tables = &max_tables; args.write(oprot_); oprot_->writeMessageEnd(); @@ -20992,7 +23623,7 @@ void ThriftHiveMetastoreClient::send_create_table(const Table& tbl) oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_create_table() +void ThriftHiveMetastoreClient::recv_get_table_names_by_filter(std::vector & _return) { int32_t rseqid = 0; @@ -21012,16 +23643,21 @@ void ThriftHiveMetastoreClient::recv_create_table() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("create_table") != 0) { + if (fname.compare("get_table_names_by_filter") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_create_table_presult result; + ThriftHiveMetastore_get_table_names_by_filter_presult result; + result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); + if (result.__isset.success) { + // _return pointer has now been filled + return; + } if (result.__isset.o1) { throw result.o1; } @@ -21031,26 +23667,24 @@ void ThriftHiveMetastoreClient::recv_create_table() if (result.__isset.o3) { throw result.o3; } - if (result.__isset.o4) { - throw result.o4; - } - return; + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_names_by_filter failed: unknown result"); } -void ThriftHiveMetastoreClient::create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context) +void ThriftHiveMetastoreClient::alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl) { - send_create_table_with_environment_context(tbl, environment_context); - recv_create_table_with_environment_context(); + send_alter_table(dbname, tbl_name, new_tbl); + recv_alter_table(); } -void ThriftHiveMetastoreClient::send_create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context) +void ThriftHiveMetastoreClient::send_alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl) { int32_t cseqid = 0; - oprot_->writeMessageBegin("create_table_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("alter_table", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_create_table_with_environment_context_pargs args; - args.tbl = &tbl; - args.environment_context = &environment_context; + ThriftHiveMetastore_alter_table_pargs args; + args.dbname = &dbname; + args.tbl_name = &tbl_name; + args.new_tbl = &new_tbl; args.write(oprot_); oprot_->writeMessageEnd(); @@ -21058,7 +23692,7 @@ void ThriftHiveMetastoreClient::send_create_table_with_environment_context(const oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_create_table_with_environment_context() +void ThriftHiveMetastoreClient::recv_alter_table() { int32_t rseqid = 0; @@ -21078,12 +23712,12 @@ void ThriftHiveMetastoreClient::recv_create_table_with_environment_context() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("create_table_with_environment_context") != 0) { + if (fname.compare("alter_table") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_create_table_with_environment_context_presult result; + ThriftHiveMetastore_alter_table_presult result; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); @@ -21094,30 +23728,25 @@ void ThriftHiveMetastoreClient::recv_create_table_with_environment_context() if (result.__isset.o2) { throw result.o2; } - if (result.__isset.o3) { - throw result.o3; - } - if (result.__isset.o4) { - throw result.o4; - } return; } -void ThriftHiveMetastoreClient::drop_table(const std::string& dbname, const std::string& name, const bool deleteData) +void ThriftHiveMetastoreClient::alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context) { - send_drop_table(dbname, name, deleteData); - recv_drop_table(); + send_alter_table_with_environment_context(dbname, tbl_name, new_tbl, environment_context); + recv_alter_table_with_environment_context(); } -void ThriftHiveMetastoreClient::send_drop_table(const std::string& dbname, const std::string& name, const bool deleteData) +void ThriftHiveMetastoreClient::send_alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context) { int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_table", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("alter_table_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_drop_table_pargs args; + ThriftHiveMetastore_alter_table_with_environment_context_pargs args; args.dbname = &dbname; - args.name = &name; - args.deleteData = &deleteData; + args.tbl_name = &tbl_name; + args.new_tbl = &new_tbl; + args.environment_context = &environment_context; args.write(oprot_); oprot_->writeMessageEnd(); @@ -21125,7 +23754,7 @@ void ThriftHiveMetastoreClient::send_drop_table(const std::string& dbname, const oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_drop_table() +void ThriftHiveMetastoreClient::recv_alter_table_with_environment_context() { int32_t rseqid = 0; @@ -21145,12 +23774,12 @@ void ThriftHiveMetastoreClient::recv_drop_table() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("drop_table") != 0) { + if (fname.compare("alter_table_with_environment_context") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_drop_table_presult result; + ThriftHiveMetastore_alter_table_with_environment_context_presult result; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); @@ -21158,28 +23787,25 @@ void ThriftHiveMetastoreClient::recv_drop_table() if (result.__isset.o1) { throw result.o1; } - if (result.__isset.o3) { - throw result.o3; + if (result.__isset.o2) { + throw result.o2; } return; } -void ThriftHiveMetastoreClient::drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context) +void ThriftHiveMetastoreClient::add_partition(Partition& _return, const Partition& new_part) { - send_drop_table_with_environment_context(dbname, name, deleteData, environment_context); - recv_drop_table_with_environment_context(); + send_add_partition(new_part); + recv_add_partition(_return); } -void ThriftHiveMetastoreClient::send_drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context) +void ThriftHiveMetastoreClient::send_add_partition(const Partition& new_part) { int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_table_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("add_partition", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_drop_table_with_environment_context_pargs args; - args.dbname = &dbname; - args.name = &name; - args.deleteData = &deleteData; - args.environment_context = &environment_context; + ThriftHiveMetastore_add_partition_pargs args; + args.new_part = &new_part; args.write(oprot_); oprot_->writeMessageEnd(); @@ -21187,7 +23813,7 @@ void ThriftHiveMetastoreClient::send_drop_table_with_environment_context(const s oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_drop_table_with_environment_context() +void ThriftHiveMetastoreClient::recv_add_partition(Partition& _return) { int32_t rseqid = 0; @@ -21207,39 +23833,47 @@ void ThriftHiveMetastoreClient::recv_drop_table_with_environment_context() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("drop_table_with_environment_context") != 0) { + if (fname.compare("add_partition") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_drop_table_with_environment_context_presult result; + ThriftHiveMetastore_add_partition_presult result; + result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); + if (result.__isset.success) { + // _return pointer has now been filled + return; + } if (result.__isset.o1) { throw result.o1; } + if (result.__isset.o2) { + throw result.o2; + } if (result.__isset.o3) { throw result.o3; } - return; + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_partition failed: unknown result"); } -void ThriftHiveMetastoreClient::get_tables(std::vector & _return, const std::string& db_name, const std::string& pattern) +void ThriftHiveMetastoreClient::add_partition_with_environment_context(Partition& _return, const Partition& new_part, const EnvironmentContext& environment_context) { - send_get_tables(db_name, pattern); - recv_get_tables(_return); + send_add_partition_with_environment_context(new_part, environment_context); + recv_add_partition_with_environment_context(_return); } -void ThriftHiveMetastoreClient::send_get_tables(const std::string& db_name, const std::string& pattern) +void ThriftHiveMetastoreClient::send_add_partition_with_environment_context(const Partition& new_part, const EnvironmentContext& environment_context) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_tables", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("add_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_tables_pargs args; - args.db_name = &db_name; - args.pattern = &pattern; + ThriftHiveMetastore_add_partition_with_environment_context_pargs args; + args.new_part = &new_part; + args.environment_context = &environment_context; args.write(oprot_); oprot_->writeMessageEnd(); @@ -21247,7 +23881,7 @@ void ThriftHiveMetastoreClient::send_get_tables(const std::string& db_name, cons oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_tables(std::vector & _return) +void ThriftHiveMetastoreClient::recv_add_partition_with_environment_context(Partition& _return) { int32_t rseqid = 0; @@ -21267,12 +23901,12 @@ void ThriftHiveMetastoreClient::recv_get_tables(std::vector & _retu iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_tables") != 0) { + if (fname.compare("add_partition_with_environment_context") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_tables_presult result; + ThriftHiveMetastore_add_partition_with_environment_context_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -21285,22 +23919,28 @@ void ThriftHiveMetastoreClient::recv_get_tables(std::vector & _retu if (result.__isset.o1) { throw result.o1; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_tables failed: unknown result"); + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_partition_with_environment_context failed: unknown result"); } -void ThriftHiveMetastoreClient::get_all_tables(std::vector & _return, const std::string& db_name) +int32_t ThriftHiveMetastoreClient::add_partitions(const std::vector & new_parts) { - send_get_all_tables(db_name); - recv_get_all_tables(_return); + send_add_partitions(new_parts); + return recv_add_partitions(); } -void ThriftHiveMetastoreClient::send_get_all_tables(const std::string& db_name) +void ThriftHiveMetastoreClient::send_add_partitions(const std::vector & new_parts) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_all_tables", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("add_partitions", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_all_tables_pargs args; - args.db_name = &db_name; + ThriftHiveMetastore_add_partitions_pargs args; + args.new_parts = &new_parts; args.write(oprot_); oprot_->writeMessageEnd(); @@ -21308,7 +23948,7 @@ void ThriftHiveMetastoreClient::send_get_all_tables(const std::string& db_name) oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_all_tables(std::vector & _return) +int32_t ThriftHiveMetastoreClient::recv_add_partitions() { int32_t rseqid = 0; @@ -21328,41 +23968,48 @@ void ThriftHiveMetastoreClient::recv_get_all_tables(std::vector & _ iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_all_tables") != 0) { + if (fname.compare("add_partitions") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_all_tables_presult result; + int32_t _return; + ThriftHiveMetastore_add_partitions_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); if (result.__isset.success) { - // _return pointer has now been filled - return; + return _return; } if (result.__isset.o1) { throw result.o1; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_all_tables failed: unknown result"); + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_partitions failed: unknown result"); } -void ThriftHiveMetastoreClient::get_table(Table& _return, const std::string& dbname, const std::string& tbl_name) +void ThriftHiveMetastoreClient::append_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) { - send_get_table(dbname, tbl_name); - recv_get_table(_return); + send_append_partition(db_name, tbl_name, part_vals); + recv_append_partition(_return); } -void ThriftHiveMetastoreClient::send_get_table(const std::string& dbname, const std::string& tbl_name) +void ThriftHiveMetastoreClient::send_append_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_table", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("append_partition", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_table_pargs args; - args.dbname = &dbname; + ThriftHiveMetastore_append_partition_pargs args; + args.db_name = &db_name; args.tbl_name = &tbl_name; + args.part_vals = &part_vals; args.write(oprot_); oprot_->writeMessageEnd(); @@ -21370,7 +24017,7 @@ void ThriftHiveMetastoreClient::send_get_table(const std::string& dbname, const oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_table(Table& _return) +void ThriftHiveMetastoreClient::recv_append_partition(Partition& _return) { int32_t rseqid = 0; @@ -21390,12 +24037,12 @@ void ThriftHiveMetastoreClient::recv_get_table(Table& _return) iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_table") != 0) { + if (fname.compare("append_partition") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_table_presult result; + ThriftHiveMetastore_append_partition_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -21411,23 +24058,28 @@ void ThriftHiveMetastoreClient::recv_get_table(Table& _return) if (result.__isset.o2) { throw result.o2; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table failed: unknown result"); + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "append_partition failed: unknown result"); } -void ThriftHiveMetastoreClient::get_table_objects_by_name(std::vector
& _return, const std::string& dbname, const std::vector & tbl_names) +void ThriftHiveMetastoreClient::append_partition_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const EnvironmentContext& environment_context) { - send_get_table_objects_by_name(dbname, tbl_names); - recv_get_table_objects_by_name(_return); + send_append_partition_with_environment_context(db_name, tbl_name, part_vals, environment_context); + recv_append_partition_with_environment_context(_return); } -void ThriftHiveMetastoreClient::send_get_table_objects_by_name(const std::string& dbname, const std::vector & tbl_names) +void ThriftHiveMetastoreClient::send_append_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const EnvironmentContext& environment_context) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_table_objects_by_name", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("append_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_table_objects_by_name_pargs args; - args.dbname = &dbname; - args.tbl_names = &tbl_names; + ThriftHiveMetastore_append_partition_with_environment_context_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_vals = &part_vals; + args.environment_context = &environment_context; args.write(oprot_); oprot_->writeMessageEnd(); @@ -21435,7 +24087,7 @@ void ThriftHiveMetastoreClient::send_get_table_objects_by_name(const std::string oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_table_objects_by_name(std::vector
& _return) +void ThriftHiveMetastoreClient::recv_append_partition_with_environment_context(Partition& _return) { int32_t rseqid = 0; @@ -21455,12 +24107,12 @@ void ThriftHiveMetastoreClient::recv_get_table_objects_by_name(std::vector
readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_table_objects_by_name") != 0) { + if (fname.compare("append_partition_with_environment_context") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_table_objects_by_name_presult result; + ThriftHiveMetastore_append_partition_with_environment_context_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -21479,24 +24131,24 @@ void ThriftHiveMetastoreClient::recv_get_table_objects_by_name(std::vector
& _return, const std::string& dbname, const std::string& filter, const int16_t max_tables) +void ThriftHiveMetastoreClient::append_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name) { - send_get_table_names_by_filter(dbname, filter, max_tables); - recv_get_table_names_by_filter(_return); + send_append_partition_by_name(db_name, tbl_name, part_name); + recv_append_partition_by_name(_return); } -void ThriftHiveMetastoreClient::send_get_table_names_by_filter(const std::string& dbname, const std::string& filter, const int16_t max_tables) +void ThriftHiveMetastoreClient::send_append_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_table_names_by_filter", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("append_partition_by_name", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_table_names_by_filter_pargs args; - args.dbname = &dbname; - args.filter = &filter; - args.max_tables = &max_tables; + ThriftHiveMetastore_append_partition_by_name_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_name = &part_name; args.write(oprot_); oprot_->writeMessageEnd(); @@ -21504,7 +24156,7 @@ void ThriftHiveMetastoreClient::send_get_table_names_by_filter(const std::string oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_table_names_by_filter(std::vector & _return) +void ThriftHiveMetastoreClient::recv_append_partition_by_name(Partition& _return) { int32_t rseqid = 0; @@ -21524,12 +24176,12 @@ void ThriftHiveMetastoreClient::recv_get_table_names_by_filter(std::vectorreadMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_table_names_by_filter") != 0) { + if (fname.compare("append_partition_by_name") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_table_names_by_filter_presult result; + ThriftHiveMetastore_append_partition_by_name_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -21548,24 +24200,25 @@ void ThriftHiveMetastoreClient::recv_get_table_names_by_filter(std::vectorwriteMessageBegin("alter_table", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("append_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_alter_table_pargs args; - args.dbname = &dbname; + ThriftHiveMetastore_append_partition_by_name_with_environment_context_pargs args; + args.db_name = &db_name; args.tbl_name = &tbl_name; - args.new_tbl = &new_tbl; + args.part_name = &part_name; + args.environment_context = &environment_context; args.write(oprot_); oprot_->writeMessageEnd(); @@ -21573,7 +24226,7 @@ void ThriftHiveMetastoreClient::send_alter_table(const std::string& dbname, cons oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_alter_table() +void ThriftHiveMetastoreClient::recv_append_partition_by_name_with_environment_context(Partition& _return) { int32_t rseqid = 0; @@ -21593,41 +24246,49 @@ void ThriftHiveMetastoreClient::recv_alter_table() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("alter_table") != 0) { + if (fname.compare("append_partition_by_name_with_environment_context") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_alter_table_presult result; + ThriftHiveMetastore_append_partition_by_name_with_environment_context_presult result; + result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); + if (result.__isset.success) { + // _return pointer has now been filled + return; + } if (result.__isset.o1) { throw result.o1; } if (result.__isset.o2) { throw result.o2; } - return; + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "append_partition_by_name_with_environment_context failed: unknown result"); } -void ThriftHiveMetastoreClient::alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context) +bool ThriftHiveMetastoreClient::drop_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData) { - send_alter_table_with_environment_context(dbname, tbl_name, new_tbl, environment_context); - recv_alter_table_with_environment_context(); + send_drop_partition(db_name, tbl_name, part_vals, deleteData); + return recv_drop_partition(); } -void ThriftHiveMetastoreClient::send_alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context) +void ThriftHiveMetastoreClient::send_drop_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData) { int32_t cseqid = 0; - oprot_->writeMessageBegin("alter_table_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("drop_partition", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_alter_table_with_environment_context_pargs args; - args.dbname = &dbname; + ThriftHiveMetastore_drop_partition_pargs args; + args.db_name = &db_name; args.tbl_name = &tbl_name; - args.new_tbl = &new_tbl; - args.environment_context = &environment_context; + args.part_vals = &part_vals; + args.deleteData = &deleteData; args.write(oprot_); oprot_->writeMessageEnd(); @@ -21635,7 +24296,7 @@ void ThriftHiveMetastoreClient::send_alter_table_with_environment_context(const oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_alter_table_with_environment_context() +bool ThriftHiveMetastoreClient::recv_drop_partition() { int32_t rseqid = 0; @@ -21655,38 +24316,47 @@ void ThriftHiveMetastoreClient::recv_alter_table_with_environment_context() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("alter_table_with_environment_context") != 0) { + if (fname.compare("drop_partition") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_alter_table_with_environment_context_presult result; + bool _return; + ThriftHiveMetastore_drop_partition_presult result; + result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); + if (result.__isset.success) { + return _return; + } if (result.__isset.o1) { throw result.o1; } if (result.__isset.o2) { throw result.o2; } - return; + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_partition failed: unknown result"); } -void ThriftHiveMetastoreClient::add_partition(Partition& _return, const Partition& new_part) +bool ThriftHiveMetastoreClient::drop_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData, const EnvironmentContext& environment_context) { - send_add_partition(new_part); - recv_add_partition(_return); + send_drop_partition_with_environment_context(db_name, tbl_name, part_vals, deleteData, environment_context); + return recv_drop_partition_with_environment_context(); } -void ThriftHiveMetastoreClient::send_add_partition(const Partition& new_part) +void ThriftHiveMetastoreClient::send_drop_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData, const EnvironmentContext& environment_context) { int32_t cseqid = 0; - oprot_->writeMessageBegin("add_partition", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("drop_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_add_partition_pargs args; - args.new_part = &new_part; + ThriftHiveMetastore_drop_partition_with_environment_context_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_vals = &part_vals; + args.deleteData = &deleteData; + args.environment_context = &environment_context; args.write(oprot_); oprot_->writeMessageEnd(); @@ -21694,7 +24364,7 @@ void ThriftHiveMetastoreClient::send_add_partition(const Partition& new_part) oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_add_partition(Partition& _return) +bool ThriftHiveMetastoreClient::recv_drop_partition_with_environment_context() { int32_t rseqid = 0; @@ -21714,20 +24384,20 @@ void ThriftHiveMetastoreClient::recv_add_partition(Partition& _return) iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("add_partition") != 0) { + if (fname.compare("drop_partition_with_environment_context") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_add_partition_presult result; + bool _return; + ThriftHiveMetastore_drop_partition_with_environment_context_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); if (result.__isset.success) { - // _return pointer has now been filled - return; + return _return; } if (result.__isset.o1) { throw result.o1; @@ -21735,26 +24405,25 @@ void ThriftHiveMetastoreClient::recv_add_partition(Partition& _return) if (result.__isset.o2) { throw result.o2; } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_partition failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_partition_with_environment_context failed: unknown result"); } -void ThriftHiveMetastoreClient::add_partition_with_environment_context(Partition& _return, const Partition& new_part, const EnvironmentContext& environment_context) +bool ThriftHiveMetastoreClient::drop_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData) { - send_add_partition_with_environment_context(new_part, environment_context); - recv_add_partition_with_environment_context(_return); + send_drop_partition_by_name(db_name, tbl_name, part_name, deleteData); + return recv_drop_partition_by_name(); } -void ThriftHiveMetastoreClient::send_add_partition_with_environment_context(const Partition& new_part, const EnvironmentContext& environment_context) +void ThriftHiveMetastoreClient::send_drop_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData) { int32_t cseqid = 0; - oprot_->writeMessageBegin("add_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("drop_partition_by_name", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_add_partition_with_environment_context_pargs args; - args.new_part = &new_part; - args.environment_context = &environment_context; + ThriftHiveMetastore_drop_partition_by_name_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_name = &part_name; + args.deleteData = &deleteData; args.write(oprot_); oprot_->writeMessageEnd(); @@ -21762,7 +24431,7 @@ void ThriftHiveMetastoreClient::send_add_partition_with_environment_context(cons oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_add_partition_with_environment_context(Partition& _return) +bool ThriftHiveMetastoreClient::recv_drop_partition_by_name() { int32_t rseqid = 0; @@ -21782,20 +24451,20 @@ void ThriftHiveMetastoreClient::recv_add_partition_with_environment_context(Part iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("add_partition_with_environment_context") != 0) { + if (fname.compare("drop_partition_by_name") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_add_partition_with_environment_context_presult result; + bool _return; + ThriftHiveMetastore_drop_partition_by_name_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); if (result.__isset.success) { - // _return pointer has now been filled - return; + return _return; } if (result.__isset.o1) { throw result.o1; @@ -21803,25 +24472,26 @@ void ThriftHiveMetastoreClient::recv_add_partition_with_environment_context(Part if (result.__isset.o2) { throw result.o2; } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_partition_with_environment_context failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_partition_by_name failed: unknown result"); } -int32_t ThriftHiveMetastoreClient::add_partitions(const std::vector & new_parts) +bool ThriftHiveMetastoreClient::drop_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData, const EnvironmentContext& environment_context) { - send_add_partitions(new_parts); - return recv_add_partitions(); + send_drop_partition_by_name_with_environment_context(db_name, tbl_name, part_name, deleteData, environment_context); + return recv_drop_partition_by_name_with_environment_context(); } -void ThriftHiveMetastoreClient::send_add_partitions(const std::vector & new_parts) +void ThriftHiveMetastoreClient::send_drop_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData, const EnvironmentContext& environment_context) { int32_t cseqid = 0; - oprot_->writeMessageBegin("add_partitions", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("drop_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_add_partitions_pargs args; - args.new_parts = &new_parts; + ThriftHiveMetastore_drop_partition_by_name_with_environment_context_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_name = &part_name; + args.deleteData = &deleteData; + args.environment_context = &environment_context; args.write(oprot_); oprot_->writeMessageEnd(); @@ -21829,7 +24499,7 @@ void ThriftHiveMetastoreClient::send_add_partitions(const std::vector oprot_->getTransport()->flush(); } -int32_t ThriftHiveMetastoreClient::recv_add_partitions() +bool ThriftHiveMetastoreClient::recv_drop_partition_by_name_with_environment_context() { int32_t rseqid = 0; @@ -21849,13 +24519,13 @@ int32_t ThriftHiveMetastoreClient::recv_add_partitions() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("add_partitions") != 0) { + if (fname.compare("drop_partition_by_name_with_environment_context") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - int32_t _return; - ThriftHiveMetastore_add_partitions_presult result; + bool _return; + ThriftHiveMetastore_drop_partition_by_name_with_environment_context_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -21870,24 +24540,21 @@ int32_t ThriftHiveMetastoreClient::recv_add_partitions() if (result.__isset.o2) { throw result.o2; } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_partitions failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_partition_by_name_with_environment_context failed: unknown result"); } -void ThriftHiveMetastoreClient::append_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) +void ThriftHiveMetastoreClient::get_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) { - send_append_partition(db_name, tbl_name, part_vals); - recv_append_partition(_return); + send_get_partition(db_name, tbl_name, part_vals); + recv_get_partition(_return); } -void ThriftHiveMetastoreClient::send_append_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) +void ThriftHiveMetastoreClient::send_get_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) { int32_t cseqid = 0; - oprot_->writeMessageBegin("append_partition", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_partition", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_append_partition_pargs args; + ThriftHiveMetastore_get_partition_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; args.part_vals = &part_vals; @@ -21898,7 +24565,7 @@ void ThriftHiveMetastoreClient::send_append_partition(const std::string& db_name oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_append_partition(Partition& _return) +void ThriftHiveMetastoreClient::recv_get_partition(Partition& _return) { int32_t rseqid = 0; @@ -21918,12 +24585,12 @@ void ThriftHiveMetastoreClient::recv_append_partition(Partition& _return) iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("append_partition") != 0) { + if (fname.compare("get_partition") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_append_partition_presult result; + ThriftHiveMetastore_get_partition_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -21939,28 +24606,26 @@ void ThriftHiveMetastoreClient::recv_append_partition(Partition& _return) if (result.__isset.o2) { throw result.o2; } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "append_partition failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition failed: unknown result"); } -void ThriftHiveMetastoreClient::append_partition_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const EnvironmentContext& environment_context) +void ThriftHiveMetastoreClient::exchange_partition(Partition& _return, const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name) { - send_append_partition_with_environment_context(db_name, tbl_name, part_vals, environment_context); - recv_append_partition_with_environment_context(_return); + send_exchange_partition(partitionSpecs, source_db, source_table_name, dest_db, dest_table_name); + recv_exchange_partition(_return); } -void ThriftHiveMetastoreClient::send_append_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const EnvironmentContext& environment_context) +void ThriftHiveMetastoreClient::send_exchange_partition(const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name) { int32_t cseqid = 0; - oprot_->writeMessageBegin("append_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("exchange_partition", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_append_partition_with_environment_context_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.environment_context = &environment_context; + ThriftHiveMetastore_exchange_partition_pargs args; + args.partitionSpecs = &partitionSpecs; + args.source_db = &source_db; + args.source_table_name = &source_table_name; + args.dest_db = &dest_db; + args.dest_table_name = &dest_table_name; args.write(oprot_); oprot_->writeMessageEnd(); @@ -21968,7 +24633,7 @@ void ThriftHiveMetastoreClient::send_append_partition_with_environment_context(c oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_append_partition_with_environment_context(Partition& _return) +void ThriftHiveMetastoreClient::recv_exchange_partition(Partition& _return) { int32_t rseqid = 0; @@ -21988,12 +24653,12 @@ void ThriftHiveMetastoreClient::recv_append_partition_with_environment_context(P iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("append_partition_with_environment_context") != 0) { + if (fname.compare("exchange_partition") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_append_partition_with_environment_context_presult result; + ThriftHiveMetastore_exchange_partition_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -22012,24 +24677,29 @@ void ThriftHiveMetastoreClient::recv_append_partition_with_environment_context(P if (result.__isset.o3) { throw result.o3; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "append_partition_with_environment_context failed: unknown result"); + if (result.__isset.o4) { + throw result.o4; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "exchange_partition failed: unknown result"); } -void ThriftHiveMetastoreClient::append_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name) +void ThriftHiveMetastoreClient::get_partition_with_auth(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const std::string& user_name, const std::vector & group_names) { - send_append_partition_by_name(db_name, tbl_name, part_name); - recv_append_partition_by_name(_return); + send_get_partition_with_auth(db_name, tbl_name, part_vals, user_name, group_names); + recv_get_partition_with_auth(_return); } -void ThriftHiveMetastoreClient::send_append_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name) +void ThriftHiveMetastoreClient::send_get_partition_with_auth(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const std::string& user_name, const std::vector & group_names) { int32_t cseqid = 0; - oprot_->writeMessageBegin("append_partition_by_name", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_partition_with_auth", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_append_partition_by_name_pargs args; + ThriftHiveMetastore_get_partition_with_auth_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; - args.part_name = &part_name; + args.part_vals = &part_vals; + args.user_name = &user_name; + args.group_names = &group_names; args.write(oprot_); oprot_->writeMessageEnd(); @@ -22037,7 +24707,7 @@ void ThriftHiveMetastoreClient::send_append_partition_by_name(const std::string& oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_append_partition_by_name(Partition& _return) +void ThriftHiveMetastoreClient::recv_get_partition_with_auth(Partition& _return) { int32_t rseqid = 0; @@ -22057,12 +24727,12 @@ void ThriftHiveMetastoreClient::recv_append_partition_by_name(Partition& _return iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("append_partition_by_name") != 0) { + if (fname.compare("get_partition_with_auth") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_append_partition_by_name_presult result; + ThriftHiveMetastore_get_partition_with_auth_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -22078,28 +24748,24 @@ void ThriftHiveMetastoreClient::recv_append_partition_by_name(Partition& _return if (result.__isset.o2) { throw result.o2; } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "append_partition_by_name failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_with_auth failed: unknown result"); } -void ThriftHiveMetastoreClient::append_partition_by_name_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const EnvironmentContext& environment_context) +void ThriftHiveMetastoreClient::get_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name) { - send_append_partition_by_name_with_environment_context(db_name, tbl_name, part_name, environment_context); - recv_append_partition_by_name_with_environment_context(_return); + send_get_partition_by_name(db_name, tbl_name, part_name); + recv_get_partition_by_name(_return); } -void ThriftHiveMetastoreClient::send_append_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const EnvironmentContext& environment_context) +void ThriftHiveMetastoreClient::send_get_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name) { int32_t cseqid = 0; - oprot_->writeMessageBegin("append_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_partition_by_name", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_append_partition_by_name_with_environment_context_pargs args; + ThriftHiveMetastore_get_partition_by_name_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; args.part_name = &part_name; - args.environment_context = &environment_context; args.write(oprot_); oprot_->writeMessageEnd(); @@ -22107,7 +24773,7 @@ void ThriftHiveMetastoreClient::send_append_partition_by_name_with_environment_c oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_append_partition_by_name_with_environment_context(Partition& _return) +void ThriftHiveMetastoreClient::recv_get_partition_by_name(Partition& _return) { int32_t rseqid = 0; @@ -22127,12 +24793,12 @@ void ThriftHiveMetastoreClient::recv_append_partition_by_name_with_environment_c iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("append_partition_by_name_with_environment_context") != 0) { + if (fname.compare("get_partition_by_name") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_append_partition_by_name_with_environment_context_presult result; + ThriftHiveMetastore_get_partition_by_name_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -22148,28 +24814,24 @@ void ThriftHiveMetastoreClient::recv_append_partition_by_name_with_environment_c if (result.__isset.o2) { throw result.o2; } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "append_partition_by_name_with_environment_context failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_by_name failed: unknown result"); } -bool ThriftHiveMetastoreClient::drop_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData) +void ThriftHiveMetastoreClient::get_partitions(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) { - send_drop_partition(db_name, tbl_name, part_vals, deleteData); - return recv_drop_partition(); + send_get_partitions(db_name, tbl_name, max_parts); + recv_get_partitions(_return); } -void ThriftHiveMetastoreClient::send_drop_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData) +void ThriftHiveMetastoreClient::send_get_partitions(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) { int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_partition", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_partitions", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_drop_partition_pargs args; + ThriftHiveMetastore_get_partitions_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.deleteData = &deleteData; + args.max_parts = &max_parts; args.write(oprot_); oprot_->writeMessageEnd(); @@ -22177,7 +24839,7 @@ void ThriftHiveMetastoreClient::send_drop_partition(const std::string& db_name, oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_drop_partition() +void ThriftHiveMetastoreClient::recv_get_partitions(std::vector & _return) { int32_t rseqid = 0; @@ -22197,20 +24859,20 @@ bool ThriftHiveMetastoreClient::recv_drop_partition() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("drop_partition") != 0) { + if (fname.compare("get_partitions") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - bool _return; - ThriftHiveMetastore_drop_partition_presult result; + ThriftHiveMetastore_get_partitions_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); if (result.__isset.success) { - return _return; + // _return pointer has now been filled + return; } if (result.__isset.o1) { throw result.o1; @@ -22218,26 +24880,26 @@ bool ThriftHiveMetastoreClient::recv_drop_partition() if (result.__isset.o2) { throw result.o2; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_partition failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions failed: unknown result"); } -bool ThriftHiveMetastoreClient::drop_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData, const EnvironmentContext& environment_context) +void ThriftHiveMetastoreClient::get_partitions_with_auth(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) { - send_drop_partition_with_environment_context(db_name, tbl_name, part_vals, deleteData, environment_context); - return recv_drop_partition_with_environment_context(); + send_get_partitions_with_auth(db_name, tbl_name, max_parts, user_name, group_names); + recv_get_partitions_with_auth(_return); } -void ThriftHiveMetastoreClient::send_drop_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData, const EnvironmentContext& environment_context) +void ThriftHiveMetastoreClient::send_get_partitions_with_auth(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) { int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_partitions_with_auth", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_drop_partition_with_environment_context_pargs args; + ThriftHiveMetastore_get_partitions_with_auth_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.deleteData = &deleteData; - args.environment_context = &environment_context; + args.max_parts = &max_parts; + args.user_name = &user_name; + args.group_names = &group_names; args.write(oprot_); oprot_->writeMessageEnd(); @@ -22245,7 +24907,7 @@ void ThriftHiveMetastoreClient::send_drop_partition_with_environment_context(con oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_drop_partition_with_environment_context() +void ThriftHiveMetastoreClient::recv_get_partitions_with_auth(std::vector & _return) { int32_t rseqid = 0; @@ -22265,20 +24927,20 @@ bool ThriftHiveMetastoreClient::recv_drop_partition_with_environment_context() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("drop_partition_with_environment_context") != 0) { + if (fname.compare("get_partitions_with_auth") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - bool _return; - ThriftHiveMetastore_drop_partition_with_environment_context_presult result; + ThriftHiveMetastore_get_partitions_with_auth_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); if (result.__isset.success) { - return _return; + // _return pointer has now been filled + return; } if (result.__isset.o1) { throw result.o1; @@ -22286,25 +24948,24 @@ bool ThriftHiveMetastoreClient::recv_drop_partition_with_environment_context() if (result.__isset.o2) { throw result.o2; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_partition_with_environment_context failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_with_auth failed: unknown result"); } -bool ThriftHiveMetastoreClient::drop_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData) +void ThriftHiveMetastoreClient::get_partition_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) { - send_drop_partition_by_name(db_name, tbl_name, part_name, deleteData); - return recv_drop_partition_by_name(); + send_get_partition_names(db_name, tbl_name, max_parts); + recv_get_partition_names(_return); } -void ThriftHiveMetastoreClient::send_drop_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData) +void ThriftHiveMetastoreClient::send_get_partition_names(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) { int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_partition_by_name", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_partition_names", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_drop_partition_by_name_pargs args; + ThriftHiveMetastore_get_partition_names_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; - args.part_name = &part_name; - args.deleteData = &deleteData; + args.max_parts = &max_parts; args.write(oprot_); oprot_->writeMessageEnd(); @@ -22312,7 +24973,7 @@ void ThriftHiveMetastoreClient::send_drop_partition_by_name(const std::string& d oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_drop_partition_by_name() +void ThriftHiveMetastoreClient::recv_get_partition_names(std::vector & _return) { int32_t rseqid = 0; @@ -22332,47 +24993,43 @@ bool ThriftHiveMetastoreClient::recv_drop_partition_by_name() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("drop_partition_by_name") != 0) { + if (fname.compare("get_partition_names") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - bool _return; - ThriftHiveMetastore_drop_partition_by_name_presult result; + ThriftHiveMetastore_get_partition_names_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); if (result.__isset.success) { - return _return; - } - if (result.__isset.o1) { - throw result.o1; + // _return pointer has now been filled + return; } if (result.__isset.o2) { throw result.o2; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_partition_by_name failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_names failed: unknown result"); } -bool ThriftHiveMetastoreClient::drop_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData, const EnvironmentContext& environment_context) +void ThriftHiveMetastoreClient::get_partitions_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) { - send_drop_partition_by_name_with_environment_context(db_name, tbl_name, part_name, deleteData, environment_context); - return recv_drop_partition_by_name_with_environment_context(); + send_get_partitions_ps(db_name, tbl_name, part_vals, max_parts); + recv_get_partitions_ps(_return); } -void ThriftHiveMetastoreClient::send_drop_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData, const EnvironmentContext& environment_context) +void ThriftHiveMetastoreClient::send_get_partitions_ps(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) { int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_partitions_ps", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_drop_partition_by_name_with_environment_context_pargs args; + ThriftHiveMetastore_get_partitions_ps_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; - args.part_name = &part_name; - args.deleteData = &deleteData; - args.environment_context = &environment_context; + args.part_vals = &part_vals; + args.max_parts = &max_parts; args.write(oprot_); oprot_->writeMessageEnd(); @@ -22380,7 +25037,7 @@ void ThriftHiveMetastoreClient::send_drop_partition_by_name_with_environment_con oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_drop_partition_by_name_with_environment_context() +void ThriftHiveMetastoreClient::recv_get_partitions_ps(std::vector & _return) { int32_t rseqid = 0; @@ -22400,20 +25057,20 @@ bool ThriftHiveMetastoreClient::recv_drop_partition_by_name_with_environment_con iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("drop_partition_by_name_with_environment_context") != 0) { + if (fname.compare("get_partitions_ps") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - bool _return; - ThriftHiveMetastore_drop_partition_by_name_with_environment_context_presult result; + ThriftHiveMetastore_get_partitions_ps_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); if (result.__isset.success) { - return _return; + // _return pointer has now been filled + return; } if (result.__isset.o1) { throw result.o1; @@ -22421,24 +25078,27 @@ bool ThriftHiveMetastoreClient::recv_drop_partition_by_name_with_environment_con if (result.__isset.o2) { throw result.o2; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_partition_by_name_with_environment_context failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_ps failed: unknown result"); } -void ThriftHiveMetastoreClient::get_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) +void ThriftHiveMetastoreClient::get_partitions_ps_with_auth(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) { - send_get_partition(db_name, tbl_name, part_vals); - recv_get_partition(_return); + send_get_partitions_ps_with_auth(db_name, tbl_name, part_vals, max_parts, user_name, group_names); + recv_get_partitions_ps_with_auth(_return); } -void ThriftHiveMetastoreClient::send_get_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) +void ThriftHiveMetastoreClient::send_get_partitions_ps_with_auth(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partition", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_partitions_ps_with_auth", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_partition_pargs args; + ThriftHiveMetastore_get_partitions_ps_with_auth_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; args.part_vals = &part_vals; + args.max_parts = &max_parts; + args.user_name = &user_name; + args.group_names = &group_names; args.write(oprot_); oprot_->writeMessageEnd(); @@ -22446,7 +25106,7 @@ void ThriftHiveMetastoreClient::send_get_partition(const std::string& db_name, c oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_partition(Partition& _return) +void ThriftHiveMetastoreClient::recv_get_partitions_ps_with_auth(std::vector & _return) { int32_t rseqid = 0; @@ -22466,12 +25126,12 @@ void ThriftHiveMetastoreClient::recv_get_partition(Partition& _return) iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_partition") != 0) { + if (fname.compare("get_partitions_ps_with_auth") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_partition_presult result; + ThriftHiveMetastore_get_partitions_ps_with_auth_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -22487,26 +25147,25 @@ void ThriftHiveMetastoreClient::recv_get_partition(Partition& _return) if (result.__isset.o2) { throw result.o2; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_ps_with_auth failed: unknown result"); } -void ThriftHiveMetastoreClient::exchange_partition(Partition& _return, const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name) +void ThriftHiveMetastoreClient::get_partition_names_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) { - send_exchange_partition(partitionSpecs, source_db, source_table_name, dest_db, dest_table_name); - recv_exchange_partition(_return); + send_get_partition_names_ps(db_name, tbl_name, part_vals, max_parts); + recv_get_partition_names_ps(_return); } -void ThriftHiveMetastoreClient::send_exchange_partition(const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name) +void ThriftHiveMetastoreClient::send_get_partition_names_ps(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) { int32_t cseqid = 0; - oprot_->writeMessageBegin("exchange_partition", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_partition_names_ps", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_exchange_partition_pargs args; - args.partitionSpecs = &partitionSpecs; - args.source_db = &source_db; - args.source_table_name = &source_table_name; - args.dest_db = &dest_db; - args.dest_table_name = &dest_table_name; + ThriftHiveMetastore_get_partition_names_ps_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_vals = &part_vals; + args.max_parts = &max_parts; args.write(oprot_); oprot_->writeMessageEnd(); @@ -22514,7 +25173,7 @@ void ThriftHiveMetastoreClient::send_exchange_partition(const std::mapgetTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_exchange_partition(Partition& _return) +void ThriftHiveMetastoreClient::recv_get_partition_names_ps(std::vector & _return) { int32_t rseqid = 0; @@ -22534,12 +25193,12 @@ void ThriftHiveMetastoreClient::recv_exchange_partition(Partition& _return) iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("exchange_partition") != 0) { + if (fname.compare("get_partition_names_ps") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_exchange_partition_presult result; + ThriftHiveMetastore_get_partition_names_ps_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -22555,32 +25214,25 @@ void ThriftHiveMetastoreClient::recv_exchange_partition(Partition& _return) if (result.__isset.o2) { throw result.o2; } - if (result.__isset.o3) { - throw result.o3; - } - if (result.__isset.o4) { - throw result.o4; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "exchange_partition failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_names_ps failed: unknown result"); } -void ThriftHiveMetastoreClient::get_partition_with_auth(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const std::string& user_name, const std::vector & group_names) +void ThriftHiveMetastoreClient::get_partitions_by_filter(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int16_t max_parts) { - send_get_partition_with_auth(db_name, tbl_name, part_vals, user_name, group_names); - recv_get_partition_with_auth(_return); + send_get_partitions_by_filter(db_name, tbl_name, filter, max_parts); + recv_get_partitions_by_filter(_return); } -void ThriftHiveMetastoreClient::send_get_partition_with_auth(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const std::string& user_name, const std::vector & group_names) +void ThriftHiveMetastoreClient::send_get_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int16_t max_parts) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partition_with_auth", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_partitions_by_filter", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_partition_with_auth_pargs args; + ThriftHiveMetastore_get_partitions_by_filter_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.user_name = &user_name; - args.group_names = &group_names; + args.filter = &filter; + args.max_parts = &max_parts; args.write(oprot_); oprot_->writeMessageEnd(); @@ -22588,7 +25240,7 @@ void ThriftHiveMetastoreClient::send_get_partition_with_auth(const std::string& oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_partition_with_auth(Partition& _return) +void ThriftHiveMetastoreClient::recv_get_partitions_by_filter(std::vector & _return) { int32_t rseqid = 0; @@ -22608,12 +25260,12 @@ void ThriftHiveMetastoreClient::recv_get_partition_with_auth(Partition& _return) iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_partition_with_auth") != 0) { + if (fname.compare("get_partitions_by_filter") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_partition_with_auth_presult result; + ThriftHiveMetastore_get_partitions_by_filter_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -22629,24 +25281,24 @@ void ThriftHiveMetastoreClient::recv_get_partition_with_auth(Partition& _return) if (result.__isset.o2) { throw result.o2; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_with_auth failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_by_filter failed: unknown result"); } -void ThriftHiveMetastoreClient::get_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name) +void ThriftHiveMetastoreClient::get_partitions_by_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & names) { - send_get_partition_by_name(db_name, tbl_name, part_name); - recv_get_partition_by_name(_return); + send_get_partitions_by_names(db_name, tbl_name, names); + recv_get_partitions_by_names(_return); } -void ThriftHiveMetastoreClient::send_get_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name) +void ThriftHiveMetastoreClient::send_get_partitions_by_names(const std::string& db_name, const std::string& tbl_name, const std::vector & names) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partition_by_name", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_partitions_by_names", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_partition_by_name_pargs args; + ThriftHiveMetastore_get_partitions_by_names_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; - args.part_name = &part_name; + args.names = &names; args.write(oprot_); oprot_->writeMessageEnd(); @@ -22654,7 +25306,7 @@ void ThriftHiveMetastoreClient::send_get_partition_by_name(const std::string& db oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_partition_by_name(Partition& _return) +void ThriftHiveMetastoreClient::recv_get_partitions_by_names(std::vector & _return) { int32_t rseqid = 0; @@ -22674,12 +25326,12 @@ void ThriftHiveMetastoreClient::recv_get_partition_by_name(Partition& _return) iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_partition_by_name") != 0) { + if (fname.compare("get_partitions_by_names") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_partition_by_name_presult result; + ThriftHiveMetastore_get_partitions_by_names_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -22695,24 +25347,24 @@ void ThriftHiveMetastoreClient::recv_get_partition_by_name(Partition& _return) if (result.__isset.o2) { throw result.o2; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_by_name failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_by_names failed: unknown result"); } -void ThriftHiveMetastoreClient::get_partitions(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) +void ThriftHiveMetastoreClient::alter_partition(const std::string& db_name, const std::string& tbl_name, const Partition& new_part) { - send_get_partitions(db_name, tbl_name, max_parts); - recv_get_partitions(_return); + send_alter_partition(db_name, tbl_name, new_part); + recv_alter_partition(); } -void ThriftHiveMetastoreClient::send_get_partitions(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) +void ThriftHiveMetastoreClient::send_alter_partition(const std::string& db_name, const std::string& tbl_name, const Partition& new_part) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partitions", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("alter_partition", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_partitions_pargs args; + ThriftHiveMetastore_alter_partition_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; - args.max_parts = &max_parts; + args.new_part = &new_part; args.write(oprot_); oprot_->writeMessageEnd(); @@ -22720,7 +25372,7 @@ void ThriftHiveMetastoreClient::send_get_partitions(const std::string& db_name, oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_partitions(std::vector & _return) +void ThriftHiveMetastoreClient::recv_alter_partition() { int32_t rseqid = 0; @@ -22740,47 +25392,40 @@ void ThriftHiveMetastoreClient::recv_get_partitions(std::vector & _re iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_partitions") != 0) { + if (fname.compare("alter_partition") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_partitions_presult result; - result.success = &_return; + ThriftHiveMetastore_alter_partition_presult result; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); - if (result.__isset.success) { - // _return pointer has now been filled - return; - } if (result.__isset.o1) { throw result.o1; } if (result.__isset.o2) { throw result.o2; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions failed: unknown result"); + return; } -void ThriftHiveMetastoreClient::get_partitions_with_auth(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) +void ThriftHiveMetastoreClient::alter_partitions(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts) { - send_get_partitions_with_auth(db_name, tbl_name, max_parts, user_name, group_names); - recv_get_partitions_with_auth(_return); + send_alter_partitions(db_name, tbl_name, new_parts); + recv_alter_partitions(); } -void ThriftHiveMetastoreClient::send_get_partitions_with_auth(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) +void ThriftHiveMetastoreClient::send_alter_partitions(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partitions_with_auth", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("alter_partitions", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_partitions_with_auth_pargs args; + ThriftHiveMetastore_alter_partitions_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; - args.max_parts = &max_parts; - args.user_name = &user_name; - args.group_names = &group_names; + args.new_parts = &new_parts; args.write(oprot_); oprot_->writeMessageEnd(); @@ -22788,7 +25433,7 @@ void ThriftHiveMetastoreClient::send_get_partitions_with_auth(const std::string& oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_partitions_with_auth(std::vector & _return) +void ThriftHiveMetastoreClient::recv_alter_partitions() { int32_t rseqid = 0; @@ -22808,45 +25453,41 @@ void ThriftHiveMetastoreClient::recv_get_partitions_with_auth(std::vectorreadMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_partitions_with_auth") != 0) { + if (fname.compare("alter_partitions") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_partitions_with_auth_presult result; - result.success = &_return; + ThriftHiveMetastore_alter_partitions_presult result; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); - if (result.__isset.success) { - // _return pointer has now been filled - return; - } if (result.__isset.o1) { throw result.o1; } if (result.__isset.o2) { throw result.o2; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_with_auth failed: unknown result"); + return; } -void ThriftHiveMetastoreClient::get_partition_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) +void ThriftHiveMetastoreClient::alter_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const Partition& new_part, const EnvironmentContext& environment_context) { - send_get_partition_names(db_name, tbl_name, max_parts); - recv_get_partition_names(_return); + send_alter_partition_with_environment_context(db_name, tbl_name, new_part, environment_context); + recv_alter_partition_with_environment_context(); } -void ThriftHiveMetastoreClient::send_get_partition_names(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) +void ThriftHiveMetastoreClient::send_alter_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const Partition& new_part, const EnvironmentContext& environment_context) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partition_names", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("alter_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_partition_names_pargs args; + ThriftHiveMetastore_alter_partition_with_environment_context_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; - args.max_parts = &max_parts; + args.new_part = &new_part; + args.environment_context = &environment_context; args.write(oprot_); oprot_->writeMessageEnd(); @@ -22854,7 +25495,7 @@ void ThriftHiveMetastoreClient::send_get_partition_names(const std::string& db_n oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_partition_names(std::vector & _return) +void ThriftHiveMetastoreClient::recv_alter_partition_with_environment_context() { int32_t rseqid = 0; @@ -22874,43 +25515,41 @@ void ThriftHiveMetastoreClient::recv_get_partition_names(std::vectorreadMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_partition_names") != 0) { + if (fname.compare("alter_partition_with_environment_context") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_partition_names_presult result; - result.success = &_return; + ThriftHiveMetastore_alter_partition_with_environment_context_presult result; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); - if (result.__isset.success) { - // _return pointer has now been filled - return; + if (result.__isset.o1) { + throw result.o1; } if (result.__isset.o2) { throw result.o2; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_names failed: unknown result"); + return; } -void ThriftHiveMetastoreClient::get_partitions_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) +void ThriftHiveMetastoreClient::rename_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const Partition& new_part) { - send_get_partitions_ps(db_name, tbl_name, part_vals, max_parts); - recv_get_partitions_ps(_return); + send_rename_partition(db_name, tbl_name, part_vals, new_part); + recv_rename_partition(); } -void ThriftHiveMetastoreClient::send_get_partitions_ps(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) +void ThriftHiveMetastoreClient::send_rename_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const Partition& new_part) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partitions_ps", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("rename_partition", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_partitions_ps_pargs args; + ThriftHiveMetastore_rename_partition_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; args.part_vals = &part_vals; - args.max_parts = &max_parts; + args.new_part = &new_part; args.write(oprot_); oprot_->writeMessageEnd(); @@ -22918,7 +25557,7 @@ void ThriftHiveMetastoreClient::send_get_partitions_ps(const std::string& db_nam oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_partitions_ps(std::vector & _return) +void ThriftHiveMetastoreClient::recv_rename_partition() { int32_t rseqid = 0; @@ -22938,48 +25577,39 @@ void ThriftHiveMetastoreClient::recv_get_partitions_ps(std::vector & iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_partitions_ps") != 0) { + if (fname.compare("rename_partition") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_partitions_ps_presult result; - result.success = &_return; + ThriftHiveMetastore_rename_partition_presult result; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); - if (result.__isset.success) { - // _return pointer has now been filled - return; - } if (result.__isset.o1) { throw result.o1; } if (result.__isset.o2) { throw result.o2; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_ps failed: unknown result"); + return; } -void ThriftHiveMetastoreClient::get_partitions_ps_with_auth(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) +bool ThriftHiveMetastoreClient::partition_name_has_valid_characters(const std::vector & part_vals, const bool throw_exception) { - send_get_partitions_ps_with_auth(db_name, tbl_name, part_vals, max_parts, user_name, group_names); - recv_get_partitions_ps_with_auth(_return); + send_partition_name_has_valid_characters(part_vals, throw_exception); + return recv_partition_name_has_valid_characters(); } -void ThriftHiveMetastoreClient::send_get_partitions_ps_with_auth(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) +void ThriftHiveMetastoreClient::send_partition_name_has_valid_characters(const std::vector & part_vals, const bool throw_exception) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partitions_ps_with_auth", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("partition_name_has_valid_characters", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_partitions_ps_with_auth_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; + ThriftHiveMetastore_partition_name_has_valid_characters_pargs args; args.part_vals = &part_vals; - args.max_parts = &max_parts; - args.user_name = &user_name; - args.group_names = &group_names; + args.throw_exception = &throw_exception; args.write(oprot_); oprot_->writeMessageEnd(); @@ -22987,7 +25617,7 @@ void ThriftHiveMetastoreClient::send_get_partitions_ps_with_auth(const std::stri oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_partitions_ps_with_auth(std::vector & _return) +bool ThriftHiveMetastoreClient::recv_partition_name_has_valid_characters() { int32_t rseqid = 0; @@ -23007,46 +25637,41 @@ void ThriftHiveMetastoreClient::recv_get_partitions_ps_with_auth(std::vectorreadMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_partitions_ps_with_auth") != 0) { + if (fname.compare("partition_name_has_valid_characters") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_partitions_ps_with_auth_presult result; + bool _return; + ThriftHiveMetastore_partition_name_has_valid_characters_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); if (result.__isset.success) { - // _return pointer has now been filled - return; + return _return; } if (result.__isset.o1) { throw result.o1; } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_ps_with_auth failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "partition_name_has_valid_characters failed: unknown result"); } -void ThriftHiveMetastoreClient::get_partition_names_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) +void ThriftHiveMetastoreClient::get_config_value(std::string& _return, const std::string& name, const std::string& defaultValue) { - send_get_partition_names_ps(db_name, tbl_name, part_vals, max_parts); - recv_get_partition_names_ps(_return); + send_get_config_value(name, defaultValue); + recv_get_config_value(_return); } -void ThriftHiveMetastoreClient::send_get_partition_names_ps(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) +void ThriftHiveMetastoreClient::send_get_config_value(const std::string& name, const std::string& defaultValue) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partition_names_ps", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_config_value", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_partition_names_ps_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.max_parts = &max_parts; + ThriftHiveMetastore_get_config_value_pargs args; + args.name = &name; + args.defaultValue = &defaultValue; args.write(oprot_); oprot_->writeMessageEnd(); @@ -23054,7 +25679,7 @@ void ThriftHiveMetastoreClient::send_get_partition_names_ps(const std::string& d oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_partition_names_ps(std::vector & _return) +void ThriftHiveMetastoreClient::recv_get_config_value(std::string& _return) { int32_t rseqid = 0; @@ -23074,12 +25699,12 @@ void ThriftHiveMetastoreClient::recv_get_partition_names_ps(std::vectorreadMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_partition_names_ps") != 0) { + if (fname.compare("get_config_value") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_partition_names_ps_presult result; + ThriftHiveMetastore_get_config_value_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -23092,28 +25717,22 @@ void ThriftHiveMetastoreClient::recv_get_partition_names_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int16_t max_parts) +void ThriftHiveMetastoreClient::partition_name_to_vals(std::vector & _return, const std::string& part_name) { - send_get_partitions_by_filter(db_name, tbl_name, filter, max_parts); - recv_get_partitions_by_filter(_return); + send_partition_name_to_vals(part_name); + recv_partition_name_to_vals(_return); } -void ThriftHiveMetastoreClient::send_get_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int16_t max_parts) +void ThriftHiveMetastoreClient::send_partition_name_to_vals(const std::string& part_name) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partitions_by_filter", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("partition_name_to_vals", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_partitions_by_filter_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.filter = &filter; - args.max_parts = &max_parts; + ThriftHiveMetastore_partition_name_to_vals_pargs args; + args.part_name = &part_name; args.write(oprot_); oprot_->writeMessageEnd(); @@ -23121,7 +25740,7 @@ void ThriftHiveMetastoreClient::send_get_partitions_by_filter(const std::string& oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_partitions_by_filter(std::vector & _return) +void ThriftHiveMetastoreClient::recv_partition_name_to_vals(std::vector & _return) { int32_t rseqid = 0; @@ -23141,12 +25760,12 @@ void ThriftHiveMetastoreClient::recv_get_partitions_by_filter(std::vectorreadMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_partitions_by_filter") != 0) { + if (fname.compare("partition_name_to_vals") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_partitions_by_filter_presult result; + ThriftHiveMetastore_partition_name_to_vals_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -23159,27 +25778,22 @@ void ThriftHiveMetastoreClient::recv_get_partitions_by_filter(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & names) +void ThriftHiveMetastoreClient::partition_name_to_spec(std::map & _return, const std::string& part_name) { - send_get_partitions_by_names(db_name, tbl_name, names); - recv_get_partitions_by_names(_return); + send_partition_name_to_spec(part_name); + recv_partition_name_to_spec(_return); } -void ThriftHiveMetastoreClient::send_get_partitions_by_names(const std::string& db_name, const std::string& tbl_name, const std::vector & names) +void ThriftHiveMetastoreClient::send_partition_name_to_spec(const std::string& part_name) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partitions_by_names", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("partition_name_to_spec", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_partitions_by_names_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.names = &names; + ThriftHiveMetastore_partition_name_to_spec_pargs args; + args.part_name = &part_name; args.write(oprot_); oprot_->writeMessageEnd(); @@ -23187,7 +25801,7 @@ void ThriftHiveMetastoreClient::send_get_partitions_by_names(const std::string& oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_partitions_by_names(std::vector & _return) +void ThriftHiveMetastoreClient::recv_partition_name_to_spec(std::map & _return) { int32_t rseqid = 0; @@ -23207,12 +25821,12 @@ void ThriftHiveMetastoreClient::recv_get_partitions_by_names(std::vectorreadMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_partitions_by_names") != 0) { + if (fname.compare("partition_name_to_spec") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_partitions_by_names_presult result; + ThriftHiveMetastore_partition_name_to_spec_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -23225,27 +25839,25 @@ void ThriftHiveMetastoreClient::recv_get_partitions_by_names(std::vector & part_vals, const PartitionEventType::type eventType) { - send_alter_partition(db_name, tbl_name, new_part); - recv_alter_partition(); + send_markPartitionForEvent(db_name, tbl_name, part_vals, eventType); + recv_markPartitionForEvent(); } -void ThriftHiveMetastoreClient::send_alter_partition(const std::string& db_name, const std::string& tbl_name, const Partition& new_part) +void ThriftHiveMetastoreClient::send_markPartitionForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) { int32_t cseqid = 0; - oprot_->writeMessageBegin("alter_partition", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("markPartitionForEvent", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_alter_partition_pargs args; + ThriftHiveMetastore_markPartitionForEvent_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; - args.new_part = &new_part; + args.part_vals = &part_vals; + args.eventType = &eventType; args.write(oprot_); oprot_->writeMessageEnd(); @@ -23253,7 +25865,7 @@ void ThriftHiveMetastoreClient::send_alter_partition(const std::string& db_name, oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_alter_partition() +void ThriftHiveMetastoreClient::recv_markPartitionForEvent() { int32_t rseqid = 0; @@ -23273,12 +25885,12 @@ void ThriftHiveMetastoreClient::recv_alter_partition() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("alter_partition") != 0) { + if (fname.compare("markPartitionForEvent") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_alter_partition_presult result; + ThriftHiveMetastore_markPartitionForEvent_presult result; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); @@ -23289,24 +25901,37 @@ void ThriftHiveMetastoreClient::recv_alter_partition() if (result.__isset.o2) { throw result.o2; } + if (result.__isset.o3) { + throw result.o3; + } + if (result.__isset.o4) { + throw result.o4; + } + if (result.__isset.o5) { + throw result.o5; + } + if (result.__isset.o6) { + throw result.o6; + } return; } -void ThriftHiveMetastoreClient::alter_partitions(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts) +bool ThriftHiveMetastoreClient::isPartitionMarkedForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) { - send_alter_partitions(db_name, tbl_name, new_parts); - recv_alter_partitions(); + send_isPartitionMarkedForEvent(db_name, tbl_name, part_vals, eventType); + return recv_isPartitionMarkedForEvent(); } -void ThriftHiveMetastoreClient::send_alter_partitions(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts) +void ThriftHiveMetastoreClient::send_isPartitionMarkedForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) { int32_t cseqid = 0; - oprot_->writeMessageBegin("alter_partitions", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("isPartitionMarkedForEvent", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_alter_partitions_pargs args; + ThriftHiveMetastore_isPartitionMarkedForEvent_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; - args.new_parts = &new_parts; + args.part_vals = &part_vals; + args.eventType = &eventType; args.write(oprot_); oprot_->writeMessageEnd(); @@ -23314,7 +25939,7 @@ void ThriftHiveMetastoreClient::send_alter_partitions(const std::string& db_name oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_alter_partitions() +bool ThriftHiveMetastoreClient::recv_isPartitionMarkedForEvent() { int32_t rseqid = 0; @@ -23334,41 +25959,56 @@ void ThriftHiveMetastoreClient::recv_alter_partitions() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("alter_partitions") != 0) { + if (fname.compare("isPartitionMarkedForEvent") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_alter_partitions_presult result; + bool _return; + ThriftHiveMetastore_isPartitionMarkedForEvent_presult result; + result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); + if (result.__isset.success) { + return _return; + } if (result.__isset.o1) { throw result.o1; } if (result.__isset.o2) { throw result.o2; } - return; + if (result.__isset.o3) { + throw result.o3; + } + if (result.__isset.o4) { + throw result.o4; + } + if (result.__isset.o5) { + throw result.o5; + } + if (result.__isset.o6) { + throw result.o6; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "isPartitionMarkedForEvent failed: unknown result"); } -void ThriftHiveMetastoreClient::alter_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const Partition& new_part, const EnvironmentContext& environment_context) +void ThriftHiveMetastoreClient::add_index(Index& _return, const Index& new_index, const Table& index_table) { - send_alter_partition_with_environment_context(db_name, tbl_name, new_part, environment_context); - recv_alter_partition_with_environment_context(); + send_add_index(new_index, index_table); + recv_add_index(_return); } -void ThriftHiveMetastoreClient::send_alter_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const Partition& new_part, const EnvironmentContext& environment_context) +void ThriftHiveMetastoreClient::send_add_index(const Index& new_index, const Table& index_table) { int32_t cseqid = 0; - oprot_->writeMessageBegin("alter_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("add_index", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_alter_partition_with_environment_context_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.new_part = &new_part; - args.environment_context = &environment_context; + ThriftHiveMetastore_add_index_pargs args; + args.new_index = &new_index; + args.index_table = &index_table; args.write(oprot_); oprot_->writeMessageEnd(); @@ -23376,7 +26016,7 @@ void ThriftHiveMetastoreClient::send_alter_partition_with_environment_context(co oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_alter_partition_with_environment_context() +void ThriftHiveMetastoreClient::recv_add_index(Index& _return) { int32_t rseqid = 0; @@ -23396,41 +26036,49 @@ void ThriftHiveMetastoreClient::recv_alter_partition_with_environment_context() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("alter_partition_with_environment_context") != 0) { + if (fname.compare("add_index") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_alter_partition_with_environment_context_presult result; + ThriftHiveMetastore_add_index_presult result; + result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); + if (result.__isset.success) { + // _return pointer has now been filled + return; + } if (result.__isset.o1) { throw result.o1; } if (result.__isset.o2) { throw result.o2; } - return; + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_index failed: unknown result"); } -void ThriftHiveMetastoreClient::rename_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const Partition& new_part) +void ThriftHiveMetastoreClient::alter_index(const std::string& dbname, const std::string& base_tbl_name, const std::string& idx_name, const Index& new_idx) { - send_rename_partition(db_name, tbl_name, part_vals, new_part); - recv_rename_partition(); + send_alter_index(dbname, base_tbl_name, idx_name, new_idx); + recv_alter_index(); } -void ThriftHiveMetastoreClient::send_rename_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const Partition& new_part) +void ThriftHiveMetastoreClient::send_alter_index(const std::string& dbname, const std::string& base_tbl_name, const std::string& idx_name, const Index& new_idx) { int32_t cseqid = 0; - oprot_->writeMessageBegin("rename_partition", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("alter_index", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_rename_partition_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.new_part = &new_part; + ThriftHiveMetastore_alter_index_pargs args; + args.dbname = &dbname; + args.base_tbl_name = &base_tbl_name; + args.idx_name = &idx_name; + args.new_idx = &new_idx; args.write(oprot_); oprot_->writeMessageEnd(); @@ -23438,7 +26086,7 @@ void ThriftHiveMetastoreClient::send_rename_partition(const std::string& db_name oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_rename_partition() +void ThriftHiveMetastoreClient::recv_alter_index() { int32_t rseqid = 0; @@ -23458,12 +26106,12 @@ void ThriftHiveMetastoreClient::recv_rename_partition() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("rename_partition") != 0) { + if (fname.compare("alter_index") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_rename_partition_presult result; + ThriftHiveMetastore_alter_index_presult result; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); @@ -23477,20 +26125,22 @@ void ThriftHiveMetastoreClient::recv_rename_partition() return; } -bool ThriftHiveMetastoreClient::partition_name_has_valid_characters(const std::vector & part_vals, const bool throw_exception) +bool ThriftHiveMetastoreClient::drop_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name, const bool deleteData) { - send_partition_name_has_valid_characters(part_vals, throw_exception); - return recv_partition_name_has_valid_characters(); + send_drop_index_by_name(db_name, tbl_name, index_name, deleteData); + return recv_drop_index_by_name(); } -void ThriftHiveMetastoreClient::send_partition_name_has_valid_characters(const std::vector & part_vals, const bool throw_exception) +void ThriftHiveMetastoreClient::send_drop_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name, const bool deleteData) { int32_t cseqid = 0; - oprot_->writeMessageBegin("partition_name_has_valid_characters", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("drop_index_by_name", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_partition_name_has_valid_characters_pargs args; - args.part_vals = &part_vals; - args.throw_exception = &throw_exception; + ThriftHiveMetastore_drop_index_by_name_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.index_name = &index_name; + args.deleteData = &deleteData; args.write(oprot_); oprot_->writeMessageEnd(); @@ -23498,7 +26148,7 @@ void ThriftHiveMetastoreClient::send_partition_name_has_valid_characters(const s oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_partition_name_has_valid_characters() +bool ThriftHiveMetastoreClient::recv_drop_index_by_name() { int32_t rseqid = 0; @@ -23518,13 +26168,13 @@ bool ThriftHiveMetastoreClient::recv_partition_name_has_valid_characters() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("partition_name_has_valid_characters") != 0) { + if (fname.compare("drop_index_by_name") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } bool _return; - ThriftHiveMetastore_partition_name_has_valid_characters_presult result; + ThriftHiveMetastore_drop_index_by_name_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -23536,23 +26186,27 @@ bool ThriftHiveMetastoreClient::recv_partition_name_has_valid_characters() if (result.__isset.o1) { throw result.o1; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "partition_name_has_valid_characters failed: unknown result"); + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_index_by_name failed: unknown result"); } -void ThriftHiveMetastoreClient::get_config_value(std::string& _return, const std::string& name, const std::string& defaultValue) +void ThriftHiveMetastoreClient::get_index_by_name(Index& _return, const std::string& db_name, const std::string& tbl_name, const std::string& index_name) { - send_get_config_value(name, defaultValue); - recv_get_config_value(_return); + send_get_index_by_name(db_name, tbl_name, index_name); + recv_get_index_by_name(_return); } -void ThriftHiveMetastoreClient::send_get_config_value(const std::string& name, const std::string& defaultValue) +void ThriftHiveMetastoreClient::send_get_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_config_value", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_index_by_name", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_config_value_pargs args; - args.name = &name; - args.defaultValue = &defaultValue; + ThriftHiveMetastore_get_index_by_name_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.index_name = &index_name; args.write(oprot_); oprot_->writeMessageEnd(); @@ -23560,7 +26214,7 @@ void ThriftHiveMetastoreClient::send_get_config_value(const std::string& name, c oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_config_value(std::string& _return) +void ThriftHiveMetastoreClient::recv_get_index_by_name(Index& _return) { int32_t rseqid = 0; @@ -23580,12 +26234,12 @@ void ThriftHiveMetastoreClient::recv_get_config_value(std::string& _return) iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_config_value") != 0) { + if (fname.compare("get_index_by_name") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_config_value_presult result; + ThriftHiveMetastore_get_index_by_name_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -23598,22 +26252,27 @@ void ThriftHiveMetastoreClient::recv_get_config_value(std::string& _return) if (result.__isset.o1) { throw result.o1; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_config_value failed: unknown result"); + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_index_by_name failed: unknown result"); } -void ThriftHiveMetastoreClient::partition_name_to_vals(std::vector & _return, const std::string& part_name) +void ThriftHiveMetastoreClient::get_indexes(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) { - send_partition_name_to_vals(part_name); - recv_partition_name_to_vals(_return); + send_get_indexes(db_name, tbl_name, max_indexes); + recv_get_indexes(_return); } -void ThriftHiveMetastoreClient::send_partition_name_to_vals(const std::string& part_name) +void ThriftHiveMetastoreClient::send_get_indexes(const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) { int32_t cseqid = 0; - oprot_->writeMessageBegin("partition_name_to_vals", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_indexes", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_partition_name_to_vals_pargs args; - args.part_name = &part_name; + ThriftHiveMetastore_get_indexes_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.max_indexes = &max_indexes; args.write(oprot_); oprot_->writeMessageEnd(); @@ -23621,7 +26280,7 @@ void ThriftHiveMetastoreClient::send_partition_name_to_vals(const std::string& p oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_partition_name_to_vals(std::vector & _return) +void ThriftHiveMetastoreClient::recv_get_indexes(std::vector & _return) { int32_t rseqid = 0; @@ -23641,12 +26300,12 @@ void ThriftHiveMetastoreClient::recv_partition_name_to_vals(std::vectorreadMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("partition_name_to_vals") != 0) { + if (fname.compare("get_indexes") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_partition_name_to_vals_presult result; + ThriftHiveMetastore_get_indexes_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -23659,22 +26318,27 @@ void ThriftHiveMetastoreClient::recv_partition_name_to_vals(std::vector & _return, const std::string& part_name) +void ThriftHiveMetastoreClient::get_index_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) { - send_partition_name_to_spec(part_name); - recv_partition_name_to_spec(_return); + send_get_index_names(db_name, tbl_name, max_indexes); + recv_get_index_names(_return); } -void ThriftHiveMetastoreClient::send_partition_name_to_spec(const std::string& part_name) +void ThriftHiveMetastoreClient::send_get_index_names(const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) { int32_t cseqid = 0; - oprot_->writeMessageBegin("partition_name_to_spec", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_index_names", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_partition_name_to_spec_pargs args; - args.part_name = &part_name; + ThriftHiveMetastore_get_index_names_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.max_indexes = &max_indexes; args.write(oprot_); oprot_->writeMessageEnd(); @@ -23682,7 +26346,7 @@ void ThriftHiveMetastoreClient::send_partition_name_to_spec(const std::string& p oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_partition_name_to_spec(std::map & _return) +void ThriftHiveMetastoreClient::recv_get_index_names(std::vector & _return) { int32_t rseqid = 0; @@ -23702,12 +26366,12 @@ void ThriftHiveMetastoreClient::recv_partition_name_to_spec(std::mapreadMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("partition_name_to_spec") != 0) { + if (fname.compare("get_index_names") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_partition_name_to_spec_presult result; + ThriftHiveMetastore_get_index_names_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -23717,28 +26381,25 @@ void ThriftHiveMetastoreClient::recv_partition_name_to_spec(std::map & part_vals, const PartitionEventType::type eventType) +bool ThriftHiveMetastoreClient::update_table_column_statistics(const ColumnStatistics& stats_obj) { - send_markPartitionForEvent(db_name, tbl_name, part_vals, eventType); - recv_markPartitionForEvent(); + send_update_table_column_statistics(stats_obj); + return recv_update_table_column_statistics(); } -void ThriftHiveMetastoreClient::send_markPartitionForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) +void ThriftHiveMetastoreClient::send_update_table_column_statistics(const ColumnStatistics& stats_obj) { int32_t cseqid = 0; - oprot_->writeMessageBegin("markPartitionForEvent", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("update_table_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_markPartitionForEvent_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.eventType = &eventType; + ThriftHiveMetastore_update_table_column_statistics_pargs args; + args.stats_obj = &stats_obj; args.write(oprot_); oprot_->writeMessageEnd(); @@ -23746,7 +26407,7 @@ void ThriftHiveMetastoreClient::send_markPartitionForEvent(const std::string& db oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_markPartitionForEvent() +bool ThriftHiveMetastoreClient::recv_update_table_column_statistics() { int32_t rseqid = 0; @@ -23766,16 +26427,21 @@ void ThriftHiveMetastoreClient::recv_markPartitionForEvent() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("markPartitionForEvent") != 0) { + if (fname.compare("update_table_column_statistics") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_markPartitionForEvent_presult result; + bool _return; + ThriftHiveMetastore_update_table_column_statistics_presult result; + result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); + if (result.__isset.success) { + return _return; + } if (result.__isset.o1) { throw result.o1; } @@ -23788,31 +26454,22 @@ void ThriftHiveMetastoreClient::recv_markPartitionForEvent() if (result.__isset.o4) { throw result.o4; } - if (result.__isset.o5) { - throw result.o5; - } - if (result.__isset.o6) { - throw result.o6; - } - return; + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "update_table_column_statistics failed: unknown result"); } -bool ThriftHiveMetastoreClient::isPartitionMarkedForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) +bool ThriftHiveMetastoreClient::update_partition_column_statistics(const ColumnStatistics& stats_obj) { - send_isPartitionMarkedForEvent(db_name, tbl_name, part_vals, eventType); - return recv_isPartitionMarkedForEvent(); + send_update_partition_column_statistics(stats_obj); + return recv_update_partition_column_statistics(); } -void ThriftHiveMetastoreClient::send_isPartitionMarkedForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) +void ThriftHiveMetastoreClient::send_update_partition_column_statistics(const ColumnStatistics& stats_obj) { int32_t cseqid = 0; - oprot_->writeMessageBegin("isPartitionMarkedForEvent", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("update_partition_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_isPartitionMarkedForEvent_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.eventType = &eventType; + ThriftHiveMetastore_update_partition_column_statistics_pargs args; + args.stats_obj = &stats_obj; args.write(oprot_); oprot_->writeMessageEnd(); @@ -23820,7 +26477,7 @@ void ThriftHiveMetastoreClient::send_isPartitionMarkedForEvent(const std::string oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_isPartitionMarkedForEvent() +bool ThriftHiveMetastoreClient::recv_update_partition_column_statistics() { int32_t rseqid = 0; @@ -23840,13 +26497,13 @@ bool ThriftHiveMetastoreClient::recv_isPartitionMarkedForEvent() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("isPartitionMarkedForEvent") != 0) { + if (fname.compare("update_partition_column_statistics") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } bool _return; - ThriftHiveMetastore_isPartitionMarkedForEvent_presult result; + ThriftHiveMetastore_update_partition_column_statistics_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -23867,29 +26524,24 @@ bool ThriftHiveMetastoreClient::recv_isPartitionMarkedForEvent() if (result.__isset.o4) { throw result.o4; } - if (result.__isset.o5) { - throw result.o5; - } - if (result.__isset.o6) { - throw result.o6; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "isPartitionMarkedForEvent failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "update_partition_column_statistics failed: unknown result"); } -void ThriftHiveMetastoreClient::add_index(Index& _return, const Index& new_index, const Table& index_table) +void ThriftHiveMetastoreClient::get_table_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& col_name) { - send_add_index(new_index, index_table); - recv_add_index(_return); + send_get_table_column_statistics(db_name, tbl_name, col_name); + recv_get_table_column_statistics(_return); } -void ThriftHiveMetastoreClient::send_add_index(const Index& new_index, const Table& index_table) +void ThriftHiveMetastoreClient::send_get_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name) { int32_t cseqid = 0; - oprot_->writeMessageBegin("add_index", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_table_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_add_index_pargs args; - args.new_index = &new_index; - args.index_table = &index_table; + ThriftHiveMetastore_get_table_column_statistics_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.col_name = &col_name; args.write(oprot_); oprot_->writeMessageEnd(); @@ -23897,7 +26549,7 @@ void ThriftHiveMetastoreClient::send_add_index(const Index& new_index, const Tab oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_add_index(Index& _return) +void ThriftHiveMetastoreClient::recv_get_table_column_statistics(ColumnStatistics& _return) { int32_t rseqid = 0; @@ -23917,12 +26569,12 @@ void ThriftHiveMetastoreClient::recv_add_index(Index& _return) iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("add_index") != 0) { + if (fname.compare("get_table_column_statistics") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_add_index_presult result; + ThriftHiveMetastore_get_table_column_statistics_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -23941,25 +26593,28 @@ void ThriftHiveMetastoreClient::recv_add_index(Index& _return) if (result.__isset.o3) { throw result.o3; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_index failed: unknown result"); + if (result.__isset.o4) { + throw result.o4; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_column_statistics failed: unknown result"); } -void ThriftHiveMetastoreClient::alter_index(const std::string& dbname, const std::string& base_tbl_name, const std::string& idx_name, const Index& new_idx) +void ThriftHiveMetastoreClient::get_partition_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) { - send_alter_index(dbname, base_tbl_name, idx_name, new_idx); - recv_alter_index(); + send_get_partition_column_statistics(db_name, tbl_name, part_name, col_name); + recv_get_partition_column_statistics(_return); } -void ThriftHiveMetastoreClient::send_alter_index(const std::string& dbname, const std::string& base_tbl_name, const std::string& idx_name, const Index& new_idx) +void ThriftHiveMetastoreClient::send_get_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) { int32_t cseqid = 0; - oprot_->writeMessageBegin("alter_index", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_partition_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_alter_index_pargs args; - args.dbname = &dbname; - args.base_tbl_name = &base_tbl_name; - args.idx_name = &idx_name; - args.new_idx = &new_idx; + ThriftHiveMetastore_get_partition_column_statistics_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_name = &part_name; + args.col_name = &col_name; args.write(oprot_); oprot_->writeMessageEnd(); @@ -23967,7 +26622,7 @@ void ThriftHiveMetastoreClient::send_alter_index(const std::string& dbname, cons oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_alter_index() +void ThriftHiveMetastoreClient::recv_get_partition_column_statistics(ColumnStatistics& _return) { int32_t rseqid = 0; @@ -23987,41 +26642,52 @@ void ThriftHiveMetastoreClient::recv_alter_index() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("alter_index") != 0) { + if (fname.compare("get_partition_column_statistics") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_alter_index_presult result; + ThriftHiveMetastore_get_partition_column_statistics_presult result; + result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); + if (result.__isset.success) { + // _return pointer has now been filled + return; + } if (result.__isset.o1) { throw result.o1; } if (result.__isset.o2) { throw result.o2; } - return; + if (result.__isset.o3) { + throw result.o3; + } + if (result.__isset.o4) { + throw result.o4; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_column_statistics failed: unknown result"); } -bool ThriftHiveMetastoreClient::drop_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name, const bool deleteData) +bool ThriftHiveMetastoreClient::delete_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) { - send_drop_index_by_name(db_name, tbl_name, index_name, deleteData); - return recv_drop_index_by_name(); + send_delete_partition_column_statistics(db_name, tbl_name, part_name, col_name); + return recv_delete_partition_column_statistics(); } -void ThriftHiveMetastoreClient::send_drop_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name, const bool deleteData) +void ThriftHiveMetastoreClient::send_delete_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) { int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_index_by_name", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("delete_partition_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_drop_index_by_name_pargs args; + ThriftHiveMetastore_delete_partition_column_statistics_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; - args.index_name = &index_name; - args.deleteData = &deleteData; + args.part_name = &part_name; + args.col_name = &col_name; args.write(oprot_); oprot_->writeMessageEnd(); @@ -24029,7 +26695,7 @@ void ThriftHiveMetastoreClient::send_drop_index_by_name(const std::string& db_na oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_drop_index_by_name() +bool ThriftHiveMetastoreClient::recv_delete_partition_column_statistics() { int32_t rseqid = 0; @@ -24049,13 +26715,13 @@ bool ThriftHiveMetastoreClient::recv_drop_index_by_name() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("drop_index_by_name") != 0) { + if (fname.compare("delete_partition_column_statistics") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } bool _return; - ThriftHiveMetastore_drop_index_by_name_presult result; + ThriftHiveMetastore_delete_partition_column_statistics_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -24070,24 +26736,30 @@ bool ThriftHiveMetastoreClient::recv_drop_index_by_name() if (result.__isset.o2) { throw result.o2; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_index_by_name failed: unknown result"); + if (result.__isset.o3) { + throw result.o3; + } + if (result.__isset.o4) { + throw result.o4; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "delete_partition_column_statistics failed: unknown result"); } -void ThriftHiveMetastoreClient::get_index_by_name(Index& _return, const std::string& db_name, const std::string& tbl_name, const std::string& index_name) +bool ThriftHiveMetastoreClient::delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name) { - send_get_index_by_name(db_name, tbl_name, index_name); - recv_get_index_by_name(_return); + send_delete_table_column_statistics(db_name, tbl_name, col_name); + return recv_delete_table_column_statistics(); } -void ThriftHiveMetastoreClient::send_get_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name) +void ThriftHiveMetastoreClient::send_delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_index_by_name", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("delete_table_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_index_by_name_pargs args; + ThriftHiveMetastore_delete_table_column_statistics_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; - args.index_name = &index_name; + args.col_name = &col_name; args.write(oprot_); oprot_->writeMessageEnd(); @@ -24095,7 +26767,7 @@ void ThriftHiveMetastoreClient::send_get_index_by_name(const std::string& db_nam oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_index_by_name(Index& _return) +bool ThriftHiveMetastoreClient::recv_delete_table_column_statistics() { int32_t rseqid = 0; @@ -24115,20 +26787,20 @@ void ThriftHiveMetastoreClient::recv_get_index_by_name(Index& _return) iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_index_by_name") != 0) { + if (fname.compare("delete_table_column_statistics") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_index_by_name_presult result; + bool _return; + ThriftHiveMetastore_delete_table_column_statistics_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); if (result.__isset.success) { - // _return pointer has now been filled - return; + return _return; } if (result.__isset.o1) { throw result.o1; @@ -24136,24 +26808,29 @@ void ThriftHiveMetastoreClient::recv_get_index_by_name(Index& _return) if (result.__isset.o2) { throw result.o2; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_index_by_name failed: unknown result"); + if (result.__isset.o3) { + throw result.o3; + } + if (result.__isset.o4) { + throw result.o4; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "delete_table_column_statistics failed: unknown result"); } -void ThriftHiveMetastoreClient::get_indexes(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) +void ThriftHiveMetastoreClient::streamingStatus(std::map & _return, const std::string& db_name, const std::string& table_name) { - send_get_indexes(db_name, tbl_name, max_indexes); - recv_get_indexes(_return); + send_streamingStatus(db_name, table_name); + recv_streamingStatus(_return); } -void ThriftHiveMetastoreClient::send_get_indexes(const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) +void ThriftHiveMetastoreClient::send_streamingStatus(const std::string& db_name, const std::string& table_name) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_indexes", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("streamingStatus", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_indexes_pargs args; + ThriftHiveMetastore_streamingStatus_pargs args; args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.max_indexes = &max_indexes; + args.table_name = &table_name; args.write(oprot_); oprot_->writeMessageEnd(); @@ -24161,7 +26838,7 @@ void ThriftHiveMetastoreClient::send_get_indexes(const std::string& db_name, con oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_indexes(std::vector & _return) +void ThriftHiveMetastoreClient::recv_streamingStatus(std::map & _return) { int32_t rseqid = 0; @@ -24181,12 +26858,12 @@ void ThriftHiveMetastoreClient::recv_get_indexes(std::vector & _return) iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_indexes") != 0) { + if (fname.compare("streamingStatus") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_indexes_presult result; + ThriftHiveMetastore_streamingStatus_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -24202,24 +26879,29 @@ void ThriftHiveMetastoreClient::recv_get_indexes(std::vector & _return) if (result.__isset.o2) { throw result.o2; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_indexes failed: unknown result"); + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "streamingStatus failed: unknown result"); } -void ThriftHiveMetastoreClient::get_index_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) +void ThriftHiveMetastoreClient::enableStreaming(const std::string& db_name, const std::string& table_name, const std::string& streaming_tmp_dir, const std::string& partition_path, const std::string& partition_val) { - send_get_index_names(db_name, tbl_name, max_indexes); - recv_get_index_names(_return); + send_enableStreaming(db_name, table_name, streaming_tmp_dir, partition_path, partition_val); + recv_enableStreaming(); } -void ThriftHiveMetastoreClient::send_get_index_names(const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) +void ThriftHiveMetastoreClient::send_enableStreaming(const std::string& db_name, const std::string& table_name, const std::string& streaming_tmp_dir, const std::string& partition_path, const std::string& partition_val) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_index_names", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("enableStreaming", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_index_names_pargs args; + ThriftHiveMetastore_enableStreaming_pargs args; args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.max_indexes = &max_indexes; + args.table_name = &table_name; + args.streaming_tmp_dir = &streaming_tmp_dir; + args.partition_path = &partition_path; + args.partition_val = &partition_val; args.write(oprot_); oprot_->writeMessageEnd(); @@ -24227,7 +26909,7 @@ void ThriftHiveMetastoreClient::send_get_index_names(const std::string& db_name, oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_index_names(std::vector & _return) +void ThriftHiveMetastoreClient::recv_enableStreaming() { int32_t rseqid = 0; @@ -24247,40 +26929,45 @@ void ThriftHiveMetastoreClient::recv_get_index_names(std::vector & iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_index_names") != 0) { + if (fname.compare("enableStreaming") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_index_names_presult result; - result.success = &_return; + ThriftHiveMetastore_enableStreaming_presult result; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); - if (result.__isset.success) { - // _return pointer has now been filled - return; + if (result.__isset.o1) { + throw result.o1; } if (result.__isset.o2) { throw result.o2; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_index_names failed: unknown result"); + if (result.__isset.o3) { + throw result.o3; + } + if (result.__isset.o4) { + throw result.o4; + } + return; } -bool ThriftHiveMetastoreClient::update_table_column_statistics(const ColumnStatistics& stats_obj) +void ThriftHiveMetastoreClient::disableStreaming(const std::string& db_name, const std::string& table_name) { - send_update_table_column_statistics(stats_obj); - return recv_update_table_column_statistics(); + send_disableStreaming(db_name, table_name); + recv_disableStreaming(); } -void ThriftHiveMetastoreClient::send_update_table_column_statistics(const ColumnStatistics& stats_obj) +void ThriftHiveMetastoreClient::send_disableStreaming(const std::string& db_name, const std::string& table_name) { int32_t cseqid = 0; - oprot_->writeMessageBegin("update_table_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("disableStreaming", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_update_table_column_statistics_pargs args; - args.stats_obj = &stats_obj; + ThriftHiveMetastore_disableStreaming_pargs args; + args.db_name = &db_name; + args.table_name = &table_name; args.write(oprot_); oprot_->writeMessageEnd(); @@ -24288,7 +26975,7 @@ void ThriftHiveMetastoreClient::send_update_table_column_statistics(const Column oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_update_table_column_statistics() +void ThriftHiveMetastoreClient::recv_disableStreaming() { int32_t rseqid = 0; @@ -24308,21 +26995,16 @@ bool ThriftHiveMetastoreClient::recv_update_table_column_statistics() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("update_table_column_statistics") != 0) { + if (fname.compare("disableStreaming") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - bool _return; - ThriftHiveMetastore_update_table_column_statistics_presult result; - result.success = &_return; + ThriftHiveMetastore_disableStreaming_presult result; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); - if (result.__isset.success) { - return _return; - } if (result.__isset.o1) { throw result.o1; } @@ -24335,22 +27017,23 @@ bool ThriftHiveMetastoreClient::recv_update_table_column_statistics() if (result.__isset.o4) { throw result.o4; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "update_table_column_statistics failed: unknown result"); + return; } -bool ThriftHiveMetastoreClient::update_partition_column_statistics(const ColumnStatistics& stats_obj) +void ThriftHiveMetastoreClient::getCurrentStreamingPartitionPath(std::string& _return, const std::string& dbName, const std::string& tableName) { - send_update_partition_column_statistics(stats_obj); - return recv_update_partition_column_statistics(); + send_getCurrentStreamingPartitionPath(dbName, tableName); + recv_getCurrentStreamingPartitionPath(_return); } -void ThriftHiveMetastoreClient::send_update_partition_column_statistics(const ColumnStatistics& stats_obj) +void ThriftHiveMetastoreClient::send_getCurrentStreamingPartitionPath(const std::string& dbName, const std::string& tableName) { int32_t cseqid = 0; - oprot_->writeMessageBegin("update_partition_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("getCurrentStreamingPartitionPath", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_update_partition_column_statistics_pargs args; - args.stats_obj = &stats_obj; + ThriftHiveMetastore_getCurrentStreamingPartitionPath_pargs args; + args.dbName = &dbName; + args.tableName = &tableName; args.write(oprot_); oprot_->writeMessageEnd(); @@ -24358,7 +27041,7 @@ void ThriftHiveMetastoreClient::send_update_partition_column_statistics(const Co oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_update_partition_column_statistics() +void ThriftHiveMetastoreClient::recv_getCurrentStreamingPartitionPath(std::string& _return) { int32_t rseqid = 0; @@ -24378,20 +27061,20 @@ bool ThriftHiveMetastoreClient::recv_update_partition_column_statistics() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("update_partition_column_statistics") != 0) { + if (fname.compare("getCurrentStreamingPartitionPath") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - bool _return; - ThriftHiveMetastore_update_partition_column_statistics_presult result; + ThriftHiveMetastore_getCurrentStreamingPartitionPath_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); if (result.__isset.success) { - return _return; + // _return pointer has now been filled + return; } if (result.__isset.o1) { throw result.o1; @@ -24405,24 +27088,25 @@ bool ThriftHiveMetastoreClient::recv_update_partition_column_statistics() if (result.__isset.o4) { throw result.o4; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "update_partition_column_statistics failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "getCurrentStreamingPartitionPath failed: unknown result"); } -void ThriftHiveMetastoreClient::get_table_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& col_name) +void ThriftHiveMetastoreClient::rollStreamingPartition(std::string& _return, const std::string& dbName, const std::string& tableName, const std::string& newPartitionPath, const std::string& newPartitionVal) { - send_get_table_column_statistics(db_name, tbl_name, col_name); - recv_get_table_column_statistics(_return); + send_rollStreamingPartition(dbName, tableName, newPartitionPath, newPartitionVal); + recv_rollStreamingPartition(_return); } -void ThriftHiveMetastoreClient::send_get_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name) +void ThriftHiveMetastoreClient::send_rollStreamingPartition(const std::string& dbName, const std::string& tableName, const std::string& newPartitionPath, const std::string& newPartitionVal) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_table_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("rollStreamingPartition", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_table_column_statistics_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.col_name = &col_name; + ThriftHiveMetastore_rollStreamingPartition_pargs args; + args.dbName = &dbName; + args.tableName = &tableName; + args.newPartitionPath = &newPartitionPath; + args.newPartitionVal = &newPartitionVal; args.write(oprot_); oprot_->writeMessageEnd(); @@ -24430,7 +27114,7 @@ void ThriftHiveMetastoreClient::send_get_table_column_statistics(const std::stri oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_table_column_statistics(ColumnStatistics& _return) +void ThriftHiveMetastoreClient::recv_rollStreamingPartition(std::string& _return) { int32_t rseqid = 0; @@ -24450,12 +27134,12 @@ void ThriftHiveMetastoreClient::recv_get_table_column_statistics(ColumnStatistic iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_table_column_statistics") != 0) { + if (fname.compare("rollStreamingPartition") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_table_column_statistics_presult result; + ThriftHiveMetastore_rollStreamingPartition_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -24477,25 +27161,27 @@ void ThriftHiveMetastoreClient::recv_get_table_column_statistics(ColumnStatistic if (result.__isset.o4) { throw result.o4; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_column_statistics failed: unknown result"); + if (result.__isset.o5) { + throw result.o5; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "rollStreamingPartition failed: unknown result"); } -void ThriftHiveMetastoreClient::get_partition_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) +void ThriftHiveMetastoreClient::updateStreamingTempLocation(const std::string& dbName, const std::string& tableName, const std::string& streamingTempDir) { - send_get_partition_column_statistics(db_name, tbl_name, part_name, col_name); - recv_get_partition_column_statistics(_return); + send_updateStreamingTempLocation(dbName, tableName, streamingTempDir); + recv_updateStreamingTempLocation(); } -void ThriftHiveMetastoreClient::send_get_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) +void ThriftHiveMetastoreClient::send_updateStreamingTempLocation(const std::string& dbName, const std::string& tableName, const std::string& streamingTempDir) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partition_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("updateStreamingTempLocation", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_partition_column_statistics_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_name = &part_name; - args.col_name = &col_name; + ThriftHiveMetastore_updateStreamingTempLocation_pargs args; + args.dbName = &dbName; + args.tableName = &tableName; + args.streamingTempDir = &streamingTempDir; args.write(oprot_); oprot_->writeMessageEnd(); @@ -24503,7 +27189,7 @@ void ThriftHiveMetastoreClient::send_get_partition_column_statistics(const std:: oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_partition_column_statistics(ColumnStatistics& _return) +void ThriftHiveMetastoreClient::recv_updateStreamingTempLocation() { int32_t rseqid = 0; @@ -24523,21 +27209,16 @@ void ThriftHiveMetastoreClient::recv_get_partition_column_statistics(ColumnStati iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_partition_column_statistics") != 0) { + if (fname.compare("updateStreamingTempLocation") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_partition_column_statistics_presult result; - result.success = &_return; + ThriftHiveMetastore_updateStreamingTempLocation_presult result; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); - if (result.__isset.success) { - // _return pointer has now been filled - return; - } if (result.__isset.o1) { throw result.o1; } @@ -24547,28 +27228,23 @@ void ThriftHiveMetastoreClient::recv_get_partition_column_statistics(ColumnStati if (result.__isset.o3) { throw result.o3; } - if (result.__isset.o4) { - throw result.o4; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_column_statistics failed: unknown result"); + return; } -bool ThriftHiveMetastoreClient::delete_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) +int64_t ThriftHiveMetastoreClient::getNextChunkID(const std::string& db_name, const std::string& table_name) { - send_delete_partition_column_statistics(db_name, tbl_name, part_name, col_name); - return recv_delete_partition_column_statistics(); + send_getNextChunkID(db_name, table_name); + return recv_getNextChunkID(); } -void ThriftHiveMetastoreClient::send_delete_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) +void ThriftHiveMetastoreClient::send_getNextChunkID(const std::string& db_name, const std::string& table_name) { int32_t cseqid = 0; - oprot_->writeMessageBegin("delete_partition_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("getNextChunkID", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_delete_partition_column_statistics_pargs args; + ThriftHiveMetastore_getNextChunkID_pargs args; args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_name = &part_name; - args.col_name = &col_name; + args.table_name = &table_name; args.write(oprot_); oprot_->writeMessageEnd(); @@ -24576,7 +27252,7 @@ void ThriftHiveMetastoreClient::send_delete_partition_column_statistics(const st oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_delete_partition_column_statistics() +int64_t ThriftHiveMetastoreClient::recv_getNextChunkID() { int32_t rseqid = 0; @@ -24596,13 +27272,13 @@ bool ThriftHiveMetastoreClient::recv_delete_partition_column_statistics() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("delete_partition_column_statistics") != 0) { + if (fname.compare("getNextChunkID") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - bool _return; - ThriftHiveMetastore_delete_partition_column_statistics_presult result; + int64_t _return; + ThriftHiveMetastore_getNextChunkID_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -24623,24 +27299,23 @@ bool ThriftHiveMetastoreClient::recv_delete_partition_column_statistics() if (result.__isset.o4) { throw result.o4; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "delete_partition_column_statistics failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "getNextChunkID failed: unknown result"); } -bool ThriftHiveMetastoreClient::delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name) +void ThriftHiveMetastoreClient::getStreamingTmpDir(std::string& _return, const std::string& dbName, const std::string& tableName) { - send_delete_table_column_statistics(db_name, tbl_name, col_name); - return recv_delete_table_column_statistics(); + send_getStreamingTmpDir(dbName, tableName); + recv_getStreamingTmpDir(_return); } -void ThriftHiveMetastoreClient::send_delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name) +void ThriftHiveMetastoreClient::send_getStreamingTmpDir(const std::string& dbName, const std::string& tableName) { int32_t cseqid = 0; - oprot_->writeMessageBegin("delete_table_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_delete_table_column_statistics_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.col_name = &col_name; + oprot_->writeMessageBegin("getStreamingTmpDir", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_getStreamingTmpDir_pargs args; + args.dbName = &dbName; + args.tableName = &tableName; args.write(oprot_); oprot_->writeMessageEnd(); @@ -24648,7 +27323,7 @@ void ThriftHiveMetastoreClient::send_delete_table_column_statistics(const std::s oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_delete_table_column_statistics() +void ThriftHiveMetastoreClient::recv_getStreamingTmpDir(std::string& _return) { int32_t rseqid = 0; @@ -24668,20 +27343,20 @@ bool ThriftHiveMetastoreClient::recv_delete_table_column_statistics() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("delete_table_column_statistics") != 0) { + if (fname.compare("getStreamingTmpDir") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - bool _return; - ThriftHiveMetastore_delete_table_column_statistics_presult result; + ThriftHiveMetastore_getStreamingTmpDir_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); if (result.__isset.success) { - return _return; + // _return pointer has now been filled + return; } if (result.__isset.o1) { throw result.o1; @@ -24695,7 +27370,7 @@ bool ThriftHiveMetastoreClient::recv_delete_table_column_statistics() if (result.__isset.o4) { throw result.o4; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "delete_table_column_statistics failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "getStreamingTmpDir failed: unknown result"); } bool ThriftHiveMetastoreClient::create_role(const Role& role) @@ -25093,7 +27768,254 @@ void ThriftHiveMetastoreClient::send_get_privilege_set(const HiveObjectRef& hive oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_privilege_set(PrincipalPrivilegeSet& _return) +void ThriftHiveMetastoreClient::recv_get_privilege_set(PrincipalPrivilegeSet& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_privilege_set") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_privilege_set_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_privilege_set failed: unknown result"); +} + +void ThriftHiveMetastoreClient::list_privileges(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject) +{ + send_list_privileges(principal_name, principal_type, hiveObject); + recv_list_privileges(_return); +} + +void ThriftHiveMetastoreClient::send_list_privileges(const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("list_privileges", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_list_privileges_pargs args; + args.principal_name = &principal_name; + args.principal_type = &principal_type; + args.hiveObject = &hiveObject; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_list_privileges(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("list_privileges") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_list_privileges_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "list_privileges failed: unknown result"); +} + +bool ThriftHiveMetastoreClient::grant_privileges(const PrivilegeBag& privileges) +{ + send_grant_privileges(privileges); + return recv_grant_privileges(); +} + +void ThriftHiveMetastoreClient::send_grant_privileges(const PrivilegeBag& privileges) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("grant_privileges", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_grant_privileges_pargs args; + args.privileges = &privileges; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +bool ThriftHiveMetastoreClient::recv_grant_privileges() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("grant_privileges") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + bool _return; + ThriftHiveMetastore_grant_privileges_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "grant_privileges failed: unknown result"); +} + +bool ThriftHiveMetastoreClient::revoke_privileges(const PrivilegeBag& privileges) +{ + send_revoke_privileges(privileges); + return recv_revoke_privileges(); +} + +void ThriftHiveMetastoreClient::send_revoke_privileges(const PrivilegeBag& privileges) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("revoke_privileges", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_revoke_privileges_pargs args; + args.privileges = &privileges; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +bool ThriftHiveMetastoreClient::recv_revoke_privileges() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("revoke_privileges") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + bool _return; + ThriftHiveMetastore_revoke_privileges_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "revoke_privileges failed: unknown result"); +} + +void ThriftHiveMetastoreClient::set_ugi(std::vector & _return, const std::string& user_name, const std::vector & group_names) +{ + send_set_ugi(user_name, group_names); + recv_set_ugi(_return); +} + +void ThriftHiveMetastoreClient::send_set_ugi(const std::string& user_name, const std::vector & group_names) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("set_ugi", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_set_ugi_pargs args; + args.user_name = &user_name; + args.group_names = &group_names; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_set_ugi(std::vector & _return) { int32_t rseqid = 0; @@ -25113,12 +28035,12 @@ void ThriftHiveMetastoreClient::recv_get_privilege_set(PrincipalPrivilegeSet& _r iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_privilege_set") != 0) { + if (fname.compare("set_ugi") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_privilege_set_presult result; + ThriftHiveMetastore_set_ugi_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -25131,24 +28053,23 @@ void ThriftHiveMetastoreClient::recv_get_privilege_set(PrincipalPrivilegeSet& _r if (result.__isset.o1) { throw result.o1; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_privilege_set failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "set_ugi failed: unknown result"); } -void ThriftHiveMetastoreClient::list_privileges(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject) +void ThriftHiveMetastoreClient::get_delegation_token(std::string& _return, const std::string& token_owner, const std::string& renewer_kerberos_principal_name) { - send_list_privileges(principal_name, principal_type, hiveObject); - recv_list_privileges(_return); + send_get_delegation_token(token_owner, renewer_kerberos_principal_name); + recv_get_delegation_token(_return); } -void ThriftHiveMetastoreClient::send_list_privileges(const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject) +void ThriftHiveMetastoreClient::send_get_delegation_token(const std::string& token_owner, const std::string& renewer_kerberos_principal_name) { int32_t cseqid = 0; - oprot_->writeMessageBegin("list_privileges", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_delegation_token", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_list_privileges_pargs args; - args.principal_name = &principal_name; - args.principal_type = &principal_type; - args.hiveObject = &hiveObject; + ThriftHiveMetastore_get_delegation_token_pargs args; + args.token_owner = &token_owner; + args.renewer_kerberos_principal_name = &renewer_kerberos_principal_name; args.write(oprot_); oprot_->writeMessageEnd(); @@ -25156,7 +28077,7 @@ void ThriftHiveMetastoreClient::send_list_privileges(const std::string& principa oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_list_privileges(std::vector & _return) +void ThriftHiveMetastoreClient::recv_get_delegation_token(std::string& _return) { int32_t rseqid = 0; @@ -25176,12 +28097,12 @@ void ThriftHiveMetastoreClient::recv_list_privileges(std::vectorreadMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("list_privileges") != 0) { + if (fname.compare("get_delegation_token") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_list_privileges_presult result; + ThriftHiveMetastore_get_delegation_token_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -25194,22 +28115,22 @@ void ThriftHiveMetastoreClient::recv_list_privileges(std::vectorwriteMessageBegin("grant_privileges", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("renew_delegation_token", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_grant_privileges_pargs args; - args.privileges = &privileges; + ThriftHiveMetastore_renew_delegation_token_pargs args; + args.token_str_form = &token_str_form; args.write(oprot_); oprot_->writeMessageEnd(); @@ -25217,7 +28138,7 @@ void ThriftHiveMetastoreClient::send_grant_privileges(const PrivilegeBag& privil oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_grant_privileges() +int64_t ThriftHiveMetastoreClient::recv_renew_delegation_token() { int32_t rseqid = 0; @@ -25237,13 +28158,13 @@ bool ThriftHiveMetastoreClient::recv_grant_privileges() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("grant_privileges") != 0) { + if (fname.compare("renew_delegation_token") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - bool _return; - ThriftHiveMetastore_grant_privileges_presult result; + int64_t _return; + ThriftHiveMetastore_renew_delegation_token_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -25255,22 +28176,22 @@ bool ThriftHiveMetastoreClient::recv_grant_privileges() if (result.__isset.o1) { throw result.o1; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "grant_privileges failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "renew_delegation_token failed: unknown result"); } -bool ThriftHiveMetastoreClient::revoke_privileges(const PrivilegeBag& privileges) +void ThriftHiveMetastoreClient::cancel_delegation_token(const std::string& token_str_form) { - send_revoke_privileges(privileges); - return recv_revoke_privileges(); + send_cancel_delegation_token(token_str_form); + recv_cancel_delegation_token(); } -void ThriftHiveMetastoreClient::send_revoke_privileges(const PrivilegeBag& privileges) +void ThriftHiveMetastoreClient::send_cancel_delegation_token(const std::string& token_str_form) { int32_t cseqid = 0; - oprot_->writeMessageBegin("revoke_privileges", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("cancel_delegation_token", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_revoke_privileges_pargs args; - args.privileges = &privileges; + ThriftHiveMetastore_cancel_delegation_token_pargs args; + args.token_str_form = &token_str_form; args.write(oprot_); oprot_->writeMessageEnd(); @@ -25278,7 +28199,7 @@ void ThriftHiveMetastoreClient::send_revoke_privileges(const PrivilegeBag& privi oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_revoke_privileges() +void ThriftHiveMetastoreClient::recv_cancel_delegation_token() { int32_t rseqid = 0; @@ -25298,318 +28219,550 @@ bool ThriftHiveMetastoreClient::recv_revoke_privileges() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("revoke_privileges") != 0) { + if (fname.compare("cancel_delegation_token") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - bool _return; - ThriftHiveMetastore_revoke_privileges_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); + ThriftHiveMetastore_cancel_delegation_token_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + return; +} + +bool ThriftHiveMetastoreProcessor::dispatchCall(::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, const std::string& fname, int32_t seqid, void* callContext) { + ProcessMap::iterator pfn; + pfn = processMap_.find(fname); + if (pfn == processMap_.end()) { + return ::facebook::fb303::FacebookServiceProcessor::dispatchCall(iprot, oprot, fname, seqid, callContext); + } + (this->*(pfn->second))(seqid, iprot, oprot, callContext); + return true; +} + +void ThriftHiveMetastoreProcessor::process_create_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_database", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_database"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_database"); + } + + ThriftHiveMetastore_create_database_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_database", bytes); + } + + ThriftHiveMetastore_create_database_result result; + try { + iface_->create_database(args.database); + } catch (AlreadyExistsException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (InvalidObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_database"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("create_database", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_database"); + } + + oprot->writeMessageBegin("create_database", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_database", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_database", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_database"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_database"); + } + + ThriftHiveMetastore_get_database_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_database", bytes); + } + + ThriftHiveMetastore_get_database_result result; + try { + iface_->get_database(result.success, args.name); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_database"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_database", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_database"); + } + + oprot->writeMessageBegin("get_database", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_database", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_drop_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_database", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_database"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_database"); + } + + ThriftHiveMetastore_drop_database_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_database", bytes); + } + + ThriftHiveMetastore_drop_database_result result; + try { + iface_->drop_database(args.name, args.deleteData, args.cascade); + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (InvalidOperationException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_database"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("drop_database", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } - if (result.__isset.success) { - return _return; + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_database"); } - if (result.__isset.o1) { - throw result.o1; + + oprot->writeMessageBegin("drop_database", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_database", bytes); } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "revoke_privileges failed: unknown result"); } -void ThriftHiveMetastoreClient::set_ugi(std::vector & _return, const std::string& user_name, const std::vector & group_names) +void ThriftHiveMetastoreProcessor::process_get_databases(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { - send_set_ugi(user_name, group_names); - recv_set_ugi(_return); -} + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_databases", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_databases"); -void ThriftHiveMetastoreClient::send_set_ugi(const std::string& user_name, const std::vector & group_names) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("set_ugi", ::apache::thrift::protocol::T_CALL, cseqid); + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_databases"); + } - ThriftHiveMetastore_set_ugi_pargs args; - args.user_name = &user_name; - args.group_names = &group_names; - args.write(oprot_); + ThriftHiveMetastore_get_databases_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_databases", bytes); + } -void ThriftHiveMetastoreClient::recv_set_ugi(std::vector & _return) -{ + ThriftHiveMetastore_get_databases_result result; + try { + iface_->get_databases(result.success, args.pattern); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_databases"); + } - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_databases", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_databases"); } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); + + oprot->writeMessageBegin("get_databases", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_databases", bytes); } - if (fname.compare("set_ugi") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); +} + +void ThriftHiveMetastoreProcessor::process_get_all_databases(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_all_databases", callContext); } - ThriftHiveMetastore_set_ugi_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_all_databases"); - if (result.__isset.success) { - // _return pointer has now been filled - return; + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_all_databases"); } - if (result.__isset.o1) { - throw result.o1; + + ThriftHiveMetastore_get_all_databases_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_all_databases", bytes); } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "set_ugi failed: unknown result"); -} -void ThriftHiveMetastoreClient::get_delegation_token(std::string& _return, const std::string& token_owner, const std::string& renewer_kerberos_principal_name) -{ - send_get_delegation_token(token_owner, renewer_kerberos_principal_name); - recv_get_delegation_token(_return); -} + ThriftHiveMetastore_get_all_databases_result result; + try { + iface_->get_all_databases(result.success); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_all_databases"); + } -void ThriftHiveMetastoreClient::send_get_delegation_token(const std::string& token_owner, const std::string& renewer_kerberos_principal_name) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_delegation_token", ::apache::thrift::protocol::T_CALL, cseqid); + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_all_databases", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } - ThriftHiveMetastore_get_delegation_token_pargs args; - args.token_owner = &token_owner; - args.renewer_kerberos_principal_name = &renewer_kerberos_principal_name; - args.write(oprot_); + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_all_databases"); + } - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); + oprot->writeMessageBegin("get_all_databases", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_all_databases", bytes); + } } -void ThriftHiveMetastoreClient::recv_get_delegation_token(std::string& _return) +void ThriftHiveMetastoreProcessor::process_alter_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_database", callContext); } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_database"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_database"); } - if (fname.compare("get_delegation_token") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); + + ThriftHiveMetastore_alter_database_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_database", bytes); } - ThriftHiveMetastore_get_delegation_token_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - if (result.__isset.success) { - // _return pointer has now been filled + ThriftHiveMetastore_alter_database_result result; + try { + iface_->alter_database(args.dbname, args.db); + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_database"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("alter_database", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); return; } - if (result.__isset.o1) { - throw result.o1; + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_database"); } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_delegation_token failed: unknown result"); -} -int64_t ThriftHiveMetastoreClient::renew_delegation_token(const std::string& token_str_form) -{ - send_renew_delegation_token(token_str_form); - return recv_renew_delegation_token(); + oprot->writeMessageBegin("alter_database", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_database", bytes); + } } -void ThriftHiveMetastoreClient::send_renew_delegation_token(const std::string& token_str_form) +void ThriftHiveMetastoreProcessor::process_get_type(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { - int32_t cseqid = 0; - oprot_->writeMessageBegin("renew_delegation_token", ::apache::thrift::protocol::T_CALL, cseqid); + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_type", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_type"); - ThriftHiveMetastore_renew_delegation_token_pargs args; - args.token_str_form = &token_str_form; - args.write(oprot_); + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_type"); + } - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} + ThriftHiveMetastore_get_type_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); -int64_t ThriftHiveMetastoreClient::recv_renew_delegation_token() -{ + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_type", bytes); + } - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; + ThriftHiveMetastore_get_type_result result; + try { + iface_->get_type(result.success, args.name); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_type"); + } - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("renew_delegation_token") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_type", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; } - int64_t _return; - ThriftHiveMetastore_renew_delegation_token_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - if (result.__isset.success) { - return _return; - } - if (result.__isset.o1) { - throw result.o1; + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_type"); } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "renew_delegation_token failed: unknown result"); -} -void ThriftHiveMetastoreClient::cancel_delegation_token(const std::string& token_str_form) -{ - send_cancel_delegation_token(token_str_form); - recv_cancel_delegation_token(); + oprot->writeMessageBegin("get_type", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_type", bytes); + } } -void ThriftHiveMetastoreClient::send_cancel_delegation_token(const std::string& token_str_form) +void ThriftHiveMetastoreProcessor::process_create_type(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { - int32_t cseqid = 0; - oprot_->writeMessageBegin("cancel_delegation_token", ::apache::thrift::protocol::T_CALL, cseqid); + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_type", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_type"); - ThriftHiveMetastore_cancel_delegation_token_pargs args; - args.token_str_form = &token_str_form; - args.write(oprot_); + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_type"); + } - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} + ThriftHiveMetastore_create_type_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); -void ThriftHiveMetastoreClient::recv_cancel_delegation_token() -{ + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_type", bytes); + } - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; + ThriftHiveMetastore_create_type_result result; + try { + result.success = iface_->create_type(args.type); + result.__isset.success = true; + } catch (AlreadyExistsException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (InvalidObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_type"); + } - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("cancel_delegation_token") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("create_type", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; } - ThriftHiveMetastore_cancel_delegation_token_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - if (result.__isset.o1) { - throw result.o1; + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_type"); } - return; -} -bool ThriftHiveMetastoreProcessor::dispatchCall(::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, const std::string& fname, int32_t seqid, void* callContext) { - ProcessMap::iterator pfn; - pfn = processMap_.find(fname); - if (pfn == processMap_.end()) { - return ::facebook::fb303::FacebookServiceProcessor::dispatchCall(iprot, oprot, fname, seqid, callContext); + oprot->writeMessageBegin("create_type", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_type", bytes); } - (this->*(pfn->second))(seqid, iprot, oprot, callContext); - return true; } -void ThriftHiveMetastoreProcessor::process_create_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_drop_type(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_database", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_type", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_database"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_type"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_database"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_type"); } - ThriftHiveMetastore_create_database_args args; + ThriftHiveMetastore_drop_type_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_database", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_type", bytes); } - ThriftHiveMetastore_create_database_result result; + ThriftHiveMetastore_drop_type_result result; try { - iface_->create_database(args.database); - } catch (AlreadyExistsException &o1) { + result.success = iface_->drop_type(args.type); + result.__isset.success = true; + } catch (MetaException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (InvalidObjectException &o2) { + } catch (NoSuchObjectException &o2) { result.o2 = o2; result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_database"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_type"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("create_database", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("drop_type", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -25618,58 +28771,55 @@ void ThriftHiveMetastoreProcessor::process_create_database(int32_t seqid, ::apac } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_database"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_type"); } - oprot->writeMessageBegin("create_database", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("drop_type", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_database", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_type", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_get_type_all(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_database", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_type_all", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_database"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_type_all"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_database"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_type_all"); } - ThriftHiveMetastore_get_database_args args; + ThriftHiveMetastore_get_type_all_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_database", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_type_all", bytes); } - ThriftHiveMetastore_get_database_result result; + ThriftHiveMetastore_get_type_all_result result; try { - iface_->get_database(result.success, args.name); + iface_->get_type_all(result.success, args.name); result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; } catch (MetaException &o2) { result.o2 = o2; result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_database"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_type_all"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_database", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_type_all", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -25678,60 +28828,61 @@ void ThriftHiveMetastoreProcessor::process_get_database(int32_t seqid, ::apache: } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_database"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_type_all"); } - oprot->writeMessageBegin("get_database", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("get_type_all", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_database", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_type_all", bytes); } } -void ThriftHiveMetastoreProcessor::process_drop_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_get_fields(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_database", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_fields", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_database"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_fields"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_database"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_fields"); } - ThriftHiveMetastore_drop_database_args args; + ThriftHiveMetastore_get_fields_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_database", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_fields", bytes); } - ThriftHiveMetastore_drop_database_result result; + ThriftHiveMetastore_get_fields_result result; try { - iface_->drop_database(args.name, args.deleteData, args.cascade); - } catch (NoSuchObjectException &o1) { + iface_->get_fields(result.success, args.db_name, args.table_name); + result.__isset.success = true; + } catch (MetaException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (InvalidOperationException &o2) { + } catch (UnknownTableException &o2) { result.o2 = o2; result.__isset.o2 = true; - } catch (MetaException &o3) { + } catch (UnknownDBException &o3) { result.o3 = o3; result.__isset.o3 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_database"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_fields"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_database", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_fields", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -25740,55 +28891,61 @@ void ThriftHiveMetastoreProcessor::process_drop_database(int32_t seqid, ::apache } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_database"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_fields"); } - oprot->writeMessageBegin("drop_database", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("get_fields", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_database", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_fields", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_databases(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_get_schema(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_databases", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_schema", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_databases"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_schema"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_databases"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_schema"); } - ThriftHiveMetastore_get_databases_args args; + ThriftHiveMetastore_get_schema_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_databases", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_schema", bytes); } - ThriftHiveMetastore_get_databases_result result; + ThriftHiveMetastore_get_schema_result result; try { - iface_->get_databases(result.success, args.pattern); + iface_->get_schema(result.success, args.db_name, args.table_name); result.__isset.success = true; } catch (MetaException &o1) { result.o1 = o1; result.__isset.o1 = true; + } catch (UnknownTableException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (UnknownDBException &o3) { + result.o3 = o3; + result.__isset.o3 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_databases"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_schema"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_databases", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_schema", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -25797,55 +28954,63 @@ void ThriftHiveMetastoreProcessor::process_get_databases(int32_t seqid, ::apache } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_databases"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_schema"); } - oprot->writeMessageBegin("get_databases", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("get_schema", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_databases", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_schema", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_all_databases(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_create_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_all_databases", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_table", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_all_databases"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_table"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_all_databases"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_table"); } - ThriftHiveMetastore_get_all_databases_args args; + ThriftHiveMetastore_create_table_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_all_databases", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_table", bytes); } - ThriftHiveMetastore_get_all_databases_result result; + ThriftHiveMetastore_create_table_result result; try { - iface_->get_all_databases(result.success); - result.__isset.success = true; - } catch (MetaException &o1) { + iface_->create_table(args.tbl); + } catch (AlreadyExistsException &o1) { result.o1 = o1; result.__isset.o1 = true; + } catch (InvalidObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (NoSuchObjectException &o4) { + result.o4 = o4; + result.__isset.o4 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_all_databases"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_table"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_all_databases", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("create_table", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -25854,57 +29019,63 @@ void ThriftHiveMetastoreProcessor::process_get_all_databases(int32_t seqid, ::ap } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_all_databases"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_table"); } - oprot->writeMessageBegin("get_all_databases", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("create_table", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_all_databases", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_table", bytes); } } -void ThriftHiveMetastoreProcessor::process_alter_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_create_table_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_database", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_table_with_environment_context", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_database"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_table_with_environment_context"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_database"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_table_with_environment_context"); } - ThriftHiveMetastore_alter_database_args args; + ThriftHiveMetastore_create_table_with_environment_context_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_database", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_table_with_environment_context", bytes); } - ThriftHiveMetastore_alter_database_result result; + ThriftHiveMetastore_create_table_with_environment_context_result result; try { - iface_->alter_database(args.dbname, args.db); - } catch (MetaException &o1) { + iface_->create_table_with_environment_context(args.tbl, args.environment_context); + } catch (AlreadyExistsException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { + } catch (InvalidObjectException &o2) { result.o2 = o2; result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (NoSuchObjectException &o4) { + result.o4 = o4; + result.__isset.o4 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_database"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_table_with_environment_context"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("alter_database", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("create_table_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -25913,58 +29084,57 @@ void ThriftHiveMetastoreProcessor::process_alter_database(int32_t seqid, ::apach } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_database"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_table_with_environment_context"); } - oprot->writeMessageBegin("alter_database", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("create_table_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_database", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_table_with_environment_context", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_type(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_drop_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_type", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_table", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_type"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_table"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_type"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_table"); } - ThriftHiveMetastore_get_type_args args; + ThriftHiveMetastore_drop_table_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_type", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_table", bytes); } - ThriftHiveMetastore_get_type_result result; + ThriftHiveMetastore_drop_table_result result; try { - iface_->get_type(result.success, args.name); - result.__isset.success = true; - } catch (MetaException &o1) { + iface_->drop_table(args.dbname, args.name, args.deleteData); + } catch (NoSuchObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_type"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_table"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_type", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("drop_table", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -25973,61 +29143,57 @@ void ThriftHiveMetastoreProcessor::process_get_type(int32_t seqid, ::apache::thr } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_type"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_table"); } - oprot->writeMessageBegin("get_type", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("drop_table", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_type", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_table", bytes); } } -void ThriftHiveMetastoreProcessor::process_create_type(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_drop_table_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_type", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_table_with_environment_context", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_type"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_table_with_environment_context"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_type"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_table_with_environment_context"); } - ThriftHiveMetastore_create_type_args args; + ThriftHiveMetastore_drop_table_with_environment_context_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_type", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_table_with_environment_context", bytes); } - ThriftHiveMetastore_create_type_result result; + ThriftHiveMetastore_drop_table_with_environment_context_result result; try { - result.success = iface_->create_type(args.type); - result.__isset.success = true; - } catch (AlreadyExistsException &o1) { + iface_->drop_table_with_environment_context(args.dbname, args.name, args.deleteData, args.environment_context); + } catch (NoSuchObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (InvalidObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; } catch (MetaException &o3) { result.o3 = o3; result.__isset.o3 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_type"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_table_with_environment_context"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("create_type", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("drop_table_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -26036,58 +29202,55 @@ void ThriftHiveMetastoreProcessor::process_create_type(int32_t seqid, ::apache:: } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_type"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_table_with_environment_context"); } - oprot->writeMessageBegin("create_type", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("drop_table_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_type", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_table_with_environment_context", bytes); } } -void ThriftHiveMetastoreProcessor::process_drop_type(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_get_tables(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_type", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_tables", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_type"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_tables"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_type"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_tables"); } - ThriftHiveMetastore_drop_type_args args; + ThriftHiveMetastore_get_tables_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_type", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_tables", bytes); } - ThriftHiveMetastore_drop_type_result result; + ThriftHiveMetastore_get_tables_result result; try { - result.success = iface_->drop_type(args.type); + iface_->get_tables(result.success, args.db_name, args.pattern); result.__isset.success = true; } catch (MetaException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_type"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_tables"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_type", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_tables", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -26096,55 +29259,55 @@ void ThriftHiveMetastoreProcessor::process_drop_type(int32_t seqid, ::apache::th } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_type"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_tables"); } - oprot->writeMessageBegin("drop_type", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("get_tables", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_type", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_tables", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_type_all(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_get_all_tables(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_type_all", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_all_tables", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_type_all"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_all_tables"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_type_all"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_all_tables"); } - ThriftHiveMetastore_get_type_all_args args; + ThriftHiveMetastore_get_all_tables_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_type_all", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_all_tables", bytes); } - ThriftHiveMetastore_get_type_all_result result; + ThriftHiveMetastore_get_all_tables_result result; try { - iface_->get_type_all(result.success, args.name); + iface_->get_all_tables(result.success, args.db_name); result.__isset.success = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_type_all"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_all_tables"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_type_all", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_all_tables", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -26153,61 +29316,58 @@ void ThriftHiveMetastoreProcessor::process_get_type_all(int32_t seqid, ::apache: } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_type_all"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_all_tables"); } - oprot->writeMessageBegin("get_type_all", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("get_all_tables", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_type_all", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_all_tables", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_fields(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_get_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_fields", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_fields"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_fields"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table"); } - ThriftHiveMetastore_get_fields_args args; + ThriftHiveMetastore_get_table_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_fields", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table", bytes); } - ThriftHiveMetastore_get_fields_result result; + ThriftHiveMetastore_get_table_result result; try { - iface_->get_fields(result.success, args.db_name, args.table_name); + iface_->get_table(result.success, args.dbname, args.tbl_name); result.__isset.success = true; } catch (MetaException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (UnknownTableException &o2) { + } catch (NoSuchObjectException &o2) { result.o2 = o2; result.__isset.o2 = true; - } catch (UnknownDBException &o3) { - result.o3 = o3; - result.__isset.o3 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_fields"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_fields", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_table", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -26216,49 +29376,49 @@ void ThriftHiveMetastoreProcessor::process_get_fields(int32_t seqid, ::apache::t } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_fields"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table"); } - oprot->writeMessageBegin("get_fields", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("get_table", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_fields", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_schema(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_get_table_objects_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_schema", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table_objects_by_name", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_schema"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table_objects_by_name"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_schema"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table_objects_by_name"); } - ThriftHiveMetastore_get_schema_args args; + ThriftHiveMetastore_get_table_objects_by_name_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_schema", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table_objects_by_name", bytes); } - ThriftHiveMetastore_get_schema_result result; + ThriftHiveMetastore_get_table_objects_by_name_result result; try { - iface_->get_schema(result.success, args.db_name, args.table_name); + iface_->get_table_objects_by_name(result.success, args.dbname, args.tbl_names); result.__isset.success = true; } catch (MetaException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (UnknownTableException &o2) { + } catch (InvalidOperationException &o2) { result.o2 = o2; result.__isset.o2 = true; } catch (UnknownDBException &o3) { @@ -26266,11 +29426,11 @@ void ThriftHiveMetastoreProcessor::process_get_schema(int32_t seqid, ::apache::t result.__isset.o3 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_schema"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table_objects_by_name"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_schema", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_table_objects_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -26279,63 +29439,61 @@ void ThriftHiveMetastoreProcessor::process_get_schema(int32_t seqid, ::apache::t } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_schema"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table_objects_by_name"); } - oprot->writeMessageBegin("get_schema", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("get_table_objects_by_name", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_schema", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table_objects_by_name", bytes); } } -void ThriftHiveMetastoreProcessor::process_create_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_get_table_names_by_filter(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_table", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table_names_by_filter", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_table"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table_names_by_filter"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_table"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table_names_by_filter"); } - ThriftHiveMetastore_create_table_args args; + ThriftHiveMetastore_get_table_names_by_filter_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_table", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table_names_by_filter", bytes); } - ThriftHiveMetastore_create_table_result result; + ThriftHiveMetastore_get_table_names_by_filter_result result; try { - iface_->create_table(args.tbl); - } catch (AlreadyExistsException &o1) { + iface_->get_table_names_by_filter(result.success, args.dbname, args.filter, args.max_tables); + result.__isset.success = true; + } catch (MetaException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (InvalidObjectException &o2) { + } catch (InvalidOperationException &o2) { result.o2 = o2; result.__isset.o2 = true; - } catch (MetaException &o3) { + } catch (UnknownDBException &o3) { result.o3 = o3; result.__isset.o3 = true; - } catch (NoSuchObjectException &o4) { - result.o4 = o4; - result.__isset.o4 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_table"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table_names_by_filter"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("create_table", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_table_names_by_filter", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -26344,63 +29502,57 @@ void ThriftHiveMetastoreProcessor::process_create_table(int32_t seqid, ::apache: } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_table"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table_names_by_filter"); } - oprot->writeMessageBegin("create_table", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("get_table_names_by_filter", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_table", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table_names_by_filter", bytes); } } -void ThriftHiveMetastoreProcessor::process_create_table_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_alter_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_table_with_environment_context", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_table", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_table_with_environment_context"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_table"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_table_with_environment_context"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_table"); } - ThriftHiveMetastore_create_table_with_environment_context_args args; + ThriftHiveMetastore_alter_table_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_table_with_environment_context", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_table", bytes); } - ThriftHiveMetastore_create_table_with_environment_context_result result; + ThriftHiveMetastore_alter_table_result result; try { - iface_->create_table_with_environment_context(args.tbl, args.environment_context); - } catch (AlreadyExistsException &o1) { + iface_->alter_table(args.dbname, args.tbl_name, args.new_tbl); + } catch (InvalidOperationException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (InvalidObjectException &o2) { + } catch (MetaException &o2) { result.o2 = o2; result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (NoSuchObjectException &o4) { - result.o4 = o4; - result.__isset.o4 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_table_with_environment_context"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_table"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("create_table_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("alter_table", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -26409,57 +29561,57 @@ void ThriftHiveMetastoreProcessor::process_create_table_with_environment_context } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_table_with_environment_context"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_table"); } - oprot->writeMessageBegin("create_table_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("alter_table", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_table_with_environment_context", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_table", bytes); } } -void ThriftHiveMetastoreProcessor::process_drop_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_alter_table_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_table", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_table_with_environment_context", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_table"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_table_with_environment_context"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_table"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_table_with_environment_context"); } - ThriftHiveMetastore_drop_table_args args; + ThriftHiveMetastore_alter_table_with_environment_context_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_table", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_table_with_environment_context", bytes); } - ThriftHiveMetastore_drop_table_result result; + ThriftHiveMetastore_alter_table_with_environment_context_result result; try { - iface_->drop_table(args.dbname, args.name, args.deleteData); - } catch (NoSuchObjectException &o1) { + iface_->alter_table_with_environment_context(args.dbname, args.tbl_name, args.new_tbl, args.environment_context); + } catch (InvalidOperationException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_table"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_table_with_environment_context"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_table", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("alter_table_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -26468,57 +29620,61 @@ void ThriftHiveMetastoreProcessor::process_drop_table(int32_t seqid, ::apache::t } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_table"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_table_with_environment_context"); } - oprot->writeMessageBegin("drop_table", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("alter_table_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_table", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_table_with_environment_context", bytes); } } -void ThriftHiveMetastoreProcessor::process_drop_table_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_add_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_table_with_environment_context", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_partition", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_table_with_environment_context"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_partition"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_table_with_environment_context"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_partition"); } - ThriftHiveMetastore_drop_table_with_environment_context_args args; + ThriftHiveMetastore_add_partition_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_table_with_environment_context", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_partition", bytes); } - ThriftHiveMetastore_drop_table_with_environment_context_result result; + ThriftHiveMetastore_add_partition_result result; try { - iface_->drop_table_with_environment_context(args.dbname, args.name, args.deleteData, args.environment_context); - } catch (NoSuchObjectException &o1) { + iface_->add_partition(result.success, args.new_part); + result.__isset.success = true; + } catch (InvalidObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; + } catch (AlreadyExistsException &o2) { + result.o2 = o2; + result.__isset.o2 = true; } catch (MetaException &o3) { result.o3 = o3; result.__isset.o3 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_table_with_environment_context"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_partition"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_table_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("add_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -26527,55 +29683,61 @@ void ThriftHiveMetastoreProcessor::process_drop_table_with_environment_context(i } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_table_with_environment_context"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_partition"); } - oprot->writeMessageBegin("drop_table_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("add_partition", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_table_with_environment_context", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_partition", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_tables(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_add_partition_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_tables", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_partition_with_environment_context", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_tables"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_partition_with_environment_context"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_tables"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_partition_with_environment_context"); } - ThriftHiveMetastore_get_tables_args args; + ThriftHiveMetastore_add_partition_with_environment_context_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_tables", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_partition_with_environment_context", bytes); } - ThriftHiveMetastore_get_tables_result result; + ThriftHiveMetastore_add_partition_with_environment_context_result result; try { - iface_->get_tables(result.success, args.db_name, args.pattern); + iface_->add_partition_with_environment_context(result.success, args.new_part, args.environment_context); result.__isset.success = true; - } catch (MetaException &o1) { + } catch (InvalidObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; + } catch (AlreadyExistsException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_tables"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_partition_with_environment_context"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_tables", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("add_partition_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -26584,55 +29746,61 @@ void ThriftHiveMetastoreProcessor::process_get_tables(int32_t seqid, ::apache::t } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_tables"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_partition_with_environment_context"); } - oprot->writeMessageBegin("get_tables", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("add_partition_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_tables", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_partition_with_environment_context", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_all_tables(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_add_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_all_tables", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_partitions", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_all_tables"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_partitions"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_all_tables"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_partitions"); } - ThriftHiveMetastore_get_all_tables_args args; + ThriftHiveMetastore_add_partitions_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_all_tables", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_partitions", bytes); } - ThriftHiveMetastore_get_all_tables_result result; + ThriftHiveMetastore_add_partitions_result result; try { - iface_->get_all_tables(result.success, args.db_name); + result.success = iface_->add_partitions(args.new_parts); result.__isset.success = true; - } catch (MetaException &o1) { + } catch (InvalidObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; + } catch (AlreadyExistsException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_all_tables"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_partitions"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_all_tables", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("add_partitions", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -26641,58 +29809,61 @@ void ThriftHiveMetastoreProcessor::process_get_all_tables(int32_t seqid, ::apach } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_all_tables"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_partitions"); } - oprot->writeMessageBegin("get_all_tables", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("add_partitions", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_all_tables", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_partitions", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_append_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.append_partition", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.append_partition"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.append_partition"); } - ThriftHiveMetastore_get_table_args args; + ThriftHiveMetastore_append_partition_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.append_partition", bytes); } - ThriftHiveMetastore_get_table_result result; + ThriftHiveMetastore_append_partition_result result; try { - iface_->get_table(result.success, args.dbname, args.tbl_name); + iface_->append_partition(result.success, args.db_name, args.tbl_name, args.part_vals); result.__isset.success = true; - } catch (MetaException &o1) { + } catch (InvalidObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { + } catch (AlreadyExistsException &o2) { result.o2 = o2; result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.append_partition"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_table", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("append_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -26701,61 +29872,61 @@ void ThriftHiveMetastoreProcessor::process_get_table(int32_t seqid, ::apache::th } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.append_partition"); } - oprot->writeMessageBegin("get_table", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("append_partition", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.append_partition", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_table_objects_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_append_partition_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table_objects_by_name", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.append_partition_with_environment_context", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table_objects_by_name"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.append_partition_with_environment_context"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table_objects_by_name"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.append_partition_with_environment_context"); } - ThriftHiveMetastore_get_table_objects_by_name_args args; + ThriftHiveMetastore_append_partition_with_environment_context_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table_objects_by_name", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.append_partition_with_environment_context", bytes); } - ThriftHiveMetastore_get_table_objects_by_name_result result; + ThriftHiveMetastore_append_partition_with_environment_context_result result; try { - iface_->get_table_objects_by_name(result.success, args.dbname, args.tbl_names); + iface_->append_partition_with_environment_context(result.success, args.db_name, args.tbl_name, args.part_vals, args.environment_context); result.__isset.success = true; - } catch (MetaException &o1) { + } catch (InvalidObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (InvalidOperationException &o2) { + } catch (AlreadyExistsException &o2) { result.o2 = o2; result.__isset.o2 = true; - } catch (UnknownDBException &o3) { + } catch (MetaException &o3) { result.o3 = o3; result.__isset.o3 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table_objects_by_name"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.append_partition_with_environment_context"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_table_objects_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("append_partition_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -26764,61 +29935,61 @@ void ThriftHiveMetastoreProcessor::process_get_table_objects_by_name(int32_t seq } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table_objects_by_name"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.append_partition_with_environment_context"); } - oprot->writeMessageBegin("get_table_objects_by_name", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("append_partition_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table_objects_by_name", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.append_partition_with_environment_context", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_table_names_by_filter(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_append_partition_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table_names_by_filter", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.append_partition_by_name", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table_names_by_filter"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.append_partition_by_name"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table_names_by_filter"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.append_partition_by_name"); } - ThriftHiveMetastore_get_table_names_by_filter_args args; + ThriftHiveMetastore_append_partition_by_name_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table_names_by_filter", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.append_partition_by_name", bytes); } - ThriftHiveMetastore_get_table_names_by_filter_result result; + ThriftHiveMetastore_append_partition_by_name_result result; try { - iface_->get_table_names_by_filter(result.success, args.dbname, args.filter, args.max_tables); + iface_->append_partition_by_name(result.success, args.db_name, args.tbl_name, args.part_name); result.__isset.success = true; - } catch (MetaException &o1) { + } catch (InvalidObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (InvalidOperationException &o2) { + } catch (AlreadyExistsException &o2) { result.o2 = o2; result.__isset.o2 = true; - } catch (UnknownDBException &o3) { + } catch (MetaException &o3) { result.o3 = o3; result.__isset.o3 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table_names_by_filter"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.append_partition_by_name"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_table_names_by_filter", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("append_partition_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -26827,57 +29998,61 @@ void ThriftHiveMetastoreProcessor::process_get_table_names_by_filter(int32_t seq } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table_names_by_filter"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.append_partition_by_name"); } - oprot->writeMessageBegin("get_table_names_by_filter", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("append_partition_by_name", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table_names_by_filter", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.append_partition_by_name", bytes); } } -void ThriftHiveMetastoreProcessor::process_alter_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_append_partition_by_name_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_table", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.append_partition_by_name_with_environment_context", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_table"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_table"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context"); } - ThriftHiveMetastore_alter_table_args args; + ThriftHiveMetastore_append_partition_by_name_with_environment_context_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_table", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context", bytes); } - ThriftHiveMetastore_alter_table_result result; + ThriftHiveMetastore_append_partition_by_name_with_environment_context_result result; try { - iface_->alter_table(args.dbname, args.tbl_name, args.new_tbl); - } catch (InvalidOperationException &o1) { + iface_->append_partition_by_name_with_environment_context(result.success, args.db_name, args.tbl_name, args.part_name, args.environment_context); + result.__isset.success = true; + } catch (InvalidObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (MetaException &o2) { + } catch (AlreadyExistsException &o2) { result.o2 = o2; result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_table"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("alter_table", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("append_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -26886,45 +30061,46 @@ void ThriftHiveMetastoreProcessor::process_alter_table(int32_t seqid, ::apache:: } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_table"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context"); } - oprot->writeMessageBegin("alter_table", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("append_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_table", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context", bytes); } } -void ThriftHiveMetastoreProcessor::process_alter_table_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_drop_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_table_with_environment_context", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_partition", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_table_with_environment_context"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_partition"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_table_with_environment_context"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_partition"); } - ThriftHiveMetastore_alter_table_with_environment_context_args args; + ThriftHiveMetastore_drop_partition_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_table_with_environment_context", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_partition", bytes); } - ThriftHiveMetastore_alter_table_with_environment_context_result result; + ThriftHiveMetastore_drop_partition_result result; try { - iface_->alter_table_with_environment_context(args.dbname, args.tbl_name, args.new_tbl, args.environment_context); - } catch (InvalidOperationException &o1) { + result.success = iface_->drop_partition(args.db_name, args.tbl_name, args.part_vals, args.deleteData); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; } catch (MetaException &o2) { @@ -26932,11 +30108,11 @@ void ThriftHiveMetastoreProcessor::process_alter_table_with_environment_context( result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_table_with_environment_context"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_partition"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("alter_table_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("drop_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -26945,61 +30121,58 @@ void ThriftHiveMetastoreProcessor::process_alter_table_with_environment_context( } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_table_with_environment_context"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_partition"); } - oprot->writeMessageBegin("alter_table_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("drop_partition", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_table_with_environment_context", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_partition", bytes); } } -void ThriftHiveMetastoreProcessor::process_add_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_drop_partition_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_partition", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_partition_with_environment_context", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_partition"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_partition_with_environment_context"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_partition"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_partition_with_environment_context"); } - ThriftHiveMetastore_add_partition_args args; + ThriftHiveMetastore_drop_partition_with_environment_context_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_partition", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_partition_with_environment_context", bytes); } - ThriftHiveMetastore_add_partition_result result; + ThriftHiveMetastore_drop_partition_with_environment_context_result result; try { - iface_->add_partition(result.success, args.new_part); + result.success = iface_->drop_partition_with_environment_context(args.db_name, args.tbl_name, args.part_vals, args.deleteData, args.environment_context); result.__isset.success = true; - } catch (InvalidObjectException &o1) { + } catch (NoSuchObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (AlreadyExistsException &o2) { + } catch (MetaException &o2) { result.o2 = o2; result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_partition"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_partition_with_environment_context"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("add_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("drop_partition_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -27008,61 +30181,58 @@ void ThriftHiveMetastoreProcessor::process_add_partition(int32_t seqid, ::apache } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_partition"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_partition_with_environment_context"); } - oprot->writeMessageBegin("add_partition", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("drop_partition_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_partition", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_partition_with_environment_context", bytes); } } -void ThriftHiveMetastoreProcessor::process_add_partition_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_drop_partition_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_partition_with_environment_context", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_partition_by_name", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_partition_with_environment_context"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_partition_by_name"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_partition_with_environment_context"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_partition_by_name"); } - ThriftHiveMetastore_add_partition_with_environment_context_args args; + ThriftHiveMetastore_drop_partition_by_name_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_partition_with_environment_context", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_partition_by_name", bytes); } - ThriftHiveMetastore_add_partition_with_environment_context_result result; + ThriftHiveMetastore_drop_partition_by_name_result result; try { - iface_->add_partition_with_environment_context(result.success, args.new_part, args.environment_context); + result.success = iface_->drop_partition_by_name(args.db_name, args.tbl_name, args.part_name, args.deleteData); result.__isset.success = true; - } catch (InvalidObjectException &o1) { + } catch (NoSuchObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (AlreadyExistsException &o2) { + } catch (MetaException &o2) { result.o2 = o2; result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_partition_with_environment_context"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_partition_by_name"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("add_partition_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("drop_partition_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -27071,61 +30241,58 @@ void ThriftHiveMetastoreProcessor::process_add_partition_with_environment_contex } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_partition_with_environment_context"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_partition_by_name"); } - oprot->writeMessageBegin("add_partition_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("drop_partition_by_name", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_partition_with_environment_context", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_partition_by_name", bytes); } } -void ThriftHiveMetastoreProcessor::process_add_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_drop_partition_by_name_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_partitions", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_partition_by_name_with_environment_context", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_partitions"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_partitions"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context"); } - ThriftHiveMetastore_add_partitions_args args; + ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_partitions", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context", bytes); } - ThriftHiveMetastore_add_partitions_result result; + ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result result; try { - result.success = iface_->add_partitions(args.new_parts); + result.success = iface_->drop_partition_by_name_with_environment_context(args.db_name, args.tbl_name, args.part_name, args.deleteData, args.environment_context); result.__isset.success = true; - } catch (InvalidObjectException &o1) { + } catch (NoSuchObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (AlreadyExistsException &o2) { + } catch (MetaException &o2) { result.o2 = o2; result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_partitions"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("add_partitions", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("drop_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -27134,61 +30301,58 @@ void ThriftHiveMetastoreProcessor::process_add_partitions(int32_t seqid, ::apach } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_partitions"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context"); } - oprot->writeMessageBegin("add_partitions", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("drop_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_partitions", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context", bytes); } } -void ThriftHiveMetastoreProcessor::process_append_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_get_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.append_partition", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.append_partition"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.append_partition"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition"); } - ThriftHiveMetastore_append_partition_args args; + ThriftHiveMetastore_get_partition_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.append_partition", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition", bytes); } - ThriftHiveMetastore_append_partition_result result; + ThriftHiveMetastore_get_partition_result result; try { - iface_->append_partition(result.success, args.db_name, args.tbl_name, args.part_vals); + iface_->get_partition(result.success, args.db_name, args.tbl_name, args.part_vals); result.__isset.success = true; - } catch (InvalidObjectException &o1) { + } catch (MetaException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (AlreadyExistsException &o2) { + } catch (NoSuchObjectException &o2) { result.o2 = o2; result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.append_partition"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("append_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -27197,61 +30361,64 @@ void ThriftHiveMetastoreProcessor::process_append_partition(int32_t seqid, ::apa } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.append_partition"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition"); } - oprot->writeMessageBegin("append_partition", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("get_partition", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.append_partition", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition", bytes); } } -void ThriftHiveMetastoreProcessor::process_append_partition_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_exchange_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.append_partition_with_environment_context", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.exchange_partition", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.append_partition_with_environment_context"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.exchange_partition"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.append_partition_with_environment_context"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.exchange_partition"); } - ThriftHiveMetastore_append_partition_with_environment_context_args args; + ThriftHiveMetastore_exchange_partition_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.append_partition_with_environment_context", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.exchange_partition", bytes); } - ThriftHiveMetastore_append_partition_with_environment_context_result result; + ThriftHiveMetastore_exchange_partition_result result; try { - iface_->append_partition_with_environment_context(result.success, args.db_name, args.tbl_name, args.part_vals, args.environment_context); + iface_->exchange_partition(result.success, args.partitionSpecs, args.source_db, args.source_table_name, args.dest_db, args.dest_table_name); result.__isset.success = true; - } catch (InvalidObjectException &o1) { + } catch (MetaException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (AlreadyExistsException &o2) { + } catch (NoSuchObjectException &o2) { result.o2 = o2; result.__isset.o2 = true; - } catch (MetaException &o3) { + } catch (InvalidObjectException &o3) { result.o3 = o3; result.__isset.o3 = true; + } catch (InvalidInputException &o4) { + result.o4 = o4; + result.__isset.o4 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.append_partition_with_environment_context"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.exchange_partition"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("append_partition_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("exchange_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -27260,61 +30427,58 @@ void ThriftHiveMetastoreProcessor::process_append_partition_with_environment_con } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.append_partition_with_environment_context"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.exchange_partition"); } - oprot->writeMessageBegin("append_partition_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("exchange_partition", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.append_partition_with_environment_context", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.exchange_partition", bytes); } } -void ThriftHiveMetastoreProcessor::process_append_partition_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_get_partition_with_auth(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.append_partition_by_name", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition_with_auth", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.append_partition_by_name"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_with_auth"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.append_partition_by_name"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_with_auth"); } - ThriftHiveMetastore_append_partition_by_name_args args; + ThriftHiveMetastore_get_partition_with_auth_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.append_partition_by_name", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition_with_auth", bytes); } - ThriftHiveMetastore_append_partition_by_name_result result; + ThriftHiveMetastore_get_partition_with_auth_result result; try { - iface_->append_partition_by_name(result.success, args.db_name, args.tbl_name, args.part_name); + iface_->get_partition_with_auth(result.success, args.db_name, args.tbl_name, args.part_vals, args.user_name, args.group_names); result.__isset.success = true; - } catch (InvalidObjectException &o1) { + } catch (MetaException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (AlreadyExistsException &o2) { + } catch (NoSuchObjectException &o2) { result.o2 = o2; result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.append_partition_by_name"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition_with_auth"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("append_partition_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_partition_with_auth", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -27323,61 +30487,58 @@ void ThriftHiveMetastoreProcessor::process_append_partition_by_name(int32_t seqi } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.append_partition_by_name"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_with_auth"); } - oprot->writeMessageBegin("append_partition_by_name", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("get_partition_with_auth", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.append_partition_by_name", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition_with_auth", bytes); } } -void ThriftHiveMetastoreProcessor::process_append_partition_by_name_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_get_partition_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.append_partition_by_name_with_environment_context", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition_by_name", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_by_name"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_by_name"); } - ThriftHiveMetastore_append_partition_by_name_with_environment_context_args args; + ThriftHiveMetastore_get_partition_by_name_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition_by_name", bytes); } - ThriftHiveMetastore_append_partition_by_name_with_environment_context_result result; + ThriftHiveMetastore_get_partition_by_name_result result; try { - iface_->append_partition_by_name_with_environment_context(result.success, args.db_name, args.tbl_name, args.part_name, args.environment_context); + iface_->get_partition_by_name(result.success, args.db_name, args.tbl_name, args.part_name); result.__isset.success = true; - } catch (InvalidObjectException &o1) { + } catch (MetaException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (AlreadyExistsException &o2) { + } catch (NoSuchObjectException &o2) { result.o2 = o2; result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition_by_name"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("append_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_partition_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -27386,44 +30547,44 @@ void ThriftHiveMetastoreProcessor::process_append_partition_by_name_with_environ } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_by_name"); } - oprot->writeMessageBegin("append_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("get_partition_by_name", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition_by_name", bytes); } } -void ThriftHiveMetastoreProcessor::process_drop_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_get_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_partition", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_partition"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_partition"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions"); } - ThriftHiveMetastore_drop_partition_args args; + ThriftHiveMetastore_get_partitions_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_partition", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions", bytes); } - ThriftHiveMetastore_drop_partition_result result; + ThriftHiveMetastore_get_partitions_result result; try { - result.success = iface_->drop_partition(args.db_name, args.tbl_name, args.part_vals, args.deleteData); + iface_->get_partitions(result.success, args.db_name, args.tbl_name, args.max_parts); result.__isset.success = true; } catch (NoSuchObjectException &o1) { result.o1 = o1; @@ -27433,11 +30594,11 @@ void ThriftHiveMetastoreProcessor::process_drop_partition(int32_t seqid, ::apach result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_partition"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_partitions", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -27446,44 +30607,44 @@ void ThriftHiveMetastoreProcessor::process_drop_partition(int32_t seqid, ::apach } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_partition"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions"); } - oprot->writeMessageBegin("drop_partition", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("get_partitions", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_partition", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions", bytes); } } -void ThriftHiveMetastoreProcessor::process_drop_partition_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_get_partitions_with_auth(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_partition_with_environment_context", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_with_auth", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_partition_with_environment_context"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_with_auth"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_partition_with_environment_context"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_with_auth"); } - ThriftHiveMetastore_drop_partition_with_environment_context_args args; + ThriftHiveMetastore_get_partitions_with_auth_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_partition_with_environment_context", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_with_auth", bytes); } - ThriftHiveMetastore_drop_partition_with_environment_context_result result; + ThriftHiveMetastore_get_partitions_with_auth_result result; try { - result.success = iface_->drop_partition_with_environment_context(args.db_name, args.tbl_name, args.part_vals, args.deleteData, args.environment_context); + iface_->get_partitions_with_auth(result.success, args.db_name, args.tbl_name, args.max_parts, args.user_name, args.group_names); result.__isset.success = true; } catch (NoSuchObjectException &o1) { result.o1 = o1; @@ -27493,11 +30654,11 @@ void ThriftHiveMetastoreProcessor::process_drop_partition_with_environment_conte result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_partition_with_environment_context"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_with_auth"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_partition_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_partitions_with_auth", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -27506,58 +30667,55 @@ void ThriftHiveMetastoreProcessor::process_drop_partition_with_environment_conte } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_partition_with_environment_context"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_with_auth"); } - oprot->writeMessageBegin("drop_partition_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("get_partitions_with_auth", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_partition_with_environment_context", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_with_auth", bytes); } } -void ThriftHiveMetastoreProcessor::process_drop_partition_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_get_partition_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_partition_by_name", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition_names", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_partition_by_name"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_names"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_partition_by_name"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_names"); } - ThriftHiveMetastore_drop_partition_by_name_args args; + ThriftHiveMetastore_get_partition_names_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_partition_by_name", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition_names", bytes); } - ThriftHiveMetastore_drop_partition_by_name_result result; + ThriftHiveMetastore_get_partition_names_result result; try { - result.success = iface_->drop_partition_by_name(args.db_name, args.tbl_name, args.part_name, args.deleteData); + iface_->get_partition_names(result.success, args.db_name, args.tbl_name, args.max_parts); result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; } catch (MetaException &o2) { result.o2 = o2; result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_partition_by_name"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition_names"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_partition_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_partition_names", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -27566,58 +30724,58 @@ void ThriftHiveMetastoreProcessor::process_drop_partition_by_name(int32_t seqid, } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_partition_by_name"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_names"); } - oprot->writeMessageBegin("drop_partition_by_name", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("get_partition_names", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_partition_by_name", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition_names", bytes); } } -void ThriftHiveMetastoreProcessor::process_drop_partition_by_name_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_get_partitions_ps(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_partition_by_name_with_environment_context", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_ps", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_ps"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_ps"); } - ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args args; + ThriftHiveMetastore_get_partitions_ps_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_ps", bytes); } - ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result result; + ThriftHiveMetastore_get_partitions_ps_result result; try { - result.success = iface_->drop_partition_by_name_with_environment_context(args.db_name, args.tbl_name, args.part_name, args.deleteData, args.environment_context); + iface_->get_partitions_ps(result.success, args.db_name, args.tbl_name, args.part_vals, args.max_parts); result.__isset.success = true; - } catch (NoSuchObjectException &o1) { + } catch (MetaException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (MetaException &o2) { + } catch (NoSuchObjectException &o2) { result.o2 = o2; result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_ps"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_partitions_ps", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -27626,58 +30784,58 @@ void ThriftHiveMetastoreProcessor::process_drop_partition_by_name_with_environme } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_ps"); } - oprot->writeMessageBegin("drop_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("get_partitions_ps", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_ps", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_get_partitions_ps_with_auth(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_ps_with_auth", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth"); } - ThriftHiveMetastore_get_partition_args args; + ThriftHiveMetastore_get_partitions_ps_with_auth_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth", bytes); } - ThriftHiveMetastore_get_partition_result result; + ThriftHiveMetastore_get_partitions_ps_with_auth_result result; try { - iface_->get_partition(result.success, args.db_name, args.tbl_name, args.part_vals); + iface_->get_partitions_ps_with_auth(result.success, args.db_name, args.tbl_name, args.part_vals, args.max_parts, args.user_name, args.group_names); result.__isset.success = true; - } catch (MetaException &o1) { + } catch (NoSuchObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { + } catch (MetaException &o2) { result.o2 = o2; result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_partitions_ps_with_auth", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -27686,44 +30844,44 @@ void ThriftHiveMetastoreProcessor::process_get_partition(int32_t seqid, ::apache } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth"); } - oprot->writeMessageBegin("get_partition", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("get_partitions_ps_with_auth", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth", bytes); } } -void ThriftHiveMetastoreProcessor::process_exchange_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_get_partition_names_ps(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.exchange_partition", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition_names_ps", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.exchange_partition"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_names_ps"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.exchange_partition"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_names_ps"); } - ThriftHiveMetastore_exchange_partition_args args; + ThriftHiveMetastore_get_partition_names_ps_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.exchange_partition", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition_names_ps", bytes); } - ThriftHiveMetastore_exchange_partition_result result; + ThriftHiveMetastore_get_partition_names_ps_result result; try { - iface_->exchange_partition(result.success, args.partitionSpecs, args.source_db, args.source_table_name, args.dest_db, args.dest_table_name); + iface_->get_partition_names_ps(result.success, args.db_name, args.tbl_name, args.part_vals, args.max_parts); result.__isset.success = true; } catch (MetaException &o1) { result.o1 = o1; @@ -27731,19 +30889,13 @@ void ThriftHiveMetastoreProcessor::process_exchange_partition(int32_t seqid, ::a } catch (NoSuchObjectException &o2) { result.o2 = o2; result.__isset.o2 = true; - } catch (InvalidObjectException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (InvalidInputException &o4) { - result.o4 = o4; - result.__isset.o4 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.exchange_partition"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition_names_ps"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("exchange_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_partition_names_ps", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -27752,44 +30904,44 @@ void ThriftHiveMetastoreProcessor::process_exchange_partition(int32_t seqid, ::a } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.exchange_partition"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_names_ps"); } - oprot->writeMessageBegin("exchange_partition", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("get_partition_names_ps", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.exchange_partition", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition_names_ps", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_partition_with_auth(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_get_partitions_by_filter(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition_with_auth", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_by_filter", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_with_auth"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_by_filter"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_with_auth"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_by_filter"); } - ThriftHiveMetastore_get_partition_with_auth_args args; + ThriftHiveMetastore_get_partitions_by_filter_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition_with_auth", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_by_filter", bytes); } - ThriftHiveMetastore_get_partition_with_auth_result result; + ThriftHiveMetastore_get_partitions_by_filter_result result; try { - iface_->get_partition_with_auth(result.success, args.db_name, args.tbl_name, args.part_vals, args.user_name, args.group_names); + iface_->get_partitions_by_filter(result.success, args.db_name, args.tbl_name, args.filter, args.max_parts); result.__isset.success = true; } catch (MetaException &o1) { result.o1 = o1; @@ -27799,11 +30951,11 @@ void ThriftHiveMetastoreProcessor::process_get_partition_with_auth(int32_t seqid result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition_with_auth"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_by_filter"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partition_with_auth", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_partitions_by_filter", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -27812,44 +30964,44 @@ void ThriftHiveMetastoreProcessor::process_get_partition_with_auth(int32_t seqid } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_with_auth"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_by_filter"); } - oprot->writeMessageBegin("get_partition_with_auth", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("get_partitions_by_filter", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition_with_auth", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_by_filter", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_partition_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_get_partitions_by_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition_by_name", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_by_names", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_by_name"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_by_names"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_by_name"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_by_names"); } - ThriftHiveMetastore_get_partition_by_name_args args; + ThriftHiveMetastore_get_partitions_by_names_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition_by_name", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_by_names", bytes); } - ThriftHiveMetastore_get_partition_by_name_result result; + ThriftHiveMetastore_get_partitions_by_names_result result; try { - iface_->get_partition_by_name(result.success, args.db_name, args.tbl_name, args.part_name); + iface_->get_partitions_by_names(result.success, args.db_name, args.tbl_name, args.names); result.__isset.success = true; } catch (MetaException &o1) { result.o1 = o1; @@ -27859,11 +31011,11 @@ void ThriftHiveMetastoreProcessor::process_get_partition_by_name(int32_t seqid, result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition_by_name"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_by_names"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partition_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_partitions_by_names", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -27872,46 +31024,45 @@ void ThriftHiveMetastoreProcessor::process_get_partition_by_name(int32_t seqid, } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_by_name"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_by_names"); } - oprot->writeMessageBegin("get_partition_by_name", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("get_partitions_by_names", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition_by_name", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_by_names", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_alter_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_partition", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_partition"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_partition"); } - ThriftHiveMetastore_get_partitions_args args; + ThriftHiveMetastore_alter_partition_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_partition", bytes); } - ThriftHiveMetastore_get_partitions_result result; + ThriftHiveMetastore_alter_partition_result result; try { - iface_->get_partitions(result.success, args.db_name, args.tbl_name, args.max_parts); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { + iface_->alter_partition(args.db_name, args.tbl_name, args.new_part); + } catch (InvalidOperationException &o1) { result.o1 = o1; result.__isset.o1 = true; } catch (MetaException &o2) { @@ -27919,11 +31070,11 @@ void ThriftHiveMetastoreProcessor::process_get_partitions(int32_t seqid, ::apach result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_partition"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partitions", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("alter_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -27932,46 +31083,45 @@ void ThriftHiveMetastoreProcessor::process_get_partitions(int32_t seqid, ::apach } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_partition"); } - oprot->writeMessageBegin("get_partitions", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("alter_partition", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_partition", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_partitions_with_auth(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_alter_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_with_auth", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_partitions", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_with_auth"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_partitions"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_with_auth"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_partitions"); } - ThriftHiveMetastore_get_partitions_with_auth_args args; + ThriftHiveMetastore_alter_partitions_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_with_auth", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_partitions", bytes); } - ThriftHiveMetastore_get_partitions_with_auth_result result; + ThriftHiveMetastore_alter_partitions_result result; try { - iface_->get_partitions_with_auth(result.success, args.db_name, args.tbl_name, args.max_parts, args.user_name, args.group_names); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { + iface_->alter_partitions(args.db_name, args.tbl_name, args.new_parts); + } catch (InvalidOperationException &o1) { result.o1 = o1; result.__isset.o1 = true; } catch (MetaException &o2) { @@ -27979,11 +31129,11 @@ void ThriftHiveMetastoreProcessor::process_get_partitions_with_auth(int32_t seqi result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_with_auth"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_partitions"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partitions_with_auth", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("alter_partitions", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -27992,55 +31142,57 @@ void ThriftHiveMetastoreProcessor::process_get_partitions_with_auth(int32_t seqi } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_with_auth"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_partitions"); } - oprot->writeMessageBegin("get_partitions_with_auth", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("alter_partitions", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_with_auth", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_partitions", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_partition_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_alter_partition_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition_names", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_partition_with_environment_context", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_names"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_partition_with_environment_context"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_names"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_partition_with_environment_context"); } - ThriftHiveMetastore_get_partition_names_args args; + ThriftHiveMetastore_alter_partition_with_environment_context_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition_names", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_partition_with_environment_context", bytes); } - ThriftHiveMetastore_get_partition_names_result result; + ThriftHiveMetastore_alter_partition_with_environment_context_result result; try { - iface_->get_partition_names(result.success, args.db_name, args.tbl_name, args.max_parts); - result.__isset.success = true; + iface_->alter_partition_with_environment_context(args.db_name, args.tbl_name, args.new_part, args.environment_context); + } catch (InvalidOperationException &o1) { + result.o1 = o1; + result.__isset.o1 = true; } catch (MetaException &o2) { result.o2 = o2; result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition_names"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_partition_with_environment_context"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partition_names", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("alter_partition_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -28049,58 +31201,57 @@ void ThriftHiveMetastoreProcessor::process_get_partition_names(int32_t seqid, :: } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_names"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_partition_with_environment_context"); } - oprot->writeMessageBegin("get_partition_names", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("alter_partition_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition_names", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_partition_with_environment_context", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_partitions_ps(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_rename_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_ps", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.rename_partition", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_ps"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.rename_partition"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_ps"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.rename_partition"); } - ThriftHiveMetastore_get_partitions_ps_args args; + ThriftHiveMetastore_rename_partition_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_ps", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.rename_partition", bytes); } - ThriftHiveMetastore_get_partitions_ps_result result; + ThriftHiveMetastore_rename_partition_result result; try { - iface_->get_partitions_ps(result.success, args.db_name, args.tbl_name, args.part_vals, args.max_parts); - result.__isset.success = true; - } catch (MetaException &o1) { + iface_->rename_partition(args.db_name, args.tbl_name, args.part_vals, args.new_part); + } catch (InvalidOperationException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { + } catch (MetaException &o2) { result.o2 = o2; result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_ps"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.rename_partition"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partitions_ps", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("rename_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -28109,58 +31260,55 @@ void ThriftHiveMetastoreProcessor::process_get_partitions_ps(int32_t seqid, ::ap } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_ps"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.rename_partition"); } - oprot->writeMessageBegin("get_partitions_ps", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("rename_partition", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_ps", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.rename_partition", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_partitions_ps_with_auth(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_partition_name_has_valid_characters(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_ps_with_auth", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.partition_name_has_valid_characters", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.partition_name_has_valid_characters"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.partition_name_has_valid_characters"); } - ThriftHiveMetastore_get_partitions_ps_with_auth_args args; + ThriftHiveMetastore_partition_name_has_valid_characters_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.partition_name_has_valid_characters", bytes); } - ThriftHiveMetastore_get_partitions_ps_with_auth_result result; + ThriftHiveMetastore_partition_name_has_valid_characters_result result; try { - iface_->get_partitions_ps_with_auth(result.success, args.db_name, args.tbl_name, args.part_vals, args.max_parts, args.user_name, args.group_names); + result.success = iface_->partition_name_has_valid_characters(args.part_vals, args.throw_exception); result.__isset.success = true; - } catch (NoSuchObjectException &o1) { + } catch (MetaException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.partition_name_has_valid_characters"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partitions_ps_with_auth", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("partition_name_has_valid_characters", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -28169,58 +31317,55 @@ void ThriftHiveMetastoreProcessor::process_get_partitions_ps_with_auth(int32_t s } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.partition_name_has_valid_characters"); } - oprot->writeMessageBegin("get_partitions_ps_with_auth", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("partition_name_has_valid_characters", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.partition_name_has_valid_characters", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_partition_names_ps(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_get_config_value(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition_names_ps", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_config_value", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_names_ps"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_config_value"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_names_ps"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_config_value"); } - ThriftHiveMetastore_get_partition_names_ps_args args; + ThriftHiveMetastore_get_config_value_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition_names_ps", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_config_value", bytes); } - ThriftHiveMetastore_get_partition_names_ps_result result; + ThriftHiveMetastore_get_config_value_result result; try { - iface_->get_partition_names_ps(result.success, args.db_name, args.tbl_name, args.part_vals, args.max_parts); + iface_->get_config_value(result.success, args.name, args.defaultValue); result.__isset.success = true; - } catch (MetaException &o1) { + } catch (ConfigValSecurityException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition_names_ps"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_config_value"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partition_names_ps", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_config_value", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -28229,58 +31374,55 @@ void ThriftHiveMetastoreProcessor::process_get_partition_names_ps(int32_t seqid, } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_names_ps"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_config_value"); } - oprot->writeMessageBegin("get_partition_names_ps", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("get_config_value", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition_names_ps", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_config_value", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_partitions_by_filter(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_partition_name_to_vals(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_by_filter", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.partition_name_to_vals", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_by_filter"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.partition_name_to_vals"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_by_filter"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.partition_name_to_vals"); } - ThriftHiveMetastore_get_partitions_by_filter_args args; + ThriftHiveMetastore_partition_name_to_vals_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_by_filter", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.partition_name_to_vals", bytes); } - ThriftHiveMetastore_get_partitions_by_filter_result result; + ThriftHiveMetastore_partition_name_to_vals_result result; try { - iface_->get_partitions_by_filter(result.success, args.db_name, args.tbl_name, args.filter, args.max_parts); + iface_->partition_name_to_vals(result.success, args.part_name); result.__isset.success = true; } catch (MetaException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_by_filter"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.partition_name_to_vals"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partitions_by_filter", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("partition_name_to_vals", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -28289,58 +31431,55 @@ void ThriftHiveMetastoreProcessor::process_get_partitions_by_filter(int32_t seqi } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_by_filter"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.partition_name_to_vals"); } - oprot->writeMessageBegin("get_partitions_by_filter", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("partition_name_to_vals", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_by_filter", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.partition_name_to_vals", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_partitions_by_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_partition_name_to_spec(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_by_names", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.partition_name_to_spec", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_by_names"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.partition_name_to_spec"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_by_names"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.partition_name_to_spec"); } - ThriftHiveMetastore_get_partitions_by_names_args args; + ThriftHiveMetastore_partition_name_to_spec_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_by_names", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.partition_name_to_spec", bytes); } - ThriftHiveMetastore_get_partitions_by_names_result result; + ThriftHiveMetastore_partition_name_to_spec_result result; try { - iface_->get_partitions_by_names(result.success, args.db_name, args.tbl_name, args.names); + iface_->partition_name_to_spec(result.success, args.part_name); result.__isset.success = true; } catch (MetaException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_by_names"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.partition_name_to_spec"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partitions_by_names", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("partition_name_to_spec", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -28349,57 +31488,69 @@ void ThriftHiveMetastoreProcessor::process_get_partitions_by_names(int32_t seqid } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_by_names"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.partition_name_to_spec"); } - oprot->writeMessageBegin("get_partitions_by_names", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("partition_name_to_spec", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_by_names", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.partition_name_to_spec", bytes); } } -void ThriftHiveMetastoreProcessor::process_alter_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_markPartitionForEvent(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_partition", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.markPartitionForEvent", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_partition"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.markPartitionForEvent"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_partition"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.markPartitionForEvent"); } - ThriftHiveMetastore_alter_partition_args args; + ThriftHiveMetastore_markPartitionForEvent_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_partition", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.markPartitionForEvent", bytes); } - ThriftHiveMetastore_alter_partition_result result; + ThriftHiveMetastore_markPartitionForEvent_result result; try { - iface_->alter_partition(args.db_name, args.tbl_name, args.new_part); - } catch (InvalidOperationException &o1) { + iface_->markPartitionForEvent(args.db_name, args.tbl_name, args.part_vals, args.eventType); + } catch (MetaException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (MetaException &o2) { + } catch (NoSuchObjectException &o2) { result.o2 = o2; result.__isset.o2 = true; + } catch (UnknownDBException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (UnknownTableException &o4) { + result.o4 = o4; + result.__isset.o4 = true; + } catch (UnknownPartitionException &o5) { + result.o5 = o5; + result.__isset.o5 = true; + } catch (InvalidPartitionException &o6) { + result.o6 = o6; + result.__isset.o6 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_partition"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.markPartitionForEvent"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("alter_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("markPartitionForEvent", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -28408,57 +31559,70 @@ void ThriftHiveMetastoreProcessor::process_alter_partition(int32_t seqid, ::apac } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_partition"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.markPartitionForEvent"); } - oprot->writeMessageBegin("alter_partition", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("markPartitionForEvent", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_partition", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.markPartitionForEvent", bytes); } } -void ThriftHiveMetastoreProcessor::process_alter_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_isPartitionMarkedForEvent(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_partitions", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.isPartitionMarkedForEvent", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_partitions"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_partitions"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent"); } - ThriftHiveMetastore_alter_partitions_args args; + ThriftHiveMetastore_isPartitionMarkedForEvent_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_partitions", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent", bytes); } - ThriftHiveMetastore_alter_partitions_result result; + ThriftHiveMetastore_isPartitionMarkedForEvent_result result; try { - iface_->alter_partitions(args.db_name, args.tbl_name, args.new_parts); - } catch (InvalidOperationException &o1) { + result.success = iface_->isPartitionMarkedForEvent(args.db_name, args.tbl_name, args.part_vals, args.eventType); + result.__isset.success = true; + } catch (MetaException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (MetaException &o2) { + } catch (NoSuchObjectException &o2) { result.o2 = o2; result.__isset.o2 = true; + } catch (UnknownDBException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (UnknownTableException &o4) { + result.o4 = o4; + result.__isset.o4 = true; + } catch (UnknownPartitionException &o5) { + result.o5 = o5; + result.__isset.o5 = true; + } catch (InvalidPartitionException &o6) { + result.o6 = o6; + result.__isset.o6 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_partitions"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("alter_partitions", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("isPartitionMarkedForEvent", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -28467,57 +31631,61 @@ void ThriftHiveMetastoreProcessor::process_alter_partitions(int32_t seqid, ::apa } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_partitions"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent"); } - oprot->writeMessageBegin("alter_partitions", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("isPartitionMarkedForEvent", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_partitions", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent", bytes); } } -void ThriftHiveMetastoreProcessor::process_alter_partition_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_add_index(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_partition_with_environment_context", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_index", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_partition_with_environment_context"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_index"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_partition_with_environment_context"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_index"); } - ThriftHiveMetastore_alter_partition_with_environment_context_args args; + ThriftHiveMetastore_add_index_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_partition_with_environment_context", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_index", bytes); } - ThriftHiveMetastore_alter_partition_with_environment_context_result result; + ThriftHiveMetastore_add_index_result result; try { - iface_->alter_partition_with_environment_context(args.db_name, args.tbl_name, args.new_part, args.environment_context); - } catch (InvalidOperationException &o1) { + iface_->add_index(result.success, args.new_index, args.index_table); + result.__isset.success = true; + } catch (InvalidObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (MetaException &o2) { + } catch (AlreadyExistsException &o2) { result.o2 = o2; result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_partition_with_environment_context"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_index"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("alter_partition_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("add_index", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -28526,44 +31694,44 @@ void ThriftHiveMetastoreProcessor::process_alter_partition_with_environment_cont } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_partition_with_environment_context"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_index"); } - oprot->writeMessageBegin("alter_partition_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("add_index", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_partition_with_environment_context", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_index", bytes); } } -void ThriftHiveMetastoreProcessor::process_rename_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_alter_index(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.rename_partition", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_index", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.rename_partition"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_index"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.rename_partition"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_index"); } - ThriftHiveMetastore_rename_partition_args args; + ThriftHiveMetastore_alter_index_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.rename_partition", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_index", bytes); } - ThriftHiveMetastore_rename_partition_result result; + ThriftHiveMetastore_alter_index_result result; try { - iface_->rename_partition(args.db_name, args.tbl_name, args.part_vals, args.new_part); + iface_->alter_index(args.dbname, args.base_tbl_name, args.idx_name, args.new_idx); } catch (InvalidOperationException &o1) { result.o1 = o1; result.__isset.o1 = true; @@ -28572,11 +31740,11 @@ void ThriftHiveMetastoreProcessor::process_rename_partition(int32_t seqid, ::apa result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.rename_partition"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_index"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("rename_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("alter_index", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -28585,55 +31753,58 @@ void ThriftHiveMetastoreProcessor::process_rename_partition(int32_t seqid, ::apa } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.rename_partition"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_index"); } - oprot->writeMessageBegin("rename_partition", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("alter_index", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.rename_partition", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_index", bytes); } } -void ThriftHiveMetastoreProcessor::process_partition_name_has_valid_characters(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_drop_index_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.partition_name_has_valid_characters", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_index_by_name", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.partition_name_has_valid_characters"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_index_by_name"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.partition_name_has_valid_characters"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_index_by_name"); } - ThriftHiveMetastore_partition_name_has_valid_characters_args args; + ThriftHiveMetastore_drop_index_by_name_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.partition_name_has_valid_characters", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_index_by_name", bytes); } - ThriftHiveMetastore_partition_name_has_valid_characters_result result; + ThriftHiveMetastore_drop_index_by_name_result result; try { - result.success = iface_->partition_name_has_valid_characters(args.part_vals, args.throw_exception); + result.success = iface_->drop_index_by_name(args.db_name, args.tbl_name, args.index_name, args.deleteData); result.__isset.success = true; - } catch (MetaException &o1) { + } catch (NoSuchObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.partition_name_has_valid_characters"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_index_by_name"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("partition_name_has_valid_characters", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("drop_index_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -28642,55 +31813,58 @@ void ThriftHiveMetastoreProcessor::process_partition_name_has_valid_characters(i } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.partition_name_has_valid_characters"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_index_by_name"); } - oprot->writeMessageBegin("partition_name_has_valid_characters", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("drop_index_by_name", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.partition_name_has_valid_characters", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_index_by_name", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_config_value(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_get_index_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_config_value", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_index_by_name", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_config_value"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_index_by_name"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_config_value"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_index_by_name"); } - ThriftHiveMetastore_get_config_value_args args; + ThriftHiveMetastore_get_index_by_name_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_config_value", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_index_by_name", bytes); } - ThriftHiveMetastore_get_config_value_result result; + ThriftHiveMetastore_get_index_by_name_result result; try { - iface_->get_config_value(result.success, args.name, args.defaultValue); + iface_->get_index_by_name(result.success, args.db_name, args.tbl_name, args.index_name); result.__isset.success = true; - } catch (ConfigValSecurityException &o1) { + } catch (MetaException &o1) { result.o1 = o1; result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_config_value"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_index_by_name"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_config_value", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_index_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -28699,55 +31873,58 @@ void ThriftHiveMetastoreProcessor::process_get_config_value(int32_t seqid, ::apa } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_config_value"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_index_by_name"); } - oprot->writeMessageBegin("get_config_value", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("get_index_by_name", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_config_value", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_index_by_name", bytes); } } -void ThriftHiveMetastoreProcessor::process_partition_name_to_vals(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_get_indexes(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.partition_name_to_vals", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_indexes", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.partition_name_to_vals"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_indexes"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.partition_name_to_vals"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_indexes"); } - ThriftHiveMetastore_partition_name_to_vals_args args; + ThriftHiveMetastore_get_indexes_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.partition_name_to_vals", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_indexes", bytes); } - ThriftHiveMetastore_partition_name_to_vals_result result; + ThriftHiveMetastore_get_indexes_result result; try { - iface_->partition_name_to_vals(result.success, args.part_name); + iface_->get_indexes(result.success, args.db_name, args.tbl_name, args.max_indexes); result.__isset.success = true; - } catch (MetaException &o1) { + } catch (NoSuchObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.partition_name_to_vals"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_indexes"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("partition_name_to_vals", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_indexes", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -28756,55 +31933,55 @@ void ThriftHiveMetastoreProcessor::process_partition_name_to_vals(int32_t seqid, } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.partition_name_to_vals"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_indexes"); } - oprot->writeMessageBegin("partition_name_to_vals", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("get_indexes", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.partition_name_to_vals", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_indexes", bytes); } } -void ThriftHiveMetastoreProcessor::process_partition_name_to_spec(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_get_index_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.partition_name_to_spec", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_index_names", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.partition_name_to_spec"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_index_names"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.partition_name_to_spec"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_index_names"); } - ThriftHiveMetastore_partition_name_to_spec_args args; + ThriftHiveMetastore_get_index_names_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.partition_name_to_spec", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_index_names", bytes); } - ThriftHiveMetastore_partition_name_to_spec_result result; + ThriftHiveMetastore_get_index_names_result result; try { - iface_->partition_name_to_spec(result.success, args.part_name); + iface_->get_index_names(result.success, args.db_name, args.tbl_name, args.max_indexes); result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.partition_name_to_spec"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_index_names"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("partition_name_to_spec", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_index_names", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -28813,69 +31990,64 @@ void ThriftHiveMetastoreProcessor::process_partition_name_to_spec(int32_t seqid, } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.partition_name_to_spec"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_index_names"); } - oprot->writeMessageBegin("partition_name_to_spec", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("get_index_names", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.partition_name_to_spec", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_index_names", bytes); } } -void ThriftHiveMetastoreProcessor::process_markPartitionForEvent(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_update_table_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.markPartitionForEvent", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.update_table_column_statistics", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.markPartitionForEvent"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.update_table_column_statistics"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.markPartitionForEvent"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.update_table_column_statistics"); } - ThriftHiveMetastore_markPartitionForEvent_args args; + ThriftHiveMetastore_update_table_column_statistics_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.markPartitionForEvent", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.update_table_column_statistics", bytes); } - ThriftHiveMetastore_markPartitionForEvent_result result; + ThriftHiveMetastore_update_table_column_statistics_result result; try { - iface_->markPartitionForEvent(args.db_name, args.tbl_name, args.part_vals, args.eventType); - } catch (MetaException &o1) { + result.success = iface_->update_table_column_statistics(args.stats_obj); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { + } catch (InvalidObjectException &o2) { result.o2 = o2; result.__isset.o2 = true; - } catch (UnknownDBException &o3) { + } catch (MetaException &o3) { result.o3 = o3; result.__isset.o3 = true; - } catch (UnknownTableException &o4) { + } catch (InvalidInputException &o4) { result.o4 = o4; result.__isset.o4 = true; - } catch (UnknownPartitionException &o5) { - result.o5 = o5; - result.__isset.o5 = true; - } catch (InvalidPartitionException &o6) { - result.o6 = o6; - result.__isset.o6 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.markPartitionForEvent"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.update_table_column_statistics"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("markPartitionForEvent", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("update_table_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -28884,70 +32056,64 @@ void ThriftHiveMetastoreProcessor::process_markPartitionForEvent(int32_t seqid, } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.markPartitionForEvent"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.update_table_column_statistics"); } - oprot->writeMessageBegin("markPartitionForEvent", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("update_table_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.markPartitionForEvent", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.update_table_column_statistics", bytes); } } -void ThriftHiveMetastoreProcessor::process_isPartitionMarkedForEvent(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_update_partition_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.isPartitionMarkedForEvent", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.update_partition_column_statistics", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.update_partition_column_statistics"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.update_partition_column_statistics"); } - ThriftHiveMetastore_isPartitionMarkedForEvent_args args; + ThriftHiveMetastore_update_partition_column_statistics_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.update_partition_column_statistics", bytes); } - ThriftHiveMetastore_isPartitionMarkedForEvent_result result; + ThriftHiveMetastore_update_partition_column_statistics_result result; try { - result.success = iface_->isPartitionMarkedForEvent(args.db_name, args.tbl_name, args.part_vals, args.eventType); + result.success = iface_->update_partition_column_statistics(args.stats_obj); result.__isset.success = true; - } catch (MetaException &o1) { + } catch (NoSuchObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { + } catch (InvalidObjectException &o2) { result.o2 = o2; result.__isset.o2 = true; - } catch (UnknownDBException &o3) { + } catch (MetaException &o3) { result.o3 = o3; result.__isset.o3 = true; - } catch (UnknownTableException &o4) { + } catch (InvalidInputException &o4) { result.o4 = o4; result.__isset.o4 = true; - } catch (UnknownPartitionException &o5) { - result.o5 = o5; - result.__isset.o5 = true; - } catch (InvalidPartitionException &o6) { - result.o6 = o6; - result.__isset.o6 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.update_partition_column_statistics"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("isPartitionMarkedForEvent", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("update_partition_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -28956,61 +32122,64 @@ void ThriftHiveMetastoreProcessor::process_isPartitionMarkedForEvent(int32_t seq } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.update_partition_column_statistics"); } - oprot->writeMessageBegin("isPartitionMarkedForEvent", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("update_partition_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.update_partition_column_statistics", bytes); } } -void ThriftHiveMetastoreProcessor::process_add_index(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_get_table_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_index", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table_column_statistics", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_index"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table_column_statistics"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_index"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table_column_statistics"); } - ThriftHiveMetastore_add_index_args args; + ThriftHiveMetastore_get_table_column_statistics_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_index", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table_column_statistics", bytes); } - ThriftHiveMetastore_add_index_result result; + ThriftHiveMetastore_get_table_column_statistics_result result; try { - iface_->add_index(result.success, args.new_index, args.index_table); + iface_->get_table_column_statistics(result.success, args.db_name, args.tbl_name, args.col_name); result.__isset.success = true; - } catch (InvalidObjectException &o1) { + } catch (NoSuchObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (AlreadyExistsException &o2) { + } catch (MetaException &o2) { result.o2 = o2; result.__isset.o2 = true; - } catch (MetaException &o3) { + } catch (InvalidInputException &o3) { result.o3 = o3; result.__isset.o3 = true; + } catch (InvalidObjectException &o4) { + result.o4 = o4; + result.__isset.o4 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_index"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table_column_statistics"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("add_index", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_table_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -29019,57 +32188,64 @@ void ThriftHiveMetastoreProcessor::process_add_index(int32_t seqid, ::apache::th } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_index"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table_column_statistics"); } - oprot->writeMessageBegin("add_index", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("get_table_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_index", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table_column_statistics", bytes); } } -void ThriftHiveMetastoreProcessor::process_alter_index(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_get_partition_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_index", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition_column_statistics", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_index"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_column_statistics"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_index"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_column_statistics"); } - ThriftHiveMetastore_alter_index_args args; + ThriftHiveMetastore_get_partition_column_statistics_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_index", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition_column_statistics", bytes); } - ThriftHiveMetastore_alter_index_result result; + ThriftHiveMetastore_get_partition_column_statistics_result result; try { - iface_->alter_index(args.dbname, args.base_tbl_name, args.idx_name, args.new_idx); - } catch (InvalidOperationException &o1) { + iface_->get_partition_column_statistics(result.success, args.db_name, args.tbl_name, args.part_name, args.col_name); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; } catch (MetaException &o2) { result.o2 = o2; result.__isset.o2 = true; + } catch (InvalidInputException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (InvalidObjectException &o4) { + result.o4 = o4; + result.__isset.o4 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_index"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition_column_statistics"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("alter_index", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_partition_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -29078,44 +32254,44 @@ void ThriftHiveMetastoreProcessor::process_alter_index(int32_t seqid, ::apache:: } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_index"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_column_statistics"); } - oprot->writeMessageBegin("alter_index", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("get_partition_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_index", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition_column_statistics", bytes); } } -void ThriftHiveMetastoreProcessor::process_drop_index_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_delete_partition_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_index_by_name", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.delete_partition_column_statistics", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_index_by_name"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.delete_partition_column_statistics"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_index_by_name"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.delete_partition_column_statistics"); } - ThriftHiveMetastore_drop_index_by_name_args args; + ThriftHiveMetastore_delete_partition_column_statistics_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_index_by_name", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.delete_partition_column_statistics", bytes); } - ThriftHiveMetastore_drop_index_by_name_result result; + ThriftHiveMetastore_delete_partition_column_statistics_result result; try { - result.success = iface_->drop_index_by_name(args.db_name, args.tbl_name, args.index_name, args.deleteData); + result.success = iface_->delete_partition_column_statistics(args.db_name, args.tbl_name, args.part_name, args.col_name); result.__isset.success = true; } catch (NoSuchObjectException &o1) { result.o1 = o1; @@ -29123,13 +32299,19 @@ void ThriftHiveMetastoreProcessor::process_drop_index_by_name(int32_t seqid, ::a } catch (MetaException &o2) { result.o2 = o2; result.__isset.o2 = true; + } catch (InvalidObjectException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (InvalidInputException &o4) { + result.o4 = o4; + result.__isset.o4 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_index_by_name"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.delete_partition_column_statistics"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_index_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("delete_partition_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -29138,58 +32320,64 @@ void ThriftHiveMetastoreProcessor::process_drop_index_by_name(int32_t seqid, ::a } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_index_by_name"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.delete_partition_column_statistics"); } - oprot->writeMessageBegin("drop_index_by_name", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("delete_partition_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_index_by_name", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.delete_partition_column_statistics", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_index_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_delete_table_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_index_by_name", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.delete_table_column_statistics", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_index_by_name"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.delete_table_column_statistics"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_index_by_name"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.delete_table_column_statistics"); } - ThriftHiveMetastore_get_index_by_name_args args; + ThriftHiveMetastore_delete_table_column_statistics_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_index_by_name", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.delete_table_column_statistics", bytes); } - ThriftHiveMetastore_get_index_by_name_result result; + ThriftHiveMetastore_delete_table_column_statistics_result result; try { - iface_->get_index_by_name(result.success, args.db_name, args.tbl_name, args.index_name); + result.success = iface_->delete_table_column_statistics(args.db_name, args.tbl_name, args.col_name); result.__isset.success = true; - } catch (MetaException &o1) { + } catch (NoSuchObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { + } catch (MetaException &o2) { result.o2 = o2; result.__isset.o2 = true; + } catch (InvalidObjectException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (InvalidInputException &o4) { + result.o4 = o4; + result.__isset.o4 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_index_by_name"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.delete_table_column_statistics"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_index_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("delete_table_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -29198,58 +32386,61 @@ void ThriftHiveMetastoreProcessor::process_get_index_by_name(int32_t seqid, ::ap } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_index_by_name"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.delete_table_column_statistics"); } - oprot->writeMessageBegin("get_index_by_name", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("delete_table_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_index_by_name", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.delete_table_column_statistics", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_indexes(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_streamingStatus(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_indexes", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.streamingStatus", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_indexes"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.streamingStatus"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_indexes"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.streamingStatus"); } - ThriftHiveMetastore_get_indexes_args args; + ThriftHiveMetastore_streamingStatus_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_indexes", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.streamingStatus", bytes); } - ThriftHiveMetastore_get_indexes_result result; + ThriftHiveMetastore_streamingStatus_result result; try { - iface_->get_indexes(result.success, args.db_name, args.tbl_name, args.max_indexes); + iface_->streamingStatus(result.success, args.db_name, args.table_name); result.__isset.success = true; - } catch (NoSuchObjectException &o1) { + } catch (InvalidInputException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (MetaException &o2) { + } catch (InvalidObjectException &o2) { result.o2 = o2; result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_indexes"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.streamingStatus"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_indexes", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("streamingStatus", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -29258,55 +32449,63 @@ void ThriftHiveMetastoreProcessor::process_get_indexes(int32_t seqid, ::apache:: } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_indexes"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.streamingStatus"); } - oprot->writeMessageBegin("get_indexes", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("streamingStatus", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_indexes", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.streamingStatus", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_index_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_enableStreaming(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_index_names", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.enableStreaming", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_index_names"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.enableStreaming"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_index_names"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.enableStreaming"); } - ThriftHiveMetastore_get_index_names_args args; + ThriftHiveMetastore_enableStreaming_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_index_names", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.enableStreaming", bytes); } - ThriftHiveMetastore_get_index_names_result result; + ThriftHiveMetastore_enableStreaming_result result; try { - iface_->get_index_names(result.success, args.db_name, args.tbl_name, args.max_indexes); - result.__isset.success = true; - } catch (MetaException &o2) { + iface_->enableStreaming(args.db_name, args.table_name, args.streaming_tmp_dir, args.partition_path, args.partition_val); + } catch (InvalidInputException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (InvalidObjectException &o2) { result.o2 = o2; result.__isset.o2 = true; + } catch (InvalidOperationException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (MetaException &o4) { + result.o4 = o4; + result.__isset.o4 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_index_names"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.enableStreaming"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_index_names", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("enableStreaming", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -29315,64 +32514,63 @@ void ThriftHiveMetastoreProcessor::process_get_index_names(int32_t seqid, ::apac } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_index_names"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.enableStreaming"); } - oprot->writeMessageBegin("get_index_names", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("enableStreaming", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_index_names", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.enableStreaming", bytes); } } -void ThriftHiveMetastoreProcessor::process_update_table_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_disableStreaming(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.update_table_column_statistics", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.disableStreaming", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.update_table_column_statistics"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.disableStreaming"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.update_table_column_statistics"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.disableStreaming"); } - ThriftHiveMetastore_update_table_column_statistics_args args; + ThriftHiveMetastore_disableStreaming_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.update_table_column_statistics", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.disableStreaming", bytes); } - ThriftHiveMetastore_update_table_column_statistics_result result; + ThriftHiveMetastore_disableStreaming_result result; try { - result.success = iface_->update_table_column_statistics(args.stats_obj); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { + iface_->disableStreaming(args.db_name, args.table_name); + } catch (InvalidInputException &o1) { result.o1 = o1; result.__isset.o1 = true; } catch (InvalidObjectException &o2) { result.o2 = o2; result.__isset.o2 = true; - } catch (MetaException &o3) { + } catch (InvalidOperationException &o3) { result.o3 = o3; result.__isset.o3 = true; - } catch (InvalidInputException &o4) { + } catch (MetaException &o4) { result.o4 = o4; result.__isset.o4 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.update_table_column_statistics"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.disableStreaming"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("update_table_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("disableStreaming", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -29381,64 +32579,64 @@ void ThriftHiveMetastoreProcessor::process_update_table_column_statistics(int32_ } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.update_table_column_statistics"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.disableStreaming"); } - oprot->writeMessageBegin("update_table_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("disableStreaming", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.update_table_column_statistics", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.disableStreaming", bytes); } } -void ThriftHiveMetastoreProcessor::process_update_partition_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_getCurrentStreamingPartitionPath(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.update_partition_column_statistics", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.getCurrentStreamingPartitionPath", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.update_partition_column_statistics"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.getCurrentStreamingPartitionPath"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.update_partition_column_statistics"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.getCurrentStreamingPartitionPath"); } - ThriftHiveMetastore_update_partition_column_statistics_args args; + ThriftHiveMetastore_getCurrentStreamingPartitionPath_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.update_partition_column_statistics", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.getCurrentStreamingPartitionPath", bytes); } - ThriftHiveMetastore_update_partition_column_statistics_result result; + ThriftHiveMetastore_getCurrentStreamingPartitionPath_result result; try { - result.success = iface_->update_partition_column_statistics(args.stats_obj); + iface_->getCurrentStreamingPartitionPath(result.success, args.dbName, args.tableName); result.__isset.success = true; - } catch (NoSuchObjectException &o1) { + } catch (InvalidInputException &o1) { result.o1 = o1; result.__isset.o1 = true; } catch (InvalidObjectException &o2) { result.o2 = o2; result.__isset.o2 = true; - } catch (MetaException &o3) { + } catch (InvalidOperationException &o3) { result.o3 = o3; result.__isset.o3 = true; - } catch (InvalidInputException &o4) { + } catch (MetaException &o4) { result.o4 = o4; result.__isset.o4 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.update_partition_column_statistics"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.getCurrentStreamingPartitionPath"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("update_partition_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("getCurrentStreamingPartitionPath", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -29447,64 +32645,67 @@ void ThriftHiveMetastoreProcessor::process_update_partition_column_statistics(in } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.update_partition_column_statistics"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.getCurrentStreamingPartitionPath"); } - oprot->writeMessageBegin("update_partition_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("getCurrentStreamingPartitionPath", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.update_partition_column_statistics", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.getCurrentStreamingPartitionPath", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_table_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_rollStreamingPartition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table_column_statistics", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.rollStreamingPartition", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table_column_statistics"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.rollStreamingPartition"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table_column_statistics"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.rollStreamingPartition"); } - ThriftHiveMetastore_get_table_column_statistics_args args; + ThriftHiveMetastore_rollStreamingPartition_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table_column_statistics", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.rollStreamingPartition", bytes); } - ThriftHiveMetastore_get_table_column_statistics_result result; + ThriftHiveMetastore_rollStreamingPartition_result result; try { - iface_->get_table_column_statistics(result.success, args.db_name, args.tbl_name, args.col_name); + iface_->rollStreamingPartition(result.success, args.dbName, args.tableName, args.newPartitionPath, args.newPartitionVal); result.__isset.success = true; - } catch (NoSuchObjectException &o1) { + } catch (InvalidInputException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (MetaException &o2) { + } catch (InvalidObjectException &o2) { result.o2 = o2; result.__isset.o2 = true; - } catch (InvalidInputException &o3) { + } catch (InvalidOperationException &o3) { result.o3 = o3; result.__isset.o3 = true; - } catch (InvalidObjectException &o4) { + } catch (AlreadyExistsException &o4) { result.o4 = o4; result.__isset.o4 = true; + } catch (MetaException &o5) { + result.o5 = o5; + result.__isset.o5 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table_column_statistics"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.rollStreamingPartition"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_table_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("rollStreamingPartition", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -29513,64 +32714,60 @@ void ThriftHiveMetastoreProcessor::process_get_table_column_statistics(int32_t s } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table_column_statistics"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.rollStreamingPartition"); } - oprot->writeMessageBegin("get_table_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("rollStreamingPartition", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table_column_statistics", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.rollStreamingPartition", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_partition_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_updateStreamingTempLocation(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition_column_statistics", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.updateStreamingTempLocation", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_column_statistics"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.updateStreamingTempLocation"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_column_statistics"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.updateStreamingTempLocation"); } - ThriftHiveMetastore_get_partition_column_statistics_args args; + ThriftHiveMetastore_updateStreamingTempLocation_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition_column_statistics", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.updateStreamingTempLocation", bytes); } - ThriftHiveMetastore_get_partition_column_statistics_result result; + ThriftHiveMetastore_updateStreamingTempLocation_result result; try { - iface_->get_partition_column_statistics(result.success, args.db_name, args.tbl_name, args.part_name, args.col_name); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { + iface_->updateStreamingTempLocation(args.dbName, args.tableName, args.streamingTempDir); + } catch (InvalidInputException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (MetaException &o2) { + } catch (InvalidObjectException &o2) { result.o2 = o2; result.__isset.o2 = true; - } catch (InvalidInputException &o3) { + } catch (MetaException &o3) { result.o3 = o3; result.__isset.o3 = true; - } catch (InvalidObjectException &o4) { - result.o4 = o4; - result.__isset.o4 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition_column_statistics"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.updateStreamingTempLocation"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partition_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("updateStreamingTempLocation", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -29579,64 +32776,64 @@ void ThriftHiveMetastoreProcessor::process_get_partition_column_statistics(int32 } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_column_statistics"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.updateStreamingTempLocation"); } - oprot->writeMessageBegin("get_partition_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("updateStreamingTempLocation", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition_column_statistics", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.updateStreamingTempLocation", bytes); } } -void ThriftHiveMetastoreProcessor::process_delete_partition_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_getNextChunkID(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.delete_partition_column_statistics", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.getNextChunkID", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.delete_partition_column_statistics"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.getNextChunkID"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.delete_partition_column_statistics"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.getNextChunkID"); } - ThriftHiveMetastore_delete_partition_column_statistics_args args; + ThriftHiveMetastore_getNextChunkID_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.delete_partition_column_statistics", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.getNextChunkID", bytes); } - ThriftHiveMetastore_delete_partition_column_statistics_result result; + ThriftHiveMetastore_getNextChunkID_result result; try { - result.success = iface_->delete_partition_column_statistics(args.db_name, args.tbl_name, args.part_name, args.col_name); + result.success = iface_->getNextChunkID(args.db_name, args.table_name); result.__isset.success = true; - } catch (NoSuchObjectException &o1) { + } catch (InvalidInputException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (MetaException &o2) { + } catch (InvalidObjectException &o2) { result.o2 = o2; result.__isset.o2 = true; - } catch (InvalidObjectException &o3) { + } catch (InvalidOperationException &o3) { result.o3 = o3; result.__isset.o3 = true; - } catch (InvalidInputException &o4) { + } catch (MetaException &o4) { result.o4 = o4; result.__isset.o4 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.delete_partition_column_statistics"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.getNextChunkID"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("delete_partition_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("getNextChunkID", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -29645,64 +32842,64 @@ void ThriftHiveMetastoreProcessor::process_delete_partition_column_statistics(in } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.delete_partition_column_statistics"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.getNextChunkID"); } - oprot->writeMessageBegin("delete_partition_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("getNextChunkID", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.delete_partition_column_statistics", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.getNextChunkID", bytes); } } -void ThriftHiveMetastoreProcessor::process_delete_table_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +void ThriftHiveMetastoreProcessor::process_getStreamingTmpDir(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { void* ctx = NULL; if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.delete_table_column_statistics", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.getStreamingTmpDir", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.delete_table_column_statistics"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.getStreamingTmpDir"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.delete_table_column_statistics"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.getStreamingTmpDir"); } - ThriftHiveMetastore_delete_table_column_statistics_args args; + ThriftHiveMetastore_getStreamingTmpDir_args args; args.read(iprot); iprot->readMessageEnd(); uint32_t bytes = iprot->getTransport()->readEnd(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.delete_table_column_statistics", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.getStreamingTmpDir", bytes); } - ThriftHiveMetastore_delete_table_column_statistics_result result; + ThriftHiveMetastore_getStreamingTmpDir_result result; try { - result.success = iface_->delete_table_column_statistics(args.db_name, args.tbl_name, args.col_name); + iface_->getStreamingTmpDir(result.success, args.dbName, args.tableName); result.__isset.success = true; - } catch (NoSuchObjectException &o1) { + } catch (InvalidInputException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (MetaException &o2) { + } catch (InvalidObjectException &o2) { result.o2 = o2; result.__isset.o2 = true; - } catch (InvalidObjectException &o3) { + } catch (InvalidOperationException &o3) { result.o3 = o3; result.__isset.o3 = true; - } catch (InvalidInputException &o4) { + } catch (MetaException &o4) { result.o4 = o4; result.__isset.o4 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.delete_table_column_statistics"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.getStreamingTmpDir"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("delete_table_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("getStreamingTmpDir", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -29711,17 +32908,17 @@ void ThriftHiveMetastoreProcessor::process_delete_table_column_statistics(int32_ } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.delete_table_column_statistics"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.getStreamingTmpDir"); } - oprot->writeMessageBegin("delete_table_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); + oprot->writeMessageBegin("getStreamingTmpDir", ::apache::thrift::protocol::T_REPLY, seqid); result.write(oprot); oprot->writeMessageEnd(); bytes = oprot->getTransport()->writeEnd(); oprot->getTransport()->flush(); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.delete_table_column_statistics", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.getStreamingTmpDir", bytes); } } diff --git metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h index 7b31d28..3a0f6e5 100644 --- metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h +++ metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h @@ -84,6 +84,14 @@ class ThriftHiveMetastoreIf : virtual public ::facebook::fb303::FacebookService virtual void get_partition_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) = 0; virtual bool delete_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) = 0; virtual bool delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name) = 0; + virtual void streamingStatus(std::map & _return, const std::string& db_name, const std::string& table_name) = 0; + virtual void enableStreaming(const std::string& db_name, const std::string& table_name, const std::string& streaming_tmp_dir, const std::string& partition_path, const std::string& partition_val) = 0; + virtual void disableStreaming(const std::string& db_name, const std::string& table_name) = 0; + virtual void getCurrentStreamingPartitionPath(std::string& _return, const std::string& dbName, const std::string& tableName) = 0; + virtual void rollStreamingPartition(std::string& _return, const std::string& dbName, const std::string& tableName, const std::string& newPartitionPath, const std::string& newPartitionVal) = 0; + virtual void updateStreamingTempLocation(const std::string& dbName, const std::string& tableName, const std::string& streamingTempDir) = 0; + virtual int64_t getNextChunkID(const std::string& db_name, const std::string& table_name) = 0; + virtual void getStreamingTmpDir(std::string& _return, const std::string& dbName, const std::string& tableName) = 0; virtual bool create_role(const Role& role) = 0; virtual bool drop_role(const std::string& role_name) = 0; virtual void get_role_names(std::vector & _return) = 0; @@ -345,6 +353,31 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p bool _return = false; return _return; } + void streamingStatus(std::map & /* _return */, const std::string& /* db_name */, const std::string& /* table_name */) { + return; + } + void enableStreaming(const std::string& /* db_name */, const std::string& /* table_name */, const std::string& /* streaming_tmp_dir */, const std::string& /* partition_path */, const std::string& /* partition_val */) { + return; + } + void disableStreaming(const std::string& /* db_name */, const std::string& /* table_name */) { + return; + } + void getCurrentStreamingPartitionPath(std::string& /* _return */, const std::string& /* dbName */, const std::string& /* tableName */) { + return; + } + void rollStreamingPartition(std::string& /* _return */, const std::string& /* dbName */, const std::string& /* tableName */, const std::string& /* newPartitionPath */, const std::string& /* newPartitionVal */) { + return; + } + void updateStreamingTempLocation(const std::string& /* dbName */, const std::string& /* tableName */, const std::string& /* streamingTempDir */) { + return; + } + int64_t getNextChunkID(const std::string& /* db_name */, const std::string& /* table_name */) { + int64_t _return = 0; + return _return; + } + void getStreamingTmpDir(std::string& /* _return */, const std::string& /* dbName */, const std::string& /* tableName */) { + return; + } bool create_role(const Role& /* role */) { bool _return = false; return _return; @@ -10343,38 +10376,46 @@ class ThriftHiveMetastore_delete_table_column_statistics_presult { }; -typedef struct _ThriftHiveMetastore_create_role_args__isset { - _ThriftHiveMetastore_create_role_args__isset() : role(false) {} - bool role; -} _ThriftHiveMetastore_create_role_args__isset; +typedef struct _ThriftHiveMetastore_streamingStatus_args__isset { + _ThriftHiveMetastore_streamingStatus_args__isset() : db_name(false), table_name(false) {} + bool db_name; + bool table_name; +} _ThriftHiveMetastore_streamingStatus_args__isset; -class ThriftHiveMetastore_create_role_args { +class ThriftHiveMetastore_streamingStatus_args { public: - ThriftHiveMetastore_create_role_args() { + ThriftHiveMetastore_streamingStatus_args() : db_name(), table_name() { } - virtual ~ThriftHiveMetastore_create_role_args() throw() {} + virtual ~ThriftHiveMetastore_streamingStatus_args() throw() {} - Role role; + std::string db_name; + std::string table_name; - _ThriftHiveMetastore_create_role_args__isset __isset; + _ThriftHiveMetastore_streamingStatus_args__isset __isset; - void __set_role(const Role& val) { - role = val; + void __set_db_name(const std::string& val) { + db_name = val; } - bool operator == (const ThriftHiveMetastore_create_role_args & rhs) const + void __set_table_name(const std::string& val) { + table_name = val; + } + + bool operator == (const ThriftHiveMetastore_streamingStatus_args & rhs) const { - if (!(role == rhs.role)) + if (!(db_name == rhs.db_name)) + return false; + if (!(table_name == rhs.table_name)) return false; return true; } - bool operator != (const ThriftHiveMetastore_create_role_args &rhs) const { + bool operator != (const ThriftHiveMetastore_streamingStatus_args &rhs) const { return !(*this == rhs); } - bool operator < (const ThriftHiveMetastore_create_role_args & ) const; + bool operator < (const ThriftHiveMetastore_streamingStatus_args & ) const; uint32_t read(::apache::thrift::protocol::TProtocol* iprot); uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; @@ -10382,117 +10423,170 @@ class ThriftHiveMetastore_create_role_args { }; -class ThriftHiveMetastore_create_role_pargs { +class ThriftHiveMetastore_streamingStatus_pargs { public: - virtual ~ThriftHiveMetastore_create_role_pargs() throw() {} + virtual ~ThriftHiveMetastore_streamingStatus_pargs() throw() {} - const Role* role; + const std::string* db_name; + const std::string* table_name; uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; }; -typedef struct _ThriftHiveMetastore_create_role_result__isset { - _ThriftHiveMetastore_create_role_result__isset() : success(false), o1(false) {} +typedef struct _ThriftHiveMetastore_streamingStatus_result__isset { + _ThriftHiveMetastore_streamingStatus_result__isset() : success(false), o1(false), o2(false), o3(false) {} bool success; bool o1; -} _ThriftHiveMetastore_create_role_result__isset; + bool o2; + bool o3; +} _ThriftHiveMetastore_streamingStatus_result__isset; -class ThriftHiveMetastore_create_role_result { +class ThriftHiveMetastore_streamingStatus_result { public: - ThriftHiveMetastore_create_role_result() : success(0) { + ThriftHiveMetastore_streamingStatus_result() { } - virtual ~ThriftHiveMetastore_create_role_result() throw() {} + virtual ~ThriftHiveMetastore_streamingStatus_result() throw() {} - bool success; - MetaException o1; + std::map success; + InvalidInputException o1; + InvalidObjectException o2; + MetaException o3; - _ThriftHiveMetastore_create_role_result__isset __isset; + _ThriftHiveMetastore_streamingStatus_result__isset __isset; - void __set_success(const bool val) { + void __set_success(const std::map & val) { success = val; } - void __set_o1(const MetaException& val) { + void __set_o1(const InvalidInputException& val) { o1 = val; } - bool operator == (const ThriftHiveMetastore_create_role_result & rhs) const + void __set_o2(const InvalidObjectException& val) { + o2 = val; + } + + void __set_o3(const MetaException& val) { + o3 = val; + } + + bool operator == (const ThriftHiveMetastore_streamingStatus_result & rhs) const { if (!(success == rhs.success)) return false; if (!(o1 == rhs.o1)) return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; return true; } - bool operator != (const ThriftHiveMetastore_create_role_result &rhs) const { + bool operator != (const ThriftHiveMetastore_streamingStatus_result &rhs) const { return !(*this == rhs); } - bool operator < (const ThriftHiveMetastore_create_role_result & ) const; + bool operator < (const ThriftHiveMetastore_streamingStatus_result & ) const; uint32_t read(::apache::thrift::protocol::TProtocol* iprot); uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; }; -typedef struct _ThriftHiveMetastore_create_role_presult__isset { - _ThriftHiveMetastore_create_role_presult__isset() : success(false), o1(false) {} +typedef struct _ThriftHiveMetastore_streamingStatus_presult__isset { + _ThriftHiveMetastore_streamingStatus_presult__isset() : success(false), o1(false), o2(false), o3(false) {} bool success; bool o1; -} _ThriftHiveMetastore_create_role_presult__isset; + bool o2; + bool o3; +} _ThriftHiveMetastore_streamingStatus_presult__isset; -class ThriftHiveMetastore_create_role_presult { +class ThriftHiveMetastore_streamingStatus_presult { public: - virtual ~ThriftHiveMetastore_create_role_presult() throw() {} + virtual ~ThriftHiveMetastore_streamingStatus_presult() throw() {} - bool* success; - MetaException o1; + std::map * success; + InvalidInputException o1; + InvalidObjectException o2; + MetaException o3; - _ThriftHiveMetastore_create_role_presult__isset __isset; + _ThriftHiveMetastore_streamingStatus_presult__isset __isset; uint32_t read(::apache::thrift::protocol::TProtocol* iprot); }; -typedef struct _ThriftHiveMetastore_drop_role_args__isset { - _ThriftHiveMetastore_drop_role_args__isset() : role_name(false) {} - bool role_name; -} _ThriftHiveMetastore_drop_role_args__isset; +typedef struct _ThriftHiveMetastore_enableStreaming_args__isset { + _ThriftHiveMetastore_enableStreaming_args__isset() : db_name(false), table_name(false), streaming_tmp_dir(false), partition_path(false), partition_val(false) {} + bool db_name; + bool table_name; + bool streaming_tmp_dir; + bool partition_path; + bool partition_val; +} _ThriftHiveMetastore_enableStreaming_args__isset; -class ThriftHiveMetastore_drop_role_args { +class ThriftHiveMetastore_enableStreaming_args { public: - ThriftHiveMetastore_drop_role_args() : role_name() { + ThriftHiveMetastore_enableStreaming_args() : db_name(), table_name(), streaming_tmp_dir(), partition_path(), partition_val() { } - virtual ~ThriftHiveMetastore_drop_role_args() throw() {} + virtual ~ThriftHiveMetastore_enableStreaming_args() throw() {} - std::string role_name; + std::string db_name; + std::string table_name; + std::string streaming_tmp_dir; + std::string partition_path; + std::string partition_val; - _ThriftHiveMetastore_drop_role_args__isset __isset; + _ThriftHiveMetastore_enableStreaming_args__isset __isset; - void __set_role_name(const std::string& val) { - role_name = val; + void __set_db_name(const std::string& val) { + db_name = val; } - bool operator == (const ThriftHiveMetastore_drop_role_args & rhs) const + void __set_table_name(const std::string& val) { + table_name = val; + } + + void __set_streaming_tmp_dir(const std::string& val) { + streaming_tmp_dir = val; + } + + void __set_partition_path(const std::string& val) { + partition_path = val; + } + + void __set_partition_val(const std::string& val) { + partition_val = val; + } + + bool operator == (const ThriftHiveMetastore_enableStreaming_args & rhs) const { - if (!(role_name == rhs.role_name)) + if (!(db_name == rhs.db_name)) + return false; + if (!(table_name == rhs.table_name)) + return false; + if (!(streaming_tmp_dir == rhs.streaming_tmp_dir)) + return false; + if (!(partition_path == rhs.partition_path)) + return false; + if (!(partition_val == rhs.partition_val)) return false; return true; } - bool operator != (const ThriftHiveMetastore_drop_role_args &rhs) const { + bool operator != (const ThriftHiveMetastore_enableStreaming_args &rhs) const { return !(*this == rhs); } - bool operator < (const ThriftHiveMetastore_drop_role_args & ) const; + bool operator < (const ThriftHiveMetastore_enableStreaming_args & ) const; uint32_t read(::apache::thrift::protocol::TProtocol* iprot); uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; @@ -10500,104 +10594,149 @@ class ThriftHiveMetastore_drop_role_args { }; -class ThriftHiveMetastore_drop_role_pargs { +class ThriftHiveMetastore_enableStreaming_pargs { public: - virtual ~ThriftHiveMetastore_drop_role_pargs() throw() {} + virtual ~ThriftHiveMetastore_enableStreaming_pargs() throw() {} - const std::string* role_name; + const std::string* db_name; + const std::string* table_name; + const std::string* streaming_tmp_dir; + const std::string* partition_path; + const std::string* partition_val; uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; }; -typedef struct _ThriftHiveMetastore_drop_role_result__isset { - _ThriftHiveMetastore_drop_role_result__isset() : success(false), o1(false) {} - bool success; +typedef struct _ThriftHiveMetastore_enableStreaming_result__isset { + _ThriftHiveMetastore_enableStreaming_result__isset() : o1(false), o2(false), o3(false), o4(false) {} bool o1; -} _ThriftHiveMetastore_drop_role_result__isset; + bool o2; + bool o3; + bool o4; +} _ThriftHiveMetastore_enableStreaming_result__isset; -class ThriftHiveMetastore_drop_role_result { +class ThriftHiveMetastore_enableStreaming_result { public: - ThriftHiveMetastore_drop_role_result() : success(0) { + ThriftHiveMetastore_enableStreaming_result() { } - virtual ~ThriftHiveMetastore_drop_role_result() throw() {} + virtual ~ThriftHiveMetastore_enableStreaming_result() throw() {} - bool success; - MetaException o1; + InvalidInputException o1; + InvalidObjectException o2; + InvalidOperationException o3; + MetaException o4; - _ThriftHiveMetastore_drop_role_result__isset __isset; + _ThriftHiveMetastore_enableStreaming_result__isset __isset; - void __set_success(const bool val) { - success = val; + void __set_o1(const InvalidInputException& val) { + o1 = val; } - void __set_o1(const MetaException& val) { - o1 = val; + void __set_o2(const InvalidObjectException& val) { + o2 = val; } - bool operator == (const ThriftHiveMetastore_drop_role_result & rhs) const + void __set_o3(const InvalidOperationException& val) { + o3 = val; + } + + void __set_o4(const MetaException& val) { + o4 = val; + } + + bool operator == (const ThriftHiveMetastore_enableStreaming_result & rhs) const { - if (!(success == rhs.success)) - return false; if (!(o1 == rhs.o1)) return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + if (!(o4 == rhs.o4)) + return false; return true; } - bool operator != (const ThriftHiveMetastore_drop_role_result &rhs) const { + bool operator != (const ThriftHiveMetastore_enableStreaming_result &rhs) const { return !(*this == rhs); } - bool operator < (const ThriftHiveMetastore_drop_role_result & ) const; + bool operator < (const ThriftHiveMetastore_enableStreaming_result & ) const; uint32_t read(::apache::thrift::protocol::TProtocol* iprot); uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; }; -typedef struct _ThriftHiveMetastore_drop_role_presult__isset { - _ThriftHiveMetastore_drop_role_presult__isset() : success(false), o1(false) {} - bool success; +typedef struct _ThriftHiveMetastore_enableStreaming_presult__isset { + _ThriftHiveMetastore_enableStreaming_presult__isset() : o1(false), o2(false), o3(false), o4(false) {} bool o1; -} _ThriftHiveMetastore_drop_role_presult__isset; + bool o2; + bool o3; + bool o4; +} _ThriftHiveMetastore_enableStreaming_presult__isset; -class ThriftHiveMetastore_drop_role_presult { +class ThriftHiveMetastore_enableStreaming_presult { public: - virtual ~ThriftHiveMetastore_drop_role_presult() throw() {} + virtual ~ThriftHiveMetastore_enableStreaming_presult() throw() {} - bool* success; - MetaException o1; + InvalidInputException o1; + InvalidObjectException o2; + InvalidOperationException o3; + MetaException o4; - _ThriftHiveMetastore_drop_role_presult__isset __isset; + _ThriftHiveMetastore_enableStreaming_presult__isset __isset; uint32_t read(::apache::thrift::protocol::TProtocol* iprot); }; +typedef struct _ThriftHiveMetastore_disableStreaming_args__isset { + _ThriftHiveMetastore_disableStreaming_args__isset() : db_name(false), table_name(false) {} + bool db_name; + bool table_name; +} _ThriftHiveMetastore_disableStreaming_args__isset; -class ThriftHiveMetastore_get_role_names_args { +class ThriftHiveMetastore_disableStreaming_args { public: - ThriftHiveMetastore_get_role_names_args() { + ThriftHiveMetastore_disableStreaming_args() : db_name(), table_name() { } - virtual ~ThriftHiveMetastore_get_role_names_args() throw() {} + virtual ~ThriftHiveMetastore_disableStreaming_args() throw() {} + std::string db_name; + std::string table_name; - bool operator == (const ThriftHiveMetastore_get_role_names_args & /* rhs */) const + _ThriftHiveMetastore_disableStreaming_args__isset __isset; + + void __set_db_name(const std::string& val) { + db_name = val; + } + + void __set_table_name(const std::string& val) { + table_name = val; + } + + bool operator == (const ThriftHiveMetastore_disableStreaming_args & rhs) const { + if (!(db_name == rhs.db_name)) + return false; + if (!(table_name == rhs.table_name)) + return false; return true; } - bool operator != (const ThriftHiveMetastore_get_role_names_args &rhs) const { + bool operator != (const ThriftHiveMetastore_disableStreaming_args &rhs) const { return !(*this == rhs); } - bool operator < (const ThriftHiveMetastore_get_role_names_args & ) const; + bool operator < (const ThriftHiveMetastore_disableStreaming_args & ) const; uint32_t read(::apache::thrift::protocol::TProtocol* iprot); uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; @@ -10605,66 +10744,1230 @@ class ThriftHiveMetastore_get_role_names_args { }; -class ThriftHiveMetastore_get_role_names_pargs { +class ThriftHiveMetastore_disableStreaming_pargs { public: - virtual ~ThriftHiveMetastore_get_role_names_pargs() throw() {} + virtual ~ThriftHiveMetastore_disableStreaming_pargs() throw() {} + const std::string* db_name; + const std::string* table_name; uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; }; -typedef struct _ThriftHiveMetastore_get_role_names_result__isset { - _ThriftHiveMetastore_get_role_names_result__isset() : success(false), o1(false) {} - bool success; +typedef struct _ThriftHiveMetastore_disableStreaming_result__isset { + _ThriftHiveMetastore_disableStreaming_result__isset() : o1(false), o2(false), o3(false), o4(false) {} bool o1; -} _ThriftHiveMetastore_get_role_names_result__isset; + bool o2; + bool o3; + bool o4; +} _ThriftHiveMetastore_disableStreaming_result__isset; -class ThriftHiveMetastore_get_role_names_result { +class ThriftHiveMetastore_disableStreaming_result { public: - ThriftHiveMetastore_get_role_names_result() { + ThriftHiveMetastore_disableStreaming_result() { } - virtual ~ThriftHiveMetastore_get_role_names_result() throw() {} + virtual ~ThriftHiveMetastore_disableStreaming_result() throw() {} - std::vector success; - MetaException o1; + InvalidInputException o1; + InvalidObjectException o2; + InvalidOperationException o3; + MetaException o4; - _ThriftHiveMetastore_get_role_names_result__isset __isset; + _ThriftHiveMetastore_disableStreaming_result__isset __isset; - void __set_success(const std::vector & val) { - success = val; + void __set_o1(const InvalidInputException& val) { + o1 = val; } - void __set_o1(const MetaException& val) { - o1 = val; + void __set_o2(const InvalidObjectException& val) { + o2 = val; } - bool operator == (const ThriftHiveMetastore_get_role_names_result & rhs) const + void __set_o3(const InvalidOperationException& val) { + o3 = val; + } + + void __set_o4(const MetaException& val) { + o4 = val; + } + + bool operator == (const ThriftHiveMetastore_disableStreaming_result & rhs) const { - if (!(success == rhs.success)) - return false; if (!(o1 == rhs.o1)) return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + if (!(o4 == rhs.o4)) + return false; return true; } - bool operator != (const ThriftHiveMetastore_get_role_names_result &rhs) const { + bool operator != (const ThriftHiveMetastore_disableStreaming_result &rhs) const { return !(*this == rhs); } - bool operator < (const ThriftHiveMetastore_get_role_names_result & ) const; + bool operator < (const ThriftHiveMetastore_disableStreaming_result & ) const; uint32_t read(::apache::thrift::protocol::TProtocol* iprot); uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; }; -typedef struct _ThriftHiveMetastore_get_role_names_presult__isset { - _ThriftHiveMetastore_get_role_names_presult__isset() : success(false), o1(false) {} - bool success; +typedef struct _ThriftHiveMetastore_disableStreaming_presult__isset { + _ThriftHiveMetastore_disableStreaming_presult__isset() : o1(false), o2(false), o3(false), o4(false) {} + bool o1; + bool o2; + bool o3; + bool o4; +} _ThriftHiveMetastore_disableStreaming_presult__isset; + +class ThriftHiveMetastore_disableStreaming_presult { + public: + + + virtual ~ThriftHiveMetastore_disableStreaming_presult() throw() {} + + InvalidInputException o1; + InvalidObjectException o2; + InvalidOperationException o3; + MetaException o4; + + _ThriftHiveMetastore_disableStreaming_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_getCurrentStreamingPartitionPath_args__isset { + _ThriftHiveMetastore_getCurrentStreamingPartitionPath_args__isset() : dbName(false), tableName(false) {} + bool dbName; + bool tableName; +} _ThriftHiveMetastore_getCurrentStreamingPartitionPath_args__isset; + +class ThriftHiveMetastore_getCurrentStreamingPartitionPath_args { + public: + + ThriftHiveMetastore_getCurrentStreamingPartitionPath_args() : dbName(), tableName() { + } + + virtual ~ThriftHiveMetastore_getCurrentStreamingPartitionPath_args() throw() {} + + std::string dbName; + std::string tableName; + + _ThriftHiveMetastore_getCurrentStreamingPartitionPath_args__isset __isset; + + void __set_dbName(const std::string& val) { + dbName = val; + } + + void __set_tableName(const std::string& val) { + tableName = val; + } + + bool operator == (const ThriftHiveMetastore_getCurrentStreamingPartitionPath_args & rhs) const + { + if (!(dbName == rhs.dbName)) + return false; + if (!(tableName == rhs.tableName)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_getCurrentStreamingPartitionPath_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_getCurrentStreamingPartitionPath_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_getCurrentStreamingPartitionPath_pargs { + public: + + + virtual ~ThriftHiveMetastore_getCurrentStreamingPartitionPath_pargs() throw() {} + + const std::string* dbName; + const std::string* tableName; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_getCurrentStreamingPartitionPath_result__isset { + _ThriftHiveMetastore_getCurrentStreamingPartitionPath_result__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} + bool success; + bool o1; + bool o2; + bool o3; + bool o4; +} _ThriftHiveMetastore_getCurrentStreamingPartitionPath_result__isset; + +class ThriftHiveMetastore_getCurrentStreamingPartitionPath_result { + public: + + ThriftHiveMetastore_getCurrentStreamingPartitionPath_result() : success() { + } + + virtual ~ThriftHiveMetastore_getCurrentStreamingPartitionPath_result() throw() {} + + std::string success; + InvalidInputException o1; + InvalidObjectException o2; + InvalidOperationException o3; + MetaException o4; + + _ThriftHiveMetastore_getCurrentStreamingPartitionPath_result__isset __isset; + + void __set_success(const std::string& val) { + success = val; + } + + void __set_o1(const InvalidInputException& val) { + o1 = val; + } + + void __set_o2(const InvalidObjectException& val) { + o2 = val; + } + + void __set_o3(const InvalidOperationException& val) { + o3 = val; + } + + void __set_o4(const MetaException& val) { + o4 = val; + } + + bool operator == (const ThriftHiveMetastore_getCurrentStreamingPartitionPath_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + if (!(o4 == rhs.o4)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_getCurrentStreamingPartitionPath_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_getCurrentStreamingPartitionPath_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_getCurrentStreamingPartitionPath_presult__isset { + _ThriftHiveMetastore_getCurrentStreamingPartitionPath_presult__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} + bool success; + bool o1; + bool o2; + bool o3; + bool o4; +} _ThriftHiveMetastore_getCurrentStreamingPartitionPath_presult__isset; + +class ThriftHiveMetastore_getCurrentStreamingPartitionPath_presult { + public: + + + virtual ~ThriftHiveMetastore_getCurrentStreamingPartitionPath_presult() throw() {} + + std::string* success; + InvalidInputException o1; + InvalidObjectException o2; + InvalidOperationException o3; + MetaException o4; + + _ThriftHiveMetastore_getCurrentStreamingPartitionPath_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_rollStreamingPartition_args__isset { + _ThriftHiveMetastore_rollStreamingPartition_args__isset() : dbName(false), tableName(false), newPartitionPath(false), newPartitionVal(false) {} + bool dbName; + bool tableName; + bool newPartitionPath; + bool newPartitionVal; +} _ThriftHiveMetastore_rollStreamingPartition_args__isset; + +class ThriftHiveMetastore_rollStreamingPartition_args { + public: + + ThriftHiveMetastore_rollStreamingPartition_args() : dbName(), tableName(), newPartitionPath(), newPartitionVal() { + } + + virtual ~ThriftHiveMetastore_rollStreamingPartition_args() throw() {} + + std::string dbName; + std::string tableName; + std::string newPartitionPath; + std::string newPartitionVal; + + _ThriftHiveMetastore_rollStreamingPartition_args__isset __isset; + + void __set_dbName(const std::string& val) { + dbName = val; + } + + void __set_tableName(const std::string& val) { + tableName = val; + } + + void __set_newPartitionPath(const std::string& val) { + newPartitionPath = val; + } + + void __set_newPartitionVal(const std::string& val) { + newPartitionVal = val; + } + + bool operator == (const ThriftHiveMetastore_rollStreamingPartition_args & rhs) const + { + if (!(dbName == rhs.dbName)) + return false; + if (!(tableName == rhs.tableName)) + return false; + if (!(newPartitionPath == rhs.newPartitionPath)) + return false; + if (!(newPartitionVal == rhs.newPartitionVal)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_rollStreamingPartition_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_rollStreamingPartition_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_rollStreamingPartition_pargs { + public: + + + virtual ~ThriftHiveMetastore_rollStreamingPartition_pargs() throw() {} + + const std::string* dbName; + const std::string* tableName; + const std::string* newPartitionPath; + const std::string* newPartitionVal; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_rollStreamingPartition_result__isset { + _ThriftHiveMetastore_rollStreamingPartition_result__isset() : success(false), o1(false), o2(false), o3(false), o4(false), o5(false) {} + bool success; + bool o1; + bool o2; + bool o3; + bool o4; + bool o5; +} _ThriftHiveMetastore_rollStreamingPartition_result__isset; + +class ThriftHiveMetastore_rollStreamingPartition_result { + public: + + ThriftHiveMetastore_rollStreamingPartition_result() : success() { + } + + virtual ~ThriftHiveMetastore_rollStreamingPartition_result() throw() {} + + std::string success; + InvalidInputException o1; + InvalidObjectException o2; + InvalidOperationException o3; + AlreadyExistsException o4; + MetaException o5; + + _ThriftHiveMetastore_rollStreamingPartition_result__isset __isset; + + void __set_success(const std::string& val) { + success = val; + } + + void __set_o1(const InvalidInputException& val) { + o1 = val; + } + + void __set_o2(const InvalidObjectException& val) { + o2 = val; + } + + void __set_o3(const InvalidOperationException& val) { + o3 = val; + } + + void __set_o4(const AlreadyExistsException& val) { + o4 = val; + } + + void __set_o5(const MetaException& val) { + o5 = val; + } + + bool operator == (const ThriftHiveMetastore_rollStreamingPartition_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + if (!(o4 == rhs.o4)) + return false; + if (!(o5 == rhs.o5)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_rollStreamingPartition_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_rollStreamingPartition_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_rollStreamingPartition_presult__isset { + _ThriftHiveMetastore_rollStreamingPartition_presult__isset() : success(false), o1(false), o2(false), o3(false), o4(false), o5(false) {} + bool success; + bool o1; + bool o2; + bool o3; + bool o4; + bool o5; +} _ThriftHiveMetastore_rollStreamingPartition_presult__isset; + +class ThriftHiveMetastore_rollStreamingPartition_presult { + public: + + + virtual ~ThriftHiveMetastore_rollStreamingPartition_presult() throw() {} + + std::string* success; + InvalidInputException o1; + InvalidObjectException o2; + InvalidOperationException o3; + AlreadyExistsException o4; + MetaException o5; + + _ThriftHiveMetastore_rollStreamingPartition_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_updateStreamingTempLocation_args__isset { + _ThriftHiveMetastore_updateStreamingTempLocation_args__isset() : dbName(false), tableName(false), streamingTempDir(false) {} + bool dbName; + bool tableName; + bool streamingTempDir; +} _ThriftHiveMetastore_updateStreamingTempLocation_args__isset; + +class ThriftHiveMetastore_updateStreamingTempLocation_args { + public: + + ThriftHiveMetastore_updateStreamingTempLocation_args() : dbName(), tableName(), streamingTempDir() { + } + + virtual ~ThriftHiveMetastore_updateStreamingTempLocation_args() throw() {} + + std::string dbName; + std::string tableName; + std::string streamingTempDir; + + _ThriftHiveMetastore_updateStreamingTempLocation_args__isset __isset; + + void __set_dbName(const std::string& val) { + dbName = val; + } + + void __set_tableName(const std::string& val) { + tableName = val; + } + + void __set_streamingTempDir(const std::string& val) { + streamingTempDir = val; + } + + bool operator == (const ThriftHiveMetastore_updateStreamingTempLocation_args & rhs) const + { + if (!(dbName == rhs.dbName)) + return false; + if (!(tableName == rhs.tableName)) + return false; + if (!(streamingTempDir == rhs.streamingTempDir)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_updateStreamingTempLocation_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_updateStreamingTempLocation_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_updateStreamingTempLocation_pargs { + public: + + + virtual ~ThriftHiveMetastore_updateStreamingTempLocation_pargs() throw() {} + + const std::string* dbName; + const std::string* tableName; + const std::string* streamingTempDir; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_updateStreamingTempLocation_result__isset { + _ThriftHiveMetastore_updateStreamingTempLocation_result__isset() : o1(false), o2(false), o3(false) {} + bool o1; + bool o2; + bool o3; +} _ThriftHiveMetastore_updateStreamingTempLocation_result__isset; + +class ThriftHiveMetastore_updateStreamingTempLocation_result { + public: + + ThriftHiveMetastore_updateStreamingTempLocation_result() { + } + + virtual ~ThriftHiveMetastore_updateStreamingTempLocation_result() throw() {} + + InvalidInputException o1; + InvalidObjectException o2; + MetaException o3; + + _ThriftHiveMetastore_updateStreamingTempLocation_result__isset __isset; + + void __set_o1(const InvalidInputException& val) { + o1 = val; + } + + void __set_o2(const InvalidObjectException& val) { + o2 = val; + } + + void __set_o3(const MetaException& val) { + o3 = val; + } + + bool operator == (const ThriftHiveMetastore_updateStreamingTempLocation_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_updateStreamingTempLocation_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_updateStreamingTempLocation_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_updateStreamingTempLocation_presult__isset { + _ThriftHiveMetastore_updateStreamingTempLocation_presult__isset() : o1(false), o2(false), o3(false) {} + bool o1; + bool o2; + bool o3; +} _ThriftHiveMetastore_updateStreamingTempLocation_presult__isset; + +class ThriftHiveMetastore_updateStreamingTempLocation_presult { + public: + + + virtual ~ThriftHiveMetastore_updateStreamingTempLocation_presult() throw() {} + + InvalidInputException o1; + InvalidObjectException o2; + MetaException o3; + + _ThriftHiveMetastore_updateStreamingTempLocation_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_getNextChunkID_args__isset { + _ThriftHiveMetastore_getNextChunkID_args__isset() : db_name(false), table_name(false) {} + bool db_name; + bool table_name; +} _ThriftHiveMetastore_getNextChunkID_args__isset; + +class ThriftHiveMetastore_getNextChunkID_args { + public: + + ThriftHiveMetastore_getNextChunkID_args() : db_name(), table_name() { + } + + virtual ~ThriftHiveMetastore_getNextChunkID_args() throw() {} + + std::string db_name; + std::string table_name; + + _ThriftHiveMetastore_getNextChunkID_args__isset __isset; + + void __set_db_name(const std::string& val) { + db_name = val; + } + + void __set_table_name(const std::string& val) { + table_name = val; + } + + bool operator == (const ThriftHiveMetastore_getNextChunkID_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(table_name == rhs.table_name)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_getNextChunkID_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_getNextChunkID_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_getNextChunkID_pargs { + public: + + + virtual ~ThriftHiveMetastore_getNextChunkID_pargs() throw() {} + + const std::string* db_name; + const std::string* table_name; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_getNextChunkID_result__isset { + _ThriftHiveMetastore_getNextChunkID_result__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} + bool success; + bool o1; + bool o2; + bool o3; + bool o4; +} _ThriftHiveMetastore_getNextChunkID_result__isset; + +class ThriftHiveMetastore_getNextChunkID_result { + public: + + ThriftHiveMetastore_getNextChunkID_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_getNextChunkID_result() throw() {} + + int64_t success; + InvalidInputException o1; + InvalidObjectException o2; + InvalidOperationException o3; + MetaException o4; + + _ThriftHiveMetastore_getNextChunkID_result__isset __isset; + + void __set_success(const int64_t val) { + success = val; + } + + void __set_o1(const InvalidInputException& val) { + o1 = val; + } + + void __set_o2(const InvalidObjectException& val) { + o2 = val; + } + + void __set_o3(const InvalidOperationException& val) { + o3 = val; + } + + void __set_o4(const MetaException& val) { + o4 = val; + } + + bool operator == (const ThriftHiveMetastore_getNextChunkID_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + if (!(o4 == rhs.o4)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_getNextChunkID_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_getNextChunkID_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_getNextChunkID_presult__isset { + _ThriftHiveMetastore_getNextChunkID_presult__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} + bool success; + bool o1; + bool o2; + bool o3; + bool o4; +} _ThriftHiveMetastore_getNextChunkID_presult__isset; + +class ThriftHiveMetastore_getNextChunkID_presult { + public: + + + virtual ~ThriftHiveMetastore_getNextChunkID_presult() throw() {} + + int64_t* success; + InvalidInputException o1; + InvalidObjectException o2; + InvalidOperationException o3; + MetaException o4; + + _ThriftHiveMetastore_getNextChunkID_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_getStreamingTmpDir_args__isset { + _ThriftHiveMetastore_getStreamingTmpDir_args__isset() : dbName(false), tableName(false) {} + bool dbName; + bool tableName; +} _ThriftHiveMetastore_getStreamingTmpDir_args__isset; + +class ThriftHiveMetastore_getStreamingTmpDir_args { + public: + + ThriftHiveMetastore_getStreamingTmpDir_args() : dbName(), tableName() { + } + + virtual ~ThriftHiveMetastore_getStreamingTmpDir_args() throw() {} + + std::string dbName; + std::string tableName; + + _ThriftHiveMetastore_getStreamingTmpDir_args__isset __isset; + + void __set_dbName(const std::string& val) { + dbName = val; + } + + void __set_tableName(const std::string& val) { + tableName = val; + } + + bool operator == (const ThriftHiveMetastore_getStreamingTmpDir_args & rhs) const + { + if (!(dbName == rhs.dbName)) + return false; + if (!(tableName == rhs.tableName)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_getStreamingTmpDir_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_getStreamingTmpDir_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_getStreamingTmpDir_pargs { + public: + + + virtual ~ThriftHiveMetastore_getStreamingTmpDir_pargs() throw() {} + + const std::string* dbName; + const std::string* tableName; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_getStreamingTmpDir_result__isset { + _ThriftHiveMetastore_getStreamingTmpDir_result__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} + bool success; + bool o1; + bool o2; + bool o3; + bool o4; +} _ThriftHiveMetastore_getStreamingTmpDir_result__isset; + +class ThriftHiveMetastore_getStreamingTmpDir_result { + public: + + ThriftHiveMetastore_getStreamingTmpDir_result() : success() { + } + + virtual ~ThriftHiveMetastore_getStreamingTmpDir_result() throw() {} + + std::string success; + InvalidInputException o1; + InvalidObjectException o2; + InvalidOperationException o3; + MetaException o4; + + _ThriftHiveMetastore_getStreamingTmpDir_result__isset __isset; + + void __set_success(const std::string& val) { + success = val; + } + + void __set_o1(const InvalidInputException& val) { + o1 = val; + } + + void __set_o2(const InvalidObjectException& val) { + o2 = val; + } + + void __set_o3(const InvalidOperationException& val) { + o3 = val; + } + + void __set_o4(const MetaException& val) { + o4 = val; + } + + bool operator == (const ThriftHiveMetastore_getStreamingTmpDir_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + if (!(o4 == rhs.o4)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_getStreamingTmpDir_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_getStreamingTmpDir_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_getStreamingTmpDir_presult__isset { + _ThriftHiveMetastore_getStreamingTmpDir_presult__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} + bool success; + bool o1; + bool o2; + bool o3; + bool o4; +} _ThriftHiveMetastore_getStreamingTmpDir_presult__isset; + +class ThriftHiveMetastore_getStreamingTmpDir_presult { + public: + + + virtual ~ThriftHiveMetastore_getStreamingTmpDir_presult() throw() {} + + std::string* success; + InvalidInputException o1; + InvalidObjectException o2; + InvalidOperationException o3; + MetaException o4; + + _ThriftHiveMetastore_getStreamingTmpDir_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_create_role_args__isset { + _ThriftHiveMetastore_create_role_args__isset() : role(false) {} + bool role; +} _ThriftHiveMetastore_create_role_args__isset; + +class ThriftHiveMetastore_create_role_args { + public: + + ThriftHiveMetastore_create_role_args() { + } + + virtual ~ThriftHiveMetastore_create_role_args() throw() {} + + Role role; + + _ThriftHiveMetastore_create_role_args__isset __isset; + + void __set_role(const Role& val) { + role = val; + } + + bool operator == (const ThriftHiveMetastore_create_role_args & rhs) const + { + if (!(role == rhs.role)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_create_role_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_create_role_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_create_role_pargs { + public: + + + virtual ~ThriftHiveMetastore_create_role_pargs() throw() {} + + const Role* role; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_create_role_result__isset { + _ThriftHiveMetastore_create_role_result__isset() : success(false), o1(false) {} + bool success; + bool o1; +} _ThriftHiveMetastore_create_role_result__isset; + +class ThriftHiveMetastore_create_role_result { + public: + + ThriftHiveMetastore_create_role_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_create_role_result() throw() {} + + bool success; + MetaException o1; + + _ThriftHiveMetastore_create_role_result__isset __isset; + + void __set_success(const bool val) { + success = val; + } + + void __set_o1(const MetaException& val) { + o1 = val; + } + + bool operator == (const ThriftHiveMetastore_create_role_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_create_role_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_create_role_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_create_role_presult__isset { + _ThriftHiveMetastore_create_role_presult__isset() : success(false), o1(false) {} + bool success; + bool o1; +} _ThriftHiveMetastore_create_role_presult__isset; + +class ThriftHiveMetastore_create_role_presult { + public: + + + virtual ~ThriftHiveMetastore_create_role_presult() throw() {} + + bool* success; + MetaException o1; + + _ThriftHiveMetastore_create_role_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_drop_role_args__isset { + _ThriftHiveMetastore_drop_role_args__isset() : role_name(false) {} + bool role_name; +} _ThriftHiveMetastore_drop_role_args__isset; + +class ThriftHiveMetastore_drop_role_args { + public: + + ThriftHiveMetastore_drop_role_args() : role_name() { + } + + virtual ~ThriftHiveMetastore_drop_role_args() throw() {} + + std::string role_name; + + _ThriftHiveMetastore_drop_role_args__isset __isset; + + void __set_role_name(const std::string& val) { + role_name = val; + } + + bool operator == (const ThriftHiveMetastore_drop_role_args & rhs) const + { + if (!(role_name == rhs.role_name)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_role_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_role_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_drop_role_pargs { + public: + + + virtual ~ThriftHiveMetastore_drop_role_pargs() throw() {} + + const std::string* role_name; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_role_result__isset { + _ThriftHiveMetastore_drop_role_result__isset() : success(false), o1(false) {} + bool success; + bool o1; +} _ThriftHiveMetastore_drop_role_result__isset; + +class ThriftHiveMetastore_drop_role_result { + public: + + ThriftHiveMetastore_drop_role_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_drop_role_result() throw() {} + + bool success; + MetaException o1; + + _ThriftHiveMetastore_drop_role_result__isset __isset; + + void __set_success(const bool val) { + success = val; + } + + void __set_o1(const MetaException& val) { + o1 = val; + } + + bool operator == (const ThriftHiveMetastore_drop_role_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_role_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_role_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_role_presult__isset { + _ThriftHiveMetastore_drop_role_presult__isset() : success(false), o1(false) {} + bool success; + bool o1; +} _ThriftHiveMetastore_drop_role_presult__isset; + +class ThriftHiveMetastore_drop_role_presult { + public: + + + virtual ~ThriftHiveMetastore_drop_role_presult() throw() {} + + bool* success; + MetaException o1; + + _ThriftHiveMetastore_drop_role_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + + +class ThriftHiveMetastore_get_role_names_args { + public: + + ThriftHiveMetastore_get_role_names_args() { + } + + virtual ~ThriftHiveMetastore_get_role_names_args() throw() {} + + + bool operator == (const ThriftHiveMetastore_get_role_names_args & /* rhs */) const + { + return true; + } + bool operator != (const ThriftHiveMetastore_get_role_names_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_role_names_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_role_names_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_role_names_pargs() throw() {} + + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_role_names_result__isset { + _ThriftHiveMetastore_get_role_names_result__isset() : success(false), o1(false) {} + bool success; + bool o1; +} _ThriftHiveMetastore_get_role_names_result__isset; + +class ThriftHiveMetastore_get_role_names_result { + public: + + ThriftHiveMetastore_get_role_names_result() { + } + + virtual ~ThriftHiveMetastore_get_role_names_result() throw() {} + + std::vector success; + MetaException o1; + + _ThriftHiveMetastore_get_role_names_result__isset __isset; + + void __set_success(const std::vector & val) { + success = val; + } + + void __set_o1(const MetaException& val) { + o1 = val; + } + + bool operator == (const ThriftHiveMetastore_get_role_names_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_role_names_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_role_names_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_role_names_presult__isset { + _ThriftHiveMetastore_get_role_names_presult__isset() : success(false), o1(false) {} + bool success; bool o1; } _ThriftHiveMetastore_get_role_names_presult__isset; @@ -12313,6 +13616,30 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public bool delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name); void send_delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name); bool recv_delete_table_column_statistics(); + void streamingStatus(std::map & _return, const std::string& db_name, const std::string& table_name); + void send_streamingStatus(const std::string& db_name, const std::string& table_name); + void recv_streamingStatus(std::map & _return); + void enableStreaming(const std::string& db_name, const std::string& table_name, const std::string& streaming_tmp_dir, const std::string& partition_path, const std::string& partition_val); + void send_enableStreaming(const std::string& db_name, const std::string& table_name, const std::string& streaming_tmp_dir, const std::string& partition_path, const std::string& partition_val); + void recv_enableStreaming(); + void disableStreaming(const std::string& db_name, const std::string& table_name); + void send_disableStreaming(const std::string& db_name, const std::string& table_name); + void recv_disableStreaming(); + void getCurrentStreamingPartitionPath(std::string& _return, const std::string& dbName, const std::string& tableName); + void send_getCurrentStreamingPartitionPath(const std::string& dbName, const std::string& tableName); + void recv_getCurrentStreamingPartitionPath(std::string& _return); + void rollStreamingPartition(std::string& _return, const std::string& dbName, const std::string& tableName, const std::string& newPartitionPath, const std::string& newPartitionVal); + void send_rollStreamingPartition(const std::string& dbName, const std::string& tableName, const std::string& newPartitionPath, const std::string& newPartitionVal); + void recv_rollStreamingPartition(std::string& _return); + void updateStreamingTempLocation(const std::string& dbName, const std::string& tableName, const std::string& streamingTempDir); + void send_updateStreamingTempLocation(const std::string& dbName, const std::string& tableName, const std::string& streamingTempDir); + void recv_updateStreamingTempLocation(); + int64_t getNextChunkID(const std::string& db_name, const std::string& table_name); + void send_getNextChunkID(const std::string& db_name, const std::string& table_name); + int64_t recv_getNextChunkID(); + void getStreamingTmpDir(std::string& _return, const std::string& dbName, const std::string& tableName); + void send_getStreamingTmpDir(const std::string& dbName, const std::string& tableName); + void recv_getStreamingTmpDir(std::string& _return); bool create_role(const Role& role); void send_create_role(const Role& role); bool recv_create_role(); @@ -12433,6 +13760,14 @@ class ThriftHiveMetastoreProcessor : public ::facebook::fb303::FacebookServiceP void process_get_partition_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); void process_delete_partition_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); void process_delete_table_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_streamingStatus(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_enableStreaming(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_disableStreaming(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_getCurrentStreamingPartitionPath(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_rollStreamingPartition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_updateStreamingTempLocation(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_getNextChunkID(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_getStreamingTmpDir(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); void process_create_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); void process_drop_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); void process_get_role_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); @@ -12519,6 +13854,14 @@ class ThriftHiveMetastoreProcessor : public ::facebook::fb303::FacebookServiceP processMap_["get_partition_column_statistics"] = &ThriftHiveMetastoreProcessor::process_get_partition_column_statistics; processMap_["delete_partition_column_statistics"] = &ThriftHiveMetastoreProcessor::process_delete_partition_column_statistics; processMap_["delete_table_column_statistics"] = &ThriftHiveMetastoreProcessor::process_delete_table_column_statistics; + processMap_["streamingStatus"] = &ThriftHiveMetastoreProcessor::process_streamingStatus; + processMap_["enableStreaming"] = &ThriftHiveMetastoreProcessor::process_enableStreaming; + processMap_["disableStreaming"] = &ThriftHiveMetastoreProcessor::process_disableStreaming; + processMap_["getCurrentStreamingPartitionPath"] = &ThriftHiveMetastoreProcessor::process_getCurrentStreamingPartitionPath; + processMap_["rollStreamingPartition"] = &ThriftHiveMetastoreProcessor::process_rollStreamingPartition; + processMap_["updateStreamingTempLocation"] = &ThriftHiveMetastoreProcessor::process_updateStreamingTempLocation; + processMap_["getNextChunkID"] = &ThriftHiveMetastoreProcessor::process_getNextChunkID; + processMap_["getStreamingTmpDir"] = &ThriftHiveMetastoreProcessor::process_getStreamingTmpDir; processMap_["create_role"] = &ThriftHiveMetastoreProcessor::process_create_role; processMap_["drop_role"] = &ThriftHiveMetastoreProcessor::process_drop_role; processMap_["get_role_names"] = &ThriftHiveMetastoreProcessor::process_get_role_names; @@ -13217,6 +14560,82 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi return ifaces_[i]->delete_table_column_statistics(db_name, tbl_name, col_name); } + void streamingStatus(std::map & _return, const std::string& db_name, const std::string& table_name) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->streamingStatus(_return, db_name, table_name); + } + ifaces_[i]->streamingStatus(_return, db_name, table_name); + return; + } + + void enableStreaming(const std::string& db_name, const std::string& table_name, const std::string& streaming_tmp_dir, const std::string& partition_path, const std::string& partition_val) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->enableStreaming(db_name, table_name, streaming_tmp_dir, partition_path, partition_val); + } + ifaces_[i]->enableStreaming(db_name, table_name, streaming_tmp_dir, partition_path, partition_val); + } + + void disableStreaming(const std::string& db_name, const std::string& table_name) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->disableStreaming(db_name, table_name); + } + ifaces_[i]->disableStreaming(db_name, table_name); + } + + void getCurrentStreamingPartitionPath(std::string& _return, const std::string& dbName, const std::string& tableName) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->getCurrentStreamingPartitionPath(_return, dbName, tableName); + } + ifaces_[i]->getCurrentStreamingPartitionPath(_return, dbName, tableName); + return; + } + + void rollStreamingPartition(std::string& _return, const std::string& dbName, const std::string& tableName, const std::string& newPartitionPath, const std::string& newPartitionVal) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->rollStreamingPartition(_return, dbName, tableName, newPartitionPath, newPartitionVal); + } + ifaces_[i]->rollStreamingPartition(_return, dbName, tableName, newPartitionPath, newPartitionVal); + return; + } + + void updateStreamingTempLocation(const std::string& dbName, const std::string& tableName, const std::string& streamingTempDir) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->updateStreamingTempLocation(dbName, tableName, streamingTempDir); + } + ifaces_[i]->updateStreamingTempLocation(dbName, tableName, streamingTempDir); + } + + int64_t getNextChunkID(const std::string& db_name, const std::string& table_name) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->getNextChunkID(db_name, table_name); + } + return ifaces_[i]->getNextChunkID(db_name, table_name); + } + + void getStreamingTmpDir(std::string& _return, const std::string& dbName, const std::string& tableName) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->getStreamingTmpDir(_return, dbName, tableName); + } + ifaces_[i]->getStreamingTmpDir(_return, dbName, tableName); + return; + } + bool create_role(const Role& role) { size_t sz = ifaces_.size(); size_t i = 0; diff --git metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp index 3b90b44..772fe68 100644 --- metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp +++ metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp @@ -362,6 +362,46 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf { printf("delete_table_column_statistics\n"); } + void streamingStatus(std::map & _return, const std::string& db_name, const std::string& table_name) { + // Your implementation goes here + printf("streamingStatus\n"); + } + + void enableStreaming(const std::string& db_name, const std::string& table_name, const std::string& streaming_tmp_dir, const std::string& partition_path, const std::string& partition_val) { + // Your implementation goes here + printf("enableStreaming\n"); + } + + void disableStreaming(const std::string& db_name, const std::string& table_name) { + // Your implementation goes here + printf("disableStreaming\n"); + } + + void getCurrentStreamingPartitionPath(std::string& _return, const std::string& dbName, const std::string& tableName) { + // Your implementation goes here + printf("getCurrentStreamingPartitionPath\n"); + } + + void rollStreamingPartition(std::string& _return, const std::string& dbName, const std::string& tableName, const std::string& newPartitionPath, const std::string& newPartitionVal) { + // Your implementation goes here + printf("rollStreamingPartition\n"); + } + + void updateStreamingTempLocation(const std::string& dbName, const std::string& tableName, const std::string& streamingTempDir) { + // Your implementation goes here + printf("updateStreamingTempLocation\n"); + } + + int64_t getNextChunkID(const std::string& db_name, const std::string& table_name) { + // Your implementation goes here + printf("getNextChunkID\n"); + } + + void getStreamingTmpDir(std::string& _return, const std::string& dbName, const std::string& tableName) { + // Your implementation goes here + printf("getStreamingTmpDir\n"); + } + bool create_role(const Role& role) { // Your implementation goes here printf("create_role\n"); diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java index 4f34b18..240f758 100644 --- metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java @@ -174,6 +174,22 @@ public boolean delete_table_column_statistics(String db_name, String tbl_name, String col_name) throws NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException, org.apache.thrift.TException; + public Map streamingStatus(String db_name, String table_name) throws InvalidInputException, InvalidObjectException, MetaException, org.apache.thrift.TException; + + public void enableStreaming(String db_name, String table_name, String streaming_tmp_dir, String partition_path, String partition_val) throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException; + + public void disableStreaming(String db_name, String table_name) throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException; + + public String getCurrentStreamingPartitionPath(String dbName, String tableName) throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException; + + public String rollStreamingPartition(String dbName, String tableName, String newPartitionPath, String newPartitionVal) throws InvalidInputException, InvalidObjectException, InvalidOperationException, AlreadyExistsException, MetaException, org.apache.thrift.TException; + + public void updateStreamingTempLocation(String dbName, String tableName, String streamingTempDir) throws InvalidInputException, InvalidObjectException, MetaException, org.apache.thrift.TException; + + public long getNextChunkID(String db_name, String table_name) throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException; + + public String getStreamingTmpDir(String dbName, String tableName) throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException; + public boolean create_role(Role role) throws MetaException, org.apache.thrift.TException; public boolean drop_role(String role_name) throws MetaException, org.apache.thrift.TException; @@ -342,6 +358,22 @@ public void delete_table_column_statistics(String db_name, String tbl_name, String col_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void streamingStatus(String db_name, String table_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void enableStreaming(String db_name, String table_name, String streaming_tmp_dir, String partition_path, String partition_val, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void disableStreaming(String db_name, String table_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void getCurrentStreamingPartitionPath(String dbName, String tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void rollStreamingPartition(String dbName, String tableName, String newPartitionPath, String newPartitionVal, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void updateStreamingTempLocation(String dbName, String tableName, String streamingTempDir, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void getNextChunkID(String db_name, String table_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void getStreamingTmpDir(String dbName, String tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void create_role(Role role, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; public void drop_role(String role_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; @@ -2531,6 +2563,288 @@ public boolean recv_delete_table_column_statistics() throws NoSuchObjectExceptio throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "delete_table_column_statistics failed: unknown result"); } + public Map streamingStatus(String db_name, String table_name) throws InvalidInputException, InvalidObjectException, MetaException, org.apache.thrift.TException + { + send_streamingStatus(db_name, table_name); + return recv_streamingStatus(); + } + + public void send_streamingStatus(String db_name, String table_name) throws org.apache.thrift.TException + { + streamingStatus_args args = new streamingStatus_args(); + args.setDb_name(db_name); + args.setTable_name(table_name); + sendBase("streamingStatus", args); + } + + public Map recv_streamingStatus() throws InvalidInputException, InvalidObjectException, MetaException, org.apache.thrift.TException + { + streamingStatus_result result = new streamingStatus_result(); + receiveBase(result, "streamingStatus"); + if (result.isSetSuccess()) { + return result.success; + } + if (result.o1 != null) { + throw result.o1; + } + if (result.o2 != null) { + throw result.o2; + } + if (result.o3 != null) { + throw result.o3; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "streamingStatus failed: unknown result"); + } + + public void enableStreaming(String db_name, String table_name, String streaming_tmp_dir, String partition_path, String partition_val) throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException + { + send_enableStreaming(db_name, table_name, streaming_tmp_dir, partition_path, partition_val); + recv_enableStreaming(); + } + + public void send_enableStreaming(String db_name, String table_name, String streaming_tmp_dir, String partition_path, String partition_val) throws org.apache.thrift.TException + { + enableStreaming_args args = new enableStreaming_args(); + args.setDb_name(db_name); + args.setTable_name(table_name); + args.setStreaming_tmp_dir(streaming_tmp_dir); + args.setPartition_path(partition_path); + args.setPartition_val(partition_val); + sendBase("enableStreaming", args); + } + + public void recv_enableStreaming() throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException + { + enableStreaming_result result = new enableStreaming_result(); + receiveBase(result, "enableStreaming"); + if (result.o1 != null) { + throw result.o1; + } + if (result.o2 != null) { + throw result.o2; + } + if (result.o3 != null) { + throw result.o3; + } + if (result.o4 != null) { + throw result.o4; + } + return; + } + + public void disableStreaming(String db_name, String table_name) throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException + { + send_disableStreaming(db_name, table_name); + recv_disableStreaming(); + } + + public void send_disableStreaming(String db_name, String table_name) throws org.apache.thrift.TException + { + disableStreaming_args args = new disableStreaming_args(); + args.setDb_name(db_name); + args.setTable_name(table_name); + sendBase("disableStreaming", args); + } + + public void recv_disableStreaming() throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException + { + disableStreaming_result result = new disableStreaming_result(); + receiveBase(result, "disableStreaming"); + if (result.o1 != null) { + throw result.o1; + } + if (result.o2 != null) { + throw result.o2; + } + if (result.o3 != null) { + throw result.o3; + } + if (result.o4 != null) { + throw result.o4; + } + return; + } + + public String getCurrentStreamingPartitionPath(String dbName, String tableName) throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException + { + send_getCurrentStreamingPartitionPath(dbName, tableName); + return recv_getCurrentStreamingPartitionPath(); + } + + public void send_getCurrentStreamingPartitionPath(String dbName, String tableName) throws org.apache.thrift.TException + { + getCurrentStreamingPartitionPath_args args = new getCurrentStreamingPartitionPath_args(); + args.setDbName(dbName); + args.setTableName(tableName); + sendBase("getCurrentStreamingPartitionPath", args); + } + + public String recv_getCurrentStreamingPartitionPath() throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException + { + getCurrentStreamingPartitionPath_result result = new getCurrentStreamingPartitionPath_result(); + receiveBase(result, "getCurrentStreamingPartitionPath"); + if (result.isSetSuccess()) { + return result.success; + } + if (result.o1 != null) { + throw result.o1; + } + if (result.o2 != null) { + throw result.o2; + } + if (result.o3 != null) { + throw result.o3; + } + if (result.o4 != null) { + throw result.o4; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getCurrentStreamingPartitionPath failed: unknown result"); + } + + public String rollStreamingPartition(String dbName, String tableName, String newPartitionPath, String newPartitionVal) throws InvalidInputException, InvalidObjectException, InvalidOperationException, AlreadyExistsException, MetaException, org.apache.thrift.TException + { + send_rollStreamingPartition(dbName, tableName, newPartitionPath, newPartitionVal); + return recv_rollStreamingPartition(); + } + + public void send_rollStreamingPartition(String dbName, String tableName, String newPartitionPath, String newPartitionVal) throws org.apache.thrift.TException + { + rollStreamingPartition_args args = new rollStreamingPartition_args(); + args.setDbName(dbName); + args.setTableName(tableName); + args.setNewPartitionPath(newPartitionPath); + args.setNewPartitionVal(newPartitionVal); + sendBase("rollStreamingPartition", args); + } + + public String recv_rollStreamingPartition() throws InvalidInputException, InvalidObjectException, InvalidOperationException, AlreadyExistsException, MetaException, org.apache.thrift.TException + { + rollStreamingPartition_result result = new rollStreamingPartition_result(); + receiveBase(result, "rollStreamingPartition"); + if (result.isSetSuccess()) { + return result.success; + } + if (result.o1 != null) { + throw result.o1; + } + if (result.o2 != null) { + throw result.o2; + } + if (result.o3 != null) { + throw result.o3; + } + if (result.o4 != null) { + throw result.o4; + } + if (result.o5 != null) { + throw result.o5; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "rollStreamingPartition failed: unknown result"); + } + + public void updateStreamingTempLocation(String dbName, String tableName, String streamingTempDir) throws InvalidInputException, InvalidObjectException, MetaException, org.apache.thrift.TException + { + send_updateStreamingTempLocation(dbName, tableName, streamingTempDir); + recv_updateStreamingTempLocation(); + } + + public void send_updateStreamingTempLocation(String dbName, String tableName, String streamingTempDir) throws org.apache.thrift.TException + { + updateStreamingTempLocation_args args = new updateStreamingTempLocation_args(); + args.setDbName(dbName); + args.setTableName(tableName); + args.setStreamingTempDir(streamingTempDir); + sendBase("updateStreamingTempLocation", args); + } + + public void recv_updateStreamingTempLocation() throws InvalidInputException, InvalidObjectException, MetaException, org.apache.thrift.TException + { + updateStreamingTempLocation_result result = new updateStreamingTempLocation_result(); + receiveBase(result, "updateStreamingTempLocation"); + if (result.o1 != null) { + throw result.o1; + } + if (result.o2 != null) { + throw result.o2; + } + if (result.o3 != null) { + throw result.o3; + } + return; + } + + public long getNextChunkID(String db_name, String table_name) throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException + { + send_getNextChunkID(db_name, table_name); + return recv_getNextChunkID(); + } + + public void send_getNextChunkID(String db_name, String table_name) throws org.apache.thrift.TException + { + getNextChunkID_args args = new getNextChunkID_args(); + args.setDb_name(db_name); + args.setTable_name(table_name); + sendBase("getNextChunkID", args); + } + + public long recv_getNextChunkID() throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException + { + getNextChunkID_result result = new getNextChunkID_result(); + receiveBase(result, "getNextChunkID"); + if (result.isSetSuccess()) { + return result.success; + } + if (result.o1 != null) { + throw result.o1; + } + if (result.o2 != null) { + throw result.o2; + } + if (result.o3 != null) { + throw result.o3; + } + if (result.o4 != null) { + throw result.o4; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getNextChunkID failed: unknown result"); + } + + public String getStreamingTmpDir(String dbName, String tableName) throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException + { + send_getStreamingTmpDir(dbName, tableName); + return recv_getStreamingTmpDir(); + } + + public void send_getStreamingTmpDir(String dbName, String tableName) throws org.apache.thrift.TException + { + getStreamingTmpDir_args args = new getStreamingTmpDir_args(); + args.setDbName(dbName); + args.setTableName(tableName); + sendBase("getStreamingTmpDir", args); + } + + public String recv_getStreamingTmpDir() throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException + { + getStreamingTmpDir_result result = new getStreamingTmpDir_result(); + receiveBase(result, "getStreamingTmpDir"); + if (result.isSetSuccess()) { + return result.success; + } + if (result.o1 != null) { + throw result.o1; + } + if (result.o2 != null) { + throw result.o2; + } + if (result.o3 != null) { + throw result.o3; + } + if (result.o4 != null) { + throw result.o4; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getStreamingTmpDir failed: unknown result"); + } + public boolean create_role(Role role) throws MetaException, org.apache.thrift.TException { send_create_role(role); @@ -5465,6 +5779,304 @@ public boolean getResult() throws NoSuchObjectException, MetaException, InvalidO } } + public void streamingStatus(String db_name, String table_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + streamingStatus_call method_call = new streamingStatus_call(db_name, table_name, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class streamingStatus_call extends org.apache.thrift.async.TAsyncMethodCall { + private String db_name; + private String table_name; + public streamingStatus_call(String db_name, String table_name, 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); + this.db_name = db_name; + this.table_name = table_name; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("streamingStatus", org.apache.thrift.protocol.TMessageType.CALL, 0)); + streamingStatus_args args = new streamingStatus_args(); + args.setDb_name(db_name); + args.setTable_name(table_name); + args.write(prot); + prot.writeMessageEnd(); + } + + public Map getResult() throws InvalidInputException, InvalidObjectException, MetaException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new 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_streamingStatus(); + } + } + + public void enableStreaming(String db_name, String table_name, String streaming_tmp_dir, String partition_path, String partition_val, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + enableStreaming_call method_call = new enableStreaming_call(db_name, table_name, streaming_tmp_dir, partition_path, partition_val, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class enableStreaming_call extends org.apache.thrift.async.TAsyncMethodCall { + private String db_name; + private String table_name; + private String streaming_tmp_dir; + private String partition_path; + private String partition_val; + public enableStreaming_call(String db_name, String table_name, String streaming_tmp_dir, String partition_path, String partition_val, 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); + this.db_name = db_name; + this.table_name = table_name; + this.streaming_tmp_dir = streaming_tmp_dir; + this.partition_path = partition_path; + this.partition_val = partition_val; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("enableStreaming", org.apache.thrift.protocol.TMessageType.CALL, 0)); + enableStreaming_args args = new enableStreaming_args(); + args.setDb_name(db_name); + args.setTable_name(table_name); + args.setStreaming_tmp_dir(streaming_tmp_dir); + args.setPartition_path(partition_path); + args.setPartition_val(partition_val); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new 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); + (new Client(prot)).recv_enableStreaming(); + } + } + + public void disableStreaming(String db_name, String table_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + disableStreaming_call method_call = new disableStreaming_call(db_name, table_name, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class disableStreaming_call extends org.apache.thrift.async.TAsyncMethodCall { + private String db_name; + private String table_name; + public disableStreaming_call(String db_name, String table_name, 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); + this.db_name = db_name; + this.table_name = table_name; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("disableStreaming", org.apache.thrift.protocol.TMessageType.CALL, 0)); + disableStreaming_args args = new disableStreaming_args(); + args.setDb_name(db_name); + args.setTable_name(table_name); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new 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); + (new Client(prot)).recv_disableStreaming(); + } + } + + public void getCurrentStreamingPartitionPath(String dbName, String tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + getCurrentStreamingPartitionPath_call method_call = new getCurrentStreamingPartitionPath_call(dbName, tableName, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class getCurrentStreamingPartitionPath_call extends org.apache.thrift.async.TAsyncMethodCall { + private String dbName; + private String tableName; + public getCurrentStreamingPartitionPath_call(String dbName, String tableName, 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); + this.dbName = dbName; + this.tableName = tableName; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getCurrentStreamingPartitionPath", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getCurrentStreamingPartitionPath_args args = new getCurrentStreamingPartitionPath_args(); + args.setDbName(dbName); + args.setTableName(tableName); + args.write(prot); + prot.writeMessageEnd(); + } + + public String getResult() throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new 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_getCurrentStreamingPartitionPath(); + } + } + + public void rollStreamingPartition(String dbName, String tableName, String newPartitionPath, String newPartitionVal, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + rollStreamingPartition_call method_call = new rollStreamingPartition_call(dbName, tableName, newPartitionPath, newPartitionVal, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class rollStreamingPartition_call extends org.apache.thrift.async.TAsyncMethodCall { + private String dbName; + private String tableName; + private String newPartitionPath; + private String newPartitionVal; + public rollStreamingPartition_call(String dbName, String tableName, String newPartitionPath, String newPartitionVal, 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); + this.dbName = dbName; + this.tableName = tableName; + this.newPartitionPath = newPartitionPath; + this.newPartitionVal = newPartitionVal; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("rollStreamingPartition", org.apache.thrift.protocol.TMessageType.CALL, 0)); + rollStreamingPartition_args args = new rollStreamingPartition_args(); + args.setDbName(dbName); + args.setTableName(tableName); + args.setNewPartitionPath(newPartitionPath); + args.setNewPartitionVal(newPartitionVal); + args.write(prot); + prot.writeMessageEnd(); + } + + public String getResult() throws InvalidInputException, InvalidObjectException, InvalidOperationException, AlreadyExistsException, MetaException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new 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_rollStreamingPartition(); + } + } + + public void updateStreamingTempLocation(String dbName, String tableName, String streamingTempDir, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + updateStreamingTempLocation_call method_call = new updateStreamingTempLocation_call(dbName, tableName, streamingTempDir, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class updateStreamingTempLocation_call extends org.apache.thrift.async.TAsyncMethodCall { + private String dbName; + private String tableName; + private String streamingTempDir; + public updateStreamingTempLocation_call(String dbName, String tableName, String streamingTempDir, 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); + this.dbName = dbName; + this.tableName = tableName; + this.streamingTempDir = streamingTempDir; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("updateStreamingTempLocation", org.apache.thrift.protocol.TMessageType.CALL, 0)); + updateStreamingTempLocation_args args = new updateStreamingTempLocation_args(); + args.setDbName(dbName); + args.setTableName(tableName); + args.setStreamingTempDir(streamingTempDir); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws InvalidInputException, InvalidObjectException, MetaException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new 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); + (new Client(prot)).recv_updateStreamingTempLocation(); + } + } + + public void getNextChunkID(String db_name, String table_name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + getNextChunkID_call method_call = new getNextChunkID_call(db_name, table_name, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class getNextChunkID_call extends org.apache.thrift.async.TAsyncMethodCall { + private String db_name; + private String table_name; + public getNextChunkID_call(String db_name, String table_name, 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); + this.db_name = db_name; + this.table_name = table_name; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getNextChunkID", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getNextChunkID_args args = new getNextChunkID_args(); + args.setDb_name(db_name); + args.setTable_name(table_name); + args.write(prot); + prot.writeMessageEnd(); + } + + public long getResult() throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new 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_getNextChunkID(); + } + } + + public void getStreamingTmpDir(String dbName, String tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + getStreamingTmpDir_call method_call = new getStreamingTmpDir_call(dbName, tableName, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class getStreamingTmpDir_call extends org.apache.thrift.async.TAsyncMethodCall { + private String dbName; + private String tableName; + public getStreamingTmpDir_call(String dbName, String tableName, 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); + this.dbName = dbName; + this.tableName = tableName; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getStreamingTmpDir", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getStreamingTmpDir_args args = new getStreamingTmpDir_args(); + args.setDbName(dbName); + args.setTableName(tableName); + args.write(prot); + prot.writeMessageEnd(); + } + + public String getResult() throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new 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_getStreamingTmpDir(); + } + } + public void create_role(Role role, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); create_role_call method_call = new create_role_call(role, resultHandler, this, ___protocolFactory, ___transport); @@ -6033,6 +6645,14 @@ protected Processor(I iface, Map extends org.apache.thrift.ProcessFunction { + public streamingStatus() { + super("streamingStatus"); + } + + public streamingStatus_args getEmptyArgsInstance() { + return new streamingStatus_args(); + } + + protected boolean isOneway() { + return false; + } + + public streamingStatus_result getResult(I iface, streamingStatus_args args) throws org.apache.thrift.TException { + streamingStatus_result result = new streamingStatus_result(); + try { + result.success = iface.streamingStatus(args.db_name, args.table_name); + } catch (InvalidInputException o1) { + result.o1 = o1; + } catch (InvalidObjectException o2) { + result.o2 = o2; + } catch (MetaException o3) { + result.o3 = o3; + } + return result; + } + } + + public static class enableStreaming extends org.apache.thrift.ProcessFunction { + public enableStreaming() { + super("enableStreaming"); + } + + public enableStreaming_args getEmptyArgsInstance() { + return new enableStreaming_args(); + } + + protected boolean isOneway() { + return false; + } + + public enableStreaming_result getResult(I iface, enableStreaming_args args) throws org.apache.thrift.TException { + enableStreaming_result result = new enableStreaming_result(); + try { + iface.enableStreaming(args.db_name, args.table_name, args.streaming_tmp_dir, args.partition_path, args.partition_val); + } catch (InvalidInputException o1) { + result.o1 = o1; + } catch (InvalidObjectException o2) { + result.o2 = o2; + } catch (InvalidOperationException o3) { + result.o3 = o3; + } catch (MetaException o4) { + result.o4 = o4; + } + return result; + } + } + + public static class disableStreaming extends org.apache.thrift.ProcessFunction { + public disableStreaming() { + super("disableStreaming"); + } + + public disableStreaming_args getEmptyArgsInstance() { + return new disableStreaming_args(); + } + + protected boolean isOneway() { + return false; + } + + public disableStreaming_result getResult(I iface, disableStreaming_args args) throws org.apache.thrift.TException { + disableStreaming_result result = new disableStreaming_result(); + try { + iface.disableStreaming(args.db_name, args.table_name); + } catch (InvalidInputException o1) { + result.o1 = o1; + } catch (InvalidObjectException o2) { + result.o2 = o2; + } catch (InvalidOperationException o3) { + result.o3 = o3; + } catch (MetaException o4) { + result.o4 = o4; + } + return result; + } + } + + public static class getCurrentStreamingPartitionPath extends org.apache.thrift.ProcessFunction { + public getCurrentStreamingPartitionPath() { + super("getCurrentStreamingPartitionPath"); + } + + public getCurrentStreamingPartitionPath_args getEmptyArgsInstance() { + return new getCurrentStreamingPartitionPath_args(); + } + + protected boolean isOneway() { + return false; + } + + public getCurrentStreamingPartitionPath_result getResult(I iface, getCurrentStreamingPartitionPath_args args) throws org.apache.thrift.TException { + getCurrentStreamingPartitionPath_result result = new getCurrentStreamingPartitionPath_result(); + try { + result.success = iface.getCurrentStreamingPartitionPath(args.dbName, args.tableName); + } catch (InvalidInputException o1) { + result.o1 = o1; + } catch (InvalidObjectException o2) { + result.o2 = o2; + } catch (InvalidOperationException o3) { + result.o3 = o3; + } catch (MetaException o4) { + result.o4 = o4; + } + return result; + } + } + + public static class rollStreamingPartition extends org.apache.thrift.ProcessFunction { + public rollStreamingPartition() { + super("rollStreamingPartition"); + } + + public rollStreamingPartition_args getEmptyArgsInstance() { + return new rollStreamingPartition_args(); + } + + protected boolean isOneway() { + return false; + } + + public rollStreamingPartition_result getResult(I iface, rollStreamingPartition_args args) throws org.apache.thrift.TException { + rollStreamingPartition_result result = new rollStreamingPartition_result(); + try { + result.success = iface.rollStreamingPartition(args.dbName, args.tableName, args.newPartitionPath, args.newPartitionVal); + } catch (InvalidInputException o1) { + result.o1 = o1; + } catch (InvalidObjectException o2) { + result.o2 = o2; + } catch (InvalidOperationException o3) { + result.o3 = o3; + } catch (AlreadyExistsException o4) { + result.o4 = o4; + } catch (MetaException o5) { + result.o5 = o5; + } + return result; + } + } + + public static class updateStreamingTempLocation extends org.apache.thrift.ProcessFunction { + public updateStreamingTempLocation() { + super("updateStreamingTempLocation"); + } + + public updateStreamingTempLocation_args getEmptyArgsInstance() { + return new updateStreamingTempLocation_args(); + } + + protected boolean isOneway() { + return false; + } + + public updateStreamingTempLocation_result getResult(I iface, updateStreamingTempLocation_args args) throws org.apache.thrift.TException { + updateStreamingTempLocation_result result = new updateStreamingTempLocation_result(); + try { + iface.updateStreamingTempLocation(args.dbName, args.tableName, args.streamingTempDir); + } catch (InvalidInputException o1) { + result.o1 = o1; + } catch (InvalidObjectException o2) { + result.o2 = o2; + } catch (MetaException o3) { + result.o3 = o3; + } + return result; + } + } + + public static class getNextChunkID extends org.apache.thrift.ProcessFunction { + public getNextChunkID() { + super("getNextChunkID"); + } + + public getNextChunkID_args getEmptyArgsInstance() { + return new getNextChunkID_args(); + } + + protected boolean isOneway() { + return false; + } + + public getNextChunkID_result getResult(I iface, getNextChunkID_args args) throws org.apache.thrift.TException { + getNextChunkID_result result = new getNextChunkID_result(); + try { + result.success = iface.getNextChunkID(args.db_name, args.table_name); + result.setSuccessIsSet(true); + } catch (InvalidInputException o1) { + result.o1 = o1; + } catch (InvalidObjectException o2) { + result.o2 = o2; + } catch (InvalidOperationException o3) { + result.o3 = o3; + } catch (MetaException o4) { + result.o4 = o4; + } + return result; + } + } + + public static class getStreamingTmpDir extends org.apache.thrift.ProcessFunction { + public getStreamingTmpDir() { + super("getStreamingTmpDir"); + } + + public getStreamingTmpDir_args getEmptyArgsInstance() { + return new getStreamingTmpDir_args(); + } + + protected boolean isOneway() { + return false; + } + + public getStreamingTmpDir_result getResult(I iface, getStreamingTmpDir_args args) throws org.apache.thrift.TException { + getStreamingTmpDir_result result = new getStreamingTmpDir_result(); + try { + result.success = iface.getStreamingTmpDir(args.dbName, args.tableName); + } catch (InvalidInputException o1) { + result.o1 = o1; + } catch (InvalidObjectException o2) { + result.o2 = o2; + } catch (InvalidOperationException o3) { + result.o3 = o3; + } catch (MetaException o4) { + result.o4 = o4; + } + return result; + } + } + public static class create_role extends org.apache.thrift.ProcessFunction { public create_role() { super("create_role"); @@ -85492,72 +86351,10533 @@ public void setDb_nameIsSet(boolean value) { } } - public String getTbl_name() { - return this.tbl_name; - } - - public void setTbl_name(String tbl_name) { - this.tbl_name = tbl_name; - } - - public void unsetTbl_name() { - this.tbl_name = null; - } - - /** Returns true if field tbl_name is set (has been assigned a value) and false otherwise */ - public boolean isSetTbl_name() { - return this.tbl_name != null; - } - - public void setTbl_nameIsSet(boolean value) { - if (!value) { - this.tbl_name = null; - } - } - - public String getPart_name() { - return this.part_name; - } - - public void setPart_name(String part_name) { - this.part_name = part_name; - } - - public void unsetPart_name() { - this.part_name = null; - } - - /** Returns true if field part_name is set (has been assigned a value) and false otherwise */ - public boolean isSetPart_name() { - return this.part_name != null; - } - - public void setPart_nameIsSet(boolean value) { - if (!value) { - this.part_name = null; - } - } - - public String getCol_name() { - return this.col_name; + public String getTbl_name() { + return this.tbl_name; + } + + public void setTbl_name(String tbl_name) { + this.tbl_name = tbl_name; + } + + public void unsetTbl_name() { + this.tbl_name = null; + } + + /** Returns true if field tbl_name is set (has been assigned a value) and false otherwise */ + public boolean isSetTbl_name() { + return this.tbl_name != null; + } + + public void setTbl_nameIsSet(boolean value) { + if (!value) { + this.tbl_name = null; + } + } + + public String getPart_name() { + return this.part_name; + } + + public void setPart_name(String part_name) { + this.part_name = part_name; + } + + public void unsetPart_name() { + this.part_name = null; + } + + /** Returns true if field part_name is set (has been assigned a value) and false otherwise */ + public boolean isSetPart_name() { + return this.part_name != null; + } + + public void setPart_nameIsSet(boolean value) { + if (!value) { + this.part_name = null; + } + } + + public String getCol_name() { + return this.col_name; + } + + public void setCol_name(String col_name) { + this.col_name = col_name; + } + + public void unsetCol_name() { + this.col_name = null; + } + + /** Returns true if field col_name is set (has been assigned a value) and false otherwise */ + public boolean isSetCol_name() { + return this.col_name != null; + } + + public void setCol_nameIsSet(boolean value) { + if (!value) { + this.col_name = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case DB_NAME: + if (value == null) { + unsetDb_name(); + } else { + setDb_name((String)value); + } + break; + + case TBL_NAME: + if (value == null) { + unsetTbl_name(); + } else { + setTbl_name((String)value); + } + break; + + case PART_NAME: + if (value == null) { + unsetPart_name(); + } else { + setPart_name((String)value); + } + break; + + case COL_NAME: + if (value == null) { + unsetCol_name(); + } else { + setCol_name((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case DB_NAME: + return getDb_name(); + + case TBL_NAME: + return getTbl_name(); + + case PART_NAME: + return getPart_name(); + + case COL_NAME: + return getCol_name(); + + } + throw new 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 IllegalArgumentException(); + } + + switch (field) { + case DB_NAME: + return isSetDb_name(); + case TBL_NAME: + return isSetTbl_name(); + case PART_NAME: + return isSetPart_name(); + case COL_NAME: + return isSetCol_name(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof delete_partition_column_statistics_args) + return this.equals((delete_partition_column_statistics_args)that); + return false; + } + + public boolean equals(delete_partition_column_statistics_args that) { + if (that == null) + return false; + + boolean this_present_db_name = true && this.isSetDb_name(); + boolean that_present_db_name = true && that.isSetDb_name(); + if (this_present_db_name || that_present_db_name) { + if (!(this_present_db_name && that_present_db_name)) + return false; + if (!this.db_name.equals(that.db_name)) + return false; + } + + boolean this_present_tbl_name = true && this.isSetTbl_name(); + boolean that_present_tbl_name = true && that.isSetTbl_name(); + if (this_present_tbl_name || that_present_tbl_name) { + if (!(this_present_tbl_name && that_present_tbl_name)) + return false; + if (!this.tbl_name.equals(that.tbl_name)) + return false; + } + + boolean this_present_part_name = true && this.isSetPart_name(); + boolean that_present_part_name = true && that.isSetPart_name(); + if (this_present_part_name || that_present_part_name) { + if (!(this_present_part_name && that_present_part_name)) + return false; + if (!this.part_name.equals(that.part_name)) + return false; + } + + boolean this_present_col_name = true && this.isSetCol_name(); + boolean that_present_col_name = true && that.isSetCol_name(); + if (this_present_col_name || that_present_col_name) { + if (!(this_present_col_name && that_present_col_name)) + return false; + if (!this.col_name.equals(that.col_name)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_db_name = true && (isSetDb_name()); + builder.append(present_db_name); + if (present_db_name) + builder.append(db_name); + + boolean present_tbl_name = true && (isSetTbl_name()); + builder.append(present_tbl_name); + if (present_tbl_name) + builder.append(tbl_name); + + boolean present_part_name = true && (isSetPart_name()); + builder.append(present_part_name); + if (present_part_name) + builder.append(part_name); + + boolean present_col_name = true && (isSetCol_name()); + builder.append(present_col_name); + if (present_col_name) + builder.append(col_name); + + return builder.toHashCode(); + } + + public int compareTo(delete_partition_column_statistics_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + delete_partition_column_statistics_args typedOther = (delete_partition_column_statistics_args)other; + + lastComparison = Boolean.valueOf(isSetDb_name()).compareTo(typedOther.isSetDb_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetDb_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.db_name, typedOther.db_name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTbl_name()).compareTo(typedOther.isSetTbl_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTbl_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tbl_name, typedOther.tbl_name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetPart_name()).compareTo(typedOther.isSetPart_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetPart_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.part_name, typedOther.part_name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetCol_name()).compareTo(typedOther.isSetCol_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetCol_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.col_name, typedOther.col_name); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("delete_partition_column_statistics_args("); + boolean first = true; + + sb.append("db_name:"); + if (this.db_name == null) { + sb.append("null"); + } else { + sb.append(this.db_name); + } + first = false; + if (!first) sb.append(", "); + sb.append("tbl_name:"); + if (this.tbl_name == null) { + sb.append("null"); + } else { + sb.append(this.tbl_name); + } + first = false; + if (!first) sb.append(", "); + sb.append("part_name:"); + if (this.part_name == null) { + sb.append("null"); + } else { + sb.append(this.part_name); + } + first = false; + if (!first) sb.append(", "); + sb.append("col_name:"); + if (this.col_name == null) { + sb.append("null"); + } else { + sb.append(this.col_name); + } + 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, 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 delete_partition_column_statistics_argsStandardSchemeFactory implements SchemeFactory { + public delete_partition_column_statistics_argsStandardScheme getScheme() { + return new delete_partition_column_statistics_argsStandardScheme(); + } + } + + private static class delete_partition_column_statistics_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, delete_partition_column_statistics_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: // DB_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.db_name = iprot.readString(); + struct.setDb_nameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // TBL_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.tbl_name = iprot.readString(); + struct.setTbl_nameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // PART_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.part_name = iprot.readString(); + struct.setPart_nameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // COL_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.col_name = iprot.readString(); + struct.setCol_nameIsSet(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(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, delete_partition_column_statistics_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.db_name != null) { + oprot.writeFieldBegin(DB_NAME_FIELD_DESC); + oprot.writeString(struct.db_name); + oprot.writeFieldEnd(); + } + if (struct.tbl_name != null) { + oprot.writeFieldBegin(TBL_NAME_FIELD_DESC); + oprot.writeString(struct.tbl_name); + oprot.writeFieldEnd(); + } + if (struct.part_name != null) { + oprot.writeFieldBegin(PART_NAME_FIELD_DESC); + oprot.writeString(struct.part_name); + oprot.writeFieldEnd(); + } + if (struct.col_name != null) { + oprot.writeFieldBegin(COL_NAME_FIELD_DESC); + oprot.writeString(struct.col_name); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class delete_partition_column_statistics_argsTupleSchemeFactory implements SchemeFactory { + public delete_partition_column_statistics_argsTupleScheme getScheme() { + return new delete_partition_column_statistics_argsTupleScheme(); + } + } + + private static class delete_partition_column_statistics_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, delete_partition_column_statistics_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetDb_name()) { + optionals.set(0); + } + if (struct.isSetTbl_name()) { + optionals.set(1); + } + if (struct.isSetPart_name()) { + optionals.set(2); + } + if (struct.isSetCol_name()) { + optionals.set(3); + } + oprot.writeBitSet(optionals, 4); + if (struct.isSetDb_name()) { + oprot.writeString(struct.db_name); + } + if (struct.isSetTbl_name()) { + oprot.writeString(struct.tbl_name); + } + if (struct.isSetPart_name()) { + oprot.writeString(struct.part_name); + } + if (struct.isSetCol_name()) { + oprot.writeString(struct.col_name); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, delete_partition_column_statistics_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(4); + if (incoming.get(0)) { + struct.db_name = iprot.readString(); + struct.setDb_nameIsSet(true); + } + if (incoming.get(1)) { + struct.tbl_name = iprot.readString(); + struct.setTbl_nameIsSet(true); + } + if (incoming.get(2)) { + struct.part_name = iprot.readString(); + struct.setPart_nameIsSet(true); + } + if (incoming.get(3)) { + struct.col_name = iprot.readString(); + struct.setCol_nameIsSet(true); + } + } + } + + } + + public static class delete_partition_column_statistics_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("delete_partition_column_statistics_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 O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField O2_FIELD_DESC = new org.apache.thrift.protocol.TField("o2", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField O3_FIELD_DESC = new org.apache.thrift.protocol.TField("o3", org.apache.thrift.protocol.TType.STRUCT, (short)3); + private static final org.apache.thrift.protocol.TField O4_FIELD_DESC = new org.apache.thrift.protocol.TField("o4", org.apache.thrift.protocol.TType.STRUCT, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new delete_partition_column_statistics_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new delete_partition_column_statistics_resultTupleSchemeFactory()); + } + + private boolean success; // required + private NoSuchObjectException o1; // required + private MetaException o2; // required + private InvalidObjectException o3; // required + private InvalidInputException o4; // 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"), + O1((short)1, "o1"), + O2((short)2, "o2"), + O3((short)3, "o3"), + O4((short)4, "o4"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + case 1: // O1 + return O1; + case 2: // O2 + return O2; + case 3: // O3 + return O3; + case 4: // O4 + return O4; + 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 IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __SUCCESS_ISSET_ID = 0; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new 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.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O2, new org.apache.thrift.meta_data.FieldMetaData("o2", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O3, new org.apache.thrift.meta_data.FieldMetaData("o3", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O4, new org.apache.thrift.meta_data.FieldMetaData("o4", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(delete_partition_column_statistics_result.class, metaDataMap); + } + + public delete_partition_column_statistics_result() { + } + + public delete_partition_column_statistics_result( + boolean success, + NoSuchObjectException o1, + MetaException o2, + InvalidObjectException o3, + InvalidInputException o4) + { + this(); + this.success = success; + setSuccessIsSet(true); + this.o1 = o1; + this.o2 = o2; + this.o3 = o3; + this.o4 = o4; + } + + /** + * Performs a deep copy on other. + */ + public delete_partition_column_statistics_result(delete_partition_column_statistics_result other) { + __isset_bitfield = other.__isset_bitfield; + this.success = other.success; + if (other.isSetO1()) { + this.o1 = new NoSuchObjectException(other.o1); + } + if (other.isSetO2()) { + this.o2 = new MetaException(other.o2); + } + if (other.isSetO3()) { + this.o3 = new InvalidObjectException(other.o3); + } + if (other.isSetO4()) { + this.o4 = new InvalidInputException(other.o4); + } + } + + public delete_partition_column_statistics_result deepCopy() { + return new delete_partition_column_statistics_result(this); + } + + @Override + public void clear() { + setSuccessIsSet(false); + this.success = false; + this.o1 = null; + this.o2 = null; + this.o3 = null; + this.o4 = null; + } + + public boolean isSuccess() { + return this.success; + } + + public void setSuccess(boolean success) { + this.success = success; + setSuccessIsSet(true); + } + + public void unsetSuccess() { + __isset_bitfield = 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 EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID); + } + + public void setSuccessIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value); + } + + public NoSuchObjectException getO1() { + return this.o1; + } + + public void setO1(NoSuchObjectException o1) { + this.o1 = o1; + } + + public void unsetO1() { + this.o1 = null; + } + + /** Returns true if field o1 is set (has been assigned a value) and false otherwise */ + public boolean isSetO1() { + return this.o1 != null; + } + + public void setO1IsSet(boolean value) { + if (!value) { + this.o1 = null; + } + } + + public MetaException getO2() { + return this.o2; + } + + public void setO2(MetaException o2) { + this.o2 = o2; + } + + public void unsetO2() { + this.o2 = null; + } + + /** Returns true if field o2 is set (has been assigned a value) and false otherwise */ + public boolean isSetO2() { + return this.o2 != null; + } + + public void setO2IsSet(boolean value) { + if (!value) { + this.o2 = null; + } + } + + public InvalidObjectException getO3() { + return this.o3; + } + + public void setO3(InvalidObjectException o3) { + this.o3 = o3; + } + + public void unsetO3() { + this.o3 = null; + } + + /** Returns true if field o3 is set (has been assigned a value) and false otherwise */ + public boolean isSetO3() { + return this.o3 != null; + } + + public void setO3IsSet(boolean value) { + if (!value) { + this.o3 = null; + } + } + + public InvalidInputException getO4() { + return this.o4; + } + + public void setO4(InvalidInputException o4) { + this.o4 = o4; + } + + public void unsetO4() { + this.o4 = null; + } + + /** Returns true if field o4 is set (has been assigned a value) and false otherwise */ + public boolean isSetO4() { + return this.o4 != null; + } + + public void setO4IsSet(boolean value) { + if (!value) { + this.o4 = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((Boolean)value); + } + break; + + case O1: + if (value == null) { + unsetO1(); + } else { + setO1((NoSuchObjectException)value); + } + break; + + case O2: + if (value == null) { + unsetO2(); + } else { + setO2((MetaException)value); + } + break; + + case O3: + if (value == null) { + unsetO3(); + } else { + setO3((InvalidObjectException)value); + } + break; + + case O4: + if (value == null) { + unsetO4(); + } else { + setO4((InvalidInputException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return Boolean.valueOf(isSuccess()); + + case O1: + return getO1(); + + case O2: + return getO2(); + + case O3: + return getO3(); + + case O4: + return getO4(); + + } + throw new 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 IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + case O1: + return isSetO1(); + case O2: + return isSetO2(); + case O3: + return isSetO3(); + case O4: + return isSetO4(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof delete_partition_column_statistics_result) + return this.equals((delete_partition_column_statistics_result)that); + return false; + } + + public boolean equals(delete_partition_column_statistics_result that) { + if (that == null) + return false; + + 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_o1 = true && this.isSetO1(); + boolean that_present_o1 = true && that.isSetO1(); + if (this_present_o1 || that_present_o1) { + if (!(this_present_o1 && that_present_o1)) + return false; + if (!this.o1.equals(that.o1)) + return false; + } + + boolean this_present_o2 = true && this.isSetO2(); + boolean that_present_o2 = true && that.isSetO2(); + if (this_present_o2 || that_present_o2) { + if (!(this_present_o2 && that_present_o2)) + return false; + if (!this.o2.equals(that.o2)) + return false; + } + + boolean this_present_o3 = true && this.isSetO3(); + boolean that_present_o3 = true && that.isSetO3(); + if (this_present_o3 || that_present_o3) { + if (!(this_present_o3 && that_present_o3)) + return false; + if (!this.o3.equals(that.o3)) + return false; + } + + boolean this_present_o4 = true && this.isSetO4(); + boolean that_present_o4 = true && that.isSetO4(); + if (this_present_o4 || that_present_o4) { + if (!(this_present_o4 && that_present_o4)) + return false; + if (!this.o4.equals(that.o4)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true; + builder.append(present_success); + if (present_success) + builder.append(success); + + boolean present_o1 = true && (isSetO1()); + builder.append(present_o1); + if (present_o1) + builder.append(o1); + + boolean present_o2 = true && (isSetO2()); + builder.append(present_o2); + if (present_o2) + builder.append(o2); + + boolean present_o3 = true && (isSetO3()); + builder.append(present_o3); + if (present_o3) + builder.append(o3); + + boolean present_o4 = true && (isSetO4()); + builder.append(present_o4); + if (present_o4) + builder.append(o4); + + return builder.toHashCode(); + } + + public int compareTo(delete_partition_column_statistics_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + delete_partition_column_statistics_result typedOther = (delete_partition_column_statistics_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO1()).compareTo(typedOther.isSetO1()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO1()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, typedOther.o1); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO2()).compareTo(typedOther.isSetO2()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO2()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o2, typedOther.o2); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO3()).compareTo(typedOther.isSetO3()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO3()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o3, typedOther.o3); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO4()).compareTo(typedOther.isSetO4()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO4()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o4, typedOther.o4); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("delete_partition_column_statistics_result("); + boolean first = true; + + sb.append("success:"); + sb.append(this.success); + first = false; + if (!first) sb.append(", "); + sb.append("o1:"); + if (this.o1 == null) { + sb.append("null"); + } else { + sb.append(this.o1); + } + first = false; + if (!first) sb.append(", "); + sb.append("o2:"); + if (this.o2 == null) { + sb.append("null"); + } else { + sb.append(this.o2); + } + first = false; + if (!first) sb.append(", "); + sb.append("o3:"); + if (this.o3 == null) { + sb.append("null"); + } else { + sb.append(this.o3); + } + first = false; + if (!first) sb.append(", "); + sb.append("o4:"); + if (this.o4 == null) { + sb.append("null"); + } else { + sb.append(this.o4); + } + 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, 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 delete_partition_column_statistics_resultStandardSchemeFactory implements SchemeFactory { + public delete_partition_column_statistics_resultStandardScheme getScheme() { + return new delete_partition_column_statistics_resultStandardScheme(); + } + } + + private static class delete_partition_column_statistics_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, delete_partition_column_statistics_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.BOOL) { + struct.success = iprot.readBool(); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 1: // O1 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o1 = new NoSuchObjectException(); + struct.o1.read(iprot); + struct.setO1IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // O2 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o2 = new MetaException(); + struct.o2.read(iprot); + struct.setO2IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // O3 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o3 = new InvalidObjectException(); + struct.o3.read(iprot); + struct.setO3IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // O4 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o4 = new InvalidInputException(); + struct.o4.read(iprot); + struct.setO4IsSet(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(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, delete_partition_column_statistics_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.o1 != null) { + oprot.writeFieldBegin(O1_FIELD_DESC); + struct.o1.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o2 != null) { + oprot.writeFieldBegin(O2_FIELD_DESC); + struct.o2.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o3 != null) { + oprot.writeFieldBegin(O3_FIELD_DESC); + struct.o3.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o4 != null) { + oprot.writeFieldBegin(O4_FIELD_DESC); + struct.o4.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class delete_partition_column_statistics_resultTupleSchemeFactory implements SchemeFactory { + public delete_partition_column_statistics_resultTupleScheme getScheme() { + return new delete_partition_column_statistics_resultTupleScheme(); + } + } + + private static class delete_partition_column_statistics_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, delete_partition_column_statistics_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + if (struct.isSetO1()) { + optionals.set(1); + } + if (struct.isSetO2()) { + optionals.set(2); + } + if (struct.isSetO3()) { + optionals.set(3); + } + if (struct.isSetO4()) { + optionals.set(4); + } + oprot.writeBitSet(optionals, 5); + if (struct.isSetSuccess()) { + oprot.writeBool(struct.success); + } + if (struct.isSetO1()) { + struct.o1.write(oprot); + } + if (struct.isSetO2()) { + struct.o2.write(oprot); + } + if (struct.isSetO3()) { + struct.o3.write(oprot); + } + if (struct.isSetO4()) { + struct.o4.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, delete_partition_column_statistics_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(5); + if (incoming.get(0)) { + struct.success = iprot.readBool(); + struct.setSuccessIsSet(true); + } + if (incoming.get(1)) { + struct.o1 = new NoSuchObjectException(); + struct.o1.read(iprot); + struct.setO1IsSet(true); + } + if (incoming.get(2)) { + struct.o2 = new MetaException(); + struct.o2.read(iprot); + struct.setO2IsSet(true); + } + if (incoming.get(3)) { + struct.o3 = new InvalidObjectException(); + struct.o3.read(iprot); + struct.setO3IsSet(true); + } + if (incoming.get(4)) { + struct.o4 = new InvalidInputException(); + struct.o4.read(iprot); + struct.setO4IsSet(true); + } + } + } + + } + + public static class delete_table_column_statistics_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("delete_table_column_statistics_args"); + + private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("db_name", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tbl_name", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField COL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("col_name", org.apache.thrift.protocol.TType.STRING, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new delete_table_column_statistics_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new delete_table_column_statistics_argsTupleSchemeFactory()); + } + + private String db_name; // required + private String tbl_name; // required + private String col_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 { + DB_NAME((short)1, "db_name"), + TBL_NAME((short)2, "tbl_name"), + COL_NAME((short)3, "col_name"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // DB_NAME + return DB_NAME; + case 2: // TBL_NAME + return TBL_NAME; + case 3: // COL_NAME + return COL_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 IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("db_name", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.TBL_NAME, new org.apache.thrift.meta_data.FieldMetaData("tbl_name", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.COL_NAME, new org.apache.thrift.meta_data.FieldMetaData("col_name", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(delete_table_column_statistics_args.class, metaDataMap); + } + + public delete_table_column_statistics_args() { + } + + public delete_table_column_statistics_args( + String db_name, + String tbl_name, + String col_name) + { + this(); + this.db_name = db_name; + this.tbl_name = tbl_name; + this.col_name = col_name; + } + + /** + * Performs a deep copy on other. + */ + public delete_table_column_statistics_args(delete_table_column_statistics_args other) { + if (other.isSetDb_name()) { + this.db_name = other.db_name; + } + if (other.isSetTbl_name()) { + this.tbl_name = other.tbl_name; + } + if (other.isSetCol_name()) { + this.col_name = other.col_name; + } + } + + public delete_table_column_statistics_args deepCopy() { + return new delete_table_column_statistics_args(this); + } + + @Override + public void clear() { + this.db_name = null; + this.tbl_name = null; + this.col_name = null; + } + + public String getDb_name() { + return this.db_name; + } + + public void setDb_name(String db_name) { + this.db_name = db_name; + } + + public void unsetDb_name() { + this.db_name = null; + } + + /** Returns true if field db_name is set (has been assigned a value) and false otherwise */ + public boolean isSetDb_name() { + return this.db_name != null; + } + + public void setDb_nameIsSet(boolean value) { + if (!value) { + this.db_name = null; + } + } + + public String getTbl_name() { + return this.tbl_name; + } + + public void setTbl_name(String tbl_name) { + this.tbl_name = tbl_name; + } + + public void unsetTbl_name() { + this.tbl_name = null; + } + + /** Returns true if field tbl_name is set (has been assigned a value) and false otherwise */ + public boolean isSetTbl_name() { + return this.tbl_name != null; + } + + public void setTbl_nameIsSet(boolean value) { + if (!value) { + this.tbl_name = null; + } + } + + public String getCol_name() { + return this.col_name; + } + + public void setCol_name(String col_name) { + this.col_name = col_name; + } + + public void unsetCol_name() { + this.col_name = null; + } + + /** Returns true if field col_name is set (has been assigned a value) and false otherwise */ + public boolean isSetCol_name() { + return this.col_name != null; + } + + public void setCol_nameIsSet(boolean value) { + if (!value) { + this.col_name = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case DB_NAME: + if (value == null) { + unsetDb_name(); + } else { + setDb_name((String)value); + } + break; + + case TBL_NAME: + if (value == null) { + unsetTbl_name(); + } else { + setTbl_name((String)value); + } + break; + + case COL_NAME: + if (value == null) { + unsetCol_name(); + } else { + setCol_name((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case DB_NAME: + return getDb_name(); + + case TBL_NAME: + return getTbl_name(); + + case COL_NAME: + return getCol_name(); + + } + throw new 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 IllegalArgumentException(); + } + + switch (field) { + case DB_NAME: + return isSetDb_name(); + case TBL_NAME: + return isSetTbl_name(); + case COL_NAME: + return isSetCol_name(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof delete_table_column_statistics_args) + return this.equals((delete_table_column_statistics_args)that); + return false; + } + + public boolean equals(delete_table_column_statistics_args that) { + if (that == null) + return false; + + boolean this_present_db_name = true && this.isSetDb_name(); + boolean that_present_db_name = true && that.isSetDb_name(); + if (this_present_db_name || that_present_db_name) { + if (!(this_present_db_name && that_present_db_name)) + return false; + if (!this.db_name.equals(that.db_name)) + return false; + } + + boolean this_present_tbl_name = true && this.isSetTbl_name(); + boolean that_present_tbl_name = true && that.isSetTbl_name(); + if (this_present_tbl_name || that_present_tbl_name) { + if (!(this_present_tbl_name && that_present_tbl_name)) + return false; + if (!this.tbl_name.equals(that.tbl_name)) + return false; + } + + boolean this_present_col_name = true && this.isSetCol_name(); + boolean that_present_col_name = true && that.isSetCol_name(); + if (this_present_col_name || that_present_col_name) { + if (!(this_present_col_name && that_present_col_name)) + return false; + if (!this.col_name.equals(that.col_name)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_db_name = true && (isSetDb_name()); + builder.append(present_db_name); + if (present_db_name) + builder.append(db_name); + + boolean present_tbl_name = true && (isSetTbl_name()); + builder.append(present_tbl_name); + if (present_tbl_name) + builder.append(tbl_name); + + boolean present_col_name = true && (isSetCol_name()); + builder.append(present_col_name); + if (present_col_name) + builder.append(col_name); + + return builder.toHashCode(); + } + + public int compareTo(delete_table_column_statistics_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + delete_table_column_statistics_args typedOther = (delete_table_column_statistics_args)other; + + lastComparison = Boolean.valueOf(isSetDb_name()).compareTo(typedOther.isSetDb_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetDb_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.db_name, typedOther.db_name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTbl_name()).compareTo(typedOther.isSetTbl_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTbl_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tbl_name, typedOther.tbl_name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetCol_name()).compareTo(typedOther.isSetCol_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetCol_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.col_name, typedOther.col_name); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("delete_table_column_statistics_args("); + boolean first = true; + + sb.append("db_name:"); + if (this.db_name == null) { + sb.append("null"); + } else { + sb.append(this.db_name); + } + first = false; + if (!first) sb.append(", "); + sb.append("tbl_name:"); + if (this.tbl_name == null) { + sb.append("null"); + } else { + sb.append(this.tbl_name); + } + first = false; + if (!first) sb.append(", "); + sb.append("col_name:"); + if (this.col_name == null) { + sb.append("null"); + } else { + sb.append(this.col_name); + } + 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, 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 delete_table_column_statistics_argsStandardSchemeFactory implements SchemeFactory { + public delete_table_column_statistics_argsStandardScheme getScheme() { + return new delete_table_column_statistics_argsStandardScheme(); + } + } + + private static class delete_table_column_statistics_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, delete_table_column_statistics_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: // DB_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.db_name = iprot.readString(); + struct.setDb_nameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // TBL_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.tbl_name = iprot.readString(); + struct.setTbl_nameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // COL_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.col_name = iprot.readString(); + struct.setCol_nameIsSet(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(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, delete_table_column_statistics_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.db_name != null) { + oprot.writeFieldBegin(DB_NAME_FIELD_DESC); + oprot.writeString(struct.db_name); + oprot.writeFieldEnd(); + } + if (struct.tbl_name != null) { + oprot.writeFieldBegin(TBL_NAME_FIELD_DESC); + oprot.writeString(struct.tbl_name); + oprot.writeFieldEnd(); + } + if (struct.col_name != null) { + oprot.writeFieldBegin(COL_NAME_FIELD_DESC); + oprot.writeString(struct.col_name); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class delete_table_column_statistics_argsTupleSchemeFactory implements SchemeFactory { + public delete_table_column_statistics_argsTupleScheme getScheme() { + return new delete_table_column_statistics_argsTupleScheme(); + } + } + + private static class delete_table_column_statistics_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, delete_table_column_statistics_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetDb_name()) { + optionals.set(0); + } + if (struct.isSetTbl_name()) { + optionals.set(1); + } + if (struct.isSetCol_name()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); + if (struct.isSetDb_name()) { + oprot.writeString(struct.db_name); + } + if (struct.isSetTbl_name()) { + oprot.writeString(struct.tbl_name); + } + if (struct.isSetCol_name()) { + oprot.writeString(struct.col_name); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, delete_table_column_statistics_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(3); + if (incoming.get(0)) { + struct.db_name = iprot.readString(); + struct.setDb_nameIsSet(true); + } + if (incoming.get(1)) { + struct.tbl_name = iprot.readString(); + struct.setTbl_nameIsSet(true); + } + if (incoming.get(2)) { + struct.col_name = iprot.readString(); + struct.setCol_nameIsSet(true); + } + } + } + + } + + public static class delete_table_column_statistics_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("delete_table_column_statistics_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 O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField O2_FIELD_DESC = new org.apache.thrift.protocol.TField("o2", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField O3_FIELD_DESC = new org.apache.thrift.protocol.TField("o3", org.apache.thrift.protocol.TType.STRUCT, (short)3); + private static final org.apache.thrift.protocol.TField O4_FIELD_DESC = new org.apache.thrift.protocol.TField("o4", org.apache.thrift.protocol.TType.STRUCT, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new delete_table_column_statistics_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new delete_table_column_statistics_resultTupleSchemeFactory()); + } + + private boolean success; // required + private NoSuchObjectException o1; // required + private MetaException o2; // required + private InvalidObjectException o3; // required + private InvalidInputException o4; // 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"), + O1((short)1, "o1"), + O2((short)2, "o2"), + O3((short)3, "o3"), + O4((short)4, "o4"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + case 1: // O1 + return O1; + case 2: // O2 + return O2; + case 3: // O3 + return O3; + case 4: // O4 + return O4; + 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 IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __SUCCESS_ISSET_ID = 0; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new 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.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O2, new org.apache.thrift.meta_data.FieldMetaData("o2", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O3, new org.apache.thrift.meta_data.FieldMetaData("o3", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O4, new org.apache.thrift.meta_data.FieldMetaData("o4", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(delete_table_column_statistics_result.class, metaDataMap); + } + + public delete_table_column_statistics_result() { + } + + public delete_table_column_statistics_result( + boolean success, + NoSuchObjectException o1, + MetaException o2, + InvalidObjectException o3, + InvalidInputException o4) + { + this(); + this.success = success; + setSuccessIsSet(true); + this.o1 = o1; + this.o2 = o2; + this.o3 = o3; + this.o4 = o4; + } + + /** + * Performs a deep copy on other. + */ + public delete_table_column_statistics_result(delete_table_column_statistics_result other) { + __isset_bitfield = other.__isset_bitfield; + this.success = other.success; + if (other.isSetO1()) { + this.o1 = new NoSuchObjectException(other.o1); + } + if (other.isSetO2()) { + this.o2 = new MetaException(other.o2); + } + if (other.isSetO3()) { + this.o3 = new InvalidObjectException(other.o3); + } + if (other.isSetO4()) { + this.o4 = new InvalidInputException(other.o4); + } + } + + public delete_table_column_statistics_result deepCopy() { + return new delete_table_column_statistics_result(this); + } + + @Override + public void clear() { + setSuccessIsSet(false); + this.success = false; + this.o1 = null; + this.o2 = null; + this.o3 = null; + this.o4 = null; + } + + public boolean isSuccess() { + return this.success; + } + + public void setSuccess(boolean success) { + this.success = success; + setSuccessIsSet(true); + } + + public void unsetSuccess() { + __isset_bitfield = 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 EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID); + } + + public void setSuccessIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value); + } + + public NoSuchObjectException getO1() { + return this.o1; + } + + public void setO1(NoSuchObjectException o1) { + this.o1 = o1; + } + + public void unsetO1() { + this.o1 = null; + } + + /** Returns true if field o1 is set (has been assigned a value) and false otherwise */ + public boolean isSetO1() { + return this.o1 != null; + } + + public void setO1IsSet(boolean value) { + if (!value) { + this.o1 = null; + } + } + + public MetaException getO2() { + return this.o2; + } + + public void setO2(MetaException o2) { + this.o2 = o2; + } + + public void unsetO2() { + this.o2 = null; + } + + /** Returns true if field o2 is set (has been assigned a value) and false otherwise */ + public boolean isSetO2() { + return this.o2 != null; + } + + public void setO2IsSet(boolean value) { + if (!value) { + this.o2 = null; + } + } + + public InvalidObjectException getO3() { + return this.o3; + } + + public void setO3(InvalidObjectException o3) { + this.o3 = o3; + } + + public void unsetO3() { + this.o3 = null; + } + + /** Returns true if field o3 is set (has been assigned a value) and false otherwise */ + public boolean isSetO3() { + return this.o3 != null; + } + + public void setO3IsSet(boolean value) { + if (!value) { + this.o3 = null; + } + } + + public InvalidInputException getO4() { + return this.o4; + } + + public void setO4(InvalidInputException o4) { + this.o4 = o4; + } + + public void unsetO4() { + this.o4 = null; + } + + /** Returns true if field o4 is set (has been assigned a value) and false otherwise */ + public boolean isSetO4() { + return this.o4 != null; + } + + public void setO4IsSet(boolean value) { + if (!value) { + this.o4 = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((Boolean)value); + } + break; + + case O1: + if (value == null) { + unsetO1(); + } else { + setO1((NoSuchObjectException)value); + } + break; + + case O2: + if (value == null) { + unsetO2(); + } else { + setO2((MetaException)value); + } + break; + + case O3: + if (value == null) { + unsetO3(); + } else { + setO3((InvalidObjectException)value); + } + break; + + case O4: + if (value == null) { + unsetO4(); + } else { + setO4((InvalidInputException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return Boolean.valueOf(isSuccess()); + + case O1: + return getO1(); + + case O2: + return getO2(); + + case O3: + return getO3(); + + case O4: + return getO4(); + + } + throw new 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 IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + case O1: + return isSetO1(); + case O2: + return isSetO2(); + case O3: + return isSetO3(); + case O4: + return isSetO4(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof delete_table_column_statistics_result) + return this.equals((delete_table_column_statistics_result)that); + return false; + } + + public boolean equals(delete_table_column_statistics_result that) { + if (that == null) + return false; + + 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_o1 = true && this.isSetO1(); + boolean that_present_o1 = true && that.isSetO1(); + if (this_present_o1 || that_present_o1) { + if (!(this_present_o1 && that_present_o1)) + return false; + if (!this.o1.equals(that.o1)) + return false; + } + + boolean this_present_o2 = true && this.isSetO2(); + boolean that_present_o2 = true && that.isSetO2(); + if (this_present_o2 || that_present_o2) { + if (!(this_present_o2 && that_present_o2)) + return false; + if (!this.o2.equals(that.o2)) + return false; + } + + boolean this_present_o3 = true && this.isSetO3(); + boolean that_present_o3 = true && that.isSetO3(); + if (this_present_o3 || that_present_o3) { + if (!(this_present_o3 && that_present_o3)) + return false; + if (!this.o3.equals(that.o3)) + return false; + } + + boolean this_present_o4 = true && this.isSetO4(); + boolean that_present_o4 = true && that.isSetO4(); + if (this_present_o4 || that_present_o4) { + if (!(this_present_o4 && that_present_o4)) + return false; + if (!this.o4.equals(that.o4)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true; + builder.append(present_success); + if (present_success) + builder.append(success); + + boolean present_o1 = true && (isSetO1()); + builder.append(present_o1); + if (present_o1) + builder.append(o1); + + boolean present_o2 = true && (isSetO2()); + builder.append(present_o2); + if (present_o2) + builder.append(o2); + + boolean present_o3 = true && (isSetO3()); + builder.append(present_o3); + if (present_o3) + builder.append(o3); + + boolean present_o4 = true && (isSetO4()); + builder.append(present_o4); + if (present_o4) + builder.append(o4); + + return builder.toHashCode(); + } + + public int compareTo(delete_table_column_statistics_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + delete_table_column_statistics_result typedOther = (delete_table_column_statistics_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO1()).compareTo(typedOther.isSetO1()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO1()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, typedOther.o1); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO2()).compareTo(typedOther.isSetO2()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO2()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o2, typedOther.o2); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO3()).compareTo(typedOther.isSetO3()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO3()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o3, typedOther.o3); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO4()).compareTo(typedOther.isSetO4()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO4()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o4, typedOther.o4); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("delete_table_column_statistics_result("); + boolean first = true; + + sb.append("success:"); + sb.append(this.success); + first = false; + if (!first) sb.append(", "); + sb.append("o1:"); + if (this.o1 == null) { + sb.append("null"); + } else { + sb.append(this.o1); + } + first = false; + if (!first) sb.append(", "); + sb.append("o2:"); + if (this.o2 == null) { + sb.append("null"); + } else { + sb.append(this.o2); + } + first = false; + if (!first) sb.append(", "); + sb.append("o3:"); + if (this.o3 == null) { + sb.append("null"); + } else { + sb.append(this.o3); + } + first = false; + if (!first) sb.append(", "); + sb.append("o4:"); + if (this.o4 == null) { + sb.append("null"); + } else { + sb.append(this.o4); + } + 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, 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 delete_table_column_statistics_resultStandardSchemeFactory implements SchemeFactory { + public delete_table_column_statistics_resultStandardScheme getScheme() { + return new delete_table_column_statistics_resultStandardScheme(); + } + } + + private static class delete_table_column_statistics_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, delete_table_column_statistics_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.BOOL) { + struct.success = iprot.readBool(); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 1: // O1 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o1 = new NoSuchObjectException(); + struct.o1.read(iprot); + struct.setO1IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // O2 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o2 = new MetaException(); + struct.o2.read(iprot); + struct.setO2IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // O3 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o3 = new InvalidObjectException(); + struct.o3.read(iprot); + struct.setO3IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // O4 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o4 = new InvalidInputException(); + struct.o4.read(iprot); + struct.setO4IsSet(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(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, delete_table_column_statistics_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.o1 != null) { + oprot.writeFieldBegin(O1_FIELD_DESC); + struct.o1.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o2 != null) { + oprot.writeFieldBegin(O2_FIELD_DESC); + struct.o2.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o3 != null) { + oprot.writeFieldBegin(O3_FIELD_DESC); + struct.o3.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o4 != null) { + oprot.writeFieldBegin(O4_FIELD_DESC); + struct.o4.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class delete_table_column_statistics_resultTupleSchemeFactory implements SchemeFactory { + public delete_table_column_statistics_resultTupleScheme getScheme() { + return new delete_table_column_statistics_resultTupleScheme(); + } + } + + private static class delete_table_column_statistics_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, delete_table_column_statistics_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + if (struct.isSetO1()) { + optionals.set(1); + } + if (struct.isSetO2()) { + optionals.set(2); + } + if (struct.isSetO3()) { + optionals.set(3); + } + if (struct.isSetO4()) { + optionals.set(4); + } + oprot.writeBitSet(optionals, 5); + if (struct.isSetSuccess()) { + oprot.writeBool(struct.success); + } + if (struct.isSetO1()) { + struct.o1.write(oprot); + } + if (struct.isSetO2()) { + struct.o2.write(oprot); + } + if (struct.isSetO3()) { + struct.o3.write(oprot); + } + if (struct.isSetO4()) { + struct.o4.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, delete_table_column_statistics_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(5); + if (incoming.get(0)) { + struct.success = iprot.readBool(); + struct.setSuccessIsSet(true); + } + if (incoming.get(1)) { + struct.o1 = new NoSuchObjectException(); + struct.o1.read(iprot); + struct.setO1IsSet(true); + } + if (incoming.get(2)) { + struct.o2 = new MetaException(); + struct.o2.read(iprot); + struct.setO2IsSet(true); + } + if (incoming.get(3)) { + struct.o3 = new InvalidObjectException(); + struct.o3.read(iprot); + struct.setO3IsSet(true); + } + if (incoming.get(4)) { + struct.o4 = new InvalidInputException(); + struct.o4.read(iprot); + struct.setO4IsSet(true); + } + } + } + + } + + public static class streamingStatus_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("streamingStatus_args"); + + private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("db_name", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("table_name", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new streamingStatus_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new streamingStatus_argsTupleSchemeFactory()); + } + + private String db_name; // required + private String table_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 { + DB_NAME((short)1, "db_name"), + TABLE_NAME((short)2, "table_name"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // DB_NAME + return DB_NAME; + case 2: // 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 IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("db_name", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("table_name", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(streamingStatus_args.class, metaDataMap); + } + + public streamingStatus_args() { + } + + public streamingStatus_args( + String db_name, + String table_name) + { + this(); + this.db_name = db_name; + this.table_name = table_name; + } + + /** + * Performs a deep copy on other. + */ + public streamingStatus_args(streamingStatus_args other) { + if (other.isSetDb_name()) { + this.db_name = other.db_name; + } + if (other.isSetTable_name()) { + this.table_name = other.table_name; + } + } + + public streamingStatus_args deepCopy() { + return new streamingStatus_args(this); + } + + @Override + public void clear() { + this.db_name = null; + this.table_name = null; + } + + public String getDb_name() { + return this.db_name; + } + + public void setDb_name(String db_name) { + this.db_name = db_name; + } + + public void unsetDb_name() { + this.db_name = null; + } + + /** Returns true if field db_name is set (has been assigned a value) and false otherwise */ + public boolean isSetDb_name() { + return this.db_name != null; + } + + public void setDb_nameIsSet(boolean value) { + if (!value) { + this.db_name = null; + } + } + + public String getTable_name() { + return this.table_name; + } + + public void setTable_name(String table_name) { + this.table_name = table_name; + } + + public void unsetTable_name() { + this.table_name = null; + } + + /** Returns true if field table_name is set (has been assigned a value) and false otherwise */ + public boolean isSetTable_name() { + return this.table_name != null; + } + + public void setTable_nameIsSet(boolean value) { + if (!value) { + this.table_name = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case DB_NAME: + if (value == null) { + unsetDb_name(); + } else { + setDb_name((String)value); + } + break; + + case TABLE_NAME: + if (value == null) { + unsetTable_name(); + } else { + setTable_name((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case DB_NAME: + return getDb_name(); + + case TABLE_NAME: + return getTable_name(); + + } + throw new 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 IllegalArgumentException(); + } + + switch (field) { + case DB_NAME: + return isSetDb_name(); + case TABLE_NAME: + return isSetTable_name(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof streamingStatus_args) + return this.equals((streamingStatus_args)that); + return false; + } + + public boolean equals(streamingStatus_args that) { + if (that == null) + return false; + + boolean this_present_db_name = true && this.isSetDb_name(); + boolean that_present_db_name = true && that.isSetDb_name(); + if (this_present_db_name || that_present_db_name) { + if (!(this_present_db_name && that_present_db_name)) + return false; + if (!this.db_name.equals(that.db_name)) + return false; + } + + boolean this_present_table_name = true && this.isSetTable_name(); + boolean that_present_table_name = true && that.isSetTable_name(); + if (this_present_table_name || that_present_table_name) { + if (!(this_present_table_name && that_present_table_name)) + return false; + if (!this.table_name.equals(that.table_name)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_db_name = true && (isSetDb_name()); + builder.append(present_db_name); + if (present_db_name) + builder.append(db_name); + + boolean present_table_name = true && (isSetTable_name()); + builder.append(present_table_name); + if (present_table_name) + builder.append(table_name); + + return builder.toHashCode(); + } + + public int compareTo(streamingStatus_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + streamingStatus_args typedOther = (streamingStatus_args)other; + + lastComparison = Boolean.valueOf(isSetDb_name()).compareTo(typedOther.isSetDb_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetDb_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.db_name, typedOther.db_name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTable_name()).compareTo(typedOther.isSetTable_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTable_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table_name, typedOther.table_name); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("streamingStatus_args("); + boolean first = true; + + sb.append("db_name:"); + if (this.db_name == null) { + sb.append("null"); + } else { + sb.append(this.db_name); + } + first = false; + if (!first) sb.append(", "); + sb.append("table_name:"); + if (this.table_name == null) { + sb.append("null"); + } else { + sb.append(this.table_name); + } + 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, 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 streamingStatus_argsStandardSchemeFactory implements SchemeFactory { + public streamingStatus_argsStandardScheme getScheme() { + return new streamingStatus_argsStandardScheme(); + } + } + + private static class streamingStatus_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, streamingStatus_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: // DB_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.db_name = iprot.readString(); + struct.setDb_nameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // TABLE_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.table_name = iprot.readString(); + struct.setTable_nameIsSet(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(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, streamingStatus_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.db_name != null) { + oprot.writeFieldBegin(DB_NAME_FIELD_DESC); + oprot.writeString(struct.db_name); + oprot.writeFieldEnd(); + } + if (struct.table_name != null) { + oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC); + oprot.writeString(struct.table_name); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class streamingStatus_argsTupleSchemeFactory implements SchemeFactory { + public streamingStatus_argsTupleScheme getScheme() { + return new streamingStatus_argsTupleScheme(); + } + } + + private static class streamingStatus_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, streamingStatus_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetDb_name()) { + optionals.set(0); + } + if (struct.isSetTable_name()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetDb_name()) { + oprot.writeString(struct.db_name); + } + if (struct.isSetTable_name()) { + oprot.writeString(struct.table_name); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, streamingStatus_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.db_name = iprot.readString(); + struct.setDb_nameIsSet(true); + } + if (incoming.get(1)) { + struct.table_name = iprot.readString(); + struct.setTable_nameIsSet(true); + } + } + } + + } + + public static class streamingStatus_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("streamingStatus_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.MAP, (short)0); + private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField O2_FIELD_DESC = new org.apache.thrift.protocol.TField("o2", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField O3_FIELD_DESC = new org.apache.thrift.protocol.TField("o3", org.apache.thrift.protocol.TType.STRUCT, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new streamingStatus_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new streamingStatus_resultTupleSchemeFactory()); + } + + private Map success; // required + private InvalidInputException o1; // required + private InvalidObjectException o2; // required + private MetaException o3; // 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"), + O1((short)1, "o1"), + O2((short)2, "o2"), + O3((short)3, "o3"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + case 1: // O1 + return O1; + case 2: // O2 + return O2; + case 3: // O3 + return O3; + 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 IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new 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.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); + tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O2, new org.apache.thrift.meta_data.FieldMetaData("o2", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O3, new org.apache.thrift.meta_data.FieldMetaData("o3", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(streamingStatus_result.class, metaDataMap); + } + + public streamingStatus_result() { + } + + public streamingStatus_result( + Map success, + InvalidInputException o1, + InvalidObjectException o2, + MetaException o3) + { + this(); + this.success = success; + this.o1 = o1; + this.o2 = o2; + this.o3 = o3; + } + + /** + * Performs a deep copy on other. + */ + public streamingStatus_result(streamingStatus_result other) { + if (other.isSetSuccess()) { + Map __this__success = new HashMap(); + for (Map.Entry other_element : other.success.entrySet()) { + + String other_element_key = other_element.getKey(); + String other_element_value = other_element.getValue(); + + String __this__success_copy_key = other_element_key; + + String __this__success_copy_value = other_element_value; + + __this__success.put(__this__success_copy_key, __this__success_copy_value); + } + this.success = __this__success; + } + if (other.isSetO1()) { + this.o1 = new InvalidInputException(other.o1); + } + if (other.isSetO2()) { + this.o2 = new InvalidObjectException(other.o2); + } + if (other.isSetO3()) { + this.o3 = new MetaException(other.o3); + } + } + + public streamingStatus_result deepCopy() { + return new streamingStatus_result(this); + } + + @Override + public void clear() { + this.success = null; + this.o1 = null; + this.o2 = null; + this.o3 = null; + } + + public int getSuccessSize() { + return (this.success == null) ? 0 : this.success.size(); + } + + public void putToSuccess(String key, String val) { + if (this.success == null) { + this.success = new HashMap(); + } + this.success.put(key, val); + } + + public Map getSuccess() { + return this.success; + } + + public void setSuccess(Map success) { + this.success = success; + } + + 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 InvalidInputException getO1() { + return this.o1; + } + + public void setO1(InvalidInputException o1) { + this.o1 = o1; + } + + public void unsetO1() { + this.o1 = null; + } + + /** Returns true if field o1 is set (has been assigned a value) and false otherwise */ + public boolean isSetO1() { + return this.o1 != null; + } + + public void setO1IsSet(boolean value) { + if (!value) { + this.o1 = null; + } + } + + public InvalidObjectException getO2() { + return this.o2; + } + + public void setO2(InvalidObjectException o2) { + this.o2 = o2; + } + + public void unsetO2() { + this.o2 = null; + } + + /** Returns true if field o2 is set (has been assigned a value) and false otherwise */ + public boolean isSetO2() { + return this.o2 != null; + } + + public void setO2IsSet(boolean value) { + if (!value) { + this.o2 = null; + } + } + + public MetaException getO3() { + return this.o3; + } + + public void setO3(MetaException o3) { + this.o3 = o3; + } + + public void unsetO3() { + this.o3 = null; + } + + /** Returns true if field o3 is set (has been assigned a value) and false otherwise */ + public boolean isSetO3() { + return this.o3 != null; + } + + public void setO3IsSet(boolean value) { + if (!value) { + this.o3 = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((Map)value); + } + break; + + case O1: + if (value == null) { + unsetO1(); + } else { + setO1((InvalidInputException)value); + } + break; + + case O2: + if (value == null) { + unsetO2(); + } else { + setO2((InvalidObjectException)value); + } + break; + + case O3: + if (value == null) { + unsetO3(); + } else { + setO3((MetaException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + case O1: + return getO1(); + + case O2: + return getO2(); + + case O3: + return getO3(); + + } + throw new 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 IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + case O1: + return isSetO1(); + case O2: + return isSetO2(); + case O3: + return isSetO3(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof streamingStatus_result) + return this.equals((streamingStatus_result)that); + return false; + } + + public boolean equals(streamingStatus_result that) { + if (that == null) + return false; + + 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_o1 = true && this.isSetO1(); + boolean that_present_o1 = true && that.isSetO1(); + if (this_present_o1 || that_present_o1) { + if (!(this_present_o1 && that_present_o1)) + return false; + if (!this.o1.equals(that.o1)) + return false; + } + + boolean this_present_o2 = true && this.isSetO2(); + boolean that_present_o2 = true && that.isSetO2(); + if (this_present_o2 || that_present_o2) { + if (!(this_present_o2 && that_present_o2)) + return false; + if (!this.o2.equals(that.o2)) + return false; + } + + boolean this_present_o3 = true && this.isSetO3(); + boolean that_present_o3 = true && that.isSetO3(); + if (this_present_o3 || that_present_o3) { + if (!(this_present_o3 && that_present_o3)) + return false; + if (!this.o3.equals(that.o3)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + boolean present_o1 = true && (isSetO1()); + builder.append(present_o1); + if (present_o1) + builder.append(o1); + + boolean present_o2 = true && (isSetO2()); + builder.append(present_o2); + if (present_o2) + builder.append(o2); + + boolean present_o3 = true && (isSetO3()); + builder.append(present_o3); + if (present_o3) + builder.append(o3); + + return builder.toHashCode(); + } + + public int compareTo(streamingStatus_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + streamingStatus_result typedOther = (streamingStatus_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO1()).compareTo(typedOther.isSetO1()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO1()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, typedOther.o1); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO2()).compareTo(typedOther.isSetO2()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO2()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o2, typedOther.o2); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO3()).compareTo(typedOther.isSetO3()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO3()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o3, typedOther.o3); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("streamingStatus_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("o1:"); + if (this.o1 == null) { + sb.append("null"); + } else { + sb.append(this.o1); + } + first = false; + if (!first) sb.append(", "); + sb.append("o2:"); + if (this.o2 == null) { + sb.append("null"); + } else { + sb.append(this.o2); + } + first = false; + if (!first) sb.append(", "); + sb.append("o3:"); + if (this.o3 == null) { + sb.append("null"); + } else { + sb.append(this.o3); + } + 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, 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 streamingStatus_resultStandardSchemeFactory implements SchemeFactory { + public streamingStatus_resultStandardScheme getScheme() { + return new streamingStatus_resultStandardScheme(); + } + } + + private static class streamingStatus_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, streamingStatus_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.MAP) { + { + org.apache.thrift.protocol.TMap _map602 = iprot.readMapBegin(); + struct.success = new HashMap(2*_map602.size); + for (int _i603 = 0; _i603 < _map602.size; ++_i603) + { + String _key604; // required + String _val605; // required + _key604 = iprot.readString(); + _val605 = iprot.readString(); + struct.success.put(_key604, _val605); + } + iprot.readMapEnd(); + } + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 1: // O1 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o1 = new InvalidInputException(); + struct.o1.read(iprot); + struct.setO1IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // O2 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o2 = new InvalidObjectException(); + struct.o2.read(iprot); + struct.setO2IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // O3 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o3 = new MetaException(); + struct.o3.read(iprot); + struct.setO3IsSet(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(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, streamingStatus_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.success.size())); + for (Map.Entry _iter606 : struct.success.entrySet()) + { + oprot.writeString(_iter606.getKey()); + oprot.writeString(_iter606.getValue()); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.o1 != null) { + oprot.writeFieldBegin(O1_FIELD_DESC); + struct.o1.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o2 != null) { + oprot.writeFieldBegin(O2_FIELD_DESC); + struct.o2.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o3 != null) { + oprot.writeFieldBegin(O3_FIELD_DESC); + struct.o3.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class streamingStatus_resultTupleSchemeFactory implements SchemeFactory { + public streamingStatus_resultTupleScheme getScheme() { + return new streamingStatus_resultTupleScheme(); + } + } + + private static class streamingStatus_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, streamingStatus_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + if (struct.isSetO1()) { + optionals.set(1); + } + if (struct.isSetO2()) { + optionals.set(2); + } + if (struct.isSetO3()) { + optionals.set(3); + } + oprot.writeBitSet(optionals, 4); + if (struct.isSetSuccess()) { + { + oprot.writeI32(struct.success.size()); + for (Map.Entry _iter607 : struct.success.entrySet()) + { + oprot.writeString(_iter607.getKey()); + oprot.writeString(_iter607.getValue()); + } + } + } + if (struct.isSetO1()) { + struct.o1.write(oprot); + } + if (struct.isSetO2()) { + struct.o2.write(oprot); + } + if (struct.isSetO3()) { + struct.o3.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, streamingStatus_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(4); + if (incoming.get(0)) { + { + org.apache.thrift.protocol.TMap _map608 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.success = new HashMap(2*_map608.size); + for (int _i609 = 0; _i609 < _map608.size; ++_i609) + { + String _key610; // required + String _val611; // required + _key610 = iprot.readString(); + _val611 = iprot.readString(); + struct.success.put(_key610, _val611); + } + } + struct.setSuccessIsSet(true); + } + if (incoming.get(1)) { + struct.o1 = new InvalidInputException(); + struct.o1.read(iprot); + struct.setO1IsSet(true); + } + if (incoming.get(2)) { + struct.o2 = new InvalidObjectException(); + struct.o2.read(iprot); + struct.setO2IsSet(true); + } + if (incoming.get(3)) { + struct.o3 = new MetaException(); + struct.o3.read(iprot); + struct.setO3IsSet(true); + } + } + } + + } + + public static class enableStreaming_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("enableStreaming_args"); + + private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("db_name", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("table_name", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField STREAMING_TMP_DIR_FIELD_DESC = new org.apache.thrift.protocol.TField("streaming_tmp_dir", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField PARTITION_PATH_FIELD_DESC = new org.apache.thrift.protocol.TField("partition_path", org.apache.thrift.protocol.TType.STRING, (short)4); + private static final org.apache.thrift.protocol.TField PARTITION_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("partition_val", org.apache.thrift.protocol.TType.STRING, (short)5); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new enableStreaming_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new enableStreaming_argsTupleSchemeFactory()); + } + + private String db_name; // required + private String table_name; // required + private String streaming_tmp_dir; // required + private String partition_path; // required + private String partition_val; // 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 { + DB_NAME((short)1, "db_name"), + TABLE_NAME((short)2, "table_name"), + STREAMING_TMP_DIR((short)3, "streaming_tmp_dir"), + PARTITION_PATH((short)4, "partition_path"), + PARTITION_VAL((short)5, "partition_val"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // DB_NAME + return DB_NAME; + case 2: // TABLE_NAME + return TABLE_NAME; + case 3: // STREAMING_TMP_DIR + return STREAMING_TMP_DIR; + case 4: // PARTITION_PATH + return PARTITION_PATH; + case 5: // PARTITION_VAL + return PARTITION_VAL; + 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 IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("db_name", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("table_name", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.STREAMING_TMP_DIR, new org.apache.thrift.meta_data.FieldMetaData("streaming_tmp_dir", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.PARTITION_PATH, new org.apache.thrift.meta_data.FieldMetaData("partition_path", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.PARTITION_VAL, new org.apache.thrift.meta_data.FieldMetaData("partition_val", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(enableStreaming_args.class, metaDataMap); + } + + public enableStreaming_args() { + } + + public enableStreaming_args( + String db_name, + String table_name, + String streaming_tmp_dir, + String partition_path, + String partition_val) + { + this(); + this.db_name = db_name; + this.table_name = table_name; + this.streaming_tmp_dir = streaming_tmp_dir; + this.partition_path = partition_path; + this.partition_val = partition_val; + } + + /** + * Performs a deep copy on other. + */ + public enableStreaming_args(enableStreaming_args other) { + if (other.isSetDb_name()) { + this.db_name = other.db_name; + } + if (other.isSetTable_name()) { + this.table_name = other.table_name; + } + if (other.isSetStreaming_tmp_dir()) { + this.streaming_tmp_dir = other.streaming_tmp_dir; + } + if (other.isSetPartition_path()) { + this.partition_path = other.partition_path; + } + if (other.isSetPartition_val()) { + this.partition_val = other.partition_val; + } + } + + public enableStreaming_args deepCopy() { + return new enableStreaming_args(this); + } + + @Override + public void clear() { + this.db_name = null; + this.table_name = null; + this.streaming_tmp_dir = null; + this.partition_path = null; + this.partition_val = null; + } + + public String getDb_name() { + return this.db_name; + } + + public void setDb_name(String db_name) { + this.db_name = db_name; + } + + public void unsetDb_name() { + this.db_name = null; + } + + /** Returns true if field db_name is set (has been assigned a value) and false otherwise */ + public boolean isSetDb_name() { + return this.db_name != null; + } + + public void setDb_nameIsSet(boolean value) { + if (!value) { + this.db_name = null; + } + } + + public String getTable_name() { + return this.table_name; + } + + public void setTable_name(String table_name) { + this.table_name = table_name; + } + + public void unsetTable_name() { + this.table_name = null; + } + + /** Returns true if field table_name is set (has been assigned a value) and false otherwise */ + public boolean isSetTable_name() { + return this.table_name != null; + } + + public void setTable_nameIsSet(boolean value) { + if (!value) { + this.table_name = null; + } + } + + public String getStreaming_tmp_dir() { + return this.streaming_tmp_dir; + } + + public void setStreaming_tmp_dir(String streaming_tmp_dir) { + this.streaming_tmp_dir = streaming_tmp_dir; + } + + public void unsetStreaming_tmp_dir() { + this.streaming_tmp_dir = null; + } + + /** Returns true if field streaming_tmp_dir is set (has been assigned a value) and false otherwise */ + public boolean isSetStreaming_tmp_dir() { + return this.streaming_tmp_dir != null; + } + + public void setStreaming_tmp_dirIsSet(boolean value) { + if (!value) { + this.streaming_tmp_dir = null; + } + } + + public String getPartition_path() { + return this.partition_path; + } + + public void setPartition_path(String partition_path) { + this.partition_path = partition_path; + } + + public void unsetPartition_path() { + this.partition_path = null; + } + + /** Returns true if field partition_path is set (has been assigned a value) and false otherwise */ + public boolean isSetPartition_path() { + return this.partition_path != null; + } + + public void setPartition_pathIsSet(boolean value) { + if (!value) { + this.partition_path = null; + } + } + + public String getPartition_val() { + return this.partition_val; + } + + public void setPartition_val(String partition_val) { + this.partition_val = partition_val; + } + + public void unsetPartition_val() { + this.partition_val = null; + } + + /** Returns true if field partition_val is set (has been assigned a value) and false otherwise */ + public boolean isSetPartition_val() { + return this.partition_val != null; + } + + public void setPartition_valIsSet(boolean value) { + if (!value) { + this.partition_val = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case DB_NAME: + if (value == null) { + unsetDb_name(); + } else { + setDb_name((String)value); + } + break; + + case TABLE_NAME: + if (value == null) { + unsetTable_name(); + } else { + setTable_name((String)value); + } + break; + + case STREAMING_TMP_DIR: + if (value == null) { + unsetStreaming_tmp_dir(); + } else { + setStreaming_tmp_dir((String)value); + } + break; + + case PARTITION_PATH: + if (value == null) { + unsetPartition_path(); + } else { + setPartition_path((String)value); + } + break; + + case PARTITION_VAL: + if (value == null) { + unsetPartition_val(); + } else { + setPartition_val((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case DB_NAME: + return getDb_name(); + + case TABLE_NAME: + return getTable_name(); + + case STREAMING_TMP_DIR: + return getStreaming_tmp_dir(); + + case PARTITION_PATH: + return getPartition_path(); + + case PARTITION_VAL: + return getPartition_val(); + + } + throw new 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 IllegalArgumentException(); + } + + switch (field) { + case DB_NAME: + return isSetDb_name(); + case TABLE_NAME: + return isSetTable_name(); + case STREAMING_TMP_DIR: + return isSetStreaming_tmp_dir(); + case PARTITION_PATH: + return isSetPartition_path(); + case PARTITION_VAL: + return isSetPartition_val(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof enableStreaming_args) + return this.equals((enableStreaming_args)that); + return false; + } + + public boolean equals(enableStreaming_args that) { + if (that == null) + return false; + + boolean this_present_db_name = true && this.isSetDb_name(); + boolean that_present_db_name = true && that.isSetDb_name(); + if (this_present_db_name || that_present_db_name) { + if (!(this_present_db_name && that_present_db_name)) + return false; + if (!this.db_name.equals(that.db_name)) + return false; + } + + boolean this_present_table_name = true && this.isSetTable_name(); + boolean that_present_table_name = true && that.isSetTable_name(); + if (this_present_table_name || that_present_table_name) { + if (!(this_present_table_name && that_present_table_name)) + return false; + if (!this.table_name.equals(that.table_name)) + return false; + } + + boolean this_present_streaming_tmp_dir = true && this.isSetStreaming_tmp_dir(); + boolean that_present_streaming_tmp_dir = true && that.isSetStreaming_tmp_dir(); + if (this_present_streaming_tmp_dir || that_present_streaming_tmp_dir) { + if (!(this_present_streaming_tmp_dir && that_present_streaming_tmp_dir)) + return false; + if (!this.streaming_tmp_dir.equals(that.streaming_tmp_dir)) + return false; + } + + boolean this_present_partition_path = true && this.isSetPartition_path(); + boolean that_present_partition_path = true && that.isSetPartition_path(); + if (this_present_partition_path || that_present_partition_path) { + if (!(this_present_partition_path && that_present_partition_path)) + return false; + if (!this.partition_path.equals(that.partition_path)) + return false; + } + + boolean this_present_partition_val = true && this.isSetPartition_val(); + boolean that_present_partition_val = true && that.isSetPartition_val(); + if (this_present_partition_val || that_present_partition_val) { + if (!(this_present_partition_val && that_present_partition_val)) + return false; + if (!this.partition_val.equals(that.partition_val)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_db_name = true && (isSetDb_name()); + builder.append(present_db_name); + if (present_db_name) + builder.append(db_name); + + boolean present_table_name = true && (isSetTable_name()); + builder.append(present_table_name); + if (present_table_name) + builder.append(table_name); + + boolean present_streaming_tmp_dir = true && (isSetStreaming_tmp_dir()); + builder.append(present_streaming_tmp_dir); + if (present_streaming_tmp_dir) + builder.append(streaming_tmp_dir); + + boolean present_partition_path = true && (isSetPartition_path()); + builder.append(present_partition_path); + if (present_partition_path) + builder.append(partition_path); + + boolean present_partition_val = true && (isSetPartition_val()); + builder.append(present_partition_val); + if (present_partition_val) + builder.append(partition_val); + + return builder.toHashCode(); + } + + public int compareTo(enableStreaming_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + enableStreaming_args typedOther = (enableStreaming_args)other; + + lastComparison = Boolean.valueOf(isSetDb_name()).compareTo(typedOther.isSetDb_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetDb_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.db_name, typedOther.db_name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTable_name()).compareTo(typedOther.isSetTable_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTable_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table_name, typedOther.table_name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetStreaming_tmp_dir()).compareTo(typedOther.isSetStreaming_tmp_dir()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStreaming_tmp_dir()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.streaming_tmp_dir, typedOther.streaming_tmp_dir); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetPartition_path()).compareTo(typedOther.isSetPartition_path()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetPartition_path()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.partition_path, typedOther.partition_path); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetPartition_val()).compareTo(typedOther.isSetPartition_val()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetPartition_val()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.partition_val, typedOther.partition_val); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("enableStreaming_args("); + boolean first = true; + + sb.append("db_name:"); + if (this.db_name == null) { + sb.append("null"); + } else { + sb.append(this.db_name); + } + first = false; + if (!first) sb.append(", "); + sb.append("table_name:"); + if (this.table_name == null) { + sb.append("null"); + } else { + sb.append(this.table_name); + } + first = false; + if (!first) sb.append(", "); + sb.append("streaming_tmp_dir:"); + if (this.streaming_tmp_dir == null) { + sb.append("null"); + } else { + sb.append(this.streaming_tmp_dir); + } + first = false; + if (!first) sb.append(", "); + sb.append("partition_path:"); + if (this.partition_path == null) { + sb.append("null"); + } else { + sb.append(this.partition_path); + } + first = false; + if (!first) sb.append(", "); + sb.append("partition_val:"); + if (this.partition_val == null) { + sb.append("null"); + } else { + sb.append(this.partition_val); + } + 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, 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 enableStreaming_argsStandardSchemeFactory implements SchemeFactory { + public enableStreaming_argsStandardScheme getScheme() { + return new enableStreaming_argsStandardScheme(); + } + } + + private static class enableStreaming_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, enableStreaming_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: // DB_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.db_name = iprot.readString(); + struct.setDb_nameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // TABLE_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.table_name = iprot.readString(); + struct.setTable_nameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // STREAMING_TMP_DIR + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.streaming_tmp_dir = iprot.readString(); + struct.setStreaming_tmp_dirIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // PARTITION_PATH + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.partition_path = iprot.readString(); + struct.setPartition_pathIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // PARTITION_VAL + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.partition_val = iprot.readString(); + struct.setPartition_valIsSet(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(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, enableStreaming_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.db_name != null) { + oprot.writeFieldBegin(DB_NAME_FIELD_DESC); + oprot.writeString(struct.db_name); + oprot.writeFieldEnd(); + } + if (struct.table_name != null) { + oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC); + oprot.writeString(struct.table_name); + oprot.writeFieldEnd(); + } + if (struct.streaming_tmp_dir != null) { + oprot.writeFieldBegin(STREAMING_TMP_DIR_FIELD_DESC); + oprot.writeString(struct.streaming_tmp_dir); + oprot.writeFieldEnd(); + } + if (struct.partition_path != null) { + oprot.writeFieldBegin(PARTITION_PATH_FIELD_DESC); + oprot.writeString(struct.partition_path); + oprot.writeFieldEnd(); + } + if (struct.partition_val != null) { + oprot.writeFieldBegin(PARTITION_VAL_FIELD_DESC); + oprot.writeString(struct.partition_val); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class enableStreaming_argsTupleSchemeFactory implements SchemeFactory { + public enableStreaming_argsTupleScheme getScheme() { + return new enableStreaming_argsTupleScheme(); + } + } + + private static class enableStreaming_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, enableStreaming_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetDb_name()) { + optionals.set(0); + } + if (struct.isSetTable_name()) { + optionals.set(1); + } + if (struct.isSetStreaming_tmp_dir()) { + optionals.set(2); + } + if (struct.isSetPartition_path()) { + optionals.set(3); + } + if (struct.isSetPartition_val()) { + optionals.set(4); + } + oprot.writeBitSet(optionals, 5); + if (struct.isSetDb_name()) { + oprot.writeString(struct.db_name); + } + if (struct.isSetTable_name()) { + oprot.writeString(struct.table_name); + } + if (struct.isSetStreaming_tmp_dir()) { + oprot.writeString(struct.streaming_tmp_dir); + } + if (struct.isSetPartition_path()) { + oprot.writeString(struct.partition_path); + } + if (struct.isSetPartition_val()) { + oprot.writeString(struct.partition_val); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, enableStreaming_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(5); + if (incoming.get(0)) { + struct.db_name = iprot.readString(); + struct.setDb_nameIsSet(true); + } + if (incoming.get(1)) { + struct.table_name = iprot.readString(); + struct.setTable_nameIsSet(true); + } + if (incoming.get(2)) { + struct.streaming_tmp_dir = iprot.readString(); + struct.setStreaming_tmp_dirIsSet(true); + } + if (incoming.get(3)) { + struct.partition_path = iprot.readString(); + struct.setPartition_pathIsSet(true); + } + if (incoming.get(4)) { + struct.partition_val = iprot.readString(); + struct.setPartition_valIsSet(true); + } + } + } + + } + + public static class enableStreaming_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("enableStreaming_result"); + + private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField O2_FIELD_DESC = new org.apache.thrift.protocol.TField("o2", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField O3_FIELD_DESC = new org.apache.thrift.protocol.TField("o3", org.apache.thrift.protocol.TType.STRUCT, (short)3); + private static final org.apache.thrift.protocol.TField O4_FIELD_DESC = new org.apache.thrift.protocol.TField("o4", org.apache.thrift.protocol.TType.STRUCT, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new enableStreaming_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new enableStreaming_resultTupleSchemeFactory()); + } + + private InvalidInputException o1; // required + private InvalidObjectException o2; // required + private InvalidOperationException o3; // required + private MetaException o4; // 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 { + O1((short)1, "o1"), + O2((short)2, "o2"), + O3((short)3, "o3"), + O4((short)4, "o4"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // O1 + return O1; + case 2: // O2 + return O2; + case 3: // O3 + return O3; + case 4: // O4 + return O4; + 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 IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O2, new org.apache.thrift.meta_data.FieldMetaData("o2", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O3, new org.apache.thrift.meta_data.FieldMetaData("o3", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O4, new org.apache.thrift.meta_data.FieldMetaData("o4", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(enableStreaming_result.class, metaDataMap); + } + + public enableStreaming_result() { + } + + public enableStreaming_result( + InvalidInputException o1, + InvalidObjectException o2, + InvalidOperationException o3, + MetaException o4) + { + this(); + this.o1 = o1; + this.o2 = o2; + this.o3 = o3; + this.o4 = o4; + } + + /** + * Performs a deep copy on other. + */ + public enableStreaming_result(enableStreaming_result other) { + if (other.isSetO1()) { + this.o1 = new InvalidInputException(other.o1); + } + if (other.isSetO2()) { + this.o2 = new InvalidObjectException(other.o2); + } + if (other.isSetO3()) { + this.o3 = new InvalidOperationException(other.o3); + } + if (other.isSetO4()) { + this.o4 = new MetaException(other.o4); + } + } + + public enableStreaming_result deepCopy() { + return new enableStreaming_result(this); + } + + @Override + public void clear() { + this.o1 = null; + this.o2 = null; + this.o3 = null; + this.o4 = null; + } + + public InvalidInputException getO1() { + return this.o1; + } + + public void setO1(InvalidInputException o1) { + this.o1 = o1; + } + + public void unsetO1() { + this.o1 = null; + } + + /** Returns true if field o1 is set (has been assigned a value) and false otherwise */ + public boolean isSetO1() { + return this.o1 != null; + } + + public void setO1IsSet(boolean value) { + if (!value) { + this.o1 = null; + } + } + + public InvalidObjectException getO2() { + return this.o2; + } + + public void setO2(InvalidObjectException o2) { + this.o2 = o2; + } + + public void unsetO2() { + this.o2 = null; + } + + /** Returns true if field o2 is set (has been assigned a value) and false otherwise */ + public boolean isSetO2() { + return this.o2 != null; + } + + public void setO2IsSet(boolean value) { + if (!value) { + this.o2 = null; + } + } + + public InvalidOperationException getO3() { + return this.o3; + } + + public void setO3(InvalidOperationException o3) { + this.o3 = o3; + } + + public void unsetO3() { + this.o3 = null; + } + + /** Returns true if field o3 is set (has been assigned a value) and false otherwise */ + public boolean isSetO3() { + return this.o3 != null; + } + + public void setO3IsSet(boolean value) { + if (!value) { + this.o3 = null; + } + } + + public MetaException getO4() { + return this.o4; + } + + public void setO4(MetaException o4) { + this.o4 = o4; + } + + public void unsetO4() { + this.o4 = null; + } + + /** Returns true if field o4 is set (has been assigned a value) and false otherwise */ + public boolean isSetO4() { + return this.o4 != null; + } + + public void setO4IsSet(boolean value) { + if (!value) { + this.o4 = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case O1: + if (value == null) { + unsetO1(); + } else { + setO1((InvalidInputException)value); + } + break; + + case O2: + if (value == null) { + unsetO2(); + } else { + setO2((InvalidObjectException)value); + } + break; + + case O3: + if (value == null) { + unsetO3(); + } else { + setO3((InvalidOperationException)value); + } + break; + + case O4: + if (value == null) { + unsetO4(); + } else { + setO4((MetaException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case O1: + return getO1(); + + case O2: + return getO2(); + + case O3: + return getO3(); + + case O4: + return getO4(); + + } + throw new 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 IllegalArgumentException(); + } + + switch (field) { + case O1: + return isSetO1(); + case O2: + return isSetO2(); + case O3: + return isSetO3(); + case O4: + return isSetO4(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof enableStreaming_result) + return this.equals((enableStreaming_result)that); + return false; + } + + public boolean equals(enableStreaming_result that) { + if (that == null) + return false; + + boolean this_present_o1 = true && this.isSetO1(); + boolean that_present_o1 = true && that.isSetO1(); + if (this_present_o1 || that_present_o1) { + if (!(this_present_o1 && that_present_o1)) + return false; + if (!this.o1.equals(that.o1)) + return false; + } + + boolean this_present_o2 = true && this.isSetO2(); + boolean that_present_o2 = true && that.isSetO2(); + if (this_present_o2 || that_present_o2) { + if (!(this_present_o2 && that_present_o2)) + return false; + if (!this.o2.equals(that.o2)) + return false; + } + + boolean this_present_o3 = true && this.isSetO3(); + boolean that_present_o3 = true && that.isSetO3(); + if (this_present_o3 || that_present_o3) { + if (!(this_present_o3 && that_present_o3)) + return false; + if (!this.o3.equals(that.o3)) + return false; + } + + boolean this_present_o4 = true && this.isSetO4(); + boolean that_present_o4 = true && that.isSetO4(); + if (this_present_o4 || that_present_o4) { + if (!(this_present_o4 && that_present_o4)) + return false; + if (!this.o4.equals(that.o4)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_o1 = true && (isSetO1()); + builder.append(present_o1); + if (present_o1) + builder.append(o1); + + boolean present_o2 = true && (isSetO2()); + builder.append(present_o2); + if (present_o2) + builder.append(o2); + + boolean present_o3 = true && (isSetO3()); + builder.append(present_o3); + if (present_o3) + builder.append(o3); + + boolean present_o4 = true && (isSetO4()); + builder.append(present_o4); + if (present_o4) + builder.append(o4); + + return builder.toHashCode(); + } + + public int compareTo(enableStreaming_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + enableStreaming_result typedOther = (enableStreaming_result)other; + + lastComparison = Boolean.valueOf(isSetO1()).compareTo(typedOther.isSetO1()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO1()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, typedOther.o1); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO2()).compareTo(typedOther.isSetO2()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO2()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o2, typedOther.o2); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO3()).compareTo(typedOther.isSetO3()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO3()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o3, typedOther.o3); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO4()).compareTo(typedOther.isSetO4()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO4()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o4, typedOther.o4); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("enableStreaming_result("); + boolean first = true; + + sb.append("o1:"); + if (this.o1 == null) { + sb.append("null"); + } else { + sb.append(this.o1); + } + first = false; + if (!first) sb.append(", "); + sb.append("o2:"); + if (this.o2 == null) { + sb.append("null"); + } else { + sb.append(this.o2); + } + first = false; + if (!first) sb.append(", "); + sb.append("o3:"); + if (this.o3 == null) { + sb.append("null"); + } else { + sb.append(this.o3); + } + first = false; + if (!first) sb.append(", "); + sb.append("o4:"); + if (this.o4 == null) { + sb.append("null"); + } else { + sb.append(this.o4); + } + 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, 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 enableStreaming_resultStandardSchemeFactory implements SchemeFactory { + public enableStreaming_resultStandardScheme getScheme() { + return new enableStreaming_resultStandardScheme(); + } + } + + private static class enableStreaming_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, enableStreaming_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: // O1 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o1 = new InvalidInputException(); + struct.o1.read(iprot); + struct.setO1IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // O2 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o2 = new InvalidObjectException(); + struct.o2.read(iprot); + struct.setO2IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // O3 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o3 = new InvalidOperationException(); + struct.o3.read(iprot); + struct.setO3IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // O4 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o4 = new MetaException(); + struct.o4.read(iprot); + struct.setO4IsSet(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(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, enableStreaming_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.o1 != null) { + oprot.writeFieldBegin(O1_FIELD_DESC); + struct.o1.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o2 != null) { + oprot.writeFieldBegin(O2_FIELD_DESC); + struct.o2.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o3 != null) { + oprot.writeFieldBegin(O3_FIELD_DESC); + struct.o3.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o4 != null) { + oprot.writeFieldBegin(O4_FIELD_DESC); + struct.o4.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class enableStreaming_resultTupleSchemeFactory implements SchemeFactory { + public enableStreaming_resultTupleScheme getScheme() { + return new enableStreaming_resultTupleScheme(); + } + } + + private static class enableStreaming_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, enableStreaming_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetO1()) { + optionals.set(0); + } + if (struct.isSetO2()) { + optionals.set(1); + } + if (struct.isSetO3()) { + optionals.set(2); + } + if (struct.isSetO4()) { + optionals.set(3); + } + oprot.writeBitSet(optionals, 4); + if (struct.isSetO1()) { + struct.o1.write(oprot); + } + if (struct.isSetO2()) { + struct.o2.write(oprot); + } + if (struct.isSetO3()) { + struct.o3.write(oprot); + } + if (struct.isSetO4()) { + struct.o4.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, enableStreaming_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(4); + if (incoming.get(0)) { + struct.o1 = new InvalidInputException(); + struct.o1.read(iprot); + struct.setO1IsSet(true); + } + if (incoming.get(1)) { + struct.o2 = new InvalidObjectException(); + struct.o2.read(iprot); + struct.setO2IsSet(true); + } + if (incoming.get(2)) { + struct.o3 = new InvalidOperationException(); + struct.o3.read(iprot); + struct.setO3IsSet(true); + } + if (incoming.get(3)) { + struct.o4 = new MetaException(); + struct.o4.read(iprot); + struct.setO4IsSet(true); + } + } + } + + } + + public static class disableStreaming_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("disableStreaming_args"); + + private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("db_name", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("table_name", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new disableStreaming_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new disableStreaming_argsTupleSchemeFactory()); + } + + private String db_name; // required + private String table_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 { + DB_NAME((short)1, "db_name"), + TABLE_NAME((short)2, "table_name"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // DB_NAME + return DB_NAME; + case 2: // 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 IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("db_name", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("table_name", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(disableStreaming_args.class, metaDataMap); + } + + public disableStreaming_args() { + } + + public disableStreaming_args( + String db_name, + String table_name) + { + this(); + this.db_name = db_name; + this.table_name = table_name; + } + + /** + * Performs a deep copy on other. + */ + public disableStreaming_args(disableStreaming_args other) { + if (other.isSetDb_name()) { + this.db_name = other.db_name; + } + if (other.isSetTable_name()) { + this.table_name = other.table_name; + } + } + + public disableStreaming_args deepCopy() { + return new disableStreaming_args(this); + } + + @Override + public void clear() { + this.db_name = null; + this.table_name = null; + } + + public String getDb_name() { + return this.db_name; + } + + public void setDb_name(String db_name) { + this.db_name = db_name; + } + + public void unsetDb_name() { + this.db_name = null; + } + + /** Returns true if field db_name is set (has been assigned a value) and false otherwise */ + public boolean isSetDb_name() { + return this.db_name != null; + } + + public void setDb_nameIsSet(boolean value) { + if (!value) { + this.db_name = null; + } + } + + public String getTable_name() { + return this.table_name; + } + + public void setTable_name(String table_name) { + this.table_name = table_name; + } + + public void unsetTable_name() { + this.table_name = null; + } + + /** Returns true if field table_name is set (has been assigned a value) and false otherwise */ + public boolean isSetTable_name() { + return this.table_name != null; + } + + public void setTable_nameIsSet(boolean value) { + if (!value) { + this.table_name = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case DB_NAME: + if (value == null) { + unsetDb_name(); + } else { + setDb_name((String)value); + } + break; + + case TABLE_NAME: + if (value == null) { + unsetTable_name(); + } else { + setTable_name((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case DB_NAME: + return getDb_name(); + + case TABLE_NAME: + return getTable_name(); + + } + throw new 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 IllegalArgumentException(); + } + + switch (field) { + case DB_NAME: + return isSetDb_name(); + case TABLE_NAME: + return isSetTable_name(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof disableStreaming_args) + return this.equals((disableStreaming_args)that); + return false; + } + + public boolean equals(disableStreaming_args that) { + if (that == null) + return false; + + boolean this_present_db_name = true && this.isSetDb_name(); + boolean that_present_db_name = true && that.isSetDb_name(); + if (this_present_db_name || that_present_db_name) { + if (!(this_present_db_name && that_present_db_name)) + return false; + if (!this.db_name.equals(that.db_name)) + return false; + } + + boolean this_present_table_name = true && this.isSetTable_name(); + boolean that_present_table_name = true && that.isSetTable_name(); + if (this_present_table_name || that_present_table_name) { + if (!(this_present_table_name && that_present_table_name)) + return false; + if (!this.table_name.equals(that.table_name)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_db_name = true && (isSetDb_name()); + builder.append(present_db_name); + if (present_db_name) + builder.append(db_name); + + boolean present_table_name = true && (isSetTable_name()); + builder.append(present_table_name); + if (present_table_name) + builder.append(table_name); + + return builder.toHashCode(); + } + + public int compareTo(disableStreaming_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + disableStreaming_args typedOther = (disableStreaming_args)other; + + lastComparison = Boolean.valueOf(isSetDb_name()).compareTo(typedOther.isSetDb_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetDb_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.db_name, typedOther.db_name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTable_name()).compareTo(typedOther.isSetTable_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTable_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table_name, typedOther.table_name); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("disableStreaming_args("); + boolean first = true; + + sb.append("db_name:"); + if (this.db_name == null) { + sb.append("null"); + } else { + sb.append(this.db_name); + } + first = false; + if (!first) sb.append(", "); + sb.append("table_name:"); + if (this.table_name == null) { + sb.append("null"); + } else { + sb.append(this.table_name); + } + 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, 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 disableStreaming_argsStandardSchemeFactory implements SchemeFactory { + public disableStreaming_argsStandardScheme getScheme() { + return new disableStreaming_argsStandardScheme(); + } + } + + private static class disableStreaming_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, disableStreaming_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: // DB_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.db_name = iprot.readString(); + struct.setDb_nameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // TABLE_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.table_name = iprot.readString(); + struct.setTable_nameIsSet(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(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, disableStreaming_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.db_name != null) { + oprot.writeFieldBegin(DB_NAME_FIELD_DESC); + oprot.writeString(struct.db_name); + oprot.writeFieldEnd(); + } + if (struct.table_name != null) { + oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC); + oprot.writeString(struct.table_name); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class disableStreaming_argsTupleSchemeFactory implements SchemeFactory { + public disableStreaming_argsTupleScheme getScheme() { + return new disableStreaming_argsTupleScheme(); + } + } + + private static class disableStreaming_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, disableStreaming_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetDb_name()) { + optionals.set(0); + } + if (struct.isSetTable_name()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetDb_name()) { + oprot.writeString(struct.db_name); + } + if (struct.isSetTable_name()) { + oprot.writeString(struct.table_name); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, disableStreaming_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.db_name = iprot.readString(); + struct.setDb_nameIsSet(true); + } + if (incoming.get(1)) { + struct.table_name = iprot.readString(); + struct.setTable_nameIsSet(true); + } + } + } + + } + + public static class disableStreaming_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("disableStreaming_result"); + + private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField O2_FIELD_DESC = new org.apache.thrift.protocol.TField("o2", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField O3_FIELD_DESC = new org.apache.thrift.protocol.TField("o3", org.apache.thrift.protocol.TType.STRUCT, (short)3); + private static final org.apache.thrift.protocol.TField O4_FIELD_DESC = new org.apache.thrift.protocol.TField("o4", org.apache.thrift.protocol.TType.STRUCT, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new disableStreaming_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new disableStreaming_resultTupleSchemeFactory()); + } + + private InvalidInputException o1; // required + private InvalidObjectException o2; // required + private InvalidOperationException o3; // required + private MetaException o4; // 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 { + O1((short)1, "o1"), + O2((short)2, "o2"), + O3((short)3, "o3"), + O4((short)4, "o4"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // O1 + return O1; + case 2: // O2 + return O2; + case 3: // O3 + return O3; + case 4: // O4 + return O4; + 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 IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O2, new org.apache.thrift.meta_data.FieldMetaData("o2", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O3, new org.apache.thrift.meta_data.FieldMetaData("o3", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O4, new org.apache.thrift.meta_data.FieldMetaData("o4", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(disableStreaming_result.class, metaDataMap); + } + + public disableStreaming_result() { + } + + public disableStreaming_result( + InvalidInputException o1, + InvalidObjectException o2, + InvalidOperationException o3, + MetaException o4) + { + this(); + this.o1 = o1; + this.o2 = o2; + this.o3 = o3; + this.o4 = o4; + } + + /** + * Performs a deep copy on other. + */ + public disableStreaming_result(disableStreaming_result other) { + if (other.isSetO1()) { + this.o1 = new InvalidInputException(other.o1); + } + if (other.isSetO2()) { + this.o2 = new InvalidObjectException(other.o2); + } + if (other.isSetO3()) { + this.o3 = new InvalidOperationException(other.o3); + } + if (other.isSetO4()) { + this.o4 = new MetaException(other.o4); + } + } + + public disableStreaming_result deepCopy() { + return new disableStreaming_result(this); + } + + @Override + public void clear() { + this.o1 = null; + this.o2 = null; + this.o3 = null; + this.o4 = null; + } + + public InvalidInputException getO1() { + return this.o1; + } + + public void setO1(InvalidInputException o1) { + this.o1 = o1; + } + + public void unsetO1() { + this.o1 = null; + } + + /** Returns true if field o1 is set (has been assigned a value) and false otherwise */ + public boolean isSetO1() { + return this.o1 != null; + } + + public void setO1IsSet(boolean value) { + if (!value) { + this.o1 = null; + } + } + + public InvalidObjectException getO2() { + return this.o2; + } + + public void setO2(InvalidObjectException o2) { + this.o2 = o2; + } + + public void unsetO2() { + this.o2 = null; + } + + /** Returns true if field o2 is set (has been assigned a value) and false otherwise */ + public boolean isSetO2() { + return this.o2 != null; + } + + public void setO2IsSet(boolean value) { + if (!value) { + this.o2 = null; + } + } + + public InvalidOperationException getO3() { + return this.o3; + } + + public void setO3(InvalidOperationException o3) { + this.o3 = o3; + } + + public void unsetO3() { + this.o3 = null; + } + + /** Returns true if field o3 is set (has been assigned a value) and false otherwise */ + public boolean isSetO3() { + return this.o3 != null; + } + + public void setO3IsSet(boolean value) { + if (!value) { + this.o3 = null; + } + } + + public MetaException getO4() { + return this.o4; + } + + public void setO4(MetaException o4) { + this.o4 = o4; + } + + public void unsetO4() { + this.o4 = null; + } + + /** Returns true if field o4 is set (has been assigned a value) and false otherwise */ + public boolean isSetO4() { + return this.o4 != null; + } + + public void setO4IsSet(boolean value) { + if (!value) { + this.o4 = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case O1: + if (value == null) { + unsetO1(); + } else { + setO1((InvalidInputException)value); + } + break; + + case O2: + if (value == null) { + unsetO2(); + } else { + setO2((InvalidObjectException)value); + } + break; + + case O3: + if (value == null) { + unsetO3(); + } else { + setO3((InvalidOperationException)value); + } + break; + + case O4: + if (value == null) { + unsetO4(); + } else { + setO4((MetaException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case O1: + return getO1(); + + case O2: + return getO2(); + + case O3: + return getO3(); + + case O4: + return getO4(); + + } + throw new 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 IllegalArgumentException(); + } + + switch (field) { + case O1: + return isSetO1(); + case O2: + return isSetO2(); + case O3: + return isSetO3(); + case O4: + return isSetO4(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof disableStreaming_result) + return this.equals((disableStreaming_result)that); + return false; + } + + public boolean equals(disableStreaming_result that) { + if (that == null) + return false; + + boolean this_present_o1 = true && this.isSetO1(); + boolean that_present_o1 = true && that.isSetO1(); + if (this_present_o1 || that_present_o1) { + if (!(this_present_o1 && that_present_o1)) + return false; + if (!this.o1.equals(that.o1)) + return false; + } + + boolean this_present_o2 = true && this.isSetO2(); + boolean that_present_o2 = true && that.isSetO2(); + if (this_present_o2 || that_present_o2) { + if (!(this_present_o2 && that_present_o2)) + return false; + if (!this.o2.equals(that.o2)) + return false; + } + + boolean this_present_o3 = true && this.isSetO3(); + boolean that_present_o3 = true && that.isSetO3(); + if (this_present_o3 || that_present_o3) { + if (!(this_present_o3 && that_present_o3)) + return false; + if (!this.o3.equals(that.o3)) + return false; + } + + boolean this_present_o4 = true && this.isSetO4(); + boolean that_present_o4 = true && that.isSetO4(); + if (this_present_o4 || that_present_o4) { + if (!(this_present_o4 && that_present_o4)) + return false; + if (!this.o4.equals(that.o4)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_o1 = true && (isSetO1()); + builder.append(present_o1); + if (present_o1) + builder.append(o1); + + boolean present_o2 = true && (isSetO2()); + builder.append(present_o2); + if (present_o2) + builder.append(o2); + + boolean present_o3 = true && (isSetO3()); + builder.append(present_o3); + if (present_o3) + builder.append(o3); + + boolean present_o4 = true && (isSetO4()); + builder.append(present_o4); + if (present_o4) + builder.append(o4); + + return builder.toHashCode(); + } + + public int compareTo(disableStreaming_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + disableStreaming_result typedOther = (disableStreaming_result)other; + + lastComparison = Boolean.valueOf(isSetO1()).compareTo(typedOther.isSetO1()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO1()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, typedOther.o1); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO2()).compareTo(typedOther.isSetO2()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO2()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o2, typedOther.o2); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO3()).compareTo(typedOther.isSetO3()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO3()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o3, typedOther.o3); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO4()).compareTo(typedOther.isSetO4()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO4()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o4, typedOther.o4); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("disableStreaming_result("); + boolean first = true; + + sb.append("o1:"); + if (this.o1 == null) { + sb.append("null"); + } else { + sb.append(this.o1); + } + first = false; + if (!first) sb.append(", "); + sb.append("o2:"); + if (this.o2 == null) { + sb.append("null"); + } else { + sb.append(this.o2); + } + first = false; + if (!first) sb.append(", "); + sb.append("o3:"); + if (this.o3 == null) { + sb.append("null"); + } else { + sb.append(this.o3); + } + first = false; + if (!first) sb.append(", "); + sb.append("o4:"); + if (this.o4 == null) { + sb.append("null"); + } else { + sb.append(this.o4); + } + 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, 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 disableStreaming_resultStandardSchemeFactory implements SchemeFactory { + public disableStreaming_resultStandardScheme getScheme() { + return new disableStreaming_resultStandardScheme(); + } + } + + private static class disableStreaming_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, disableStreaming_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: // O1 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o1 = new InvalidInputException(); + struct.o1.read(iprot); + struct.setO1IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // O2 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o2 = new InvalidObjectException(); + struct.o2.read(iprot); + struct.setO2IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // O3 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o3 = new InvalidOperationException(); + struct.o3.read(iprot); + struct.setO3IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // O4 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o4 = new MetaException(); + struct.o4.read(iprot); + struct.setO4IsSet(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(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, disableStreaming_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.o1 != null) { + oprot.writeFieldBegin(O1_FIELD_DESC); + struct.o1.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o2 != null) { + oprot.writeFieldBegin(O2_FIELD_DESC); + struct.o2.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o3 != null) { + oprot.writeFieldBegin(O3_FIELD_DESC); + struct.o3.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o4 != null) { + oprot.writeFieldBegin(O4_FIELD_DESC); + struct.o4.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class disableStreaming_resultTupleSchemeFactory implements SchemeFactory { + public disableStreaming_resultTupleScheme getScheme() { + return new disableStreaming_resultTupleScheme(); + } + } + + private static class disableStreaming_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, disableStreaming_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetO1()) { + optionals.set(0); + } + if (struct.isSetO2()) { + optionals.set(1); + } + if (struct.isSetO3()) { + optionals.set(2); + } + if (struct.isSetO4()) { + optionals.set(3); + } + oprot.writeBitSet(optionals, 4); + if (struct.isSetO1()) { + struct.o1.write(oprot); + } + if (struct.isSetO2()) { + struct.o2.write(oprot); + } + if (struct.isSetO3()) { + struct.o3.write(oprot); + } + if (struct.isSetO4()) { + struct.o4.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, disableStreaming_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(4); + if (incoming.get(0)) { + struct.o1 = new InvalidInputException(); + struct.o1.read(iprot); + struct.setO1IsSet(true); + } + if (incoming.get(1)) { + struct.o2 = new InvalidObjectException(); + struct.o2.read(iprot); + struct.setO2IsSet(true); + } + if (incoming.get(2)) { + struct.o3 = new InvalidOperationException(); + struct.o3.read(iprot); + struct.setO3IsSet(true); + } + if (incoming.get(3)) { + struct.o4 = new MetaException(); + struct.o4.read(iprot); + struct.setO4IsSet(true); + } + } + } + + } + + public static class getCurrentStreamingPartitionPath_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getCurrentStreamingPartitionPath_args"); + + private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getCurrentStreamingPartitionPath_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getCurrentStreamingPartitionPath_argsTupleSchemeFactory()); + } + + private String dbName; // required + private String 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 { + DB_NAME((short)1, "dbName"), + TABLE_NAME((short)2, "tableName"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // DB_NAME + return DB_NAME; + case 2: // 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 IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("dbName", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getCurrentStreamingPartitionPath_args.class, metaDataMap); + } + + public getCurrentStreamingPartitionPath_args() { + } + + public getCurrentStreamingPartitionPath_args( + String dbName, + String tableName) + { + this(); + this.dbName = dbName; + this.tableName = tableName; + } + + /** + * Performs a deep copy on other. + */ + public getCurrentStreamingPartitionPath_args(getCurrentStreamingPartitionPath_args other) { + if (other.isSetDbName()) { + this.dbName = other.dbName; + } + if (other.isSetTableName()) { + this.tableName = other.tableName; + } + } + + public getCurrentStreamingPartitionPath_args deepCopy() { + return new getCurrentStreamingPartitionPath_args(this); + } + + @Override + public void clear() { + this.dbName = null; + this.tableName = null; + } + + public String getDbName() { + return this.dbName; + } + + public void setDbName(String dbName) { + this.dbName = dbName; + } + + public void unsetDbName() { + this.dbName = null; + } + + /** Returns true if field dbName is set (has been assigned a value) and false otherwise */ + public boolean isSetDbName() { + return this.dbName != null; + } + + public void setDbNameIsSet(boolean value) { + if (!value) { + this.dbName = null; + } + } + + public String getTableName() { + return this.tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + 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, Object value) { + switch (field) { + case DB_NAME: + if (value == null) { + unsetDbName(); + } else { + setDbName((String)value); + } + break; + + case TABLE_NAME: + if (value == null) { + unsetTableName(); + } else { + setTableName((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case DB_NAME: + return getDbName(); + + case TABLE_NAME: + return getTableName(); + + } + throw new 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 IllegalArgumentException(); + } + + switch (field) { + case DB_NAME: + return isSetDbName(); + case TABLE_NAME: + return isSetTableName(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getCurrentStreamingPartitionPath_args) + return this.equals((getCurrentStreamingPartitionPath_args)that); + return false; + } + + public boolean equals(getCurrentStreamingPartitionPath_args that) { + if (that == null) + return false; + + boolean this_present_dbName = true && this.isSetDbName(); + boolean that_present_dbName = true && that.isSetDbName(); + if (this_present_dbName || that_present_dbName) { + if (!(this_present_dbName && that_present_dbName)) + return false; + if (!this.dbName.equals(that.dbName)) + return false; + } + + 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() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_dbName = true && (isSetDbName()); + builder.append(present_dbName); + if (present_dbName) + builder.append(dbName); + + boolean present_tableName = true && (isSetTableName()); + builder.append(present_tableName); + if (present_tableName) + builder.append(tableName); + + return builder.toHashCode(); + } + + public int compareTo(getCurrentStreamingPartitionPath_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + getCurrentStreamingPartitionPath_args typedOther = (getCurrentStreamingPartitionPath_args)other; + + lastComparison = Boolean.valueOf(isSetDbName()).compareTo(typedOther.isSetDbName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetDbName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.dbName, typedOther.dbName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTableName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getCurrentStreamingPartitionPath_args("); + boolean first = true; + + sb.append("dbName:"); + if (this.dbName == null) { + sb.append("null"); + } else { + sb.append(this.dbName); + } + first = false; + if (!first) sb.append(", "); + 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 + // 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, 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 getCurrentStreamingPartitionPath_argsStandardSchemeFactory implements SchemeFactory { + public getCurrentStreamingPartitionPath_argsStandardScheme getScheme() { + return new getCurrentStreamingPartitionPath_argsStandardScheme(); + } + } + + private static class getCurrentStreamingPartitionPath_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getCurrentStreamingPartitionPath_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: // DB_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.dbName = iprot.readString(); + struct.setDbNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // TABLE_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.tableName = iprot.readString(); + 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(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getCurrentStreamingPartitionPath_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.dbName != null) { + oprot.writeFieldBegin(DB_NAME_FIELD_DESC); + oprot.writeString(struct.dbName); + oprot.writeFieldEnd(); + } + if (struct.tableName != null) { + oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC); + oprot.writeString(struct.tableName); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getCurrentStreamingPartitionPath_argsTupleSchemeFactory implements SchemeFactory { + public getCurrentStreamingPartitionPath_argsTupleScheme getScheme() { + return new getCurrentStreamingPartitionPath_argsTupleScheme(); + } + } + + private static class getCurrentStreamingPartitionPath_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getCurrentStreamingPartitionPath_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetDbName()) { + optionals.set(0); + } + if (struct.isSetTableName()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetDbName()) { + oprot.writeString(struct.dbName); + } + if (struct.isSetTableName()) { + oprot.writeString(struct.tableName); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getCurrentStreamingPartitionPath_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.dbName = iprot.readString(); + struct.setDbNameIsSet(true); + } + if (incoming.get(1)) { + struct.tableName = iprot.readString(); + struct.setTableNameIsSet(true); + } + } + } + + } + + public static class getCurrentStreamingPartitionPath_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getCurrentStreamingPartitionPath_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.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField O2_FIELD_DESC = new org.apache.thrift.protocol.TField("o2", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField O3_FIELD_DESC = new org.apache.thrift.protocol.TField("o3", org.apache.thrift.protocol.TType.STRUCT, (short)3); + private static final org.apache.thrift.protocol.TField O4_FIELD_DESC = new org.apache.thrift.protocol.TField("o4", org.apache.thrift.protocol.TType.STRUCT, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getCurrentStreamingPartitionPath_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getCurrentStreamingPartitionPath_resultTupleSchemeFactory()); + } + + private String success; // required + private InvalidInputException o1; // required + private InvalidObjectException o2; // required + private InvalidOperationException o3; // required + private MetaException o4; // 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"), + O1((short)1, "o1"), + O2((short)2, "o2"), + O3((short)3, "o3"), + O4((short)4, "o4"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + case 1: // O1 + return O1; + case 2: // O2 + return O2; + case 3: // O3 + return O3; + case 4: // O4 + return O4; + 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 IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new 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))); + tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O2, new org.apache.thrift.meta_data.FieldMetaData("o2", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O3, new org.apache.thrift.meta_data.FieldMetaData("o3", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O4, new org.apache.thrift.meta_data.FieldMetaData("o4", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getCurrentStreamingPartitionPath_result.class, metaDataMap); + } + + public getCurrentStreamingPartitionPath_result() { + } + + public getCurrentStreamingPartitionPath_result( + String success, + InvalidInputException o1, + InvalidObjectException o2, + InvalidOperationException o3, + MetaException o4) + { + this(); + this.success = success; + this.o1 = o1; + this.o2 = o2; + this.o3 = o3; + this.o4 = o4; + } + + /** + * Performs a deep copy on other. + */ + public getCurrentStreamingPartitionPath_result(getCurrentStreamingPartitionPath_result other) { + if (other.isSetSuccess()) { + this.success = other.success; + } + if (other.isSetO1()) { + this.o1 = new InvalidInputException(other.o1); + } + if (other.isSetO2()) { + this.o2 = new InvalidObjectException(other.o2); + } + if (other.isSetO3()) { + this.o3 = new InvalidOperationException(other.o3); + } + if (other.isSetO4()) { + this.o4 = new MetaException(other.o4); + } + } + + public getCurrentStreamingPartitionPath_result deepCopy() { + return new getCurrentStreamingPartitionPath_result(this); + } + + @Override + public void clear() { + this.success = null; + this.o1 = null; + this.o2 = null; + this.o3 = null; + this.o4 = null; + } + + public String getSuccess() { + return this.success; + } + + public void setSuccess(String success) { + this.success = success; + } + + 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 InvalidInputException getO1() { + return this.o1; + } + + public void setO1(InvalidInputException o1) { + this.o1 = o1; + } + + public void unsetO1() { + this.o1 = null; + } + + /** Returns true if field o1 is set (has been assigned a value) and false otherwise */ + public boolean isSetO1() { + return this.o1 != null; + } + + public void setO1IsSet(boolean value) { + if (!value) { + this.o1 = null; + } + } + + public InvalidObjectException getO2() { + return this.o2; + } + + public void setO2(InvalidObjectException o2) { + this.o2 = o2; + } + + public void unsetO2() { + this.o2 = null; + } + + /** Returns true if field o2 is set (has been assigned a value) and false otherwise */ + public boolean isSetO2() { + return this.o2 != null; + } + + public void setO2IsSet(boolean value) { + if (!value) { + this.o2 = null; + } + } + + public InvalidOperationException getO3() { + return this.o3; + } + + public void setO3(InvalidOperationException o3) { + this.o3 = o3; + } + + public void unsetO3() { + this.o3 = null; + } + + /** Returns true if field o3 is set (has been assigned a value) and false otherwise */ + public boolean isSetO3() { + return this.o3 != null; + } + + public void setO3IsSet(boolean value) { + if (!value) { + this.o3 = null; + } + } + + public MetaException getO4() { + return this.o4; + } + + public void setO4(MetaException o4) { + this.o4 = o4; + } + + public void unsetO4() { + this.o4 = null; + } + + /** Returns true if field o4 is set (has been assigned a value) and false otherwise */ + public boolean isSetO4() { + return this.o4 != null; + } + + public void setO4IsSet(boolean value) { + if (!value) { + this.o4 = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((String)value); + } + break; + + case O1: + if (value == null) { + unsetO1(); + } else { + setO1((InvalidInputException)value); + } + break; + + case O2: + if (value == null) { + unsetO2(); + } else { + setO2((InvalidObjectException)value); + } + break; + + case O3: + if (value == null) { + unsetO3(); + } else { + setO3((InvalidOperationException)value); + } + break; + + case O4: + if (value == null) { + unsetO4(); + } else { + setO4((MetaException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + case O1: + return getO1(); + + case O2: + return getO2(); + + case O3: + return getO3(); + + case O4: + return getO4(); + + } + throw new 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 IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + case O1: + return isSetO1(); + case O2: + return isSetO2(); + case O3: + return isSetO3(); + case O4: + return isSetO4(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getCurrentStreamingPartitionPath_result) + return this.equals((getCurrentStreamingPartitionPath_result)that); + return false; + } + + public boolean equals(getCurrentStreamingPartitionPath_result that) { + if (that == null) + return false; + + 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_o1 = true && this.isSetO1(); + boolean that_present_o1 = true && that.isSetO1(); + if (this_present_o1 || that_present_o1) { + if (!(this_present_o1 && that_present_o1)) + return false; + if (!this.o1.equals(that.o1)) + return false; + } + + boolean this_present_o2 = true && this.isSetO2(); + boolean that_present_o2 = true && that.isSetO2(); + if (this_present_o2 || that_present_o2) { + if (!(this_present_o2 && that_present_o2)) + return false; + if (!this.o2.equals(that.o2)) + return false; + } + + boolean this_present_o3 = true && this.isSetO3(); + boolean that_present_o3 = true && that.isSetO3(); + if (this_present_o3 || that_present_o3) { + if (!(this_present_o3 && that_present_o3)) + return false; + if (!this.o3.equals(that.o3)) + return false; + } + + boolean this_present_o4 = true && this.isSetO4(); + boolean that_present_o4 = true && that.isSetO4(); + if (this_present_o4 || that_present_o4) { + if (!(this_present_o4 && that_present_o4)) + return false; + if (!this.o4.equals(that.o4)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + boolean present_o1 = true && (isSetO1()); + builder.append(present_o1); + if (present_o1) + builder.append(o1); + + boolean present_o2 = true && (isSetO2()); + builder.append(present_o2); + if (present_o2) + builder.append(o2); + + boolean present_o3 = true && (isSetO3()); + builder.append(present_o3); + if (present_o3) + builder.append(o3); + + boolean present_o4 = true && (isSetO4()); + builder.append(present_o4); + if (present_o4) + builder.append(o4); + + return builder.toHashCode(); + } + + public int compareTo(getCurrentStreamingPartitionPath_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + getCurrentStreamingPartitionPath_result typedOther = (getCurrentStreamingPartitionPath_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO1()).compareTo(typedOther.isSetO1()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO1()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, typedOther.o1); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO2()).compareTo(typedOther.isSetO2()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO2()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o2, typedOther.o2); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO3()).compareTo(typedOther.isSetO3()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO3()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o3, typedOther.o3); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO4()).compareTo(typedOther.isSetO4()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO4()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o4, typedOther.o4); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getCurrentStreamingPartitionPath_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("o1:"); + if (this.o1 == null) { + sb.append("null"); + } else { + sb.append(this.o1); + } + first = false; + if (!first) sb.append(", "); + sb.append("o2:"); + if (this.o2 == null) { + sb.append("null"); + } else { + sb.append(this.o2); + } + first = false; + if (!first) sb.append(", "); + sb.append("o3:"); + if (this.o3 == null) { + sb.append("null"); + } else { + sb.append(this.o3); + } + first = false; + if (!first) sb.append(", "); + sb.append("o4:"); + if (this.o4 == null) { + sb.append("null"); + } else { + sb.append(this.o4); + } + 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, 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 getCurrentStreamingPartitionPath_resultStandardSchemeFactory implements SchemeFactory { + public getCurrentStreamingPartitionPath_resultStandardScheme getScheme() { + return new getCurrentStreamingPartitionPath_resultStandardScheme(); + } + } + + private static class getCurrentStreamingPartitionPath_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getCurrentStreamingPartitionPath_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; + case 1: // O1 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o1 = new InvalidInputException(); + struct.o1.read(iprot); + struct.setO1IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // O2 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o2 = new InvalidObjectException(); + struct.o2.read(iprot); + struct.setO2IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // O3 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o3 = new InvalidOperationException(); + struct.o3.read(iprot); + struct.setO3IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // O4 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o4 = new MetaException(); + struct.o4.read(iprot); + struct.setO4IsSet(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(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getCurrentStreamingPartitionPath_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(); + } + if (struct.o1 != null) { + oprot.writeFieldBegin(O1_FIELD_DESC); + struct.o1.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o2 != null) { + oprot.writeFieldBegin(O2_FIELD_DESC); + struct.o2.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o3 != null) { + oprot.writeFieldBegin(O3_FIELD_DESC); + struct.o3.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o4 != null) { + oprot.writeFieldBegin(O4_FIELD_DESC); + struct.o4.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getCurrentStreamingPartitionPath_resultTupleSchemeFactory implements SchemeFactory { + public getCurrentStreamingPartitionPath_resultTupleScheme getScheme() { + return new getCurrentStreamingPartitionPath_resultTupleScheme(); + } + } + + private static class getCurrentStreamingPartitionPath_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getCurrentStreamingPartitionPath_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + if (struct.isSetO1()) { + optionals.set(1); + } + if (struct.isSetO2()) { + optionals.set(2); + } + if (struct.isSetO3()) { + optionals.set(3); + } + if (struct.isSetO4()) { + optionals.set(4); + } + oprot.writeBitSet(optionals, 5); + if (struct.isSetSuccess()) { + oprot.writeString(struct.success); + } + if (struct.isSetO1()) { + struct.o1.write(oprot); + } + if (struct.isSetO2()) { + struct.o2.write(oprot); + } + if (struct.isSetO3()) { + struct.o3.write(oprot); + } + if (struct.isSetO4()) { + struct.o4.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getCurrentStreamingPartitionPath_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(5); + if (incoming.get(0)) { + struct.success = iprot.readString(); + struct.setSuccessIsSet(true); + } + if (incoming.get(1)) { + struct.o1 = new InvalidInputException(); + struct.o1.read(iprot); + struct.setO1IsSet(true); + } + if (incoming.get(2)) { + struct.o2 = new InvalidObjectException(); + struct.o2.read(iprot); + struct.setO2IsSet(true); + } + if (incoming.get(3)) { + struct.o3 = new InvalidOperationException(); + struct.o3.read(iprot); + struct.setO3IsSet(true); + } + if (incoming.get(4)) { + struct.o4 = new MetaException(); + struct.o4.read(iprot); + struct.setO4IsSet(true); + } + } + } + + } + + public static class rollStreamingPartition_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("rollStreamingPartition_args"); + + private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField NEW_PARTITION_PATH_FIELD_DESC = new org.apache.thrift.protocol.TField("newPartitionPath", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField NEW_PARTITION_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("newPartitionVal", org.apache.thrift.protocol.TType.STRING, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new rollStreamingPartition_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new rollStreamingPartition_argsTupleSchemeFactory()); + } + + private String dbName; // required + private String tableName; // required + private String newPartitionPath; // required + private String newPartitionVal; // 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 { + DB_NAME((short)1, "dbName"), + TABLE_NAME((short)2, "tableName"), + NEW_PARTITION_PATH((short)3, "newPartitionPath"), + NEW_PARTITION_VAL((short)4, "newPartitionVal"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // DB_NAME + return DB_NAME; + case 2: // TABLE_NAME + return TABLE_NAME; + case 3: // NEW_PARTITION_PATH + return NEW_PARTITION_PATH; + case 4: // NEW_PARTITION_VAL + return NEW_PARTITION_VAL; + 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 IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("dbName", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.NEW_PARTITION_PATH, new org.apache.thrift.meta_data.FieldMetaData("newPartitionPath", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.NEW_PARTITION_VAL, new org.apache.thrift.meta_data.FieldMetaData("newPartitionVal", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(rollStreamingPartition_args.class, metaDataMap); + } + + public rollStreamingPartition_args() { + } + + public rollStreamingPartition_args( + String dbName, + String tableName, + String newPartitionPath, + String newPartitionVal) + { + this(); + this.dbName = dbName; + this.tableName = tableName; + this.newPartitionPath = newPartitionPath; + this.newPartitionVal = newPartitionVal; + } + + /** + * Performs a deep copy on other. + */ + public rollStreamingPartition_args(rollStreamingPartition_args other) { + if (other.isSetDbName()) { + this.dbName = other.dbName; + } + if (other.isSetTableName()) { + this.tableName = other.tableName; + } + if (other.isSetNewPartitionPath()) { + this.newPartitionPath = other.newPartitionPath; + } + if (other.isSetNewPartitionVal()) { + this.newPartitionVal = other.newPartitionVal; + } + } + + public rollStreamingPartition_args deepCopy() { + return new rollStreamingPartition_args(this); + } + + @Override + public void clear() { + this.dbName = null; + this.tableName = null; + this.newPartitionPath = null; + this.newPartitionVal = null; + } + + public String getDbName() { + return this.dbName; + } + + public void setDbName(String dbName) { + this.dbName = dbName; + } + + public void unsetDbName() { + this.dbName = null; + } + + /** Returns true if field dbName is set (has been assigned a value) and false otherwise */ + public boolean isSetDbName() { + return this.dbName != null; + } + + public void setDbNameIsSet(boolean value) { + if (!value) { + this.dbName = null; + } + } + + public String getTableName() { + return this.tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + 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 String getNewPartitionPath() { + return this.newPartitionPath; + } + + public void setNewPartitionPath(String newPartitionPath) { + this.newPartitionPath = newPartitionPath; + } + + public void unsetNewPartitionPath() { + this.newPartitionPath = null; + } + + /** Returns true if field newPartitionPath is set (has been assigned a value) and false otherwise */ + public boolean isSetNewPartitionPath() { + return this.newPartitionPath != null; + } + + public void setNewPartitionPathIsSet(boolean value) { + if (!value) { + this.newPartitionPath = null; + } + } + + public String getNewPartitionVal() { + return this.newPartitionVal; + } + + public void setNewPartitionVal(String newPartitionVal) { + this.newPartitionVal = newPartitionVal; + } + + public void unsetNewPartitionVal() { + this.newPartitionVal = null; + } + + /** Returns true if field newPartitionVal is set (has been assigned a value) and false otherwise */ + public boolean isSetNewPartitionVal() { + return this.newPartitionVal != null; + } + + public void setNewPartitionValIsSet(boolean value) { + if (!value) { + this.newPartitionVal = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case DB_NAME: + if (value == null) { + unsetDbName(); + } else { + setDbName((String)value); + } + break; + + case TABLE_NAME: + if (value == null) { + unsetTableName(); + } else { + setTableName((String)value); + } + break; + + case NEW_PARTITION_PATH: + if (value == null) { + unsetNewPartitionPath(); + } else { + setNewPartitionPath((String)value); + } + break; + + case NEW_PARTITION_VAL: + if (value == null) { + unsetNewPartitionVal(); + } else { + setNewPartitionVal((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case DB_NAME: + return getDbName(); + + case TABLE_NAME: + return getTableName(); + + case NEW_PARTITION_PATH: + return getNewPartitionPath(); + + case NEW_PARTITION_VAL: + return getNewPartitionVal(); + + } + throw new 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 IllegalArgumentException(); + } + + switch (field) { + case DB_NAME: + return isSetDbName(); + case TABLE_NAME: + return isSetTableName(); + case NEW_PARTITION_PATH: + return isSetNewPartitionPath(); + case NEW_PARTITION_VAL: + return isSetNewPartitionVal(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof rollStreamingPartition_args) + return this.equals((rollStreamingPartition_args)that); + return false; + } + + public boolean equals(rollStreamingPartition_args that) { + if (that == null) + return false; + + boolean this_present_dbName = true && this.isSetDbName(); + boolean that_present_dbName = true && that.isSetDbName(); + if (this_present_dbName || that_present_dbName) { + if (!(this_present_dbName && that_present_dbName)) + return false; + if (!this.dbName.equals(that.dbName)) + return false; + } + + 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_newPartitionPath = true && this.isSetNewPartitionPath(); + boolean that_present_newPartitionPath = true && that.isSetNewPartitionPath(); + if (this_present_newPartitionPath || that_present_newPartitionPath) { + if (!(this_present_newPartitionPath && that_present_newPartitionPath)) + return false; + if (!this.newPartitionPath.equals(that.newPartitionPath)) + return false; + } + + boolean this_present_newPartitionVal = true && this.isSetNewPartitionVal(); + boolean that_present_newPartitionVal = true && that.isSetNewPartitionVal(); + if (this_present_newPartitionVal || that_present_newPartitionVal) { + if (!(this_present_newPartitionVal && that_present_newPartitionVal)) + return false; + if (!this.newPartitionVal.equals(that.newPartitionVal)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_dbName = true && (isSetDbName()); + builder.append(present_dbName); + if (present_dbName) + builder.append(dbName); + + boolean present_tableName = true && (isSetTableName()); + builder.append(present_tableName); + if (present_tableName) + builder.append(tableName); + + boolean present_newPartitionPath = true && (isSetNewPartitionPath()); + builder.append(present_newPartitionPath); + if (present_newPartitionPath) + builder.append(newPartitionPath); + + boolean present_newPartitionVal = true && (isSetNewPartitionVal()); + builder.append(present_newPartitionVal); + if (present_newPartitionVal) + builder.append(newPartitionVal); + + return builder.toHashCode(); + } + + public int compareTo(rollStreamingPartition_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + rollStreamingPartition_args typedOther = (rollStreamingPartition_args)other; + + lastComparison = Boolean.valueOf(isSetDbName()).compareTo(typedOther.isSetDbName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetDbName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.dbName, typedOther.dbName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTableName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNewPartitionPath()).compareTo(typedOther.isSetNewPartitionPath()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNewPartitionPath()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.newPartitionPath, typedOther.newPartitionPath); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNewPartitionVal()).compareTo(typedOther.isSetNewPartitionVal()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNewPartitionVal()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.newPartitionVal, typedOther.newPartitionVal); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("rollStreamingPartition_args("); + boolean first = true; + + sb.append("dbName:"); + if (this.dbName == null) { + sb.append("null"); + } else { + sb.append(this.dbName); + } + first = false; + if (!first) sb.append(", "); + sb.append("tableName:"); + if (this.tableName == null) { + sb.append("null"); + } else { + sb.append(this.tableName); + } + first = false; + if (!first) sb.append(", "); + sb.append("newPartitionPath:"); + if (this.newPartitionPath == null) { + sb.append("null"); + } else { + sb.append(this.newPartitionPath); + } + first = false; + if (!first) sb.append(", "); + sb.append("newPartitionVal:"); + if (this.newPartitionVal == null) { + sb.append("null"); + } else { + sb.append(this.newPartitionVal); + } + 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, 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 rollStreamingPartition_argsStandardSchemeFactory implements SchemeFactory { + public rollStreamingPartition_argsStandardScheme getScheme() { + return new rollStreamingPartition_argsStandardScheme(); + } + } + + private static class rollStreamingPartition_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, rollStreamingPartition_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: // DB_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.dbName = iprot.readString(); + struct.setDbNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // TABLE_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.tableName = iprot.readString(); + struct.setTableNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // NEW_PARTITION_PATH + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.newPartitionPath = iprot.readString(); + struct.setNewPartitionPathIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // NEW_PARTITION_VAL + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.newPartitionVal = iprot.readString(); + struct.setNewPartitionValIsSet(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(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, rollStreamingPartition_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.dbName != null) { + oprot.writeFieldBegin(DB_NAME_FIELD_DESC); + oprot.writeString(struct.dbName); + oprot.writeFieldEnd(); + } + if (struct.tableName != null) { + oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC); + oprot.writeString(struct.tableName); + oprot.writeFieldEnd(); + } + if (struct.newPartitionPath != null) { + oprot.writeFieldBegin(NEW_PARTITION_PATH_FIELD_DESC); + oprot.writeString(struct.newPartitionPath); + oprot.writeFieldEnd(); + } + if (struct.newPartitionVal != null) { + oprot.writeFieldBegin(NEW_PARTITION_VAL_FIELD_DESC); + oprot.writeString(struct.newPartitionVal); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class rollStreamingPartition_argsTupleSchemeFactory implements SchemeFactory { + public rollStreamingPartition_argsTupleScheme getScheme() { + return new rollStreamingPartition_argsTupleScheme(); + } + } + + private static class rollStreamingPartition_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, rollStreamingPartition_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetDbName()) { + optionals.set(0); + } + if (struct.isSetTableName()) { + optionals.set(1); + } + if (struct.isSetNewPartitionPath()) { + optionals.set(2); + } + if (struct.isSetNewPartitionVal()) { + optionals.set(3); + } + oprot.writeBitSet(optionals, 4); + if (struct.isSetDbName()) { + oprot.writeString(struct.dbName); + } + if (struct.isSetTableName()) { + oprot.writeString(struct.tableName); + } + if (struct.isSetNewPartitionPath()) { + oprot.writeString(struct.newPartitionPath); + } + if (struct.isSetNewPartitionVal()) { + oprot.writeString(struct.newPartitionVal); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, rollStreamingPartition_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(4); + if (incoming.get(0)) { + struct.dbName = iprot.readString(); + struct.setDbNameIsSet(true); + } + if (incoming.get(1)) { + struct.tableName = iprot.readString(); + struct.setTableNameIsSet(true); + } + if (incoming.get(2)) { + struct.newPartitionPath = iprot.readString(); + struct.setNewPartitionPathIsSet(true); + } + if (incoming.get(3)) { + struct.newPartitionVal = iprot.readString(); + struct.setNewPartitionValIsSet(true); + } + } + } + + } + + public static class rollStreamingPartition_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("rollStreamingPartition_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.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField O2_FIELD_DESC = new org.apache.thrift.protocol.TField("o2", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField O3_FIELD_DESC = new org.apache.thrift.protocol.TField("o3", org.apache.thrift.protocol.TType.STRUCT, (short)3); + private static final org.apache.thrift.protocol.TField O4_FIELD_DESC = new org.apache.thrift.protocol.TField("o4", org.apache.thrift.protocol.TType.STRUCT, (short)4); + private static final org.apache.thrift.protocol.TField O5_FIELD_DESC = new org.apache.thrift.protocol.TField("o5", org.apache.thrift.protocol.TType.STRUCT, (short)5); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new rollStreamingPartition_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new rollStreamingPartition_resultTupleSchemeFactory()); + } + + private String success; // required + private InvalidInputException o1; // required + private InvalidObjectException o2; // required + private InvalidOperationException o3; // required + private AlreadyExistsException o4; // required + private MetaException o5; // 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"), + O1((short)1, "o1"), + O2((short)2, "o2"), + O3((short)3, "o3"), + O4((short)4, "o4"), + O5((short)5, "o5"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + case 1: // O1 + return O1; + case 2: // O2 + return O2; + case 3: // O3 + return O3; + case 4: // O4 + return O4; + case 5: // O5 + return O5; + 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 IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new 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))); + tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O2, new org.apache.thrift.meta_data.FieldMetaData("o2", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O3, new org.apache.thrift.meta_data.FieldMetaData("o3", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O4, new org.apache.thrift.meta_data.FieldMetaData("o4", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O5, new org.apache.thrift.meta_data.FieldMetaData("o5", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(rollStreamingPartition_result.class, metaDataMap); + } + + public rollStreamingPartition_result() { + } + + public rollStreamingPartition_result( + String success, + InvalidInputException o1, + InvalidObjectException o2, + InvalidOperationException o3, + AlreadyExistsException o4, + MetaException o5) + { + this(); + this.success = success; + this.o1 = o1; + this.o2 = o2; + this.o3 = o3; + this.o4 = o4; + this.o5 = o5; + } + + /** + * Performs a deep copy on other. + */ + public rollStreamingPartition_result(rollStreamingPartition_result other) { + if (other.isSetSuccess()) { + this.success = other.success; + } + if (other.isSetO1()) { + this.o1 = new InvalidInputException(other.o1); + } + if (other.isSetO2()) { + this.o2 = new InvalidObjectException(other.o2); + } + if (other.isSetO3()) { + this.o3 = new InvalidOperationException(other.o3); + } + if (other.isSetO4()) { + this.o4 = new AlreadyExistsException(other.o4); + } + if (other.isSetO5()) { + this.o5 = new MetaException(other.o5); + } + } + + public rollStreamingPartition_result deepCopy() { + return new rollStreamingPartition_result(this); + } + + @Override + public void clear() { + this.success = null; + this.o1 = null; + this.o2 = null; + this.o3 = null; + this.o4 = null; + this.o5 = null; + } + + public String getSuccess() { + return this.success; + } + + public void setSuccess(String success) { + this.success = success; + } + + 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 InvalidInputException getO1() { + return this.o1; + } + + public void setO1(InvalidInputException o1) { + this.o1 = o1; + } + + public void unsetO1() { + this.o1 = null; + } + + /** Returns true if field o1 is set (has been assigned a value) and false otherwise */ + public boolean isSetO1() { + return this.o1 != null; + } + + public void setO1IsSet(boolean value) { + if (!value) { + this.o1 = null; + } + } + + public InvalidObjectException getO2() { + return this.o2; + } + + public void setO2(InvalidObjectException o2) { + this.o2 = o2; + } + + public void unsetO2() { + this.o2 = null; + } + + /** Returns true if field o2 is set (has been assigned a value) and false otherwise */ + public boolean isSetO2() { + return this.o2 != null; + } + + public void setO2IsSet(boolean value) { + if (!value) { + this.o2 = null; + } + } + + public InvalidOperationException getO3() { + return this.o3; + } + + public void setO3(InvalidOperationException o3) { + this.o3 = o3; + } + + public void unsetO3() { + this.o3 = null; + } + + /** Returns true if field o3 is set (has been assigned a value) and false otherwise */ + public boolean isSetO3() { + return this.o3 != null; + } + + public void setO3IsSet(boolean value) { + if (!value) { + this.o3 = null; + } + } + + public AlreadyExistsException getO4() { + return this.o4; + } + + public void setO4(AlreadyExistsException o4) { + this.o4 = o4; + } + + public void unsetO4() { + this.o4 = null; + } + + /** Returns true if field o4 is set (has been assigned a value) and false otherwise */ + public boolean isSetO4() { + return this.o4 != null; + } + + public void setO4IsSet(boolean value) { + if (!value) { + this.o4 = null; + } + } + + public MetaException getO5() { + return this.o5; + } + + public void setO5(MetaException o5) { + this.o5 = o5; + } + + public void unsetO5() { + this.o5 = null; + } + + /** Returns true if field o5 is set (has been assigned a value) and false otherwise */ + public boolean isSetO5() { + return this.o5 != null; + } + + public void setO5IsSet(boolean value) { + if (!value) { + this.o5 = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((String)value); + } + break; + + case O1: + if (value == null) { + unsetO1(); + } else { + setO1((InvalidInputException)value); + } + break; + + case O2: + if (value == null) { + unsetO2(); + } else { + setO2((InvalidObjectException)value); + } + break; + + case O3: + if (value == null) { + unsetO3(); + } else { + setO3((InvalidOperationException)value); + } + break; + + case O4: + if (value == null) { + unsetO4(); + } else { + setO4((AlreadyExistsException)value); + } + break; + + case O5: + if (value == null) { + unsetO5(); + } else { + setO5((MetaException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + case O1: + return getO1(); + + case O2: + return getO2(); + + case O3: + return getO3(); + + case O4: + return getO4(); + + case O5: + return getO5(); + + } + throw new 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 IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + case O1: + return isSetO1(); + case O2: + return isSetO2(); + case O3: + return isSetO3(); + case O4: + return isSetO4(); + case O5: + return isSetO5(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof rollStreamingPartition_result) + return this.equals((rollStreamingPartition_result)that); + return false; + } + + public boolean equals(rollStreamingPartition_result that) { + if (that == null) + return false; + + 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_o1 = true && this.isSetO1(); + boolean that_present_o1 = true && that.isSetO1(); + if (this_present_o1 || that_present_o1) { + if (!(this_present_o1 && that_present_o1)) + return false; + if (!this.o1.equals(that.o1)) + return false; + } + + boolean this_present_o2 = true && this.isSetO2(); + boolean that_present_o2 = true && that.isSetO2(); + if (this_present_o2 || that_present_o2) { + if (!(this_present_o2 && that_present_o2)) + return false; + if (!this.o2.equals(that.o2)) + return false; + } + + boolean this_present_o3 = true && this.isSetO3(); + boolean that_present_o3 = true && that.isSetO3(); + if (this_present_o3 || that_present_o3) { + if (!(this_present_o3 && that_present_o3)) + return false; + if (!this.o3.equals(that.o3)) + return false; + } + + boolean this_present_o4 = true && this.isSetO4(); + boolean that_present_o4 = true && that.isSetO4(); + if (this_present_o4 || that_present_o4) { + if (!(this_present_o4 && that_present_o4)) + return false; + if (!this.o4.equals(that.o4)) + return false; + } + + boolean this_present_o5 = true && this.isSetO5(); + boolean that_present_o5 = true && that.isSetO5(); + if (this_present_o5 || that_present_o5) { + if (!(this_present_o5 && that_present_o5)) + return false; + if (!this.o5.equals(that.o5)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + boolean present_o1 = true && (isSetO1()); + builder.append(present_o1); + if (present_o1) + builder.append(o1); + + boolean present_o2 = true && (isSetO2()); + builder.append(present_o2); + if (present_o2) + builder.append(o2); + + boolean present_o3 = true && (isSetO3()); + builder.append(present_o3); + if (present_o3) + builder.append(o3); + + boolean present_o4 = true && (isSetO4()); + builder.append(present_o4); + if (present_o4) + builder.append(o4); + + boolean present_o5 = true && (isSetO5()); + builder.append(present_o5); + if (present_o5) + builder.append(o5); + + return builder.toHashCode(); + } + + public int compareTo(rollStreamingPartition_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + rollStreamingPartition_result typedOther = (rollStreamingPartition_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO1()).compareTo(typedOther.isSetO1()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO1()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, typedOther.o1); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO2()).compareTo(typedOther.isSetO2()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO2()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o2, typedOther.o2); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO3()).compareTo(typedOther.isSetO3()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO3()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o3, typedOther.o3); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO4()).compareTo(typedOther.isSetO4()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO4()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o4, typedOther.o4); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO5()).compareTo(typedOther.isSetO5()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO5()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o5, typedOther.o5); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("rollStreamingPartition_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("o1:"); + if (this.o1 == null) { + sb.append("null"); + } else { + sb.append(this.o1); + } + first = false; + if (!first) sb.append(", "); + sb.append("o2:"); + if (this.o2 == null) { + sb.append("null"); + } else { + sb.append(this.o2); + } + first = false; + if (!first) sb.append(", "); + sb.append("o3:"); + if (this.o3 == null) { + sb.append("null"); + } else { + sb.append(this.o3); + } + first = false; + if (!first) sb.append(", "); + sb.append("o4:"); + if (this.o4 == null) { + sb.append("null"); + } else { + sb.append(this.o4); + } + first = false; + if (!first) sb.append(", "); + sb.append("o5:"); + if (this.o5 == null) { + sb.append("null"); + } else { + sb.append(this.o5); + } + 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, 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 rollStreamingPartition_resultStandardSchemeFactory implements SchemeFactory { + public rollStreamingPartition_resultStandardScheme getScheme() { + return new rollStreamingPartition_resultStandardScheme(); + } + } + + private static class rollStreamingPartition_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, rollStreamingPartition_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; + case 1: // O1 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o1 = new InvalidInputException(); + struct.o1.read(iprot); + struct.setO1IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // O2 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o2 = new InvalidObjectException(); + struct.o2.read(iprot); + struct.setO2IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // O3 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o3 = new InvalidOperationException(); + struct.o3.read(iprot); + struct.setO3IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // O4 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o4 = new AlreadyExistsException(); + struct.o4.read(iprot); + struct.setO4IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // O5 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o5 = new MetaException(); + struct.o5.read(iprot); + struct.setO5IsSet(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(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, rollStreamingPartition_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(); + } + if (struct.o1 != null) { + oprot.writeFieldBegin(O1_FIELD_DESC); + struct.o1.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o2 != null) { + oprot.writeFieldBegin(O2_FIELD_DESC); + struct.o2.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o3 != null) { + oprot.writeFieldBegin(O3_FIELD_DESC); + struct.o3.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o4 != null) { + oprot.writeFieldBegin(O4_FIELD_DESC); + struct.o4.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o5 != null) { + oprot.writeFieldBegin(O5_FIELD_DESC); + struct.o5.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class rollStreamingPartition_resultTupleSchemeFactory implements SchemeFactory { + public rollStreamingPartition_resultTupleScheme getScheme() { + return new rollStreamingPartition_resultTupleScheme(); + } + } + + private static class rollStreamingPartition_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, rollStreamingPartition_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + if (struct.isSetO1()) { + optionals.set(1); + } + if (struct.isSetO2()) { + optionals.set(2); + } + if (struct.isSetO3()) { + optionals.set(3); + } + if (struct.isSetO4()) { + optionals.set(4); + } + if (struct.isSetO5()) { + optionals.set(5); + } + oprot.writeBitSet(optionals, 6); + if (struct.isSetSuccess()) { + oprot.writeString(struct.success); + } + if (struct.isSetO1()) { + struct.o1.write(oprot); + } + if (struct.isSetO2()) { + struct.o2.write(oprot); + } + if (struct.isSetO3()) { + struct.o3.write(oprot); + } + if (struct.isSetO4()) { + struct.o4.write(oprot); + } + if (struct.isSetO5()) { + struct.o5.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, rollStreamingPartition_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(6); + if (incoming.get(0)) { + struct.success = iprot.readString(); + struct.setSuccessIsSet(true); + } + if (incoming.get(1)) { + struct.o1 = new InvalidInputException(); + struct.o1.read(iprot); + struct.setO1IsSet(true); + } + if (incoming.get(2)) { + struct.o2 = new InvalidObjectException(); + struct.o2.read(iprot); + struct.setO2IsSet(true); + } + if (incoming.get(3)) { + struct.o3 = new InvalidOperationException(); + struct.o3.read(iprot); + struct.setO3IsSet(true); + } + if (incoming.get(4)) { + struct.o4 = new AlreadyExistsException(); + struct.o4.read(iprot); + struct.setO4IsSet(true); + } + if (incoming.get(5)) { + struct.o5 = new MetaException(); + struct.o5.read(iprot); + struct.setO5IsSet(true); + } + } + } + + } + + public static class updateStreamingTempLocation_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("updateStreamingTempLocation_args"); + + private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField STREAMING_TEMP_DIR_FIELD_DESC = new org.apache.thrift.protocol.TField("streamingTempDir", org.apache.thrift.protocol.TType.STRING, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new updateStreamingTempLocation_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new updateStreamingTempLocation_argsTupleSchemeFactory()); + } + + private String dbName; // required + private String tableName; // required + private String streamingTempDir; // 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 { + DB_NAME((short)1, "dbName"), + TABLE_NAME((short)2, "tableName"), + STREAMING_TEMP_DIR((short)3, "streamingTempDir"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // DB_NAME + return DB_NAME; + case 2: // TABLE_NAME + return TABLE_NAME; + case 3: // STREAMING_TEMP_DIR + return STREAMING_TEMP_DIR; + 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 IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("dbName", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.STREAMING_TEMP_DIR, new org.apache.thrift.meta_data.FieldMetaData("streamingTempDir", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(updateStreamingTempLocation_args.class, metaDataMap); + } + + public updateStreamingTempLocation_args() { + } + + public updateStreamingTempLocation_args( + String dbName, + String tableName, + String streamingTempDir) + { + this(); + this.dbName = dbName; + this.tableName = tableName; + this.streamingTempDir = streamingTempDir; + } + + /** + * Performs a deep copy on other. + */ + public updateStreamingTempLocation_args(updateStreamingTempLocation_args other) { + if (other.isSetDbName()) { + this.dbName = other.dbName; + } + if (other.isSetTableName()) { + this.tableName = other.tableName; + } + if (other.isSetStreamingTempDir()) { + this.streamingTempDir = other.streamingTempDir; + } + } + + public updateStreamingTempLocation_args deepCopy() { + return new updateStreamingTempLocation_args(this); + } + + @Override + public void clear() { + this.dbName = null; + this.tableName = null; + this.streamingTempDir = null; + } + + public String getDbName() { + return this.dbName; + } + + public void setDbName(String dbName) { + this.dbName = dbName; + } + + public void unsetDbName() { + this.dbName = null; + } + + /** Returns true if field dbName is set (has been assigned a value) and false otherwise */ + public boolean isSetDbName() { + return this.dbName != null; + } + + public void setDbNameIsSet(boolean value) { + if (!value) { + this.dbName = null; + } + } + + public String getTableName() { + return this.tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + 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 String getStreamingTempDir() { + return this.streamingTempDir; + } + + public void setStreamingTempDir(String streamingTempDir) { + this.streamingTempDir = streamingTempDir; + } + + public void unsetStreamingTempDir() { + this.streamingTempDir = null; + } + + /** Returns true if field streamingTempDir is set (has been assigned a value) and false otherwise */ + public boolean isSetStreamingTempDir() { + return this.streamingTempDir != null; + } + + public void setStreamingTempDirIsSet(boolean value) { + if (!value) { + this.streamingTempDir = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case DB_NAME: + if (value == null) { + unsetDbName(); + } else { + setDbName((String)value); + } + break; + + case TABLE_NAME: + if (value == null) { + unsetTableName(); + } else { + setTableName((String)value); + } + break; + + case STREAMING_TEMP_DIR: + if (value == null) { + unsetStreamingTempDir(); + } else { + setStreamingTempDir((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case DB_NAME: + return getDbName(); + + case TABLE_NAME: + return getTableName(); + + case STREAMING_TEMP_DIR: + return getStreamingTempDir(); + + } + throw new 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 IllegalArgumentException(); + } + + switch (field) { + case DB_NAME: + return isSetDbName(); + case TABLE_NAME: + return isSetTableName(); + case STREAMING_TEMP_DIR: + return isSetStreamingTempDir(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof updateStreamingTempLocation_args) + return this.equals((updateStreamingTempLocation_args)that); + return false; + } + + public boolean equals(updateStreamingTempLocation_args that) { + if (that == null) + return false; + + boolean this_present_dbName = true && this.isSetDbName(); + boolean that_present_dbName = true && that.isSetDbName(); + if (this_present_dbName || that_present_dbName) { + if (!(this_present_dbName && that_present_dbName)) + return false; + if (!this.dbName.equals(that.dbName)) + return false; + } + + 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_streamingTempDir = true && this.isSetStreamingTempDir(); + boolean that_present_streamingTempDir = true && that.isSetStreamingTempDir(); + if (this_present_streamingTempDir || that_present_streamingTempDir) { + if (!(this_present_streamingTempDir && that_present_streamingTempDir)) + return false; + if (!this.streamingTempDir.equals(that.streamingTempDir)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_dbName = true && (isSetDbName()); + builder.append(present_dbName); + if (present_dbName) + builder.append(dbName); + + boolean present_tableName = true && (isSetTableName()); + builder.append(present_tableName); + if (present_tableName) + builder.append(tableName); + + boolean present_streamingTempDir = true && (isSetStreamingTempDir()); + builder.append(present_streamingTempDir); + if (present_streamingTempDir) + builder.append(streamingTempDir); + + return builder.toHashCode(); + } + + public int compareTo(updateStreamingTempLocation_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + updateStreamingTempLocation_args typedOther = (updateStreamingTempLocation_args)other; + + lastComparison = Boolean.valueOf(isSetDbName()).compareTo(typedOther.isSetDbName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetDbName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.dbName, typedOther.dbName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTableName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetStreamingTempDir()).compareTo(typedOther.isSetStreamingTempDir()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStreamingTempDir()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.streamingTempDir, typedOther.streamingTempDir); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("updateStreamingTempLocation_args("); + boolean first = true; + + sb.append("dbName:"); + if (this.dbName == null) { + sb.append("null"); + } else { + sb.append(this.dbName); + } + first = false; + if (!first) sb.append(", "); + sb.append("tableName:"); + if (this.tableName == null) { + sb.append("null"); + } else { + sb.append(this.tableName); + } + first = false; + if (!first) sb.append(", "); + sb.append("streamingTempDir:"); + if (this.streamingTempDir == null) { + sb.append("null"); + } else { + sb.append(this.streamingTempDir); + } + 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, 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 updateStreamingTempLocation_argsStandardSchemeFactory implements SchemeFactory { + public updateStreamingTempLocation_argsStandardScheme getScheme() { + return new updateStreamingTempLocation_argsStandardScheme(); + } + } + + private static class updateStreamingTempLocation_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, updateStreamingTempLocation_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: // DB_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.dbName = iprot.readString(); + struct.setDbNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // TABLE_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.tableName = iprot.readString(); + struct.setTableNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // STREAMING_TEMP_DIR + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.streamingTempDir = iprot.readString(); + struct.setStreamingTempDirIsSet(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(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, updateStreamingTempLocation_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.dbName != null) { + oprot.writeFieldBegin(DB_NAME_FIELD_DESC); + oprot.writeString(struct.dbName); + oprot.writeFieldEnd(); + } + if (struct.tableName != null) { + oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC); + oprot.writeString(struct.tableName); + oprot.writeFieldEnd(); + } + if (struct.streamingTempDir != null) { + oprot.writeFieldBegin(STREAMING_TEMP_DIR_FIELD_DESC); + oprot.writeString(struct.streamingTempDir); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class updateStreamingTempLocation_argsTupleSchemeFactory implements SchemeFactory { + public updateStreamingTempLocation_argsTupleScheme getScheme() { + return new updateStreamingTempLocation_argsTupleScheme(); + } + } + + private static class updateStreamingTempLocation_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, updateStreamingTempLocation_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetDbName()) { + optionals.set(0); + } + if (struct.isSetTableName()) { + optionals.set(1); + } + if (struct.isSetStreamingTempDir()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); + if (struct.isSetDbName()) { + oprot.writeString(struct.dbName); + } + if (struct.isSetTableName()) { + oprot.writeString(struct.tableName); + } + if (struct.isSetStreamingTempDir()) { + oprot.writeString(struct.streamingTempDir); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, updateStreamingTempLocation_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(3); + if (incoming.get(0)) { + struct.dbName = iprot.readString(); + struct.setDbNameIsSet(true); + } + if (incoming.get(1)) { + struct.tableName = iprot.readString(); + struct.setTableNameIsSet(true); + } + if (incoming.get(2)) { + struct.streamingTempDir = iprot.readString(); + struct.setStreamingTempDirIsSet(true); + } + } + } + + } + + public static class updateStreamingTempLocation_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("updateStreamingTempLocation_result"); + + private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField O2_FIELD_DESC = new org.apache.thrift.protocol.TField("o2", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField O3_FIELD_DESC = new org.apache.thrift.protocol.TField("o3", org.apache.thrift.protocol.TType.STRUCT, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new updateStreamingTempLocation_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new updateStreamingTempLocation_resultTupleSchemeFactory()); + } + + private InvalidInputException o1; // required + private InvalidObjectException o2; // required + private MetaException o3; // 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 { + O1((short)1, "o1"), + O2((short)2, "o2"), + O3((short)3, "o3"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // O1 + return O1; + case 2: // O2 + return O2; + case 3: // O3 + return O3; + 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 IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O2, new org.apache.thrift.meta_data.FieldMetaData("o2", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O3, new org.apache.thrift.meta_data.FieldMetaData("o3", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(updateStreamingTempLocation_result.class, metaDataMap); + } + + public updateStreamingTempLocation_result() { + } + + public updateStreamingTempLocation_result( + InvalidInputException o1, + InvalidObjectException o2, + MetaException o3) + { + this(); + this.o1 = o1; + this.o2 = o2; + this.o3 = o3; + } + + /** + * Performs a deep copy on other. + */ + public updateStreamingTempLocation_result(updateStreamingTempLocation_result other) { + if (other.isSetO1()) { + this.o1 = new InvalidInputException(other.o1); + } + if (other.isSetO2()) { + this.o2 = new InvalidObjectException(other.o2); + } + if (other.isSetO3()) { + this.o3 = new MetaException(other.o3); + } + } + + public updateStreamingTempLocation_result deepCopy() { + return new updateStreamingTempLocation_result(this); + } + + @Override + public void clear() { + this.o1 = null; + this.o2 = null; + this.o3 = null; + } + + public InvalidInputException getO1() { + return this.o1; + } + + public void setO1(InvalidInputException o1) { + this.o1 = o1; + } + + public void unsetO1() { + this.o1 = null; + } + + /** Returns true if field o1 is set (has been assigned a value) and false otherwise */ + public boolean isSetO1() { + return this.o1 != null; + } + + public void setO1IsSet(boolean value) { + if (!value) { + this.o1 = null; + } + } + + public InvalidObjectException getO2() { + return this.o2; + } + + public void setO2(InvalidObjectException o2) { + this.o2 = o2; + } + + public void unsetO2() { + this.o2 = null; + } + + /** Returns true if field o2 is set (has been assigned a value) and false otherwise */ + public boolean isSetO2() { + return this.o2 != null; + } + + public void setO2IsSet(boolean value) { + if (!value) { + this.o2 = null; + } + } + + public MetaException getO3() { + return this.o3; + } + + public void setO3(MetaException o3) { + this.o3 = o3; + } + + public void unsetO3() { + this.o3 = null; + } + + /** Returns true if field o3 is set (has been assigned a value) and false otherwise */ + public boolean isSetO3() { + return this.o3 != null; + } + + public void setO3IsSet(boolean value) { + if (!value) { + this.o3 = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case O1: + if (value == null) { + unsetO1(); + } else { + setO1((InvalidInputException)value); + } + break; + + case O2: + if (value == null) { + unsetO2(); + } else { + setO2((InvalidObjectException)value); + } + break; + + case O3: + if (value == null) { + unsetO3(); + } else { + setO3((MetaException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case O1: + return getO1(); + + case O2: + return getO2(); + + case O3: + return getO3(); + + } + throw new 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 IllegalArgumentException(); + } + + switch (field) { + case O1: + return isSetO1(); + case O2: + return isSetO2(); + case O3: + return isSetO3(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof updateStreamingTempLocation_result) + return this.equals((updateStreamingTempLocation_result)that); + return false; + } + + public boolean equals(updateStreamingTempLocation_result that) { + if (that == null) + return false; + + boolean this_present_o1 = true && this.isSetO1(); + boolean that_present_o1 = true && that.isSetO1(); + if (this_present_o1 || that_present_o1) { + if (!(this_present_o1 && that_present_o1)) + return false; + if (!this.o1.equals(that.o1)) + return false; + } + + boolean this_present_o2 = true && this.isSetO2(); + boolean that_present_o2 = true && that.isSetO2(); + if (this_present_o2 || that_present_o2) { + if (!(this_present_o2 && that_present_o2)) + return false; + if (!this.o2.equals(that.o2)) + return false; + } + + boolean this_present_o3 = true && this.isSetO3(); + boolean that_present_o3 = true && that.isSetO3(); + if (this_present_o3 || that_present_o3) { + if (!(this_present_o3 && that_present_o3)) + return false; + if (!this.o3.equals(that.o3)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_o1 = true && (isSetO1()); + builder.append(present_o1); + if (present_o1) + builder.append(o1); + + boolean present_o2 = true && (isSetO2()); + builder.append(present_o2); + if (present_o2) + builder.append(o2); + + boolean present_o3 = true && (isSetO3()); + builder.append(present_o3); + if (present_o3) + builder.append(o3); + + return builder.toHashCode(); + } + + public int compareTo(updateStreamingTempLocation_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + updateStreamingTempLocation_result typedOther = (updateStreamingTempLocation_result)other; + + lastComparison = Boolean.valueOf(isSetO1()).compareTo(typedOther.isSetO1()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO1()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, typedOther.o1); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO2()).compareTo(typedOther.isSetO2()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO2()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o2, typedOther.o2); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO3()).compareTo(typedOther.isSetO3()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO3()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o3, typedOther.o3); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("updateStreamingTempLocation_result("); + boolean first = true; + + sb.append("o1:"); + if (this.o1 == null) { + sb.append("null"); + } else { + sb.append(this.o1); + } + first = false; + if (!first) sb.append(", "); + sb.append("o2:"); + if (this.o2 == null) { + sb.append("null"); + } else { + sb.append(this.o2); + } + first = false; + if (!first) sb.append(", "); + sb.append("o3:"); + if (this.o3 == null) { + sb.append("null"); + } else { + sb.append(this.o3); + } + 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, 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 updateStreamingTempLocation_resultStandardSchemeFactory implements SchemeFactory { + public updateStreamingTempLocation_resultStandardScheme getScheme() { + return new updateStreamingTempLocation_resultStandardScheme(); + } + } + + private static class updateStreamingTempLocation_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, updateStreamingTempLocation_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: // O1 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o1 = new InvalidInputException(); + struct.o1.read(iprot); + struct.setO1IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // O2 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o2 = new InvalidObjectException(); + struct.o2.read(iprot); + struct.setO2IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // O3 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o3 = new MetaException(); + struct.o3.read(iprot); + struct.setO3IsSet(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(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, updateStreamingTempLocation_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.o1 != null) { + oprot.writeFieldBegin(O1_FIELD_DESC); + struct.o1.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o2 != null) { + oprot.writeFieldBegin(O2_FIELD_DESC); + struct.o2.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o3 != null) { + oprot.writeFieldBegin(O3_FIELD_DESC); + struct.o3.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class updateStreamingTempLocation_resultTupleSchemeFactory implements SchemeFactory { + public updateStreamingTempLocation_resultTupleScheme getScheme() { + return new updateStreamingTempLocation_resultTupleScheme(); + } + } + + private static class updateStreamingTempLocation_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, updateStreamingTempLocation_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetO1()) { + optionals.set(0); + } + if (struct.isSetO2()) { + optionals.set(1); + } + if (struct.isSetO3()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); + if (struct.isSetO1()) { + struct.o1.write(oprot); + } + if (struct.isSetO2()) { + struct.o2.write(oprot); + } + if (struct.isSetO3()) { + struct.o3.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, updateStreamingTempLocation_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(3); + if (incoming.get(0)) { + struct.o1 = new InvalidInputException(); + struct.o1.read(iprot); + struct.setO1IsSet(true); + } + if (incoming.get(1)) { + struct.o2 = new InvalidObjectException(); + struct.o2.read(iprot); + struct.setO2IsSet(true); + } + if (incoming.get(2)) { + struct.o3 = new MetaException(); + struct.o3.read(iprot); + struct.setO3IsSet(true); + } + } + } + + } + + public static class getNextChunkID_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getNextChunkID_args"); + + private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("db_name", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("table_name", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getNextChunkID_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getNextChunkID_argsTupleSchemeFactory()); + } + + private String db_name; // required + private String table_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 { + DB_NAME((short)1, "db_name"), + TABLE_NAME((short)2, "table_name"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // DB_NAME + return DB_NAME; + case 2: // 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 IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("db_name", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("table_name", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getNextChunkID_args.class, metaDataMap); + } + + public getNextChunkID_args() { + } + + public getNextChunkID_args( + String db_name, + String table_name) + { + this(); + this.db_name = db_name; + this.table_name = table_name; + } + + /** + * Performs a deep copy on other. + */ + public getNextChunkID_args(getNextChunkID_args other) { + if (other.isSetDb_name()) { + this.db_name = other.db_name; + } + if (other.isSetTable_name()) { + this.table_name = other.table_name; + } + } + + public getNextChunkID_args deepCopy() { + return new getNextChunkID_args(this); + } + + @Override + public void clear() { + this.db_name = null; + this.table_name = null; + } + + public String getDb_name() { + return this.db_name; + } + + public void setDb_name(String db_name) { + this.db_name = db_name; + } + + public void unsetDb_name() { + this.db_name = null; + } + + /** Returns true if field db_name is set (has been assigned a value) and false otherwise */ + public boolean isSetDb_name() { + return this.db_name != null; + } + + public void setDb_nameIsSet(boolean value) { + if (!value) { + this.db_name = null; + } + } + + public String getTable_name() { + return this.table_name; } - public void setCol_name(String col_name) { - this.col_name = col_name; + public void setTable_name(String table_name) { + this.table_name = table_name; } - public void unsetCol_name() { - this.col_name = null; + public void unsetTable_name() { + this.table_name = null; } - /** Returns true if field col_name is set (has been assigned a value) and false otherwise */ - public boolean isSetCol_name() { - return this.col_name != null; + /** Returns true if field table_name is set (has been assigned a value) and false otherwise */ + public boolean isSetTable_name() { + return this.table_name != null; } - public void setCol_nameIsSet(boolean value) { + public void setTable_nameIsSet(boolean value) { if (!value) { - this.col_name = null; + this.table_name = null; } } @@ -85571,27 +96891,11 @@ public void setFieldValue(_Fields field, Object value) { } break; - case TBL_NAME: - if (value == null) { - unsetTbl_name(); - } else { - setTbl_name((String)value); - } - break; - - case PART_NAME: - if (value == null) { - unsetPart_name(); - } else { - setPart_name((String)value); - } - break; - - case COL_NAME: + case TABLE_NAME: if (value == null) { - unsetCol_name(); + unsetTable_name(); } else { - setCol_name((String)value); + setTable_name((String)value); } break; @@ -85603,14 +96907,8 @@ public Object getFieldValue(_Fields field) { case DB_NAME: return getDb_name(); - case TBL_NAME: - return getTbl_name(); - - case PART_NAME: - return getPart_name(); - - case COL_NAME: - return getCol_name(); + case TABLE_NAME: + return getTable_name(); } throw new IllegalStateException(); @@ -85625,12 +96923,8 @@ public boolean isSet(_Fields field) { switch (field) { case DB_NAME: return isSetDb_name(); - case TBL_NAME: - return isSetTbl_name(); - case PART_NAME: - return isSetPart_name(); - case COL_NAME: - return isSetCol_name(); + case TABLE_NAME: + return isSetTable_name(); } throw new IllegalStateException(); } @@ -85639,12 +96933,12 @@ public boolean isSet(_Fields field) { public boolean equals(Object that) { if (that == null) return false; - if (that instanceof delete_partition_column_statistics_args) - return this.equals((delete_partition_column_statistics_args)that); + if (that instanceof getNextChunkID_args) + return this.equals((getNextChunkID_args)that); return false; } - public boolean equals(delete_partition_column_statistics_args that) { + public boolean equals(getNextChunkID_args that) { if (that == null) return false; @@ -85657,30 +96951,12 @@ public boolean equals(delete_partition_column_statistics_args that) { return false; } - boolean this_present_tbl_name = true && this.isSetTbl_name(); - boolean that_present_tbl_name = true && that.isSetTbl_name(); - if (this_present_tbl_name || that_present_tbl_name) { - if (!(this_present_tbl_name && that_present_tbl_name)) - return false; - if (!this.tbl_name.equals(that.tbl_name)) - return false; - } - - boolean this_present_part_name = true && this.isSetPart_name(); - boolean that_present_part_name = true && that.isSetPart_name(); - if (this_present_part_name || that_present_part_name) { - if (!(this_present_part_name && that_present_part_name)) - return false; - if (!this.part_name.equals(that.part_name)) - return false; - } - - boolean this_present_col_name = true && this.isSetCol_name(); - boolean that_present_col_name = true && that.isSetCol_name(); - if (this_present_col_name || that_present_col_name) { - if (!(this_present_col_name && that_present_col_name)) + boolean this_present_table_name = true && this.isSetTable_name(); + boolean that_present_table_name = true && that.isSetTable_name(); + if (this_present_table_name || that_present_table_name) { + if (!(this_present_table_name && that_present_table_name)) return false; - if (!this.col_name.equals(that.col_name)) + if (!this.table_name.equals(that.table_name)) return false; } @@ -85696,31 +96972,21 @@ public int hashCode() { if (present_db_name) builder.append(db_name); - boolean present_tbl_name = true && (isSetTbl_name()); - builder.append(present_tbl_name); - if (present_tbl_name) - builder.append(tbl_name); - - boolean present_part_name = true && (isSetPart_name()); - builder.append(present_part_name); - if (present_part_name) - builder.append(part_name); - - boolean present_col_name = true && (isSetCol_name()); - builder.append(present_col_name); - if (present_col_name) - builder.append(col_name); + boolean present_table_name = true && (isSetTable_name()); + builder.append(present_table_name); + if (present_table_name) + builder.append(table_name); return builder.toHashCode(); } - public int compareTo(delete_partition_column_statistics_args other) { + public int compareTo(getNextChunkID_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - delete_partition_column_statistics_args typedOther = (delete_partition_column_statistics_args)other; + getNextChunkID_args typedOther = (getNextChunkID_args)other; lastComparison = Boolean.valueOf(isSetDb_name()).compareTo(typedOther.isSetDb_name()); if (lastComparison != 0) { @@ -85732,32 +96998,12 @@ public int compareTo(delete_partition_column_statistics_args other) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetTbl_name()).compareTo(typedOther.isSetTbl_name()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetTbl_name()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tbl_name, typedOther.tbl_name); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetPart_name()).compareTo(typedOther.isSetPart_name()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetPart_name()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.part_name, typedOther.part_name); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetCol_name()).compareTo(typedOther.isSetCol_name()); + lastComparison = Boolean.valueOf(isSetTable_name()).compareTo(typedOther.isSetTable_name()); if (lastComparison != 0) { return lastComparison; } - if (isSetCol_name()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.col_name, typedOther.col_name); + if (isSetTable_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table_name, typedOther.table_name); if (lastComparison != 0) { return lastComparison; } @@ -85779,7 +97025,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public String toString() { - StringBuilder sb = new StringBuilder("delete_partition_column_statistics_args("); + StringBuilder sb = new StringBuilder("getNextChunkID_args("); boolean first = true; sb.append("db_name:"); @@ -85790,27 +97036,11 @@ public String toString() { } first = false; if (!first) sb.append(", "); - sb.append("tbl_name:"); - if (this.tbl_name == null) { - sb.append("null"); - } else { - sb.append(this.tbl_name); - } - first = false; - if (!first) sb.append(", "); - sb.append("part_name:"); - if (this.part_name == null) { - sb.append("null"); - } else { - sb.append(this.part_name); - } - first = false; - if (!first) sb.append(", "); - sb.append("col_name:"); - if (this.col_name == null) { + sb.append("table_name:"); + if (this.table_name == null) { sb.append("null"); } else { - sb.append(this.col_name); + sb.append(this.table_name); } first = false; sb.append(")"); @@ -85838,15 +97068,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class delete_partition_column_statistics_argsStandardSchemeFactory implements SchemeFactory { - public delete_partition_column_statistics_argsStandardScheme getScheme() { - return new delete_partition_column_statistics_argsStandardScheme(); + private static class getNextChunkID_argsStandardSchemeFactory implements SchemeFactory { + public getNextChunkID_argsStandardScheme getScheme() { + return new getNextChunkID_argsStandardScheme(); } } - private static class delete_partition_column_statistics_argsStandardScheme extends StandardScheme { + private static class getNextChunkID_argsStandardScheme extends StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, delete_partition_column_statistics_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getNextChunkID_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -85864,26 +97094,10 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, delete_partition_co org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 2: // TBL_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.tbl_name = iprot.readString(); - struct.setTbl_nameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // PART_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.part_name = iprot.readString(); - struct.setPart_nameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 4: // COL_NAME + case 2: // TABLE_NAME if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.col_name = iprot.readString(); - struct.setCol_nameIsSet(true); + struct.table_name = iprot.readString(); + struct.setTable_nameIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -85897,7 +97111,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, delete_partition_co struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, delete_partition_column_statistics_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getNextChunkID_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -85906,19 +97120,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, delete_partition_c oprot.writeString(struct.db_name); oprot.writeFieldEnd(); } - if (struct.tbl_name != null) { - oprot.writeFieldBegin(TBL_NAME_FIELD_DESC); - oprot.writeString(struct.tbl_name); - oprot.writeFieldEnd(); - } - if (struct.part_name != null) { - oprot.writeFieldBegin(PART_NAME_FIELD_DESC); - oprot.writeString(struct.part_name); - oprot.writeFieldEnd(); - } - if (struct.col_name != null) { - oprot.writeFieldBegin(COL_NAME_FIELD_DESC); - oprot.writeString(struct.col_name); + if (struct.table_name != null) { + oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC); + oprot.writeString(struct.table_name); oprot.writeFieldEnd(); } oprot.writeFieldStop(); @@ -85927,74 +97131,54 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, delete_partition_c } - private static class delete_partition_column_statistics_argsTupleSchemeFactory implements SchemeFactory { - public delete_partition_column_statistics_argsTupleScheme getScheme() { - return new delete_partition_column_statistics_argsTupleScheme(); + private static class getNextChunkID_argsTupleSchemeFactory implements SchemeFactory { + public getNextChunkID_argsTupleScheme getScheme() { + return new getNextChunkID_argsTupleScheme(); } } - private static class delete_partition_column_statistics_argsTupleScheme extends TupleScheme { + private static class getNextChunkID_argsTupleScheme extends TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, delete_partition_column_statistics_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getNextChunkID_args struct) throws org.apache.thrift.TException { TTupleProtocol oprot = (TTupleProtocol) prot; BitSet optionals = new BitSet(); if (struct.isSetDb_name()) { optionals.set(0); } - if (struct.isSetTbl_name()) { + if (struct.isSetTable_name()) { optionals.set(1); } - if (struct.isSetPart_name()) { - optionals.set(2); - } - if (struct.isSetCol_name()) { - optionals.set(3); - } - oprot.writeBitSet(optionals, 4); + oprot.writeBitSet(optionals, 2); if (struct.isSetDb_name()) { oprot.writeString(struct.db_name); } - if (struct.isSetTbl_name()) { - oprot.writeString(struct.tbl_name); - } - if (struct.isSetPart_name()) { - oprot.writeString(struct.part_name); - } - if (struct.isSetCol_name()) { - oprot.writeString(struct.col_name); + if (struct.isSetTable_name()) { + oprot.writeString(struct.table_name); } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, delete_partition_column_statistics_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getNextChunkID_args struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(4); + BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { struct.db_name = iprot.readString(); struct.setDb_nameIsSet(true); } if (incoming.get(1)) { - struct.tbl_name = iprot.readString(); - struct.setTbl_nameIsSet(true); - } - if (incoming.get(2)) { - struct.part_name = iprot.readString(); - struct.setPart_nameIsSet(true); - } - if (incoming.get(3)) { - struct.col_name = iprot.readString(); - struct.setCol_nameIsSet(true); + struct.table_name = iprot.readString(); + struct.setTable_nameIsSet(true); } } } } - public static class delete_partition_column_statistics_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("delete_partition_column_statistics_result"); + public static class getNextChunkID_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getNextChunkID_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 SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.I64, (short)0); private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1); private static final org.apache.thrift.protocol.TField O2_FIELD_DESC = new org.apache.thrift.protocol.TField("o2", org.apache.thrift.protocol.TType.STRUCT, (short)2); private static final org.apache.thrift.protocol.TField O3_FIELD_DESC = new org.apache.thrift.protocol.TField("o3", org.apache.thrift.protocol.TType.STRUCT, (short)3); @@ -86002,15 +97186,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, delete_partition_col private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { - schemes.put(StandardScheme.class, new delete_partition_column_statistics_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new delete_partition_column_statistics_resultTupleSchemeFactory()); + schemes.put(StandardScheme.class, new getNextChunkID_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getNextChunkID_resultTupleSchemeFactory()); } - private boolean success; // required - private NoSuchObjectException o1; // required - private MetaException o2; // required - private InvalidObjectException o3; // required - private InvalidInputException o4; // required + private long success; // required + private InvalidInputException o1; // required + private InvalidObjectException o2; // required + private InvalidOperationException o3; // required + private MetaException o4; // 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 { @@ -86089,7 +97273,7 @@ public String getFieldName() { static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new 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))); + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); tmpMap.put(_Fields.O2, new org.apache.thrift.meta_data.FieldMetaData("o2", org.apache.thrift.TFieldRequirementType.DEFAULT, @@ -86099,18 +97283,18 @@ public String getFieldName() { tmpMap.put(_Fields.O4, new org.apache.thrift.meta_data.FieldMetaData("o4", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(delete_partition_column_statistics_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getNextChunkID_result.class, metaDataMap); } - public delete_partition_column_statistics_result() { + public getNextChunkID_result() { } - public delete_partition_column_statistics_result( - boolean success, - NoSuchObjectException o1, - MetaException o2, - InvalidObjectException o3, - InvalidInputException o4) + public getNextChunkID_result( + long success, + InvalidInputException o1, + InvalidObjectException o2, + InvalidOperationException o3, + MetaException o4) { this(); this.success = success; @@ -86124,42 +97308,42 @@ public delete_partition_column_statistics_result( /** * Performs a deep copy on other. */ - public delete_partition_column_statistics_result(delete_partition_column_statistics_result other) { + public getNextChunkID_result(getNextChunkID_result other) { __isset_bitfield = other.__isset_bitfield; this.success = other.success; if (other.isSetO1()) { - this.o1 = new NoSuchObjectException(other.o1); + this.o1 = new InvalidInputException(other.o1); } if (other.isSetO2()) { - this.o2 = new MetaException(other.o2); + this.o2 = new InvalidObjectException(other.o2); } if (other.isSetO3()) { - this.o3 = new InvalidObjectException(other.o3); + this.o3 = new InvalidOperationException(other.o3); } if (other.isSetO4()) { - this.o4 = new InvalidInputException(other.o4); + this.o4 = new MetaException(other.o4); } } - public delete_partition_column_statistics_result deepCopy() { - return new delete_partition_column_statistics_result(this); + public getNextChunkID_result deepCopy() { + return new getNextChunkID_result(this); } @Override public void clear() { setSuccessIsSet(false); - this.success = false; + this.success = 0; this.o1 = null; this.o2 = null; this.o3 = null; this.o4 = null; } - public boolean isSuccess() { + public long getSuccess() { return this.success; } - public void setSuccess(boolean success) { + public void setSuccess(long success) { this.success = success; setSuccessIsSet(true); } @@ -86177,11 +97361,11 @@ public void setSuccessIsSet(boolean value) { __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value); } - public NoSuchObjectException getO1() { + public InvalidInputException getO1() { return this.o1; } - public void setO1(NoSuchObjectException o1) { + public void setO1(InvalidInputException o1) { this.o1 = o1; } @@ -86200,11 +97384,11 @@ public void setO1IsSet(boolean value) { } } - public MetaException getO2() { + public InvalidObjectException getO2() { return this.o2; } - public void setO2(MetaException o2) { + public void setO2(InvalidObjectException o2) { this.o2 = o2; } @@ -86223,11 +97407,11 @@ public void setO2IsSet(boolean value) { } } - public InvalidObjectException getO3() { + public InvalidOperationException getO3() { return this.o3; } - public void setO3(InvalidObjectException o3) { + public void setO3(InvalidOperationException o3) { this.o3 = o3; } @@ -86246,11 +97430,11 @@ public void setO3IsSet(boolean value) { } } - public InvalidInputException getO4() { + public MetaException getO4() { return this.o4; } - public void setO4(InvalidInputException o4) { + public void setO4(MetaException o4) { this.o4 = o4; } @@ -86275,7 +97459,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unsetSuccess(); } else { - setSuccess((Boolean)value); + setSuccess((Long)value); } break; @@ -86283,7 +97467,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unsetO1(); } else { - setO1((NoSuchObjectException)value); + setO1((InvalidInputException)value); } break; @@ -86291,7 +97475,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unsetO2(); } else { - setO2((MetaException)value); + setO2((InvalidObjectException)value); } break; @@ -86299,7 +97483,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unsetO3(); } else { - setO3((InvalidObjectException)value); + setO3((InvalidOperationException)value); } break; @@ -86307,7 +97491,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unsetO4(); } else { - setO4((InvalidInputException)value); + setO4((MetaException)value); } break; @@ -86317,7 +97501,7 @@ public void setFieldValue(_Fields field, Object value) { public Object getFieldValue(_Fields field) { switch (field) { case SUCCESS: - return Boolean.valueOf(isSuccess()); + return Long.valueOf(getSuccess()); case O1: return getO1(); @@ -86360,12 +97544,12 @@ public boolean isSet(_Fields field) { public boolean equals(Object that) { if (that == null) return false; - if (that instanceof delete_partition_column_statistics_result) - return this.equals((delete_partition_column_statistics_result)that); + if (that instanceof getNextChunkID_result) + return this.equals((getNextChunkID_result)that); return false; } - public boolean equals(delete_partition_column_statistics_result that) { + public boolean equals(getNextChunkID_result that) { if (that == null) return false; @@ -86449,13 +97633,13 @@ public int hashCode() { return builder.toHashCode(); } - public int compareTo(delete_partition_column_statistics_result other) { + public int compareTo(getNextChunkID_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - delete_partition_column_statistics_result typedOther = (delete_partition_column_statistics_result)other; + getNextChunkID_result typedOther = (getNextChunkID_result)other; lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); if (lastComparison != 0) { @@ -86524,7 +97708,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public String toString() { - StringBuilder sb = new StringBuilder("delete_partition_column_statistics_result("); + StringBuilder sb = new StringBuilder("getNextChunkID_result("); boolean first = true; sb.append("success:"); @@ -86589,15 +97773,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class delete_partition_column_statistics_resultStandardSchemeFactory implements SchemeFactory { - public delete_partition_column_statistics_resultStandardScheme getScheme() { - return new delete_partition_column_statistics_resultStandardScheme(); + private static class getNextChunkID_resultStandardSchemeFactory implements SchemeFactory { + public getNextChunkID_resultStandardScheme getScheme() { + return new getNextChunkID_resultStandardScheme(); } } - private static class delete_partition_column_statistics_resultStandardScheme extends StandardScheme { + private static class getNextChunkID_resultStandardScheme extends StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, delete_partition_column_statistics_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getNextChunkID_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -86608,8 +97792,8 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, delete_partition_co } switch (schemeField.id) { case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { - struct.success = iprot.readBool(); + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.success = iprot.readI64(); struct.setSuccessIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); @@ -86617,7 +97801,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, delete_partition_co break; case 1: // O1 if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.o1 = new NoSuchObjectException(); + struct.o1 = new InvalidInputException(); struct.o1.read(iprot); struct.setO1IsSet(true); } else { @@ -86626,7 +97810,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, delete_partition_co break; case 2: // O2 if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.o2 = new MetaException(); + struct.o2 = new InvalidObjectException(); struct.o2.read(iprot); struct.setO2IsSet(true); } else { @@ -86635,7 +97819,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, delete_partition_co break; case 3: // O3 if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.o3 = new InvalidObjectException(); + struct.o3 = new InvalidOperationException(); struct.o3.read(iprot); struct.setO3IsSet(true); } else { @@ -86644,7 +97828,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, delete_partition_co break; case 4: // O4 if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.o4 = new InvalidInputException(); + struct.o4 = new MetaException(); struct.o4.read(iprot); struct.setO4IsSet(true); } else { @@ -86660,13 +97844,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, delete_partition_co struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, delete_partition_column_statistics_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getNextChunkID_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.writeI64(struct.success); oprot.writeFieldEnd(); } if (struct.o1 != null) { @@ -86695,16 +97879,16 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, delete_partition_c } - private static class delete_partition_column_statistics_resultTupleSchemeFactory implements SchemeFactory { - public delete_partition_column_statistics_resultTupleScheme getScheme() { - return new delete_partition_column_statistics_resultTupleScheme(); + private static class getNextChunkID_resultTupleSchemeFactory implements SchemeFactory { + public getNextChunkID_resultTupleScheme getScheme() { + return new getNextChunkID_resultTupleScheme(); } } - private static class delete_partition_column_statistics_resultTupleScheme extends TupleScheme { + private static class getNextChunkID_resultTupleScheme extends TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, delete_partition_column_statistics_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getNextChunkID_result struct) throws org.apache.thrift.TException { TTupleProtocol oprot = (TTupleProtocol) prot; BitSet optionals = new BitSet(); if (struct.isSetSuccess()) { @@ -86724,7 +97908,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, delete_partition_co } oprot.writeBitSet(optionals, 5); if (struct.isSetSuccess()) { - oprot.writeBool(struct.success); + oprot.writeI64(struct.success); } if (struct.isSetO1()) { struct.o1.write(oprot); @@ -86741,30 +97925,30 @@ public void write(org.apache.thrift.protocol.TProtocol prot, delete_partition_co } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, delete_partition_column_statistics_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getNextChunkID_result struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; BitSet incoming = iprot.readBitSet(5); if (incoming.get(0)) { - struct.success = iprot.readBool(); + struct.success = iprot.readI64(); struct.setSuccessIsSet(true); } if (incoming.get(1)) { - struct.o1 = new NoSuchObjectException(); + struct.o1 = new InvalidInputException(); struct.o1.read(iprot); struct.setO1IsSet(true); } if (incoming.get(2)) { - struct.o2 = new MetaException(); + struct.o2 = new InvalidObjectException(); struct.o2.read(iprot); struct.setO2IsSet(true); } if (incoming.get(3)) { - struct.o3 = new InvalidObjectException(); + struct.o3 = new InvalidOperationException(); struct.o3.read(iprot); struct.setO3IsSet(true); } if (incoming.get(4)) { - struct.o4 = new InvalidInputException(); + struct.o4 = new MetaException(); struct.o4.read(iprot); struct.setO4IsSet(true); } @@ -86773,28 +97957,25 @@ public void read(org.apache.thrift.protocol.TProtocol prot, delete_partition_col } - public static class delete_table_column_statistics_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("delete_table_column_statistics_args"); + public static class getStreamingTmpDir_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getStreamingTmpDir_args"); - private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("db_name", org.apache.thrift.protocol.TType.STRING, (short)1); - private static final org.apache.thrift.protocol.TField TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tbl_name", org.apache.thrift.protocol.TType.STRING, (short)2); - private static final org.apache.thrift.protocol.TField COL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("col_name", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)2); private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { - schemes.put(StandardScheme.class, new delete_table_column_statistics_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new delete_table_column_statistics_argsTupleSchemeFactory()); + schemes.put(StandardScheme.class, new getStreamingTmpDir_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getStreamingTmpDir_argsTupleSchemeFactory()); } - private String db_name; // required - private String tbl_name; // required - private String col_name; // required + private String dbName; // required + private String 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 { - DB_NAME((short)1, "db_name"), - TBL_NAME((short)2, "tbl_name"), - COL_NAME((short)3, "col_name"); + DB_NAME((short)1, "dbName"), + TABLE_NAME((short)2, "tableName"); private static final Map byName = new HashMap(); @@ -86811,10 +97992,8 @@ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // DB_NAME return DB_NAME; - case 2: // TBL_NAME - return TBL_NAME; - case 3: // COL_NAME - return COL_NAME; + case 2: // TABLE_NAME + return TABLE_NAME; default: return null; } @@ -86858,122 +98037,91 @@ public String getFieldName() { public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("db_name", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - tmpMap.put(_Fields.TBL_NAME, new org.apache.thrift.meta_data.FieldMetaData("tbl_name", org.apache.thrift.TFieldRequirementType.DEFAULT, + tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("dbName", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - tmpMap.put(_Fields.COL_NAME, new org.apache.thrift.meta_data.FieldMetaData("col_name", org.apache.thrift.TFieldRequirementType.DEFAULT, + tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(delete_table_column_statistics_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getStreamingTmpDir_args.class, metaDataMap); } - public delete_table_column_statistics_args() { + public getStreamingTmpDir_args() { } - public delete_table_column_statistics_args( - String db_name, - String tbl_name, - String col_name) + public getStreamingTmpDir_args( + String dbName, + String tableName) { this(); - this.db_name = db_name; - this.tbl_name = tbl_name; - this.col_name = col_name; + this.dbName = dbName; + this.tableName = tableName; } /** * Performs a deep copy on other. */ - public delete_table_column_statistics_args(delete_table_column_statistics_args other) { - if (other.isSetDb_name()) { - this.db_name = other.db_name; + public getStreamingTmpDir_args(getStreamingTmpDir_args other) { + if (other.isSetDbName()) { + this.dbName = other.dbName; } - if (other.isSetTbl_name()) { - this.tbl_name = other.tbl_name; - } - if (other.isSetCol_name()) { - this.col_name = other.col_name; + if (other.isSetTableName()) { + this.tableName = other.tableName; } } - public delete_table_column_statistics_args deepCopy() { - return new delete_table_column_statistics_args(this); + public getStreamingTmpDir_args deepCopy() { + return new getStreamingTmpDir_args(this); } @Override public void clear() { - this.db_name = null; - this.tbl_name = null; - this.col_name = null; - } - - public String getDb_name() { - return this.db_name; - } - - public void setDb_name(String db_name) { - this.db_name = db_name; - } - - public void unsetDb_name() { - this.db_name = null; - } - - /** Returns true if field db_name is set (has been assigned a value) and false otherwise */ - public boolean isSetDb_name() { - return this.db_name != null; - } - - public void setDb_nameIsSet(boolean value) { - if (!value) { - this.db_name = null; - } + this.dbName = null; + this.tableName = null; } - public String getTbl_name() { - return this.tbl_name; + public String getDbName() { + return this.dbName; } - public void setTbl_name(String tbl_name) { - this.tbl_name = tbl_name; + public void setDbName(String dbName) { + this.dbName = dbName; } - public void unsetTbl_name() { - this.tbl_name = null; + public void unsetDbName() { + this.dbName = null; } - /** Returns true if field tbl_name is set (has been assigned a value) and false otherwise */ - public boolean isSetTbl_name() { - return this.tbl_name != null; + /** Returns true if field dbName is set (has been assigned a value) and false otherwise */ + public boolean isSetDbName() { + return this.dbName != null; } - public void setTbl_nameIsSet(boolean value) { + public void setDbNameIsSet(boolean value) { if (!value) { - this.tbl_name = null; + this.dbName = null; } } - public String getCol_name() { - return this.col_name; + public String getTableName() { + return this.tableName; } - public void setCol_name(String col_name) { - this.col_name = col_name; + public void setTableName(String tableName) { + this.tableName = tableName; } - public void unsetCol_name() { - this.col_name = null; + public void unsetTableName() { + this.tableName = null; } - /** Returns true if field col_name is set (has been assigned a value) and false otherwise */ - public boolean isSetCol_name() { - return this.col_name != 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 setCol_nameIsSet(boolean value) { + public void setTableNameIsSet(boolean value) { if (!value) { - this.col_name = null; + this.tableName = null; } } @@ -86981,25 +98129,17 @@ public void setFieldValue(_Fields field, Object value) { switch (field) { case DB_NAME: if (value == null) { - unsetDb_name(); - } else { - setDb_name((String)value); - } - break; - - case TBL_NAME: - if (value == null) { - unsetTbl_name(); + unsetDbName(); } else { - setTbl_name((String)value); + setDbName((String)value); } break; - case COL_NAME: + case TABLE_NAME: if (value == null) { - unsetCol_name(); + unsetTableName(); } else { - setCol_name((String)value); + setTableName((String)value); } break; @@ -87009,13 +98149,10 @@ public void setFieldValue(_Fields field, Object value) { public Object getFieldValue(_Fields field) { switch (field) { case DB_NAME: - return getDb_name(); + return getDbName(); - case TBL_NAME: - return getTbl_name(); - - case COL_NAME: - return getCol_name(); + case TABLE_NAME: + return getTableName(); } throw new IllegalStateException(); @@ -87029,11 +98166,9 @@ public boolean isSet(_Fields field) { switch (field) { case DB_NAME: - return isSetDb_name(); - case TBL_NAME: - return isSetTbl_name(); - case COL_NAME: - return isSetCol_name(); + return isSetDbName(); + case TABLE_NAME: + return isSetTableName(); } throw new IllegalStateException(); } @@ -87042,39 +98177,30 @@ public boolean isSet(_Fields field) { public boolean equals(Object that) { if (that == null) return false; - if (that instanceof delete_table_column_statistics_args) - return this.equals((delete_table_column_statistics_args)that); + if (that instanceof getStreamingTmpDir_args) + return this.equals((getStreamingTmpDir_args)that); return false; } - public boolean equals(delete_table_column_statistics_args that) { + public boolean equals(getStreamingTmpDir_args that) { if (that == null) return false; - boolean this_present_db_name = true && this.isSetDb_name(); - boolean that_present_db_name = true && that.isSetDb_name(); - if (this_present_db_name || that_present_db_name) { - if (!(this_present_db_name && that_present_db_name)) + boolean this_present_dbName = true && this.isSetDbName(); + boolean that_present_dbName = true && that.isSetDbName(); + if (this_present_dbName || that_present_dbName) { + if (!(this_present_dbName && that_present_dbName)) return false; - if (!this.db_name.equals(that.db_name)) + if (!this.dbName.equals(that.dbName)) return false; } - boolean this_present_tbl_name = true && this.isSetTbl_name(); - boolean that_present_tbl_name = true && that.isSetTbl_name(); - if (this_present_tbl_name || that_present_tbl_name) { - if (!(this_present_tbl_name && that_present_tbl_name)) + 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.tbl_name.equals(that.tbl_name)) - return false; - } - - boolean this_present_col_name = true && this.isSetCol_name(); - boolean that_present_col_name = true && that.isSetCol_name(); - if (this_present_col_name || that_present_col_name) { - if (!(this_present_col_name && that_present_col_name)) - return false; - if (!this.col_name.equals(that.col_name)) + if (!this.tableName.equals(that.tableName)) return false; } @@ -87085,58 +98211,43 @@ public boolean equals(delete_table_column_statistics_args that) { public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); - boolean present_db_name = true && (isSetDb_name()); - builder.append(present_db_name); - if (present_db_name) - builder.append(db_name); - - boolean present_tbl_name = true && (isSetTbl_name()); - builder.append(present_tbl_name); - if (present_tbl_name) - builder.append(tbl_name); + boolean present_dbName = true && (isSetDbName()); + builder.append(present_dbName); + if (present_dbName) + builder.append(dbName); - boolean present_col_name = true && (isSetCol_name()); - builder.append(present_col_name); - if (present_col_name) - builder.append(col_name); + boolean present_tableName = true && (isSetTableName()); + builder.append(present_tableName); + if (present_tableName) + builder.append(tableName); return builder.toHashCode(); } - public int compareTo(delete_table_column_statistics_args other) { + public int compareTo(getStreamingTmpDir_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - delete_table_column_statistics_args typedOther = (delete_table_column_statistics_args)other; + getStreamingTmpDir_args typedOther = (getStreamingTmpDir_args)other; - lastComparison = Boolean.valueOf(isSetDb_name()).compareTo(typedOther.isSetDb_name()); + lastComparison = Boolean.valueOf(isSetDbName()).compareTo(typedOther.isSetDbName()); if (lastComparison != 0) { return lastComparison; } - if (isSetDb_name()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.db_name, typedOther.db_name); + if (isSetDbName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.dbName, typedOther.dbName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetTbl_name()).compareTo(typedOther.isSetTbl_name()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); if (lastComparison != 0) { return lastComparison; } - if (isSetTbl_name()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tbl_name, typedOther.tbl_name); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetCol_name()).compareTo(typedOther.isSetCol_name()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetCol_name()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.col_name, typedOther.col_name); + if (isSetTableName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); if (lastComparison != 0) { return lastComparison; } @@ -87158,30 +98269,22 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public String toString() { - StringBuilder sb = new StringBuilder("delete_table_column_statistics_args("); + StringBuilder sb = new StringBuilder("getStreamingTmpDir_args("); boolean first = true; - sb.append("db_name:"); - if (this.db_name == null) { + sb.append("dbName:"); + if (this.dbName == null) { sb.append("null"); } else { - sb.append(this.db_name); + sb.append(this.dbName); } first = false; if (!first) sb.append(", "); - sb.append("tbl_name:"); - if (this.tbl_name == null) { + sb.append("tableName:"); + if (this.tableName == null) { sb.append("null"); } else { - sb.append(this.tbl_name); - } - first = false; - if (!first) sb.append(", "); - sb.append("col_name:"); - if (this.col_name == null) { - sb.append("null"); - } else { - sb.append(this.col_name); + sb.append(this.tableName); } first = false; sb.append(")"); @@ -87209,15 +98312,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class delete_table_column_statistics_argsStandardSchemeFactory implements SchemeFactory { - public delete_table_column_statistics_argsStandardScheme getScheme() { - return new delete_table_column_statistics_argsStandardScheme(); + private static class getStreamingTmpDir_argsStandardSchemeFactory implements SchemeFactory { + public getStreamingTmpDir_argsStandardScheme getScheme() { + return new getStreamingTmpDir_argsStandardScheme(); } } - private static class delete_table_column_statistics_argsStandardScheme extends StandardScheme { + private static class getStreamingTmpDir_argsStandardScheme extends StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, delete_table_column_statistics_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getStreamingTmpDir_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -87229,24 +98332,16 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, delete_table_column switch (schemeField.id) { case 1: // DB_NAME if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.db_name = iprot.readString(); - struct.setDb_nameIsSet(true); + struct.dbName = iprot.readString(); + struct.setDbNameIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 2: // TBL_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.tbl_name = iprot.readString(); - struct.setTbl_nameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // COL_NAME + case 2: // TABLE_NAME if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.col_name = iprot.readString(); - struct.setCol_nameIsSet(true); + struct.tableName = iprot.readString(); + struct.setTableNameIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -87260,23 +98355,18 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, delete_table_column struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, delete_table_column_statistics_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getStreamingTmpDir_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - if (struct.db_name != null) { + if (struct.dbName != null) { oprot.writeFieldBegin(DB_NAME_FIELD_DESC); - oprot.writeString(struct.db_name); + oprot.writeString(struct.dbName); oprot.writeFieldEnd(); } - if (struct.tbl_name != null) { - oprot.writeFieldBegin(TBL_NAME_FIELD_DESC); - oprot.writeString(struct.tbl_name); - oprot.writeFieldEnd(); - } - if (struct.col_name != null) { - oprot.writeFieldBegin(COL_NAME_FIELD_DESC); - oprot.writeString(struct.col_name); + if (struct.tableName != null) { + oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC); + oprot.writeString(struct.tableName); oprot.writeFieldEnd(); } oprot.writeFieldStop(); @@ -87285,64 +98375,54 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, delete_table_colum } - private static class delete_table_column_statistics_argsTupleSchemeFactory implements SchemeFactory { - public delete_table_column_statistics_argsTupleScheme getScheme() { - return new delete_table_column_statistics_argsTupleScheme(); + private static class getStreamingTmpDir_argsTupleSchemeFactory implements SchemeFactory { + public getStreamingTmpDir_argsTupleScheme getScheme() { + return new getStreamingTmpDir_argsTupleScheme(); } } - private static class delete_table_column_statistics_argsTupleScheme extends TupleScheme { + private static class getStreamingTmpDir_argsTupleScheme extends TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, delete_table_column_statistics_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getStreamingTmpDir_args struct) throws org.apache.thrift.TException { TTupleProtocol oprot = (TTupleProtocol) prot; BitSet optionals = new BitSet(); - if (struct.isSetDb_name()) { + if (struct.isSetDbName()) { optionals.set(0); } - if (struct.isSetTbl_name()) { + if (struct.isSetTableName()) { optionals.set(1); } - if (struct.isSetCol_name()) { - optionals.set(2); - } - oprot.writeBitSet(optionals, 3); - if (struct.isSetDb_name()) { - oprot.writeString(struct.db_name); - } - if (struct.isSetTbl_name()) { - oprot.writeString(struct.tbl_name); + oprot.writeBitSet(optionals, 2); + if (struct.isSetDbName()) { + oprot.writeString(struct.dbName); } - if (struct.isSetCol_name()) { - oprot.writeString(struct.col_name); + if (struct.isSetTableName()) { + oprot.writeString(struct.tableName); } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, delete_table_column_statistics_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getStreamingTmpDir_args struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(3); + BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { - struct.db_name = iprot.readString(); - struct.setDb_nameIsSet(true); + struct.dbName = iprot.readString(); + struct.setDbNameIsSet(true); } if (incoming.get(1)) { - struct.tbl_name = iprot.readString(); - struct.setTbl_nameIsSet(true); - } - if (incoming.get(2)) { - struct.col_name = iprot.readString(); - struct.setCol_nameIsSet(true); + struct.tableName = iprot.readString(); + struct.setTableNameIsSet(true); } } } } - public static class delete_table_column_statistics_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("delete_table_column_statistics_result"); + public static class getStreamingTmpDir_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getStreamingTmpDir_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 SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRING, (short)0); private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1); private static final org.apache.thrift.protocol.TField O2_FIELD_DESC = new org.apache.thrift.protocol.TField("o2", org.apache.thrift.protocol.TType.STRUCT, (short)2); private static final org.apache.thrift.protocol.TField O3_FIELD_DESC = new org.apache.thrift.protocol.TField("o3", org.apache.thrift.protocol.TType.STRUCT, (short)3); @@ -87350,15 +98430,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, delete_table_column_ private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { - schemes.put(StandardScheme.class, new delete_table_column_statistics_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new delete_table_column_statistics_resultTupleSchemeFactory()); + schemes.put(StandardScheme.class, new getStreamingTmpDir_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getStreamingTmpDir_resultTupleSchemeFactory()); } - private boolean success; // required - private NoSuchObjectException o1; // required - private MetaException o2; // required - private InvalidObjectException o3; // required - private InvalidInputException o4; // required + private String success; // required + private InvalidInputException o1; // required + private InvalidObjectException o2; // required + private InvalidOperationException o3; // required + private MetaException o4; // 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 { @@ -87431,13 +98511,11 @@ public String getFieldName() { } // isset id assignments - private static final int __SUCCESS_ISSET_ID = 0; - private byte __isset_bitfield = 0; public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new 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))); + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); tmpMap.put(_Fields.O2, new org.apache.thrift.meta_data.FieldMetaData("o2", org.apache.thrift.TFieldRequirementType.DEFAULT, @@ -87447,22 +98525,21 @@ public String getFieldName() { tmpMap.put(_Fields.O4, new org.apache.thrift.meta_data.FieldMetaData("o4", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(delete_table_column_statistics_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getStreamingTmpDir_result.class, metaDataMap); } - public delete_table_column_statistics_result() { + public getStreamingTmpDir_result() { } - public delete_table_column_statistics_result( - boolean success, - NoSuchObjectException o1, - MetaException o2, - InvalidObjectException o3, - InvalidInputException o4) + public getStreamingTmpDir_result( + String success, + InvalidInputException o1, + InvalidObjectException o2, + InvalidOperationException o3, + MetaException o4) { this(); this.success = success; - setSuccessIsSet(true); this.o1 = o1; this.o2 = o2; this.o3 = o3; @@ -87472,64 +98549,65 @@ public delete_table_column_statistics_result( /** * Performs a deep copy on other. */ - public delete_table_column_statistics_result(delete_table_column_statistics_result other) { - __isset_bitfield = other.__isset_bitfield; - this.success = other.success; + public getStreamingTmpDir_result(getStreamingTmpDir_result other) { + if (other.isSetSuccess()) { + this.success = other.success; + } if (other.isSetO1()) { - this.o1 = new NoSuchObjectException(other.o1); + this.o1 = new InvalidInputException(other.o1); } if (other.isSetO2()) { - this.o2 = new MetaException(other.o2); + this.o2 = new InvalidObjectException(other.o2); } if (other.isSetO3()) { - this.o3 = new InvalidObjectException(other.o3); + this.o3 = new InvalidOperationException(other.o3); } if (other.isSetO4()) { - this.o4 = new InvalidInputException(other.o4); + this.o4 = new MetaException(other.o4); } } - public delete_table_column_statistics_result deepCopy() { - return new delete_table_column_statistics_result(this); + public getStreamingTmpDir_result deepCopy() { + return new getStreamingTmpDir_result(this); } @Override public void clear() { - setSuccessIsSet(false); - this.success = false; + this.success = null; this.o1 = null; this.o2 = null; this.o3 = null; this.o4 = null; } - public boolean isSuccess() { + public String getSuccess() { return this.success; } - public void setSuccess(boolean success) { + public void setSuccess(String success) { this.success = success; - setSuccessIsSet(true); } public void unsetSuccess() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID); + this.success = null; } /** Returns true if field success is set (has been assigned a value) and false otherwise */ public boolean isSetSuccess() { - return EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID); + return this.success != null; } public void setSuccessIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value); + if (!value) { + this.success = null; + } } - public NoSuchObjectException getO1() { + public InvalidInputException getO1() { return this.o1; } - public void setO1(NoSuchObjectException o1) { + public void setO1(InvalidInputException o1) { this.o1 = o1; } @@ -87548,11 +98626,11 @@ public void setO1IsSet(boolean value) { } } - public MetaException getO2() { + public InvalidObjectException getO2() { return this.o2; } - public void setO2(MetaException o2) { + public void setO2(InvalidObjectException o2) { this.o2 = o2; } @@ -87571,11 +98649,11 @@ public void setO2IsSet(boolean value) { } } - public InvalidObjectException getO3() { + public InvalidOperationException getO3() { return this.o3; } - public void setO3(InvalidObjectException o3) { + public void setO3(InvalidOperationException o3) { this.o3 = o3; } @@ -87594,11 +98672,11 @@ public void setO3IsSet(boolean value) { } } - public InvalidInputException getO4() { + public MetaException getO4() { return this.o4; } - public void setO4(InvalidInputException o4) { + public void setO4(MetaException o4) { this.o4 = o4; } @@ -87623,7 +98701,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unsetSuccess(); } else { - setSuccess((Boolean)value); + setSuccess((String)value); } break; @@ -87631,7 +98709,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unsetO1(); } else { - setO1((NoSuchObjectException)value); + setO1((InvalidInputException)value); } break; @@ -87639,7 +98717,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unsetO2(); } else { - setO2((MetaException)value); + setO2((InvalidObjectException)value); } break; @@ -87647,7 +98725,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unsetO3(); } else { - setO3((InvalidObjectException)value); + setO3((InvalidOperationException)value); } break; @@ -87655,7 +98733,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unsetO4(); } else { - setO4((InvalidInputException)value); + setO4((MetaException)value); } break; @@ -87665,7 +98743,7 @@ public void setFieldValue(_Fields field, Object value) { public Object getFieldValue(_Fields field) { switch (field) { case SUCCESS: - return Boolean.valueOf(isSuccess()); + return getSuccess(); case O1: return getO1(); @@ -87708,21 +98786,21 @@ public boolean isSet(_Fields field) { public boolean equals(Object that) { if (that == null) return false; - if (that instanceof delete_table_column_statistics_result) - return this.equals((delete_table_column_statistics_result)that); + if (that instanceof getStreamingTmpDir_result) + return this.equals((getStreamingTmpDir_result)that); return false; } - public boolean equals(delete_table_column_statistics_result that) { + public boolean equals(getStreamingTmpDir_result that) { if (that == null) return false; - boolean this_present_success = true; - boolean that_present_success = 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 != that.success) + if (!this.success.equals(that.success)) return false; } @@ -87769,7 +98847,7 @@ public boolean equals(delete_table_column_statistics_result that) { public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); - boolean present_success = true; + boolean present_success = true && (isSetSuccess()); builder.append(present_success); if (present_success) builder.append(success); @@ -87797,13 +98875,13 @@ public int hashCode() { return builder.toHashCode(); } - public int compareTo(delete_table_column_statistics_result other) { + public int compareTo(getStreamingTmpDir_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - delete_table_column_statistics_result typedOther = (delete_table_column_statistics_result)other; + getStreamingTmpDir_result typedOther = (getStreamingTmpDir_result)other; lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); if (lastComparison != 0) { @@ -87872,11 +98950,15 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public String toString() { - StringBuilder sb = new StringBuilder("delete_table_column_statistics_result("); + StringBuilder sb = new StringBuilder("getStreamingTmpDir_result("); boolean first = true; sb.append("success:"); - sb.append(this.success); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } first = false; if (!first) sb.append(", "); sb.append("o1:"); @@ -87929,23 +99011,21 @@ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOExcept private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, 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 delete_table_column_statistics_resultStandardSchemeFactory implements SchemeFactory { - public delete_table_column_statistics_resultStandardScheme getScheme() { - return new delete_table_column_statistics_resultStandardScheme(); + private static class getStreamingTmpDir_resultStandardSchemeFactory implements SchemeFactory { + public getStreamingTmpDir_resultStandardScheme getScheme() { + return new getStreamingTmpDir_resultStandardScheme(); } } - private static class delete_table_column_statistics_resultStandardScheme extends StandardScheme { + private static class getStreamingTmpDir_resultStandardScheme extends StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, delete_table_column_statistics_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getStreamingTmpDir_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -87956,8 +99036,8 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, delete_table_column } switch (schemeField.id) { case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { - struct.success = iprot.readBool(); + 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); @@ -87965,7 +99045,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, delete_table_column break; case 1: // O1 if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.o1 = new NoSuchObjectException(); + struct.o1 = new InvalidInputException(); struct.o1.read(iprot); struct.setO1IsSet(true); } else { @@ -87974,7 +99054,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, delete_table_column break; case 2: // O2 if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.o2 = new MetaException(); + struct.o2 = new InvalidObjectException(); struct.o2.read(iprot); struct.setO2IsSet(true); } else { @@ -87983,7 +99063,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, delete_table_column break; case 3: // O3 if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.o3 = new InvalidObjectException(); + struct.o3 = new InvalidOperationException(); struct.o3.read(iprot); struct.setO3IsSet(true); } else { @@ -87992,7 +99072,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, delete_table_column break; case 4: // O4 if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.o4 = new InvalidInputException(); + struct.o4 = new MetaException(); struct.o4.read(iprot); struct.setO4IsSet(true); } else { @@ -88008,13 +99088,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, delete_table_column struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, delete_table_column_statistics_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getStreamingTmpDir_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - if (struct.isSetSuccess()) { + if (struct.success != null) { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - oprot.writeBool(struct.success); + oprot.writeString(struct.success); oprot.writeFieldEnd(); } if (struct.o1 != null) { @@ -88043,16 +99123,16 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, delete_table_colum } - private static class delete_table_column_statistics_resultTupleSchemeFactory implements SchemeFactory { - public delete_table_column_statistics_resultTupleScheme getScheme() { - return new delete_table_column_statistics_resultTupleScheme(); + private static class getStreamingTmpDir_resultTupleSchemeFactory implements SchemeFactory { + public getStreamingTmpDir_resultTupleScheme getScheme() { + return new getStreamingTmpDir_resultTupleScheme(); } } - private static class delete_table_column_statistics_resultTupleScheme extends TupleScheme { + private static class getStreamingTmpDir_resultTupleScheme extends TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, delete_table_column_statistics_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getStreamingTmpDir_result struct) throws org.apache.thrift.TException { TTupleProtocol oprot = (TTupleProtocol) prot; BitSet optionals = new BitSet(); if (struct.isSetSuccess()) { @@ -88072,7 +99152,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, delete_table_column } oprot.writeBitSet(optionals, 5); if (struct.isSetSuccess()) { - oprot.writeBool(struct.success); + oprot.writeString(struct.success); } if (struct.isSetO1()) { struct.o1.write(oprot); @@ -88089,30 +99169,30 @@ public void write(org.apache.thrift.protocol.TProtocol prot, delete_table_column } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, delete_table_column_statistics_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getStreamingTmpDir_result struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; BitSet incoming = iprot.readBitSet(5); if (incoming.get(0)) { - struct.success = iprot.readBool(); + struct.success = iprot.readString(); struct.setSuccessIsSet(true); } if (incoming.get(1)) { - struct.o1 = new NoSuchObjectException(); + struct.o1 = new InvalidInputException(); struct.o1.read(iprot); struct.setO1IsSet(true); } if (incoming.get(2)) { - struct.o2 = new MetaException(); + struct.o2 = new InvalidObjectException(); struct.o2.read(iprot); struct.setO2IsSet(true); } if (incoming.get(3)) { - struct.o3 = new InvalidObjectException(); + struct.o3 = new InvalidOperationException(); struct.o3.read(iprot); struct.setO3IsSet(true); } if (incoming.get(4)) { - struct.o4 = new InvalidInputException(); + struct.o4 = new MetaException(); struct.o4.read(iprot); struct.setO4IsSet(true); } @@ -90412,13 +101492,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_role_names_resu case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list602 = iprot.readListBegin(); - struct.success = new ArrayList(_list602.size); - for (int _i603 = 0; _i603 < _list602.size; ++_i603) + org.apache.thrift.protocol.TList _list612 = iprot.readListBegin(); + struct.success = new ArrayList(_list612.size); + for (int _i613 = 0; _i613 < _list612.size; ++_i613) { - String _elem604; // required - _elem604 = iprot.readString(); - struct.success.add(_elem604); + String _elem614; // required + _elem614 = iprot.readString(); + struct.success.add(_elem614); } iprot.readListEnd(); } @@ -90453,9 +101533,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_role_names_res oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (String _iter605 : struct.success) + for (String _iter615 : struct.success) { - oprot.writeString(_iter605); + oprot.writeString(_iter615); } oprot.writeListEnd(); } @@ -90494,9 +101574,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_role_names_resu if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (String _iter606 : struct.success) + for (String _iter616 : struct.success) { - oprot.writeString(_iter606); + oprot.writeString(_iter616); } } } @@ -90511,13 +101591,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_role_names_resul BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list607 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.success = new ArrayList(_list607.size); - for (int _i608 = 0; _i608 < _list607.size; ++_i608) + org.apache.thrift.protocol.TList _list617 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.success = new ArrayList(_list617.size); + for (int _i618 = 0; _i618 < _list617.size; ++_i618) { - String _elem609; // required - _elem609 = iprot.readString(); - struct.success.add(_elem609); + String _elem619; // required + _elem619 = iprot.readString(); + struct.success.add(_elem619); } } struct.setSuccessIsSet(true); @@ -93808,14 +104888,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, list_roles_result s case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list610 = iprot.readListBegin(); - struct.success = new ArrayList(_list610.size); - for (int _i611 = 0; _i611 < _list610.size; ++_i611) + org.apache.thrift.protocol.TList _list620 = iprot.readListBegin(); + struct.success = new ArrayList(_list620.size); + for (int _i621 = 0; _i621 < _list620.size; ++_i621) { - Role _elem612; // required - _elem612 = new Role(); - _elem612.read(iprot); - struct.success.add(_elem612); + Role _elem622; // required + _elem622 = new Role(); + _elem622.read(iprot); + struct.success.add(_elem622); } iprot.readListEnd(); } @@ -93850,9 +104930,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, list_roles_result oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Role _iter613 : struct.success) + for (Role _iter623 : struct.success) { - _iter613.write(oprot); + _iter623.write(oprot); } oprot.writeListEnd(); } @@ -93891,9 +104971,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, list_roles_result s if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Role _iter614 : struct.success) + for (Role _iter624 : struct.success) { - _iter614.write(oprot); + _iter624.write(oprot); } } } @@ -93908,14 +104988,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, list_roles_result st BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list615 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list615.size); - for (int _i616 = 0; _i616 < _list615.size; ++_i616) + org.apache.thrift.protocol.TList _list625 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list625.size); + for (int _i626 = 0; _i626 < _list625.size; ++_i626) { - Role _elem617; // required - _elem617 = new Role(); - _elem617.read(iprot); - struct.success.add(_elem617); + Role _elem627; // required + _elem627 = new Role(); + _elem627.read(iprot); + struct.success.add(_elem627); } } struct.setSuccessIsSet(true); @@ -94427,13 +105507,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_privilege_set_a case 3: // GROUP_NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list618 = iprot.readListBegin(); - struct.group_names = new ArrayList(_list618.size); - for (int _i619 = 0; _i619 < _list618.size; ++_i619) + org.apache.thrift.protocol.TList _list628 = iprot.readListBegin(); + struct.group_names = new ArrayList(_list628.size); + for (int _i629 = 0; _i629 < _list628.size; ++_i629) { - String _elem620; // required - _elem620 = iprot.readString(); - struct.group_names.add(_elem620); + String _elem630; // required + _elem630 = iprot.readString(); + struct.group_names.add(_elem630); } iprot.readListEnd(); } @@ -94469,9 +105549,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_privilege_set_ oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.group_names.size())); - for (String _iter621 : struct.group_names) + for (String _iter631 : struct.group_names) { - oprot.writeString(_iter621); + oprot.writeString(_iter631); } oprot.writeListEnd(); } @@ -94514,9 +105594,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_privilege_set_a if (struct.isSetGroup_names()) { { oprot.writeI32(struct.group_names.size()); - for (String _iter622 : struct.group_names) + for (String _iter632 : struct.group_names) { - oprot.writeString(_iter622); + oprot.writeString(_iter632); } } } @@ -94537,13 +105617,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_privilege_set_ar } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list623 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.group_names = new ArrayList(_list623.size); - for (int _i624 = 0; _i624 < _list623.size; ++_i624) + org.apache.thrift.protocol.TList _list633 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.group_names = new ArrayList(_list633.size); + for (int _i634 = 0; _i634 < _list633.size; ++_i634) { - String _elem625; // required - _elem625 = iprot.readString(); - struct.group_names.add(_elem625); + String _elem635; // required + _elem635 = iprot.readString(); + struct.group_names.add(_elem635); } } struct.setGroup_namesIsSet(true); @@ -96001,14 +107081,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, list_privileges_res case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list626 = iprot.readListBegin(); - struct.success = new ArrayList(_list626.size); - for (int _i627 = 0; _i627 < _list626.size; ++_i627) + org.apache.thrift.protocol.TList _list636 = iprot.readListBegin(); + struct.success = new ArrayList(_list636.size); + for (int _i637 = 0; _i637 < _list636.size; ++_i637) { - HiveObjectPrivilege _elem628; // required - _elem628 = new HiveObjectPrivilege(); - _elem628.read(iprot); - struct.success.add(_elem628); + HiveObjectPrivilege _elem638; // required + _elem638 = new HiveObjectPrivilege(); + _elem638.read(iprot); + struct.success.add(_elem638); } iprot.readListEnd(); } @@ -96043,9 +107123,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, list_privileges_re oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (HiveObjectPrivilege _iter629 : struct.success) + for (HiveObjectPrivilege _iter639 : struct.success) { - _iter629.write(oprot); + _iter639.write(oprot); } oprot.writeListEnd(); } @@ -96084,9 +107164,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, list_privileges_res if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (HiveObjectPrivilege _iter630 : struct.success) + for (HiveObjectPrivilege _iter640 : struct.success) { - _iter630.write(oprot); + _iter640.write(oprot); } } } @@ -96101,14 +107181,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, list_privileges_resu BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list631 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list631.size); - for (int _i632 = 0; _i632 < _list631.size; ++_i632) + org.apache.thrift.protocol.TList _list641 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list641.size); + for (int _i642 = 0; _i642 < _list641.size; ++_i642) { - HiveObjectPrivilege _elem633; // required - _elem633 = new HiveObjectPrivilege(); - _elem633.read(iprot); - struct.success.add(_elem633); + HiveObjectPrivilege _elem643; // required + _elem643 = new HiveObjectPrivilege(); + _elem643.read(iprot); + struct.success.add(_elem643); } } struct.setSuccessIsSet(true); @@ -98181,13 +109261,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, set_ugi_args struct case 2: // GROUP_NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list634 = iprot.readListBegin(); - struct.group_names = new ArrayList(_list634.size); - for (int _i635 = 0; _i635 < _list634.size; ++_i635) + org.apache.thrift.protocol.TList _list644 = iprot.readListBegin(); + struct.group_names = new ArrayList(_list644.size); + for (int _i645 = 0; _i645 < _list644.size; ++_i645) { - String _elem636; // required - _elem636 = iprot.readString(); - struct.group_names.add(_elem636); + String _elem646; // required + _elem646 = iprot.readString(); + struct.group_names.add(_elem646); } iprot.readListEnd(); } @@ -98218,9 +109298,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, set_ugi_args struc oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.group_names.size())); - for (String _iter637 : struct.group_names) + for (String _iter647 : struct.group_names) { - oprot.writeString(_iter637); + oprot.writeString(_iter647); } oprot.writeListEnd(); } @@ -98257,9 +109337,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, set_ugi_args struct if (struct.isSetGroup_names()) { { oprot.writeI32(struct.group_names.size()); - for (String _iter638 : struct.group_names) + for (String _iter648 : struct.group_names) { - oprot.writeString(_iter638); + oprot.writeString(_iter648); } } } @@ -98275,13 +109355,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, set_ugi_args struct) } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list639 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.group_names = new ArrayList(_list639.size); - for (int _i640 = 0; _i640 < _list639.size; ++_i640) + org.apache.thrift.protocol.TList _list649 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.group_names = new ArrayList(_list649.size); + for (int _i650 = 0; _i650 < _list649.size; ++_i650) { - String _elem641; // required - _elem641 = iprot.readString(); - struct.group_names.add(_elem641); + String _elem651; // required + _elem651 = iprot.readString(); + struct.group_names.add(_elem651); } } struct.setGroup_namesIsSet(true); @@ -98687,13 +109767,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, set_ugi_result stru case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list642 = iprot.readListBegin(); - struct.success = new ArrayList(_list642.size); - for (int _i643 = 0; _i643 < _list642.size; ++_i643) + org.apache.thrift.protocol.TList _list652 = iprot.readListBegin(); + struct.success = new ArrayList(_list652.size); + for (int _i653 = 0; _i653 < _list652.size; ++_i653) { - String _elem644; // required - _elem644 = iprot.readString(); - struct.success.add(_elem644); + String _elem654; // required + _elem654 = iprot.readString(); + struct.success.add(_elem654); } iprot.readListEnd(); } @@ -98728,9 +109808,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, set_ugi_result str oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (String _iter645 : struct.success) + for (String _iter655 : struct.success) { - oprot.writeString(_iter645); + oprot.writeString(_iter655); } oprot.writeListEnd(); } @@ -98769,9 +109849,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, set_ugi_result stru if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (String _iter646 : struct.success) + for (String _iter656 : struct.success) { - oprot.writeString(_iter646); + oprot.writeString(_iter656); } } } @@ -98786,13 +109866,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, set_ugi_result struc BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list647 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.success = new ArrayList(_list647.size); - for (int _i648 = 0; _i648 < _list647.size; ++_i648) + org.apache.thrift.protocol.TList _list657 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.success = new ArrayList(_list657.size); + for (int _i658 = 0; _i658 < _list657.size; ++_i658) { - String _elem649; // required - _elem649 = iprot.readString(); - struct.success.add(_elem649); + String _elem659; // required + _elem659 = iprot.readString(); + struct.success.add(_elem659); } } struct.setSuccessIsSet(true); diff --git metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php index a69d214..cd4a0b4 100644 --- metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php +++ metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php @@ -84,6 +84,14 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf { public function get_partition_column_statistics($db_name, $tbl_name, $part_name, $col_name); public function delete_partition_column_statistics($db_name, $tbl_name, $part_name, $col_name); public function delete_table_column_statistics($db_name, $tbl_name, $col_name); + public function streamingStatus($db_name, $table_name); + public function enableStreaming($db_name, $table_name, $streaming_tmp_dir, $partition_path, $partition_val); + public function disableStreaming($db_name, $table_name); + public function getCurrentStreamingPartitionPath($dbName, $tableName); + public function rollStreamingPartition($dbName, $tableName, $newPartitionPath, $newPartitionVal); + public function updateStreamingTempLocation($dbName, $tableName, $streamingTempDir); + public function getNextChunkID($db_name, $table_name); + public function getStreamingTmpDir($dbName, $tableName); public function create_role(\metastore\Role $role); public function drop_role($role_name); public function get_role_names(); @@ -4148,34 +4156,35 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas throw new \Exception("delete_table_column_statistics failed: unknown result"); } - public function create_role(\metastore\Role $role) + public function streamingStatus($db_name, $table_name) { - $this->send_create_role($role); - return $this->recv_create_role(); + $this->send_streamingStatus($db_name, $table_name); + return $this->recv_streamingStatus(); } - public function send_create_role(\metastore\Role $role) + public function send_streamingStatus($db_name, $table_name) { - $args = new \metastore\ThriftHiveMetastore_create_role_args(); - $args->role = $role; + $args = new \metastore\ThriftHiveMetastore_streamingStatus_args(); + $args->db_name = $db_name; + $args->table_name = $table_name; $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); if ($bin_accel) { - thrift_protocol_write_binary($this->output_, 'create_role', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + thrift_protocol_write_binary($this->output_, 'streamingStatus', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); } else { - $this->output_->writeMessageBegin('create_role', TMessageType::CALL, $this->seqid_); + $this->output_->writeMessageBegin('streamingStatus', TMessageType::CALL, $this->seqid_); $args->write($this->output_); $this->output_->writeMessageEnd(); $this->output_->getTransport()->flush(); } } - public function recv_create_role() + public function recv_streamingStatus() { $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); - if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_create_role_result', $this->input_->isStrictRead()); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_streamingStatus_result', $this->input_->isStrictRead()); else { $rseqid = 0; @@ -4189,7 +4198,7 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas $this->input_->readMessageEnd(); throw $x; } - $result = new \metastore\ThriftHiveMetastore_create_role_result(); + $result = new \metastore\ThriftHiveMetastore_streamingStatus_result(); $result->read($this->input_); $this->input_->readMessageEnd(); } @@ -4199,37 +4208,47 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas if ($result->o1 !== null) { throw $result->o1; } - throw new \Exception("create_role failed: unknown result"); + if ($result->o2 !== null) { + throw $result->o2; + } + if ($result->o3 !== null) { + throw $result->o3; + } + throw new \Exception("streamingStatus failed: unknown result"); } - public function drop_role($role_name) + public function enableStreaming($db_name, $table_name, $streaming_tmp_dir, $partition_path, $partition_val) { - $this->send_drop_role($role_name); - return $this->recv_drop_role(); + $this->send_enableStreaming($db_name, $table_name, $streaming_tmp_dir, $partition_path, $partition_val); + $this->recv_enableStreaming(); } - public function send_drop_role($role_name) + public function send_enableStreaming($db_name, $table_name, $streaming_tmp_dir, $partition_path, $partition_val) { - $args = new \metastore\ThriftHiveMetastore_drop_role_args(); - $args->role_name = $role_name; + $args = new \metastore\ThriftHiveMetastore_enableStreaming_args(); + $args->db_name = $db_name; + $args->table_name = $table_name; + $args->streaming_tmp_dir = $streaming_tmp_dir; + $args->partition_path = $partition_path; + $args->partition_val = $partition_val; $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); if ($bin_accel) { - thrift_protocol_write_binary($this->output_, 'drop_role', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + thrift_protocol_write_binary($this->output_, 'enableStreaming', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); } else { - $this->output_->writeMessageBegin('drop_role', TMessageType::CALL, $this->seqid_); + $this->output_->writeMessageBegin('enableStreaming', TMessageType::CALL, $this->seqid_); $args->write($this->output_); $this->output_->writeMessageEnd(); $this->output_->getTransport()->flush(); } } - public function recv_drop_role() + public function recv_enableStreaming() { $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); - if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_drop_role_result', $this->input_->isStrictRead()); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_enableStreaming_result', $this->input_->isStrictRead()); else { $rseqid = 0; @@ -4243,46 +4262,54 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas $this->input_->readMessageEnd(); throw $x; } - $result = new \metastore\ThriftHiveMetastore_drop_role_result(); + $result = new \metastore\ThriftHiveMetastore_enableStreaming_result(); $result->read($this->input_); $this->input_->readMessageEnd(); } - if ($result->success !== null) { - return $result->success; - } if ($result->o1 !== null) { throw $result->o1; } - throw new \Exception("drop_role failed: unknown result"); + if ($result->o2 !== null) { + throw $result->o2; + } + if ($result->o3 !== null) { + throw $result->o3; + } + if ($result->o4 !== null) { + throw $result->o4; + } + return; } - public function get_role_names() + public function disableStreaming($db_name, $table_name) { - $this->send_get_role_names(); - return $this->recv_get_role_names(); + $this->send_disableStreaming($db_name, $table_name); + $this->recv_disableStreaming(); } - public function send_get_role_names() + public function send_disableStreaming($db_name, $table_name) { - $args = new \metastore\ThriftHiveMetastore_get_role_names_args(); + $args = new \metastore\ThriftHiveMetastore_disableStreaming_args(); + $args->db_name = $db_name; + $args->table_name = $table_name; $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); if ($bin_accel) { - thrift_protocol_write_binary($this->output_, 'get_role_names', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + thrift_protocol_write_binary($this->output_, 'disableStreaming', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); } else { - $this->output_->writeMessageBegin('get_role_names', TMessageType::CALL, $this->seqid_); + $this->output_->writeMessageBegin('disableStreaming', TMessageType::CALL, $this->seqid_); $args->write($this->output_); $this->output_->writeMessageEnd(); $this->output_->getTransport()->flush(); } } - public function recv_get_role_names() + public function recv_disableStreaming() { $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); - if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_get_role_names_result', $this->input_->isStrictRead()); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_disableStreaming_result', $this->input_->isStrictRead()); else { $rseqid = 0; @@ -4296,52 +4323,54 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas $this->input_->readMessageEnd(); throw $x; } - $result = new \metastore\ThriftHiveMetastore_get_role_names_result(); + $result = new \metastore\ThriftHiveMetastore_disableStreaming_result(); $result->read($this->input_); $this->input_->readMessageEnd(); } - if ($result->success !== null) { - return $result->success; - } if ($result->o1 !== null) { throw $result->o1; } - throw new \Exception("get_role_names failed: unknown result"); + if ($result->o2 !== null) { + throw $result->o2; + } + if ($result->o3 !== null) { + throw $result->o3; + } + if ($result->o4 !== null) { + throw $result->o4; + } + return; } - public function grant_role($role_name, $principal_name, $principal_type, $grantor, $grantorType, $grant_option) + public function getCurrentStreamingPartitionPath($dbName, $tableName) { - $this->send_grant_role($role_name, $principal_name, $principal_type, $grantor, $grantorType, $grant_option); - return $this->recv_grant_role(); + $this->send_getCurrentStreamingPartitionPath($dbName, $tableName); + return $this->recv_getCurrentStreamingPartitionPath(); } - public function send_grant_role($role_name, $principal_name, $principal_type, $grantor, $grantorType, $grant_option) + public function send_getCurrentStreamingPartitionPath($dbName, $tableName) { - $args = new \metastore\ThriftHiveMetastore_grant_role_args(); - $args->role_name = $role_name; - $args->principal_name = $principal_name; - $args->principal_type = $principal_type; - $args->grantor = $grantor; - $args->grantorType = $grantorType; - $args->grant_option = $grant_option; + $args = new \metastore\ThriftHiveMetastore_getCurrentStreamingPartitionPath_args(); + $args->dbName = $dbName; + $args->tableName = $tableName; $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); if ($bin_accel) { - thrift_protocol_write_binary($this->output_, 'grant_role', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + thrift_protocol_write_binary($this->output_, 'getCurrentStreamingPartitionPath', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); } else { - $this->output_->writeMessageBegin('grant_role', TMessageType::CALL, $this->seqid_); + $this->output_->writeMessageBegin('getCurrentStreamingPartitionPath', TMessageType::CALL, $this->seqid_); $args->write($this->output_); $this->output_->writeMessageEnd(); $this->output_->getTransport()->flush(); } } - public function recv_grant_role() + public function recv_getCurrentStreamingPartitionPath() { $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); - if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_grant_role_result', $this->input_->isStrictRead()); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_getCurrentStreamingPartitionPath_result', $this->input_->isStrictRead()); else { $rseqid = 0; @@ -4355,7 +4384,7 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas $this->input_->readMessageEnd(); throw $x; } - $result = new \metastore\ThriftHiveMetastore_grant_role_result(); + $result = new \metastore\ThriftHiveMetastore_getCurrentStreamingPartitionPath_result(); $result->read($this->input_); $this->input_->readMessageEnd(); } @@ -4365,39 +4394,49 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas if ($result->o1 !== null) { throw $result->o1; } - throw new \Exception("grant_role failed: unknown result"); + if ($result->o2 !== null) { + throw $result->o2; + } + if ($result->o3 !== null) { + throw $result->o3; + } + if ($result->o4 !== null) { + throw $result->o4; + } + throw new \Exception("getCurrentStreamingPartitionPath failed: unknown result"); } - public function revoke_role($role_name, $principal_name, $principal_type) + public function rollStreamingPartition($dbName, $tableName, $newPartitionPath, $newPartitionVal) { - $this->send_revoke_role($role_name, $principal_name, $principal_type); - return $this->recv_revoke_role(); + $this->send_rollStreamingPartition($dbName, $tableName, $newPartitionPath, $newPartitionVal); + return $this->recv_rollStreamingPartition(); } - public function send_revoke_role($role_name, $principal_name, $principal_type) + public function send_rollStreamingPartition($dbName, $tableName, $newPartitionPath, $newPartitionVal) { - $args = new \metastore\ThriftHiveMetastore_revoke_role_args(); - $args->role_name = $role_name; - $args->principal_name = $principal_name; - $args->principal_type = $principal_type; + $args = new \metastore\ThriftHiveMetastore_rollStreamingPartition_args(); + $args->dbName = $dbName; + $args->tableName = $tableName; + $args->newPartitionPath = $newPartitionPath; + $args->newPartitionVal = $newPartitionVal; $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); if ($bin_accel) { - thrift_protocol_write_binary($this->output_, 'revoke_role', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + thrift_protocol_write_binary($this->output_, 'rollStreamingPartition', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); } else { - $this->output_->writeMessageBegin('revoke_role', TMessageType::CALL, $this->seqid_); + $this->output_->writeMessageBegin('rollStreamingPartition', TMessageType::CALL, $this->seqid_); $args->write($this->output_); $this->output_->writeMessageEnd(); $this->output_->getTransport()->flush(); } } - public function recv_revoke_role() + public function recv_rollStreamingPartition() { $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); - if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_revoke_role_result', $this->input_->isStrictRead()); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_rollStreamingPartition_result', $this->input_->isStrictRead()); else { $rseqid = 0; @@ -4411,7 +4450,7 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas $this->input_->readMessageEnd(); throw $x; } - $result = new \metastore\ThriftHiveMetastore_revoke_role_result(); + $result = new \metastore\ThriftHiveMetastore_rollStreamingPartition_result(); $result->read($this->input_); $this->input_->readMessageEnd(); } @@ -4421,38 +4460,51 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas if ($result->o1 !== null) { throw $result->o1; } - throw new \Exception("revoke_role failed: unknown result"); + if ($result->o2 !== null) { + throw $result->o2; + } + if ($result->o3 !== null) { + throw $result->o3; + } + if ($result->o4 !== null) { + throw $result->o4; + } + if ($result->o5 !== null) { + throw $result->o5; + } + throw new \Exception("rollStreamingPartition failed: unknown result"); } - public function list_roles($principal_name, $principal_type) + public function updateStreamingTempLocation($dbName, $tableName, $streamingTempDir) { - $this->send_list_roles($principal_name, $principal_type); - return $this->recv_list_roles(); + $this->send_updateStreamingTempLocation($dbName, $tableName, $streamingTempDir); + $this->recv_updateStreamingTempLocation(); } - public function send_list_roles($principal_name, $principal_type) + public function send_updateStreamingTempLocation($dbName, $tableName, $streamingTempDir) { - $args = new \metastore\ThriftHiveMetastore_list_roles_args(); - $args->principal_name = $principal_name; - $args->principal_type = $principal_type; + $args = new \metastore\ThriftHiveMetastore_updateStreamingTempLocation_args(); + $args->dbName = $dbName; + $args->tableName = $tableName; + $args->streamingTempDir = $streamingTempDir; $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); if ($bin_accel) { - thrift_protocol_write_binary($this->output_, 'list_roles', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + thrift_protocol_write_binary($this->output_, 'updateStreamingTempLocation', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); } else { - $this->output_->writeMessageBegin('list_roles', TMessageType::CALL, $this->seqid_); + $this->output_->writeMessageBegin('updateStreamingTempLocation', TMessageType::CALL, $this->seqid_); $args->write($this->output_); $this->output_->writeMessageEnd(); $this->output_->getTransport()->flush(); } } - public function recv_list_roles() + public function recv_updateStreamingTempLocation() { $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); - if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_list_roles_result', $this->input_->isStrictRead()); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_updateStreamingTempLocation_result', $this->input_->isStrictRead()); else { $rseqid = 0; @@ -4466,49 +4518,51 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas $this->input_->readMessageEnd(); throw $x; } - $result = new \metastore\ThriftHiveMetastore_list_roles_result(); + $result = new \metastore\ThriftHiveMetastore_updateStreamingTempLocation_result(); $result->read($this->input_); $this->input_->readMessageEnd(); } - if ($result->success !== null) { - return $result->success; - } if ($result->o1 !== null) { throw $result->o1; } - throw new \Exception("list_roles failed: unknown result"); + if ($result->o2 !== null) { + throw $result->o2; + } + if ($result->o3 !== null) { + throw $result->o3; + } + return; } - public function get_privilege_set(\metastore\HiveObjectRef $hiveObject, $user_name, $group_names) + public function getNextChunkID($db_name, $table_name) { - $this->send_get_privilege_set($hiveObject, $user_name, $group_names); - return $this->recv_get_privilege_set(); + $this->send_getNextChunkID($db_name, $table_name); + return $this->recv_getNextChunkID(); } - public function send_get_privilege_set(\metastore\HiveObjectRef $hiveObject, $user_name, $group_names) + public function send_getNextChunkID($db_name, $table_name) { - $args = new \metastore\ThriftHiveMetastore_get_privilege_set_args(); - $args->hiveObject = $hiveObject; - $args->user_name = $user_name; - $args->group_names = $group_names; + $args = new \metastore\ThriftHiveMetastore_getNextChunkID_args(); + $args->db_name = $db_name; + $args->table_name = $table_name; $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); if ($bin_accel) { - thrift_protocol_write_binary($this->output_, 'get_privilege_set', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + thrift_protocol_write_binary($this->output_, 'getNextChunkID', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); } else { - $this->output_->writeMessageBegin('get_privilege_set', TMessageType::CALL, $this->seqid_); + $this->output_->writeMessageBegin('getNextChunkID', TMessageType::CALL, $this->seqid_); $args->write($this->output_); $this->output_->writeMessageEnd(); $this->output_->getTransport()->flush(); } } - public function recv_get_privilege_set() + public function recv_getNextChunkID() { $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); - if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_get_privilege_set_result', $this->input_->isStrictRead()); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_getNextChunkID_result', $this->input_->isStrictRead()); else { $rseqid = 0; @@ -4522,7 +4576,7 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas $this->input_->readMessageEnd(); throw $x; } - $result = new \metastore\ThriftHiveMetastore_get_privilege_set_result(); + $result = new \metastore\ThriftHiveMetastore_getNextChunkID_result(); $result->read($this->input_); $this->input_->readMessageEnd(); } @@ -4532,39 +4586,47 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas if ($result->o1 !== null) { throw $result->o1; } - throw new \Exception("get_privilege_set failed: unknown result"); + if ($result->o2 !== null) { + throw $result->o2; + } + if ($result->o3 !== null) { + throw $result->o3; + } + if ($result->o4 !== null) { + throw $result->o4; + } + throw new \Exception("getNextChunkID failed: unknown result"); } - public function list_privileges($principal_name, $principal_type, \metastore\HiveObjectRef $hiveObject) + public function getStreamingTmpDir($dbName, $tableName) { - $this->send_list_privileges($principal_name, $principal_type, $hiveObject); - return $this->recv_list_privileges(); + $this->send_getStreamingTmpDir($dbName, $tableName); + return $this->recv_getStreamingTmpDir(); } - public function send_list_privileges($principal_name, $principal_type, \metastore\HiveObjectRef $hiveObject) + public function send_getStreamingTmpDir($dbName, $tableName) { - $args = new \metastore\ThriftHiveMetastore_list_privileges_args(); - $args->principal_name = $principal_name; - $args->principal_type = $principal_type; - $args->hiveObject = $hiveObject; + $args = new \metastore\ThriftHiveMetastore_getStreamingTmpDir_args(); + $args->dbName = $dbName; + $args->tableName = $tableName; $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); if ($bin_accel) { - thrift_protocol_write_binary($this->output_, 'list_privileges', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + thrift_protocol_write_binary($this->output_, 'getStreamingTmpDir', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); } else { - $this->output_->writeMessageBegin('list_privileges', TMessageType::CALL, $this->seqid_); + $this->output_->writeMessageBegin('getStreamingTmpDir', TMessageType::CALL, $this->seqid_); $args->write($this->output_); $this->output_->writeMessageEnd(); $this->output_->getTransport()->flush(); } } - public function recv_list_privileges() + public function recv_getStreamingTmpDir() { $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); - if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_list_privileges_result', $this->input_->isStrictRead()); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_getStreamingTmpDir_result', $this->input_->isStrictRead()); else { $rseqid = 0; @@ -4578,7 +4640,7 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas $this->input_->readMessageEnd(); throw $x; } - $result = new \metastore\ThriftHiveMetastore_list_privileges_result(); + $result = new \metastore\ThriftHiveMetastore_getStreamingTmpDir_result(); $result->read($this->input_); $this->input_->readMessageEnd(); } @@ -4588,37 +4650,46 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas if ($result->o1 !== null) { throw $result->o1; } - throw new \Exception("list_privileges failed: unknown result"); + if ($result->o2 !== null) { + throw $result->o2; + } + if ($result->o3 !== null) { + throw $result->o3; + } + if ($result->o4 !== null) { + throw $result->o4; + } + throw new \Exception("getStreamingTmpDir failed: unknown result"); } - public function grant_privileges(\metastore\PrivilegeBag $privileges) + public function create_role(\metastore\Role $role) { - $this->send_grant_privileges($privileges); - return $this->recv_grant_privileges(); + $this->send_create_role($role); + return $this->recv_create_role(); } - public function send_grant_privileges(\metastore\PrivilegeBag $privileges) + public function send_create_role(\metastore\Role $role) { - $args = new \metastore\ThriftHiveMetastore_grant_privileges_args(); - $args->privileges = $privileges; + $args = new \metastore\ThriftHiveMetastore_create_role_args(); + $args->role = $role; $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); if ($bin_accel) { - thrift_protocol_write_binary($this->output_, 'grant_privileges', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + thrift_protocol_write_binary($this->output_, 'create_role', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); } else { - $this->output_->writeMessageBegin('grant_privileges', TMessageType::CALL, $this->seqid_); + $this->output_->writeMessageBegin('create_role', TMessageType::CALL, $this->seqid_); $args->write($this->output_); $this->output_->writeMessageEnd(); $this->output_->getTransport()->flush(); } } - public function recv_grant_privileges() + public function recv_create_role() { $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); - if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_grant_privileges_result', $this->input_->isStrictRead()); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_create_role_result', $this->input_->isStrictRead()); else { $rseqid = 0; @@ -4632,7 +4703,7 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas $this->input_->readMessageEnd(); throw $x; } - $result = new \metastore\ThriftHiveMetastore_grant_privileges_result(); + $result = new \metastore\ThriftHiveMetastore_create_role_result(); $result->read($this->input_); $this->input_->readMessageEnd(); } @@ -4642,37 +4713,37 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas if ($result->o1 !== null) { throw $result->o1; } - throw new \Exception("grant_privileges failed: unknown result"); + throw new \Exception("create_role failed: unknown result"); } - public function revoke_privileges(\metastore\PrivilegeBag $privileges) + public function drop_role($role_name) { - $this->send_revoke_privileges($privileges); - return $this->recv_revoke_privileges(); + $this->send_drop_role($role_name); + return $this->recv_drop_role(); } - public function send_revoke_privileges(\metastore\PrivilegeBag $privileges) + public function send_drop_role($role_name) { - $args = new \metastore\ThriftHiveMetastore_revoke_privileges_args(); - $args->privileges = $privileges; + $args = new \metastore\ThriftHiveMetastore_drop_role_args(); + $args->role_name = $role_name; $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); if ($bin_accel) { - thrift_protocol_write_binary($this->output_, 'revoke_privileges', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + thrift_protocol_write_binary($this->output_, 'drop_role', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); } else { - $this->output_->writeMessageBegin('revoke_privileges', TMessageType::CALL, $this->seqid_); + $this->output_->writeMessageBegin('drop_role', TMessageType::CALL, $this->seqid_); $args->write($this->output_); $this->output_->writeMessageEnd(); $this->output_->getTransport()->flush(); } } - public function recv_revoke_privileges() + public function recv_drop_role() { $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); - if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_revoke_privileges_result', $this->input_->isStrictRead()); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_drop_role_result', $this->input_->isStrictRead()); else { $rseqid = 0; @@ -4686,7 +4757,7 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas $this->input_->readMessageEnd(); throw $x; } - $result = new \metastore\ThriftHiveMetastore_revoke_privileges_result(); + $result = new \metastore\ThriftHiveMetastore_drop_role_result(); $result->read($this->input_); $this->input_->readMessageEnd(); } @@ -4696,38 +4767,36 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas if ($result->o1 !== null) { throw $result->o1; } - throw new \Exception("revoke_privileges failed: unknown result"); + throw new \Exception("drop_role failed: unknown result"); } - public function set_ugi($user_name, $group_names) + public function get_role_names() { - $this->send_set_ugi($user_name, $group_names); - return $this->recv_set_ugi(); + $this->send_get_role_names(); + return $this->recv_get_role_names(); } - public function send_set_ugi($user_name, $group_names) + public function send_get_role_names() { - $args = new \metastore\ThriftHiveMetastore_set_ugi_args(); - $args->user_name = $user_name; - $args->group_names = $group_names; + $args = new \metastore\ThriftHiveMetastore_get_role_names_args(); $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); if ($bin_accel) { - thrift_protocol_write_binary($this->output_, 'set_ugi', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + thrift_protocol_write_binary($this->output_, 'get_role_names', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); } else { - $this->output_->writeMessageBegin('set_ugi', TMessageType::CALL, $this->seqid_); + $this->output_->writeMessageBegin('get_role_names', TMessageType::CALL, $this->seqid_); $args->write($this->output_); $this->output_->writeMessageEnd(); $this->output_->getTransport()->flush(); } } - public function recv_set_ugi() + public function recv_get_role_names() { $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); - if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_set_ugi_result', $this->input_->isStrictRead()); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_get_role_names_result', $this->input_->isStrictRead()); else { $rseqid = 0; @@ -4741,7 +4810,7 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas $this->input_->readMessageEnd(); throw $x; } - $result = new \metastore\ThriftHiveMetastore_set_ugi_result(); + $result = new \metastore\ThriftHiveMetastore_get_role_names_result(); $result->read($this->input_); $this->input_->readMessageEnd(); } @@ -4751,38 +4820,42 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas if ($result->o1 !== null) { throw $result->o1; } - throw new \Exception("set_ugi failed: unknown result"); + throw new \Exception("get_role_names failed: unknown result"); } - public function get_delegation_token($token_owner, $renewer_kerberos_principal_name) + public function grant_role($role_name, $principal_name, $principal_type, $grantor, $grantorType, $grant_option) { - $this->send_get_delegation_token($token_owner, $renewer_kerberos_principal_name); - return $this->recv_get_delegation_token(); + $this->send_grant_role($role_name, $principal_name, $principal_type, $grantor, $grantorType, $grant_option); + return $this->recv_grant_role(); } - public function send_get_delegation_token($token_owner, $renewer_kerberos_principal_name) + public function send_grant_role($role_name, $principal_name, $principal_type, $grantor, $grantorType, $grant_option) { - $args = new \metastore\ThriftHiveMetastore_get_delegation_token_args(); - $args->token_owner = $token_owner; - $args->renewer_kerberos_principal_name = $renewer_kerberos_principal_name; + $args = new \metastore\ThriftHiveMetastore_grant_role_args(); + $args->role_name = $role_name; + $args->principal_name = $principal_name; + $args->principal_type = $principal_type; + $args->grantor = $grantor; + $args->grantorType = $grantorType; + $args->grant_option = $grant_option; $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); if ($bin_accel) { - thrift_protocol_write_binary($this->output_, 'get_delegation_token', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + thrift_protocol_write_binary($this->output_, 'grant_role', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); } else { - $this->output_->writeMessageBegin('get_delegation_token', TMessageType::CALL, $this->seqid_); + $this->output_->writeMessageBegin('grant_role', TMessageType::CALL, $this->seqid_); $args->write($this->output_); $this->output_->writeMessageEnd(); $this->output_->getTransport()->flush(); } } - public function recv_get_delegation_token() + public function recv_grant_role() { $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); - if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_get_delegation_token_result', $this->input_->isStrictRead()); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_grant_role_result', $this->input_->isStrictRead()); else { $rseqid = 0; @@ -4796,7 +4869,7 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas $this->input_->readMessageEnd(); throw $x; } - $result = new \metastore\ThriftHiveMetastore_get_delegation_token_result(); + $result = new \metastore\ThriftHiveMetastore_grant_role_result(); $result->read($this->input_); $this->input_->readMessageEnd(); } @@ -4806,37 +4879,39 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas if ($result->o1 !== null) { throw $result->o1; } - throw new \Exception("get_delegation_token failed: unknown result"); + throw new \Exception("grant_role failed: unknown result"); } - public function renew_delegation_token($token_str_form) + public function revoke_role($role_name, $principal_name, $principal_type) { - $this->send_renew_delegation_token($token_str_form); - return $this->recv_renew_delegation_token(); + $this->send_revoke_role($role_name, $principal_name, $principal_type); + return $this->recv_revoke_role(); } - public function send_renew_delegation_token($token_str_form) + public function send_revoke_role($role_name, $principal_name, $principal_type) { - $args = new \metastore\ThriftHiveMetastore_renew_delegation_token_args(); - $args->token_str_form = $token_str_form; + $args = new \metastore\ThriftHiveMetastore_revoke_role_args(); + $args->role_name = $role_name; + $args->principal_name = $principal_name; + $args->principal_type = $principal_type; $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); if ($bin_accel) { - thrift_protocol_write_binary($this->output_, 'renew_delegation_token', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + thrift_protocol_write_binary($this->output_, 'revoke_role', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); } else { - $this->output_->writeMessageBegin('renew_delegation_token', TMessageType::CALL, $this->seqid_); + $this->output_->writeMessageBegin('revoke_role', TMessageType::CALL, $this->seqid_); $args->write($this->output_); $this->output_->writeMessageEnd(); $this->output_->getTransport()->flush(); } } - public function recv_renew_delegation_token() + public function recv_revoke_role() { $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); - if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_renew_delegation_token_result', $this->input_->isStrictRead()); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_revoke_role_result', $this->input_->isStrictRead()); else { $rseqid = 0; @@ -4850,7 +4925,7 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas $this->input_->readMessageEnd(); throw $x; } - $result = new \metastore\ThriftHiveMetastore_renew_delegation_token_result(); + $result = new \metastore\ThriftHiveMetastore_revoke_role_result(); $result->read($this->input_); $this->input_->readMessageEnd(); } @@ -4860,37 +4935,38 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas if ($result->o1 !== null) { throw $result->o1; } - throw new \Exception("renew_delegation_token failed: unknown result"); + throw new \Exception("revoke_role failed: unknown result"); } - public function cancel_delegation_token($token_str_form) + public function list_roles($principal_name, $principal_type) { - $this->send_cancel_delegation_token($token_str_form); - $this->recv_cancel_delegation_token(); + $this->send_list_roles($principal_name, $principal_type); + return $this->recv_list_roles(); } - public function send_cancel_delegation_token($token_str_form) + public function send_list_roles($principal_name, $principal_type) { - $args = new \metastore\ThriftHiveMetastore_cancel_delegation_token_args(); - $args->token_str_form = $token_str_form; + $args = new \metastore\ThriftHiveMetastore_list_roles_args(); + $args->principal_name = $principal_name; + $args->principal_type = $principal_type; $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); if ($bin_accel) { - thrift_protocol_write_binary($this->output_, 'cancel_delegation_token', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + thrift_protocol_write_binary($this->output_, 'list_roles', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); } else { - $this->output_->writeMessageBegin('cancel_delegation_token', TMessageType::CALL, $this->seqid_); + $this->output_->writeMessageBegin('list_roles', TMessageType::CALL, $this->seqid_); $args->write($this->output_); $this->output_->writeMessageEnd(); $this->output_->getTransport()->flush(); } } - public function recv_cancel_delegation_token() + public function recv_list_roles() { $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); - if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_cancel_delegation_token_result', $this->input_->isStrictRead()); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_list_roles_result', $this->input_->isStrictRead()); else { $rseqid = 0; @@ -4904,44 +4980,2062 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas $this->input_->readMessageEnd(); throw $x; } - $result = new \metastore\ThriftHiveMetastore_cancel_delegation_token_result(); + $result = new \metastore\ThriftHiveMetastore_list_roles_result(); $result->read($this->input_); $this->input_->readMessageEnd(); } + if ($result->success !== null) { + return $result->success; + } if ($result->o1 !== null) { throw $result->o1; } - return; + throw new \Exception("list_roles failed: unknown result"); } -} + public function get_privilege_set(\metastore\HiveObjectRef $hiveObject, $user_name, $group_names) + { + $this->send_get_privilege_set($hiveObject, $user_name, $group_names); + return $this->recv_get_privilege_set(); + } -// HELPER FUNCTIONS AND STRUCTURES + public function send_get_privilege_set(\metastore\HiveObjectRef $hiveObject, $user_name, $group_names) + { + $args = new \metastore\ThriftHiveMetastore_get_privilege_set_args(); + $args->hiveObject = $hiveObject; + $args->user_name = $user_name; + $args->group_names = $group_names; + $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); + if ($bin_accel) + { + thrift_protocol_write_binary($this->output_, 'get_privilege_set', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + } + else + { + $this->output_->writeMessageBegin('get_privilege_set', TMessageType::CALL, $this->seqid_); + $args->write($this->output_); + $this->output_->writeMessageEnd(); + $this->output_->getTransport()->flush(); + } + } -class ThriftHiveMetastore_create_database_args { - static $_TSPEC; + public function recv_get_privilege_set() + { + $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_get_privilege_set_result', $this->input_->isStrictRead()); + else + { + $rseqid = 0; + $fname = null; + $mtype = 0; - public $database = null; + $this->input_->readMessageBegin($fname, $mtype, $rseqid); + if ($mtype == TMessageType::EXCEPTION) { + $x = new TApplicationException(); + $x->read($this->input_); + $this->input_->readMessageEnd(); + throw $x; + } + $result = new \metastore\ThriftHiveMetastore_get_privilege_set_result(); + $result->read($this->input_); + $this->input_->readMessageEnd(); + } + if ($result->success !== null) { + return $result->success; + } + if ($result->o1 !== null) { + throw $result->o1; + } + throw new \Exception("get_privilege_set failed: unknown result"); + } - public function __construct($vals=null) { - if (!isset(self::$_TSPEC)) { - self::$_TSPEC = array( - 1 => array( - 'var' => 'database', - 'type' => TType::STRUCT, - 'class' => '\metastore\Database', - ), - ); + public function list_privileges($principal_name, $principal_type, \metastore\HiveObjectRef $hiveObject) + { + $this->send_list_privileges($principal_name, $principal_type, $hiveObject); + return $this->recv_list_privileges(); + } + + public function send_list_privileges($principal_name, $principal_type, \metastore\HiveObjectRef $hiveObject) + { + $args = new \metastore\ThriftHiveMetastore_list_privileges_args(); + $args->principal_name = $principal_name; + $args->principal_type = $principal_type; + $args->hiveObject = $hiveObject; + $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); + if ($bin_accel) + { + thrift_protocol_write_binary($this->output_, 'list_privileges', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); } - if (is_array($vals)) { - if (isset($vals['database'])) { - $this->database = $vals['database']; + else + { + $this->output_->writeMessageBegin('list_privileges', TMessageType::CALL, $this->seqid_); + $args->write($this->output_); + $this->output_->writeMessageEnd(); + $this->output_->getTransport()->flush(); + } + } + + public function recv_list_privileges() + { + $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_list_privileges_result', $this->input_->isStrictRead()); + else + { + $rseqid = 0; + $fname = null; + $mtype = 0; + + $this->input_->readMessageBegin($fname, $mtype, $rseqid); + if ($mtype == TMessageType::EXCEPTION) { + $x = new TApplicationException(); + $x->read($this->input_); + $this->input_->readMessageEnd(); + throw $x; + } + $result = new \metastore\ThriftHiveMetastore_list_privileges_result(); + $result->read($this->input_); + $this->input_->readMessageEnd(); + } + if ($result->success !== null) { + return $result->success; + } + if ($result->o1 !== null) { + throw $result->o1; + } + throw new \Exception("list_privileges failed: unknown result"); + } + + public function grant_privileges(\metastore\PrivilegeBag $privileges) + { + $this->send_grant_privileges($privileges); + return $this->recv_grant_privileges(); + } + + public function send_grant_privileges(\metastore\PrivilegeBag $privileges) + { + $args = new \metastore\ThriftHiveMetastore_grant_privileges_args(); + $args->privileges = $privileges; + $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); + if ($bin_accel) + { + thrift_protocol_write_binary($this->output_, 'grant_privileges', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + } + else + { + $this->output_->writeMessageBegin('grant_privileges', TMessageType::CALL, $this->seqid_); + $args->write($this->output_); + $this->output_->writeMessageEnd(); + $this->output_->getTransport()->flush(); + } + } + + public function recv_grant_privileges() + { + $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_grant_privileges_result', $this->input_->isStrictRead()); + else + { + $rseqid = 0; + $fname = null; + $mtype = 0; + + $this->input_->readMessageBegin($fname, $mtype, $rseqid); + if ($mtype == TMessageType::EXCEPTION) { + $x = new TApplicationException(); + $x->read($this->input_); + $this->input_->readMessageEnd(); + throw $x; + } + $result = new \metastore\ThriftHiveMetastore_grant_privileges_result(); + $result->read($this->input_); + $this->input_->readMessageEnd(); + } + if ($result->success !== null) { + return $result->success; + } + if ($result->o1 !== null) { + throw $result->o1; + } + throw new \Exception("grant_privileges failed: unknown result"); + } + + public function revoke_privileges(\metastore\PrivilegeBag $privileges) + { + $this->send_revoke_privileges($privileges); + return $this->recv_revoke_privileges(); + } + + public function send_revoke_privileges(\metastore\PrivilegeBag $privileges) + { + $args = new \metastore\ThriftHiveMetastore_revoke_privileges_args(); + $args->privileges = $privileges; + $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); + if ($bin_accel) + { + thrift_protocol_write_binary($this->output_, 'revoke_privileges', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + } + else + { + $this->output_->writeMessageBegin('revoke_privileges', TMessageType::CALL, $this->seqid_); + $args->write($this->output_); + $this->output_->writeMessageEnd(); + $this->output_->getTransport()->flush(); + } + } + + public function recv_revoke_privileges() + { + $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_revoke_privileges_result', $this->input_->isStrictRead()); + else + { + $rseqid = 0; + $fname = null; + $mtype = 0; + + $this->input_->readMessageBegin($fname, $mtype, $rseqid); + if ($mtype == TMessageType::EXCEPTION) { + $x = new TApplicationException(); + $x->read($this->input_); + $this->input_->readMessageEnd(); + throw $x; + } + $result = new \metastore\ThriftHiveMetastore_revoke_privileges_result(); + $result->read($this->input_); + $this->input_->readMessageEnd(); + } + if ($result->success !== null) { + return $result->success; + } + if ($result->o1 !== null) { + throw $result->o1; + } + throw new \Exception("revoke_privileges failed: unknown result"); + } + + public function set_ugi($user_name, $group_names) + { + $this->send_set_ugi($user_name, $group_names); + return $this->recv_set_ugi(); + } + + public function send_set_ugi($user_name, $group_names) + { + $args = new \metastore\ThriftHiveMetastore_set_ugi_args(); + $args->user_name = $user_name; + $args->group_names = $group_names; + $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); + if ($bin_accel) + { + thrift_protocol_write_binary($this->output_, 'set_ugi', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + } + else + { + $this->output_->writeMessageBegin('set_ugi', TMessageType::CALL, $this->seqid_); + $args->write($this->output_); + $this->output_->writeMessageEnd(); + $this->output_->getTransport()->flush(); + } + } + + public function recv_set_ugi() + { + $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_set_ugi_result', $this->input_->isStrictRead()); + else + { + $rseqid = 0; + $fname = null; + $mtype = 0; + + $this->input_->readMessageBegin($fname, $mtype, $rseqid); + if ($mtype == TMessageType::EXCEPTION) { + $x = new TApplicationException(); + $x->read($this->input_); + $this->input_->readMessageEnd(); + throw $x; + } + $result = new \metastore\ThriftHiveMetastore_set_ugi_result(); + $result->read($this->input_); + $this->input_->readMessageEnd(); + } + if ($result->success !== null) { + return $result->success; + } + if ($result->o1 !== null) { + throw $result->o1; + } + throw new \Exception("set_ugi failed: unknown result"); + } + + public function get_delegation_token($token_owner, $renewer_kerberos_principal_name) + { + $this->send_get_delegation_token($token_owner, $renewer_kerberos_principal_name); + return $this->recv_get_delegation_token(); + } + + public function send_get_delegation_token($token_owner, $renewer_kerberos_principal_name) + { + $args = new \metastore\ThriftHiveMetastore_get_delegation_token_args(); + $args->token_owner = $token_owner; + $args->renewer_kerberos_principal_name = $renewer_kerberos_principal_name; + $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); + if ($bin_accel) + { + thrift_protocol_write_binary($this->output_, 'get_delegation_token', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + } + else + { + $this->output_->writeMessageBegin('get_delegation_token', TMessageType::CALL, $this->seqid_); + $args->write($this->output_); + $this->output_->writeMessageEnd(); + $this->output_->getTransport()->flush(); + } + } + + public function recv_get_delegation_token() + { + $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_get_delegation_token_result', $this->input_->isStrictRead()); + else + { + $rseqid = 0; + $fname = null; + $mtype = 0; + + $this->input_->readMessageBegin($fname, $mtype, $rseqid); + if ($mtype == TMessageType::EXCEPTION) { + $x = new TApplicationException(); + $x->read($this->input_); + $this->input_->readMessageEnd(); + throw $x; + } + $result = new \metastore\ThriftHiveMetastore_get_delegation_token_result(); + $result->read($this->input_); + $this->input_->readMessageEnd(); + } + if ($result->success !== null) { + return $result->success; + } + if ($result->o1 !== null) { + throw $result->o1; + } + throw new \Exception("get_delegation_token failed: unknown result"); + } + + public function renew_delegation_token($token_str_form) + { + $this->send_renew_delegation_token($token_str_form); + return $this->recv_renew_delegation_token(); + } + + public function send_renew_delegation_token($token_str_form) + { + $args = new \metastore\ThriftHiveMetastore_renew_delegation_token_args(); + $args->token_str_form = $token_str_form; + $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); + if ($bin_accel) + { + thrift_protocol_write_binary($this->output_, 'renew_delegation_token', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + } + else + { + $this->output_->writeMessageBegin('renew_delegation_token', TMessageType::CALL, $this->seqid_); + $args->write($this->output_); + $this->output_->writeMessageEnd(); + $this->output_->getTransport()->flush(); + } + } + + public function recv_renew_delegation_token() + { + $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_renew_delegation_token_result', $this->input_->isStrictRead()); + else + { + $rseqid = 0; + $fname = null; + $mtype = 0; + + $this->input_->readMessageBegin($fname, $mtype, $rseqid); + if ($mtype == TMessageType::EXCEPTION) { + $x = new TApplicationException(); + $x->read($this->input_); + $this->input_->readMessageEnd(); + throw $x; + } + $result = new \metastore\ThriftHiveMetastore_renew_delegation_token_result(); + $result->read($this->input_); + $this->input_->readMessageEnd(); + } + if ($result->success !== null) { + return $result->success; + } + if ($result->o1 !== null) { + throw $result->o1; + } + throw new \Exception("renew_delegation_token failed: unknown result"); + } + + public function cancel_delegation_token($token_str_form) + { + $this->send_cancel_delegation_token($token_str_form); + $this->recv_cancel_delegation_token(); + } + + public function send_cancel_delegation_token($token_str_form) + { + $args = new \metastore\ThriftHiveMetastore_cancel_delegation_token_args(); + $args->token_str_form = $token_str_form; + $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); + if ($bin_accel) + { + thrift_protocol_write_binary($this->output_, 'cancel_delegation_token', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + } + else + { + $this->output_->writeMessageBegin('cancel_delegation_token', TMessageType::CALL, $this->seqid_); + $args->write($this->output_); + $this->output_->writeMessageEnd(); + $this->output_->getTransport()->flush(); + } + } + + public function recv_cancel_delegation_token() + { + $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_cancel_delegation_token_result', $this->input_->isStrictRead()); + else + { + $rseqid = 0; + $fname = null; + $mtype = 0; + + $this->input_->readMessageBegin($fname, $mtype, $rseqid); + if ($mtype == TMessageType::EXCEPTION) { + $x = new TApplicationException(); + $x->read($this->input_); + $this->input_->readMessageEnd(); + throw $x; + } + $result = new \metastore\ThriftHiveMetastore_cancel_delegation_token_result(); + $result->read($this->input_); + $this->input_->readMessageEnd(); + } + if ($result->o1 !== null) { + throw $result->o1; + } + return; + } + +} + +// HELPER FUNCTIONS AND STRUCTURES + +class ThriftHiveMetastore_create_database_args { + static $_TSPEC; + + public $database = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'database', + 'type' => TType::STRUCT, + 'class' => '\metastore\Database', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['database'])) { + $this->database = $vals['database']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_create_database_args'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRUCT) { + $this->database = new \metastore\Database(); + $xfer += $this->database->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_database_args'); + if ($this->database !== null) { + if (!is_object($this->database)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('database', TType::STRUCT, 1); + $xfer += $this->database->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_create_database_result { + static $_TSPEC; + + public $o1 = null; + public $o2 = null; + public $o3 = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'o1', + 'type' => TType::STRUCT, + 'class' => '\metastore\AlreadyExistsException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidObjectException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, + 'class' => '\metastore\MetaException', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['o1'])) { + $this->o1 = $vals['o1']; + } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_create_database_result'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRUCT) { + $this->o1 = new \metastore\AlreadyExistsException(); + $xfer += $this->o1->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\InvalidObjectException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\MetaException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_database_result'); + if ($this->o1 !== null) { + $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); + $xfer += $this->o1->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_get_database_args { + static $_TSPEC; + + public $name = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'name', + 'type' => TType::STRING, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['name'])) { + $this->name = $vals['name']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_get_database_args'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->name); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_database_args'); + if ($this->name !== null) { + $xfer += $output->writeFieldBegin('name', TType::STRING, 1); + $xfer += $output->writeString($this->name); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_get_database_result { + static $_TSPEC; + + public $success = null; + public $o1 = null; + public $o2 = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::STRUCT, + 'class' => '\metastore\Database', + ), + 1 => array( + 'var' => 'o1', + 'type' => TType::STRUCT, + 'class' => '\metastore\NoSuchObjectException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\MetaException', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } + if (isset($vals['o1'])) { + $this->o1 = $vals['o1']; + } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_get_database_result'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 0: + if ($ftype == TType::STRUCT) { + $this->success = new \metastore\Database(); + $xfer += $this->success->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 1: + if ($ftype == TType::STRUCT) { + $this->o1 = new \metastore\NoSuchObjectException(); + $xfer += $this->o1->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\MetaException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_database_result'); + if ($this->success !== null) { + if (!is_object($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o1 !== null) { + $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); + $xfer += $this->o1->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_drop_database_args { + static $_TSPEC; + + public $name = null; + public $deleteData = null; + public $cascade = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'name', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'deleteData', + 'type' => TType::BOOL, + ), + 3 => array( + 'var' => 'cascade', + 'type' => TType::BOOL, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['name'])) { + $this->name = $vals['name']; + } + if (isset($vals['deleteData'])) { + $this->deleteData = $vals['deleteData']; + } + if (isset($vals['cascade'])) { + $this->cascade = $vals['cascade']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_drop_database_args'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->deleteData); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->cascade); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_database_args'); + if ($this->name !== null) { + $xfer += $output->writeFieldBegin('name', TType::STRING, 1); + $xfer += $output->writeString($this->name); + $xfer += $output->writeFieldEnd(); + } + if ($this->deleteData !== null) { + $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 2); + $xfer += $output->writeBool($this->deleteData); + $xfer += $output->writeFieldEnd(); + } + if ($this->cascade !== null) { + $xfer += $output->writeFieldBegin('cascade', TType::BOOL, 3); + $xfer += $output->writeBool($this->cascade); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_drop_database_result { + static $_TSPEC; + + public $o1 = null; + public $o2 = null; + public $o3 = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'o1', + 'type' => TType::STRUCT, + 'class' => '\metastore\NoSuchObjectException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidOperationException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, + 'class' => '\metastore\MetaException', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['o1'])) { + $this->o1 = $vals['o1']; + } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_drop_database_result'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRUCT) { + $this->o1 = new \metastore\NoSuchObjectException(); + $xfer += $this->o1->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\InvalidOperationException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\MetaException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_database_result'); + if ($this->o1 !== null) { + $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); + $xfer += $this->o1->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_get_databases_args { + static $_TSPEC; + + public $pattern = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'pattern', + 'type' => TType::STRING, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['pattern'])) { + $this->pattern = $vals['pattern']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_get_databases_args'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->pattern); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_databases_args'); + if ($this->pattern !== null) { + $xfer += $output->writeFieldBegin('pattern', TType::STRING, 1); + $xfer += $output->writeString($this->pattern); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_get_databases_result { + static $_TSPEC; + + public $success = null; + public $o1 = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), + ), + 1 => array( + 'var' => 'o1', + 'type' => TType::STRUCT, + 'class' => '\metastore\MetaException', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } + if (isset($vals['o1'])) { + $this->o1 = $vals['o1']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_get_databases_result'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 0: + if ($ftype == TType::LST) { + $this->success = array(); + $_size227 = 0; + $_etype230 = 0; + $xfer += $input->readListBegin($_etype230, $_size227); + for ($_i231 = 0; $_i231 < $_size227; ++$_i231) + { + $elem232 = null; + $xfer += $input->readString($elem232); + $this->success []= $elem232; + } + $xfer += $input->readListEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; + case 1: + if ($ftype == TType::STRUCT) { + $this->o1 = new \metastore\MetaException(); + $xfer += $this->o1->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_databases_result'); + if ($this->success !== null) { + if (!is_array($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRING, count($this->success)); + { + foreach ($this->success as $iter233) + { + $xfer += $output->writeString($iter233); + } + } + $output->writeListEnd(); + } + $xfer += $output->writeFieldEnd(); + } + if ($this->o1 !== null) { + $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); + $xfer += $this->o1->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_get_all_databases_args { + static $_TSPEC; + + + public function __construct() { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + ); + } + } + + public function getName() { + return 'ThriftHiveMetastore_get_all_databases_args'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_all_databases_args'); + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_get_all_databases_result { + static $_TSPEC; + + public $success = null; + public $o1 = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), + ), + 1 => array( + 'var' => 'o1', + 'type' => TType::STRUCT, + 'class' => '\metastore\MetaException', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } + if (isset($vals['o1'])) { + $this->o1 = $vals['o1']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_get_all_databases_result'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 0: + if ($ftype == TType::LST) { + $this->success = array(); + $_size234 = 0; + $_etype237 = 0; + $xfer += $input->readListBegin($_etype237, $_size234); + for ($_i238 = 0; $_i238 < $_size234; ++$_i238) + { + $elem239 = null; + $xfer += $input->readString($elem239); + $this->success []= $elem239; + } + $xfer += $input->readListEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; + case 1: + if ($ftype == TType::STRUCT) { + $this->o1 = new \metastore\MetaException(); + $xfer += $this->o1->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_all_databases_result'); + if ($this->success !== null) { + if (!is_array($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRING, count($this->success)); + { + foreach ($this->success as $iter240) + { + $xfer += $output->writeString($iter240); + } + } + $output->writeListEnd(); + } + $xfer += $output->writeFieldEnd(); + } + if ($this->o1 !== null) { + $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); + $xfer += $this->o1->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_alter_database_args { + static $_TSPEC; + + public $dbname = null; + public $db = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'dbname', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'db', + 'type' => TType::STRUCT, + 'class' => '\metastore\Database', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['dbname'])) { + $this->dbname = $vals['dbname']; + } + if (isset($vals['db'])) { + $this->db = $vals['db']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_alter_database_args'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->dbname); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRUCT) { + $this->db = new \metastore\Database(); + $xfer += $this->db->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_database_args'); + if ($this->dbname !== null) { + $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); + $xfer += $output->writeString($this->dbname); + $xfer += $output->writeFieldEnd(); + } + if ($this->db !== null) { + if (!is_object($this->db)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('db', TType::STRUCT, 2); + $xfer += $this->db->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_alter_database_result { + static $_TSPEC; + + public $o1 = null; + public $o2 = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'o1', + 'type' => TType::STRUCT, + 'class' => '\metastore\MetaException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\NoSuchObjectException', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['o1'])) { + $this->o1 = $vals['o1']; + } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_alter_database_result'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRUCT) { + $this->o1 = new \metastore\MetaException(); + $xfer += $this->o1->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\NoSuchObjectException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_database_result'); + if ($this->o1 !== null) { + $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); + $xfer += $this->o1->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_get_type_args { + static $_TSPEC; + + public $name = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'name', + 'type' => TType::STRING, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['name'])) { + $this->name = $vals['name']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_get_type_args'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->name); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_type_args'); + if ($this->name !== null) { + $xfer += $output->writeFieldBegin('name', TType::STRING, 1); + $xfer += $output->writeString($this->name); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_get_type_result { + static $_TSPEC; + + public $success = null; + public $o1 = null; + public $o2 = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::STRUCT, + 'class' => '\metastore\Type', + ), + 1 => array( + 'var' => 'o1', + 'type' => TType::STRUCT, + 'class' => '\metastore\MetaException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\NoSuchObjectException', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } + if (isset($vals['o1'])) { + $this->o1 = $vals['o1']; + } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_get_type_result'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 0: + if ($ftype == TType::STRUCT) { + $this->success = new \metastore\Type(); + $xfer += $this->success->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 1: + if ($ftype == TType::STRUCT) { + $this->o1 = new \metastore\MetaException(); + $xfer += $this->o1->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\NoSuchObjectException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_type_result'); + if ($this->success !== null) { + if (!is_object($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o1 !== null) { + $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); + $xfer += $this->o1->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_create_type_args { + static $_TSPEC; + + public $type = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'type', + 'type' => TType::STRUCT, + 'class' => '\metastore\Type', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['type'])) { + $this->type = $vals['type']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_create_type_args'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRUCT) { + $this->type = new \metastore\Type(); + $xfer += $this->type->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_type_args'); + if ($this->type !== null) { + if (!is_object($this->type)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('type', TType::STRUCT, 1); + $xfer += $this->type->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_create_type_result { + static $_TSPEC; + + public $success = null; + public $o1 = null; + public $o2 = null; + public $o3 = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::BOOL, + ), + 1 => array( + 'var' => 'o1', + 'type' => TType::STRUCT, + 'class' => '\metastore\AlreadyExistsException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidObjectException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, + 'class' => '\metastore\MetaException', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } + if (isset($vals['o1'])) { + $this->o1 = $vals['o1']; + } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_create_type_result'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 0: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->success); + } else { + $xfer += $input->skip($ftype); + } + break; + case 1: + if ($ftype == TType::STRUCT) { + $this->o1 = new \metastore\AlreadyExistsException(); + $xfer += $this->o1->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\InvalidObjectException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\MetaException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_type_result'); + if ($this->success !== null) { + $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); + $xfer += $output->writeBool($this->success); + $xfer += $output->writeFieldEnd(); + } + if ($this->o1 !== null) { + $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); + $xfer += $this->o1->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_drop_type_args { + static $_TSPEC; + + public $type = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'type', + 'type' => TType::STRING, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['type'])) { + $this->type = $vals['type']; } } } public function getName() { - return 'ThriftHiveMetastore_create_database_args'; + return 'ThriftHiveMetastore_drop_type_args'; } public function read($input) @@ -4960,9 +7054,8 @@ class ThriftHiveMetastore_create_database_args { switch ($fid) { case 1: - if ($ftype == TType::STRUCT) { - $this->database = new \metastore\Database(); - $xfer += $this->database->read($input); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->type); } else { $xfer += $input->skip($ftype); } @@ -4979,13 +7072,10 @@ class ThriftHiveMetastore_create_database_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_database_args'); - if ($this->database !== null) { - if (!is_object($this->database)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('database', TType::STRUCT, 1); - $xfer += $this->database->write($output); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_type_args'); + if ($this->type !== null) { + $xfer += $output->writeFieldBegin('type', TType::STRING, 1); + $xfer += $output->writeString($this->type); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -4995,48 +7085,47 @@ class ThriftHiveMetastore_create_database_args { } -class ThriftHiveMetastore_create_database_result { +class ThriftHiveMetastore_drop_type_result { static $_TSPEC; + public $success = null; public $o1 = null; public $o2 = null; - public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::BOOL, + ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\AlreadyExistsException', + 'class' => '\metastore\MetaException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', - ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchObjectException', ), ); } if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } } } public function getName() { - return 'ThriftHiveMetastore_create_database_result'; + return 'ThriftHiveMetastore_drop_type_result'; } public function read($input) @@ -5054,26 +7143,25 @@ class ThriftHiveMetastore_create_database_result { } switch ($fid) { - case 1: - if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\AlreadyExistsException(); - $xfer += $this->o1->read($input); + case 0: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->success); } else { $xfer += $input->skip($ftype); } break; - case 2: + case 1: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\InvalidObjectException(); - $xfer += $this->o2->read($input); + $this->o1 = new \metastore\MetaException(); + $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; - case 3: + case 2: if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); - $xfer += $this->o3->read($input); + $this->o2 = new \metastore\NoSuchObjectException(); + $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } @@ -5090,7 +7178,12 @@ class ThriftHiveMetastore_create_database_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_database_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_type_result'); + if ($this->success !== null) { + $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); + $xfer += $output->writeBool($this->success); + $xfer += $output->writeFieldEnd(); + } if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -5101,11 +7194,6 @@ class ThriftHiveMetastore_create_database_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -5113,7 +7201,7 @@ class ThriftHiveMetastore_create_database_result { } -class ThriftHiveMetastore_get_database_args { +class ThriftHiveMetastore_get_type_all_args { static $_TSPEC; public $name = null; @@ -5135,7 +7223,7 @@ class ThriftHiveMetastore_get_database_args { } public function getName() { - return 'ThriftHiveMetastore_get_database_args'; + return 'ThriftHiveMetastore_get_type_all_args'; } public function read($input) @@ -5172,7 +7260,7 @@ class ThriftHiveMetastore_get_database_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_database_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_type_all_args'); if ($this->name !== null) { $xfer += $output->writeFieldBegin('name', TType::STRING, 1); $xfer += $output->writeString($this->name); @@ -5185,11 +7273,10 @@ class ThriftHiveMetastore_get_database_args { } -class ThriftHiveMetastore_get_database_result { +class ThriftHiveMetastore_get_type_all_result { static $_TSPEC; public $success = null; - public $o1 = null; public $o2 = null; public function __construct($vals=null) { @@ -5197,15 +7284,18 @@ class ThriftHiveMetastore_get_database_result { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::STRUCT, - 'class' => '\metastore\Database', + 'type' => TType::MAP, + 'ktype' => TType::STRING, + 'vtype' => TType::STRUCT, + 'key' => array( + 'type' => TType::STRING, ), - 1 => array( - 'var' => 'o1', - 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'val' => array( + 'type' => TType::STRUCT, + 'class' => '\metastore\Type', + ), ), - 2 => array( + 1 => array( 'var' => 'o2', 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', @@ -5216,9 +7306,6 @@ class ThriftHiveMetastore_get_database_result { if (isset($vals['success'])) { $this->success = $vals['success']; } - if (isset($vals['o1'])) { - $this->o1 = $vals['o1']; - } if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } @@ -5226,7 +7313,7 @@ class ThriftHiveMetastore_get_database_result { } public function getName() { - return 'ThriftHiveMetastore_get_database_result'; + return 'ThriftHiveMetastore_get_type_all_result'; } public function read($input) @@ -5245,23 +7332,28 @@ class ThriftHiveMetastore_get_database_result { switch ($fid) { case 0: - if ($ftype == TType::STRUCT) { - $this->success = new \metastore\Database(); - $xfer += $this->success->read($input); + if ($ftype == TType::MAP) { + $this->success = array(); + $_size241 = 0; + $_ktype242 = 0; + $_vtype243 = 0; + $xfer += $input->readMapBegin($_ktype242, $_vtype243, $_size241); + for ($_i245 = 0; $_i245 < $_size241; ++$_i245) + { + $key246 = ''; + $val247 = new \metastore\Type(); + $xfer += $input->readString($key246); + $val247 = new \metastore\Type(); + $xfer += $val247->read($input); + $this->success[$key246] = $val247; + } + $xfer += $input->readMapEnd(); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); - $xfer += $this->o1->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRUCT) { $this->o2 = new \metastore\MetaException(); $xfer += $this->o2->read($input); } else { @@ -5280,22 +7372,27 @@ class ThriftHiveMetastore_get_database_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_database_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_type_all_result'); if ($this->success !== null) { - if (!is_object($this->success)) { + if (!is_array($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); - $xfer += $this->success->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->o1 !== null) { - $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); - $xfer += $this->o1->write($output); + $xfer += $output->writeFieldBegin('success', TType::MAP, 0); + { + $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success)); + { + foreach ($this->success as $kiter248 => $viter249) + { + $xfer += $output->writeString($kiter248); + $xfer += $viter249->write($output); + } + } + $output->writeMapEnd(); + } $xfer += $output->writeFieldEnd(); } if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 1); $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } @@ -5306,45 +7403,37 @@ class ThriftHiveMetastore_get_database_result { } -class ThriftHiveMetastore_drop_database_args { +class ThriftHiveMetastore_get_fields_args { static $_TSPEC; - public $name = null; - public $deleteData = null; - public $cascade = null; + public $db_name = null; + public $table_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'name', + 'var' => 'db_name', 'type' => TType::STRING, ), 2 => array( - 'var' => 'deleteData', - 'type' => TType::BOOL, - ), - 3 => array( - 'var' => 'cascade', - 'type' => TType::BOOL, + 'var' => 'table_name', + 'type' => TType::STRING, ), ); } if (is_array($vals)) { - if (isset($vals['name'])) { - $this->name = $vals['name']; - } - if (isset($vals['deleteData'])) { - $this->deleteData = $vals['deleteData']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; } - if (isset($vals['cascade'])) { - $this->cascade = $vals['cascade']; + if (isset($vals['table_name'])) { + $this->table_name = $vals['table_name']; } } } public function getName() { - return 'ThriftHiveMetastore_drop_database_args'; + return 'ThriftHiveMetastore_get_fields_args'; } public function read($input) @@ -5364,21 +7453,14 @@ class ThriftHiveMetastore_drop_database_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->name); + $xfer += $input->readString($this->db_name); } else { $xfer += $input->skip($ftype); } break; case 2: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->deleteData); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->cascade); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->table_name); } else { $xfer += $input->skip($ftype); } @@ -5395,20 +7477,15 @@ class ThriftHiveMetastore_drop_database_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_database_args'); - if ($this->name !== null) { - $xfer += $output->writeFieldBegin('name', TType::STRING, 1); - $xfer += $output->writeString($this->name); - $xfer += $output->writeFieldEnd(); - } - if ($this->deleteData !== null) { - $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 2); - $xfer += $output->writeBool($this->deleteData); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_fields_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); $xfer += $output->writeFieldEnd(); } - if ($this->cascade !== null) { - $xfer += $output->writeFieldBegin('cascade', TType::BOOL, 3); - $xfer += $output->writeBool($this->cascade); + if ($this->table_name !== null) { + $xfer += $output->writeFieldBegin('table_name', TType::STRING, 2); + $xfer += $output->writeString($this->table_name); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -5418,9 +7495,10 @@ class ThriftHiveMetastore_drop_database_args { } -class ThriftHiveMetastore_drop_database_result { +class ThriftHiveMetastore_get_fields_result { static $_TSPEC; + public $success = null; public $o1 = null; public $o2 = null; public $o3 = null; @@ -5428,24 +7506,36 @@ class ThriftHiveMetastore_drop_database_result { public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::LST, + 'etype' => TType::STRUCT, + 'elem' => array( + 'type' => TType::STRUCT, + 'class' => '\metastore\FieldSchema', + ), + ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\MetaException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidOperationException', + 'class' => '\metastore\UnknownTableException', ), 3 => array( 'var' => 'o3', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\UnknownDBException', ), ); } if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } @@ -5459,7 +7549,7 @@ class ThriftHiveMetastore_drop_database_result { } public function getName() { - return 'ThriftHiveMetastore_drop_database_result'; + return 'ThriftHiveMetastore_get_fields_result'; } public function read($input) @@ -5477,9 +7567,27 @@ class ThriftHiveMetastore_drop_database_result { } switch ($fid) { + case 0: + if ($ftype == TType::LST) { + $this->success = array(); + $_size250 = 0; + $_etype253 = 0; + $xfer += $input->readListBegin($_etype253, $_size250); + for ($_i254 = 0; $_i254 < $_size250; ++$_i254) + { + $elem255 = null; + $elem255 = new \metastore\FieldSchema(); + $xfer += $elem255->read($input); + $this->success []= $elem255; + } + $xfer += $input->readListEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -5487,7 +7595,7 @@ class ThriftHiveMetastore_drop_database_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\InvalidOperationException(); + $this->o2 = new \metastore\UnknownTableException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -5495,7 +7603,7 @@ class ThriftHiveMetastore_drop_database_result { break; case 3: if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); + $this->o3 = new \metastore\UnknownDBException(); $xfer += $this->o3->read($input); } else { $xfer += $input->skip($ftype); @@ -5513,7 +7621,24 @@ class ThriftHiveMetastore_drop_database_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_database_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_fields_result'); + if ($this->success !== null) { + if (!is_array($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRUCT, count($this->success)); + { + foreach ($this->success as $iter256) + { + $xfer += $iter256->write($output); + } + } + $output->writeListEnd(); + } + $xfer += $output->writeFieldEnd(); + } if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -5536,29 +7661,37 @@ class ThriftHiveMetastore_drop_database_result { } -class ThriftHiveMetastore_get_databases_args { +class ThriftHiveMetastore_get_schema_args { static $_TSPEC; - public $pattern = null; + public $db_name = null; + public $table_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'pattern', + 'var' => 'db_name', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'table_name', 'type' => TType::STRING, ), ); } if (is_array($vals)) { - if (isset($vals['pattern'])) { - $this->pattern = $vals['pattern']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; + } + if (isset($vals['table_name'])) { + $this->table_name = $vals['table_name']; } } } public function getName() { - return 'ThriftHiveMetastore_get_databases_args'; + return 'ThriftHiveMetastore_get_schema_args'; } public function read($input) @@ -5578,7 +7711,14 @@ class ThriftHiveMetastore_get_databases_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->pattern); + $xfer += $input->readString($this->db_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->table_name); } else { $xfer += $input->skip($ftype); } @@ -5595,10 +7735,15 @@ class ThriftHiveMetastore_get_databases_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_databases_args'); - if ($this->pattern !== null) { - $xfer += $output->writeFieldBegin('pattern', TType::STRING, 1); - $xfer += $output->writeString($this->pattern); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_schema_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->table_name !== null) { + $xfer += $output->writeFieldBegin('table_name', TType::STRING, 2); + $xfer += $output->writeString($this->table_name); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -5608,11 +7753,13 @@ class ThriftHiveMetastore_get_databases_args { } -class ThriftHiveMetastore_get_databases_result { +class ThriftHiveMetastore_get_schema_result { static $_TSPEC; public $success = null; public $o1 = null; + public $o2 = null; + public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -5620,9 +7767,10 @@ class ThriftHiveMetastore_get_databases_result { 0 => array( 'var' => 'success', 'type' => TType::LST, - 'etype' => TType::STRING, + 'etype' => TType::STRUCT, 'elem' => array( - 'type' => TType::STRING, + 'type' => TType::STRUCT, + 'class' => '\metastore\FieldSchema', ), ), 1 => array( @@ -5630,6 +7778,16 @@ class ThriftHiveMetastore_get_databases_result { 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\UnknownTableException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, + 'class' => '\metastore\UnknownDBException', + ), ); } if (is_array($vals)) { @@ -5639,11 +7797,17 @@ class ThriftHiveMetastore_get_databases_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } } } public function getName() { - return 'ThriftHiveMetastore_get_databases_result'; + return 'ThriftHiveMetastore_get_schema_result'; } public function read($input) @@ -5664,14 +7828,15 @@ class ThriftHiveMetastore_get_databases_result { case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size227 = 0; - $_etype230 = 0; - $xfer += $input->readListBegin($_etype230, $_size227); - for ($_i231 = 0; $_i231 < $_size227; ++$_i231) + $_size257 = 0; + $_etype260 = 0; + $xfer += $input->readListBegin($_etype260, $_size257); + for ($_i261 = 0; $_i261 < $_size257; ++$_i261) { - $elem232 = null; - $xfer += $input->readString($elem232); - $this->success []= $elem232; + $elem262 = null; + $elem262 = new \metastore\FieldSchema(); + $xfer += $elem262->read($input); + $this->success []= $elem262; } $xfer += $input->readListEnd(); } else { @@ -5686,6 +7851,22 @@ class ThriftHiveMetastore_get_databases_result { $xfer += $input->skip($ftype); } break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\UnknownTableException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\UnknownDBException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -5698,18 +7879,18 @@ class ThriftHiveMetastore_get_databases_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_databases_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_schema_result'); if ($this->success !== null) { if (!is_array($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } $xfer += $output->writeFieldBegin('success', TType::LST, 0); { - $output->writeListBegin(TType::STRING, count($this->success)); + $output->writeListBegin(TType::STRUCT, count($this->success)); { - foreach ($this->success as $iter233) + foreach ($this->success as $iter263) { - $xfer += $output->writeString($iter233); + $xfer += $iter263->write($output); } } $output->writeListEnd(); @@ -5721,6 +7902,16 @@ class ThriftHiveMetastore_get_databases_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -5728,19 +7919,30 @@ class ThriftHiveMetastore_get_databases_result { } -class ThriftHiveMetastore_get_all_databases_args { +class ThriftHiveMetastore_create_table_args { static $_TSPEC; + public $tbl = null; - public function __construct() { + public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( + 1 => array( + 'var' => 'tbl', + 'type' => TType::STRUCT, + 'class' => '\metastore\Table', + ), ); } + if (is_array($vals)) { + if (isset($vals['tbl'])) { + $this->tbl = $vals['tbl']; + } + } } public function getName() { - return 'ThriftHiveMetastore_get_all_databases_args'; + return 'ThriftHiveMetastore_create_table_args'; } public function read($input) @@ -5758,6 +7960,14 @@ class ThriftHiveMetastore_get_all_databases_args { } switch ($fid) { + case 1: + if ($ftype == TType::STRUCT) { + $this->tbl = new \metastore\Table(); + $xfer += $this->tbl->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -5770,7 +7980,15 @@ class ThriftHiveMetastore_get_all_databases_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_all_databases_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_table_args'); + if ($this->tbl !== null) { + if (!is_object($this->tbl)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('tbl', TType::STRUCT, 1); + $xfer += $this->tbl->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -5778,42 +7996,57 @@ class ThriftHiveMetastore_get_all_databases_args { } -class ThriftHiveMetastore_get_all_databases_result { +class ThriftHiveMetastore_create_table_result { static $_TSPEC; - public $success = null; public $o1 = null; + public $o2 = null; + public $o3 = null; + public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), - ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, + 'class' => '\metastore\AlreadyExistsException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidObjectException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), + 4 => array( + 'var' => 'o4', + 'type' => TType::STRUCT, + 'class' => '\metastore\NoSuchObjectException', + ), ); } if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; - } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } + if (isset($vals['o4'])) { + $this->o4 = $vals['o4']; + } } } public function getName() { - return 'ThriftHiveMetastore_get_all_databases_result'; + return 'ThriftHiveMetastore_create_table_result'; } public function read($input) @@ -5831,27 +8064,34 @@ class ThriftHiveMetastore_get_all_databases_result { } switch ($fid) { - case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size234 = 0; - $_etype237 = 0; - $xfer += $input->readListBegin($_etype237, $_size234); - for ($_i238 = 0; $_i238 < $_size234; ++$_i238) - { - $elem239 = null; - $xfer += $input->readString($elem239); - $this->success []= $elem239; - } - $xfer += $input->readListEnd(); + case 1: + if ($ftype == TType::STRUCT) { + $this->o1 = new \metastore\AlreadyExistsException(); + $xfer += $this->o1->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\InvalidObjectException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\MetaException(); + $xfer += $this->o3->read($input); } else { $xfer += $input->skip($ftype); } break; - case 1: + case 4: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); - $xfer += $this->o1->read($input); + $this->o4 = new \metastore\NoSuchObjectException(); + $xfer += $this->o4->read($input); } else { $xfer += $input->skip($ftype); } @@ -5868,29 +8108,27 @@ class ThriftHiveMetastore_get_all_databases_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_all_databases_result'); - if ($this->success !== null) { - if (!is_array($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRING, count($this->success)); - { - foreach ($this->success as $iter240) - { - $xfer += $output->writeString($iter240); - } - } - $output->writeListEnd(); - } - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_table_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o4 !== null) { + $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); + $xfer += $this->o4->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -5898,38 +8136,39 @@ class ThriftHiveMetastore_get_all_databases_result { } -class ThriftHiveMetastore_alter_database_args { +class ThriftHiveMetastore_create_table_with_environment_context_args { static $_TSPEC; - public $dbname = null; - public $db = null; + public $tbl = null; + public $environment_context = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'dbname', - 'type' => TType::STRING, + 'var' => 'tbl', + 'type' => TType::STRUCT, + 'class' => '\metastore\Table', ), 2 => array( - 'var' => 'db', + 'var' => 'environment_context', 'type' => TType::STRUCT, - 'class' => '\metastore\Database', + 'class' => '\metastore\EnvironmentContext', ), ); } if (is_array($vals)) { - if (isset($vals['dbname'])) { - $this->dbname = $vals['dbname']; + if (isset($vals['tbl'])) { + $this->tbl = $vals['tbl']; } - if (isset($vals['db'])) { - $this->db = $vals['db']; + if (isset($vals['environment_context'])) { + $this->environment_context = $vals['environment_context']; } } } public function getName() { - return 'ThriftHiveMetastore_alter_database_args'; + return 'ThriftHiveMetastore_create_table_with_environment_context_args'; } public function read($input) @@ -5948,16 +8187,17 @@ class ThriftHiveMetastore_alter_database_args { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->dbname); + if ($ftype == TType::STRUCT) { + $this->tbl = new \metastore\Table(); + $xfer += $this->tbl->read($input); } else { $xfer += $input->skip($ftype); } break; case 2: if ($ftype == TType::STRUCT) { - $this->db = new \metastore\Database(); - $xfer += $this->db->read($input); + $this->environment_context = new \metastore\EnvironmentContext(); + $xfer += $this->environment_context->read($input); } else { $xfer += $input->skip($ftype); } @@ -5974,18 +8214,21 @@ class ThriftHiveMetastore_alter_database_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_database_args'); - if ($this->dbname !== null) { - $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); - $xfer += $output->writeString($this->dbname); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_table_with_environment_context_args'); + if ($this->tbl !== null) { + if (!is_object($this->tbl)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('tbl', TType::STRUCT, 1); + $xfer += $this->tbl->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->db !== null) { - if (!is_object($this->db)) { + if ($this->environment_context !== null) { + if (!is_object($this->environment_context)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('db', TType::STRUCT, 2); - $xfer += $this->db->write($output); + $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 2); + $xfer += $this->environment_context->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -5995,11 +8238,13 @@ class ThriftHiveMetastore_alter_database_args { } -class ThriftHiveMetastore_alter_database_result { +class ThriftHiveMetastore_create_table_with_environment_context_result { static $_TSPEC; public $o1 = null; public $o2 = null; + public $o3 = null; + public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -6007,11 +8252,21 @@ class ThriftHiveMetastore_alter_database_result { 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\AlreadyExistsException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidObjectException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, + 'class' => '\metastore\MetaException', + ), + 4 => array( + 'var' => 'o4', + 'type' => TType::STRUCT, 'class' => '\metastore\NoSuchObjectException', ), ); @@ -6023,11 +8278,17 @@ class ThriftHiveMetastore_alter_database_result { if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } + if (isset($vals['o4'])) { + $this->o4 = $vals['o4']; + } } } public function getName() { - return 'ThriftHiveMetastore_alter_database_result'; + return 'ThriftHiveMetastore_create_table_with_environment_context_result'; } public function read($input) @@ -6047,7 +8308,7 @@ class ThriftHiveMetastore_alter_database_result { { case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\AlreadyExistsException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -6055,12 +8316,28 @@ class ThriftHiveMetastore_alter_database_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\NoSuchObjectException(); + $this->o2 = new \metastore\InvalidObjectException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\MetaException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRUCT) { + $this->o4 = new \metastore\NoSuchObjectException(); + $xfer += $this->o4->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -6073,7 +8350,7 @@ class ThriftHiveMetastore_alter_database_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_database_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_table_with_environment_context_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -6084,6 +8361,16 @@ class ThriftHiveMetastore_alter_database_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o4 !== null) { + $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); + $xfer += $this->o4->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -6091,29 +8378,45 @@ class ThriftHiveMetastore_alter_database_result { } -class ThriftHiveMetastore_get_type_args { +class ThriftHiveMetastore_drop_table_args { static $_TSPEC; + public $dbname = null; public $name = null; + public $deleteData = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( + 'var' => 'dbname', + 'type' => TType::STRING, + ), + 2 => array( 'var' => 'name', 'type' => TType::STRING, ), + 3 => array( + 'var' => 'deleteData', + 'type' => TType::BOOL, + ), ); } if (is_array($vals)) { + if (isset($vals['dbname'])) { + $this->dbname = $vals['dbname']; + } if (isset($vals['name'])) { $this->name = $vals['name']; } + if (isset($vals['deleteData'])) { + $this->deleteData = $vals['deleteData']; + } } } public function getName() { - return 'ThriftHiveMetastore_get_type_args'; + return 'ThriftHiveMetastore_drop_table_args'; } public function read($input) @@ -6133,11 +8436,25 @@ class ThriftHiveMetastore_get_type_args { { case 1: if ($ftype == TType::STRING) { + $xfer += $input->readString($this->dbname); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRING) { $xfer += $input->readString($this->name); } else { $xfer += $input->skip($ftype); } break; + case 3: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->deleteData); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -6150,12 +8467,22 @@ class ThriftHiveMetastore_get_type_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_type_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_table_args'); + if ($this->dbname !== null) { + $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); + $xfer += $output->writeString($this->dbname); + $xfer += $output->writeFieldEnd(); + } if ($this->name !== null) { - $xfer += $output->writeFieldBegin('name', TType::STRING, 1); + $xfer += $output->writeFieldBegin('name', TType::STRING, 2); $xfer += $output->writeString($this->name); $xfer += $output->writeFieldEnd(); } + if ($this->deleteData !== null) { + $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 3); + $xfer += $output->writeBool($this->deleteData); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -6163,48 +8490,39 @@ class ThriftHiveMetastore_get_type_args { } -class ThriftHiveMetastore_get_type_result { +class ThriftHiveMetastore_drop_table_result { static $_TSPEC; - public $success = null; public $o1 = null; - public $o2 = null; + public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::STRUCT, - 'class' => '\metastore\Type', - ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchObjectException', ), 2 => array( - 'var' => 'o2', + 'var' => 'o3', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\MetaException', ), ); } if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; - } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } - if (isset($vals['o2'])) { - $this->o2 = $vals['o2']; + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; } } } public function getName() { - return 'ThriftHiveMetastore_get_type_result'; + return 'ThriftHiveMetastore_drop_table_result'; } public function read($input) @@ -6222,17 +8540,9 @@ class ThriftHiveMetastore_get_type_result { } switch ($fid) { - case 0: - if ($ftype == TType::STRUCT) { - $this->success = new \metastore\Type(); - $xfer += $this->success->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -6240,8 +8550,8 @@ class ThriftHiveMetastore_get_type_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\NoSuchObjectException(); - $xfer += $this->o2->read($input); + $this->o3 = new \metastore\MetaException(); + $xfer += $this->o3->read($input); } else { $xfer += $input->skip($ftype); } @@ -6258,23 +8568,15 @@ class ThriftHiveMetastore_get_type_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_type_result'); - if ($this->success !== null) { - if (!is_object($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); - $xfer += $this->success->write($output); - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_table_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); - $xfer += $this->o2->write($output); + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 2); + $xfer += $this->o3->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -6284,30 +8586,54 @@ class ThriftHiveMetastore_get_type_result { } -class ThriftHiveMetastore_create_type_args { +class ThriftHiveMetastore_drop_table_with_environment_context_args { static $_TSPEC; - public $type = null; + public $dbname = null; + public $name = null; + public $deleteData = null; + public $environment_context = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'type', + 'var' => 'dbname', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'name', + 'type' => TType::STRING, + ), + 3 => array( + 'var' => 'deleteData', + 'type' => TType::BOOL, + ), + 4 => array( + 'var' => 'environment_context', 'type' => TType::STRUCT, - 'class' => '\metastore\Type', + 'class' => '\metastore\EnvironmentContext', ), ); } if (is_array($vals)) { - if (isset($vals['type'])) { - $this->type = $vals['type']; + if (isset($vals['dbname'])) { + $this->dbname = $vals['dbname']; + } + if (isset($vals['name'])) { + $this->name = $vals['name']; + } + if (isset($vals['deleteData'])) { + $this->deleteData = $vals['deleteData']; + } + if (isset($vals['environment_context'])) { + $this->environment_context = $vals['environment_context']; } } } public function getName() { - return 'ThriftHiveMetastore_create_type_args'; + return 'ThriftHiveMetastore_drop_table_with_environment_context_args'; } public function read($input) @@ -6326,9 +8652,30 @@ class ThriftHiveMetastore_create_type_args { switch ($fid) { case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->dbname); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->deleteData); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: if ($ftype == TType::STRUCT) { - $this->type = new \metastore\Type(); - $xfer += $this->type->read($input); + $this->environment_context = new \metastore\EnvironmentContext(); + $xfer += $this->environment_context->read($input); } else { $xfer += $input->skip($ftype); } @@ -6345,13 +8692,28 @@ class ThriftHiveMetastore_create_type_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_type_args'); - if ($this->type !== null) { - if (!is_object($this->type)) { + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_table_with_environment_context_args'); + if ($this->dbname !== null) { + $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); + $xfer += $output->writeString($this->dbname); + $xfer += $output->writeFieldEnd(); + } + if ($this->name !== null) { + $xfer += $output->writeFieldBegin('name', TType::STRING, 2); + $xfer += $output->writeString($this->name); + $xfer += $output->writeFieldEnd(); + } + if ($this->deleteData !== null) { + $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 3); + $xfer += $output->writeBool($this->deleteData); + $xfer += $output->writeFieldEnd(); + } + if ($this->environment_context !== null) { + if (!is_object($this->environment_context)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('type', TType::STRUCT, 1); - $xfer += $this->type->write($output); + $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 4); + $xfer += $this->environment_context->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -6361,32 +8723,21 @@ class ThriftHiveMetastore_create_type_args { } -class ThriftHiveMetastore_create_type_result { +class ThriftHiveMetastore_drop_table_with_environment_context_result { static $_TSPEC; - public $success = null; public $o1 = null; - public $o2 = null; public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::BOOL, - ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\AlreadyExistsException', + 'class' => '\metastore\NoSuchObjectException', ), 2 => array( - 'var' => 'o2', - 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', - ), - 3 => array( 'var' => 'o3', 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', @@ -6394,15 +8745,9 @@ class ThriftHiveMetastore_create_type_result { ); } if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; - } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } - if (isset($vals['o2'])) { - $this->o2 = $vals['o2']; - } if (isset($vals['o3'])) { $this->o3 = $vals['o3']; } @@ -6410,7 +8755,7 @@ class ThriftHiveMetastore_create_type_result { } public function getName() { - return 'ThriftHiveMetastore_create_type_result'; + return 'ThriftHiveMetastore_drop_table_with_environment_context_result'; } public function read($input) @@ -6428,16 +8773,9 @@ class ThriftHiveMetastore_create_type_result { } switch ($fid) { - case 0: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->success); - } else { - $xfer += $input->skip($ftype); - } - break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\AlreadyExistsException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -6445,14 +8783,6 @@ class ThriftHiveMetastore_create_type_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\InvalidObjectException(); - $xfer += $this->o2->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::STRUCT) { $this->o3 = new \metastore\MetaException(); $xfer += $this->o3->read($input); } else { @@ -6471,24 +8801,14 @@ class ThriftHiveMetastore_create_type_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_type_result'); - if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); - $xfer += $output->writeBool($this->success); - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_table_with_environment_context_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); - $xfer += $this->o2->write($output); - $xfer += $output->writeFieldEnd(); - } if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 2); $xfer += $this->o3->write($output); $xfer += $output->writeFieldEnd(); } @@ -6499,29 +8819,37 @@ class ThriftHiveMetastore_create_type_result { } -class ThriftHiveMetastore_drop_type_args { +class ThriftHiveMetastore_get_tables_args { static $_TSPEC; - public $type = null; + public $db_name = null; + public $pattern = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'type', + 'var' => 'db_name', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'pattern', 'type' => TType::STRING, ), ); } if (is_array($vals)) { - if (isset($vals['type'])) { - $this->type = $vals['type']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; + } + if (isset($vals['pattern'])) { + $this->pattern = $vals['pattern']; } } } public function getName() { - return 'ThriftHiveMetastore_drop_type_args'; + return 'ThriftHiveMetastore_get_tables_args'; } public function read($input) @@ -6541,7 +8869,14 @@ class ThriftHiveMetastore_drop_type_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->type); + $xfer += $input->readString($this->db_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->pattern); } else { $xfer += $input->skip($ftype); } @@ -6558,10 +8893,15 @@ class ThriftHiveMetastore_drop_type_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_type_args'); - if ($this->type !== null) { - $xfer += $output->writeFieldBegin('type', TType::STRING, 1); - $xfer += $output->writeString($this->type); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_tables_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->pattern !== null) { + $xfer += $output->writeFieldBegin('pattern', TType::STRING, 2); + $xfer += $output->writeString($this->pattern); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -6571,30 +8911,28 @@ class ThriftHiveMetastore_drop_type_args { } -class ThriftHiveMetastore_drop_type_result { +class ThriftHiveMetastore_get_tables_result { static $_TSPEC; public $success = null; public $o1 = null; - public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::BOOL, + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), - 2 => array( - 'var' => 'o2', - 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', - ), ); } if (is_array($vals)) { @@ -6604,14 +8942,11 @@ class ThriftHiveMetastore_drop_type_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } - if (isset($vals['o2'])) { - $this->o2 = $vals['o2']; - } } } public function getName() { - return 'ThriftHiveMetastore_drop_type_result'; + return 'ThriftHiveMetastore_get_tables_result'; } public function read($input) @@ -6630,8 +8965,18 @@ class ThriftHiveMetastore_drop_type_result { switch ($fid) { case 0: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->success); + if ($ftype == TType::LST) { + $this->success = array(); + $_size264 = 0; + $_etype267 = 0; + $xfer += $input->readListBegin($_etype267, $_size264); + for ($_i268 = 0; $_i268 < $_size264; ++$_i268) + { + $elem269 = null; + $xfer += $input->readString($elem269); + $this->success []= $elem269; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } @@ -6644,14 +8989,6 @@ class ThriftHiveMetastore_drop_type_result { $xfer += $input->skip($ftype); } break; - case 2: - if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\NoSuchObjectException(); - $xfer += $this->o2->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -6664,10 +9001,22 @@ class ThriftHiveMetastore_drop_type_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_type_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_tables_result'); if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); - $xfer += $output->writeBool($this->success); + if (!is_array($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRING, count($this->success)); + { + foreach ($this->success as $iter270) + { + $xfer += $output->writeString($iter270); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -6675,11 +9024,6 @@ class ThriftHiveMetastore_drop_type_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); - $xfer += $this->o2->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -6687,29 +9031,29 @@ class ThriftHiveMetastore_drop_type_result { } -class ThriftHiveMetastore_get_type_all_args { +class ThriftHiveMetastore_get_all_tables_args { static $_TSPEC; - public $name = null; + public $db_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'name', + 'var' => 'db_name', 'type' => TType::STRING, ), ); } if (is_array($vals)) { - if (isset($vals['name'])) { - $this->name = $vals['name']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; } } } public function getName() { - return 'ThriftHiveMetastore_get_type_all_args'; + return 'ThriftHiveMetastore_get_all_tables_args'; } public function read($input) @@ -6729,7 +9073,7 @@ class ThriftHiveMetastore_get_type_all_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->name); + $xfer += $input->readString($this->db_name); } else { $xfer += $input->skip($ftype); } @@ -6746,10 +9090,10 @@ class ThriftHiveMetastore_get_type_all_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_type_all_args'); - if ($this->name !== null) { - $xfer += $output->writeFieldBegin('name', TType::STRING, 1); - $xfer += $output->writeString($this->name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_all_tables_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -6759,30 +9103,25 @@ class ThriftHiveMetastore_get_type_all_args { } -class ThriftHiveMetastore_get_type_all_result { +class ThriftHiveMetastore_get_all_tables_result { static $_TSPEC; public $success = null; - public $o2 = null; + public $o1 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::MAP, - 'ktype' => TType::STRING, - 'vtype' => TType::STRUCT, - 'key' => array( + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( 'type' => TType::STRING, - ), - 'val' => array( - 'type' => TType::STRUCT, - 'class' => '\metastore\Type', ), ), 1 => array( - 'var' => 'o2', + 'var' => 'o1', 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), @@ -6792,14 +9131,14 @@ class ThriftHiveMetastore_get_type_all_result { if (isset($vals['success'])) { $this->success = $vals['success']; } - if (isset($vals['o2'])) { - $this->o2 = $vals['o2']; + if (isset($vals['o1'])) { + $this->o1 = $vals['o1']; } } } public function getName() { - return 'ThriftHiveMetastore_get_type_all_result'; + return 'ThriftHiveMetastore_get_all_tables_result'; } public function read($input) @@ -6818,30 +9157,26 @@ class ThriftHiveMetastore_get_type_all_result { switch ($fid) { case 0: - if ($ftype == TType::MAP) { + if ($ftype == TType::LST) { $this->success = array(); - $_size241 = 0; - $_ktype242 = 0; - $_vtype243 = 0; - $xfer += $input->readMapBegin($_ktype242, $_vtype243, $_size241); - for ($_i245 = 0; $_i245 < $_size241; ++$_i245) + $_size271 = 0; + $_etype274 = 0; + $xfer += $input->readListBegin($_etype274, $_size271); + for ($_i275 = 0; $_i275 < $_size271; ++$_i275) { - $key246 = ''; - $val247 = new \metastore\Type(); - $xfer += $input->readString($key246); - $val247 = new \metastore\Type(); - $xfer += $val247->read($input); - $this->success[$key246] = $val247; + $elem276 = null; + $xfer += $input->readString($elem276); + $this->success []= $elem276; } - $xfer += $input->readMapEnd(); + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); - $xfer += $this->o2->read($input); + $this->o1 = new \metastore\MetaException(); + $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } @@ -6858,28 +9193,27 @@ class ThriftHiveMetastore_get_type_all_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_type_all_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_all_tables_result'); if ($this->success !== null) { if (!is_array($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('success', TType::MAP, 0); + $xfer += $output->writeFieldBegin('success', TType::LST, 0); { - $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success)); + $output->writeListBegin(TType::STRING, count($this->success)); { - foreach ($this->success as $kiter248 => $viter249) + foreach ($this->success as $iter277) { - $xfer += $output->writeString($kiter248); - $xfer += $viter249->write($output); + $xfer += $output->writeString($iter277); } } - $output->writeMapEnd(); + $output->writeListEnd(); } $xfer += $output->writeFieldEnd(); } - if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 1); - $xfer += $this->o2->write($output); + if ($this->o1 !== null) { + $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); + $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -6889,37 +9223,37 @@ class ThriftHiveMetastore_get_type_all_result { } -class ThriftHiveMetastore_get_fields_args { +class ThriftHiveMetastore_get_table_args { static $_TSPEC; - public $db_name = null; - public $table_name = null; + public $dbname = null; + public $tbl_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', + 'var' => 'dbname', 'type' => TType::STRING, ), 2 => array( - 'var' => 'table_name', + 'var' => 'tbl_name', 'type' => TType::STRING, ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; + if (isset($vals['dbname'])) { + $this->dbname = $vals['dbname']; } - if (isset($vals['table_name'])) { - $this->table_name = $vals['table_name']; + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; } } } public function getName() { - return 'ThriftHiveMetastore_get_fields_args'; + return 'ThriftHiveMetastore_get_table_args'; } public function read($input) @@ -6939,14 +9273,14 @@ class ThriftHiveMetastore_get_fields_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); + $xfer += $input->readString($this->dbname); } else { $xfer += $input->skip($ftype); } break; case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->table_name); + $xfer += $input->readString($this->tbl_name); } else { $xfer += $input->skip($ftype); } @@ -6963,15 +9297,15 @@ class ThriftHiveMetastore_get_fields_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_fields_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_args'); + if ($this->dbname !== null) { + $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); + $xfer += $output->writeString($this->dbname); $xfer += $output->writeFieldEnd(); } - if ($this->table_name !== null) { - $xfer += $output->writeFieldBegin('table_name', TType::STRING, 2); - $xfer += $output->writeString($this->table_name); + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -6981,25 +9315,20 @@ class ThriftHiveMetastore_get_fields_args { } -class ThriftHiveMetastore_get_fields_result { +class ThriftHiveMetastore_get_table_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; - public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRUCT, - 'elem' => array( - 'type' => TType::STRUCT, - 'class' => '\metastore\FieldSchema', - ), + 'type' => TType::STRUCT, + 'class' => '\metastore\Table', ), 1 => array( 'var' => 'o1', @@ -7009,12 +9338,7 @@ class ThriftHiveMetastore_get_fields_result { 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\UnknownTableException', - ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, - 'class' => '\metastore\UnknownDBException', + 'class' => '\metastore\NoSuchObjectException', ), ); } @@ -7028,14 +9352,11 @@ class ThriftHiveMetastore_get_fields_result { if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } } } public function getName() { - return 'ThriftHiveMetastore_get_fields_result'; + return 'ThriftHiveMetastore_get_table_result'; } public function read($input) @@ -7054,19 +9375,9 @@ class ThriftHiveMetastore_get_fields_result { switch ($fid) { case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size250 = 0; - $_etype253 = 0; - $xfer += $input->readListBegin($_etype253, $_size250); - for ($_i254 = 0; $_i254 < $_size250; ++$_i254) - { - $elem255 = null; - $elem255 = new \metastore\FieldSchema(); - $xfer += $elem255->read($input); - $this->success []= $elem255; - } - $xfer += $input->readListEnd(); + if ($ftype == TType::STRUCT) { + $this->success = new \metastore\Table(); + $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); } @@ -7081,20 +9392,12 @@ class ThriftHiveMetastore_get_fields_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\UnknownTableException(); + $this->o2 = new \metastore\NoSuchObjectException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\UnknownDBException(); - $xfer += $this->o3->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -7107,22 +9410,13 @@ class ThriftHiveMetastore_get_fields_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_fields_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_result'); if ($this->success !== null) { - if (!is_array($this->success)) { + if (!is_object($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRUCT, count($this->success)); - { - foreach ($this->success as $iter256) - { - $xfer += $iter256->write($output); - } - } - $output->writeListEnd(); - } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -7135,11 +9429,6 @@ class ThriftHiveMetastore_get_fields_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -7147,37 +9436,41 @@ class ThriftHiveMetastore_get_fields_result { } -class ThriftHiveMetastore_get_schema_args { +class ThriftHiveMetastore_get_table_objects_by_name_args { static $_TSPEC; - public $db_name = null; - public $table_name = null; + public $dbname = null; + public $tbl_names = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', + 'var' => 'dbname', 'type' => TType::STRING, ), 2 => array( - 'var' => 'table_name', - 'type' => TType::STRING, + 'var' => 'tbl_names', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; + if (isset($vals['dbname'])) { + $this->dbname = $vals['dbname']; } - if (isset($vals['table_name'])) { - $this->table_name = $vals['table_name']; + if (isset($vals['tbl_names'])) { + $this->tbl_names = $vals['tbl_names']; } } } public function getName() { - return 'ThriftHiveMetastore_get_schema_args'; + return 'ThriftHiveMetastore_get_table_objects_by_name_args'; } public function read($input) @@ -7197,14 +9490,24 @@ class ThriftHiveMetastore_get_schema_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); + $xfer += $input->readString($this->dbname); } else { $xfer += $input->skip($ftype); } break; case 2: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->table_name); + if ($ftype == TType::LST) { + $this->tbl_names = array(); + $_size278 = 0; + $_etype281 = 0; + $xfer += $input->readListBegin($_etype281, $_size278); + for ($_i282 = 0; $_i282 < $_size278; ++$_i282) + { + $elem283 = null; + $xfer += $input->readString($elem283); + $this->tbl_names []= $elem283; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } @@ -7221,15 +9524,27 @@ class ThriftHiveMetastore_get_schema_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_schema_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_objects_by_name_args'); + if ($this->dbname !== null) { + $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); + $xfer += $output->writeString($this->dbname); $xfer += $output->writeFieldEnd(); } - if ($this->table_name !== null) { - $xfer += $output->writeFieldBegin('table_name', TType::STRING, 2); - $xfer += $output->writeString($this->table_name); + if ($this->tbl_names !== null) { + if (!is_array($this->tbl_names)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('tbl_names', TType::LST, 2); + { + $output->writeListBegin(TType::STRING, count($this->tbl_names)); + { + foreach ($this->tbl_names as $iter284) + { + $xfer += $output->writeString($iter284); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -7239,7 +9554,7 @@ class ThriftHiveMetastore_get_schema_args { } -class ThriftHiveMetastore_get_schema_result { +class ThriftHiveMetastore_get_table_objects_by_name_result { static $_TSPEC; public $success = null; @@ -7256,7 +9571,7 @@ class ThriftHiveMetastore_get_schema_result { 'etype' => TType::STRUCT, 'elem' => array( 'type' => TType::STRUCT, - 'class' => '\metastore\FieldSchema', + 'class' => '\metastore\Table', ), ), 1 => array( @@ -7267,7 +9582,7 @@ class ThriftHiveMetastore_get_schema_result { 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\UnknownTableException', + 'class' => '\metastore\InvalidOperationException', ), 3 => array( 'var' => 'o3', @@ -7293,7 +9608,7 @@ class ThriftHiveMetastore_get_schema_result { } public function getName() { - return 'ThriftHiveMetastore_get_schema_result'; + return 'ThriftHiveMetastore_get_table_objects_by_name_result'; } public function read($input) @@ -7314,15 +9629,15 @@ class ThriftHiveMetastore_get_schema_result { case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size257 = 0; - $_etype260 = 0; - $xfer += $input->readListBegin($_etype260, $_size257); - for ($_i261 = 0; $_i261 < $_size257; ++$_i261) + $_size285 = 0; + $_etype288 = 0; + $xfer += $input->readListBegin($_etype288, $_size285); + for ($_i289 = 0; $_i289 < $_size285; ++$_i289) { - $elem262 = null; - $elem262 = new \metastore\FieldSchema(); - $xfer += $elem262->read($input); - $this->success []= $elem262; + $elem290 = null; + $elem290 = new \metastore\Table(); + $xfer += $elem290->read($input); + $this->success []= $elem290; } $xfer += $input->readListEnd(); } else { @@ -7339,7 +9654,7 @@ class ThriftHiveMetastore_get_schema_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\UnknownTableException(); + $this->o2 = new \metastore\InvalidOperationException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -7365,7 +9680,7 @@ class ThriftHiveMetastore_get_schema_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_schema_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_objects_by_name_result'); if ($this->success !== null) { if (!is_array($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); @@ -7374,9 +9689,9 @@ class ThriftHiveMetastore_get_schema_result { { $output->writeListBegin(TType::STRUCT, count($this->success)); { - foreach ($this->success as $iter263) + foreach ($this->success as $iter291) { - $xfer += $iter263->write($output); + $xfer += $iter291->write($output); } } $output->writeListEnd(); @@ -7405,30 +9720,45 @@ class ThriftHiveMetastore_get_schema_result { } -class ThriftHiveMetastore_create_table_args { +class ThriftHiveMetastore_get_table_names_by_filter_args { static $_TSPEC; - public $tbl = null; + public $dbname = null; + public $filter = null; + public $max_tables = -1; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'tbl', - 'type' => TType::STRUCT, - 'class' => '\metastore\Table', + 'var' => 'dbname', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'filter', + 'type' => TType::STRING, + ), + 3 => array( + 'var' => 'max_tables', + 'type' => TType::I16, ), ); } if (is_array($vals)) { - if (isset($vals['tbl'])) { - $this->tbl = $vals['tbl']; + if (isset($vals['dbname'])) { + $this->dbname = $vals['dbname']; + } + if (isset($vals['filter'])) { + $this->filter = $vals['filter']; + } + if (isset($vals['max_tables'])) { + $this->max_tables = $vals['max_tables']; } } } public function getName() { - return 'ThriftHiveMetastore_create_table_args'; + return 'ThriftHiveMetastore_get_table_names_by_filter_args'; } public function read($input) @@ -7447,9 +9777,22 @@ class ThriftHiveMetastore_create_table_args { switch ($fid) { case 1: - if ($ftype == TType::STRUCT) { - $this->tbl = new \metastore\Table(); - $xfer += $this->tbl->read($input); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->dbname); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->filter); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::I16) { + $xfer += $input->readI16($this->max_tables); } else { $xfer += $input->skip($ftype); } @@ -7466,13 +9809,20 @@ class ThriftHiveMetastore_create_table_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_table_args'); - if ($this->tbl !== null) { - if (!is_object($this->tbl)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('tbl', TType::STRUCT, 1); - $xfer += $this->tbl->write($output); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_names_by_filter_args'); + if ($this->dbname !== null) { + $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); + $xfer += $output->writeString($this->dbname); + $xfer += $output->writeFieldEnd(); + } + if ($this->filter !== null) { + $xfer += $output->writeFieldBegin('filter', TType::STRING, 2); + $xfer += $output->writeString($this->filter); + $xfer += $output->writeFieldEnd(); + } + if ($this->max_tables !== null) { + $xfer += $output->writeFieldBegin('max_tables', TType::I16, 3); + $xfer += $output->writeI16($this->max_tables); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -7482,40 +9832,46 @@ class ThriftHiveMetastore_create_table_args { } -class ThriftHiveMetastore_create_table_result { +class ThriftHiveMetastore_get_table_names_by_filter_result { static $_TSPEC; + public $success = null; public $o1 = null; public $o2 = null; public $o3 = null; - public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), + ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\AlreadyExistsException', + 'class' => '\metastore\MetaException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', + 'class' => '\metastore\InvalidOperationException', ), 3 => array( 'var' => 'o3', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', - ), - 4 => array( - 'var' => 'o4', - 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\UnknownDBException', ), ); } if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } @@ -7525,14 +9881,11 @@ class ThriftHiveMetastore_create_table_result { if (isset($vals['o3'])) { $this->o3 = $vals['o3']; } - if (isset($vals['o4'])) { - $this->o4 = $vals['o4']; - } } } public function getName() { - return 'ThriftHiveMetastore_create_table_result'; + return 'ThriftHiveMetastore_get_table_names_by_filter_result'; } public function read($input) @@ -7550,9 +9903,26 @@ class ThriftHiveMetastore_create_table_result { } switch ($fid) { + case 0: + if ($ftype == TType::LST) { + $this->success = array(); + $_size292 = 0; + $_etype295 = 0; + $xfer += $input->readListBegin($_etype295, $_size292); + for ($_i296 = 0; $_i296 < $_size292; ++$_i296) + { + $elem297 = null; + $xfer += $input->readString($elem297); + $this->success []= $elem297; + } + $xfer += $input->readListEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\AlreadyExistsException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -7560,7 +9930,7 @@ class ThriftHiveMetastore_create_table_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\InvalidObjectException(); + $this->o2 = new \metastore\InvalidOperationException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -7568,20 +9938,12 @@ class ThriftHiveMetastore_create_table_result { break; case 3: if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); + $this->o3 = new \metastore\UnknownDBException(); $xfer += $this->o3->read($input); } else { $xfer += $input->skip($ftype); } break; - case 4: - if ($ftype == TType::STRUCT) { - $this->o4 = new \metastore\NoSuchObjectException(); - $xfer += $this->o4->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -7594,7 +9956,24 @@ class ThriftHiveMetastore_create_table_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_table_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_names_by_filter_result'); + if ($this->success !== null) { + if (!is_array($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRING, count($this->success)); + { + foreach ($this->success as $iter298) + { + $xfer += $output->writeString($iter298); + } + } + $output->writeListEnd(); + } + $xfer += $output->writeFieldEnd(); + } if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -7610,11 +9989,6 @@ class ThriftHiveMetastore_create_table_result { $xfer += $this->o3->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o4 !== null) { - $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); - $xfer += $this->o4->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -7622,39 +9996,46 @@ class ThriftHiveMetastore_create_table_result { } -class ThriftHiveMetastore_create_table_with_environment_context_args { +class ThriftHiveMetastore_alter_table_args { static $_TSPEC; - public $tbl = null; - public $environment_context = null; + public $dbname = null; + public $tbl_name = null; + public $new_tbl = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'tbl', - 'type' => TType::STRUCT, - 'class' => '\metastore\Table', + 'var' => 'dbname', + 'type' => TType::STRING, ), 2 => array( - 'var' => 'environment_context', + 'var' => 'tbl_name', + 'type' => TType::STRING, + ), + 3 => array( + 'var' => 'new_tbl', 'type' => TType::STRUCT, - 'class' => '\metastore\EnvironmentContext', + 'class' => '\metastore\Table', ), ); } if (is_array($vals)) { - if (isset($vals['tbl'])) { - $this->tbl = $vals['tbl']; + if (isset($vals['dbname'])) { + $this->dbname = $vals['dbname']; } - if (isset($vals['environment_context'])) { - $this->environment_context = $vals['environment_context']; + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; + } + if (isset($vals['new_tbl'])) { + $this->new_tbl = $vals['new_tbl']; } } } public function getName() { - return 'ThriftHiveMetastore_create_table_with_environment_context_args'; + return 'ThriftHiveMetastore_alter_table_args'; } public function read($input) @@ -7673,17 +10054,23 @@ class ThriftHiveMetastore_create_table_with_environment_context_args { switch ($fid) { case 1: - if ($ftype == TType::STRUCT) { - $this->tbl = new \metastore\Table(); - $xfer += $this->tbl->read($input); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->dbname); } else { $xfer += $input->skip($ftype); } break; case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->tbl_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: if ($ftype == TType::STRUCT) { - $this->environment_context = new \metastore\EnvironmentContext(); - $xfer += $this->environment_context->read($input); + $this->new_tbl = new \metastore\Table(); + $xfer += $this->new_tbl->read($input); } else { $xfer += $input->skip($ftype); } @@ -7700,21 +10087,23 @@ class ThriftHiveMetastore_create_table_with_environment_context_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_table_with_environment_context_args'); - if ($this->tbl !== null) { - if (!is_object($this->tbl)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('tbl', TType::STRUCT, 1); - $xfer += $this->tbl->write($output); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_table_args'); + if ($this->dbname !== null) { + $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); + $xfer += $output->writeString($this->dbname); + $xfer += $output->writeFieldEnd(); + } + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->environment_context !== null) { - if (!is_object($this->environment_context)) { + if ($this->new_tbl !== null) { + if (!is_object($this->new_tbl)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 2); - $xfer += $this->environment_context->write($output); + $xfer += $output->writeFieldBegin('new_tbl', TType::STRUCT, 3); + $xfer += $this->new_tbl->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -7724,13 +10113,11 @@ class ThriftHiveMetastore_create_table_with_environment_context_args { } -class ThriftHiveMetastore_create_table_with_environment_context_result { +class ThriftHiveMetastore_alter_table_result { static $_TSPEC; public $o1 = null; public $o2 = null; - public $o3 = null; - public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -7738,23 +10125,13 @@ class ThriftHiveMetastore_create_table_with_environment_context_result { 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\AlreadyExistsException', + 'class' => '\metastore\InvalidOperationException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', - ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), - 4 => array( - 'var' => 'o4', - 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', - ), ); } if (is_array($vals)) { @@ -7764,17 +10141,11 @@ class ThriftHiveMetastore_create_table_with_environment_context_result { if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } - if (isset($vals['o4'])) { - $this->o4 = $vals['o4']; - } } } public function getName() { - return 'ThriftHiveMetastore_create_table_with_environment_context_result'; + return 'ThriftHiveMetastore_alter_table_result'; } public function read($input) @@ -7794,7 +10165,7 @@ class ThriftHiveMetastore_create_table_with_environment_context_result { { case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\AlreadyExistsException(); + $this->o1 = new \metastore\InvalidOperationException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -7802,28 +10173,12 @@ class ThriftHiveMetastore_create_table_with_environment_context_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\InvalidObjectException(); + $this->o2 = new \metastore\MetaException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); - $xfer += $this->o3->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::STRUCT) { - $this->o4 = new \metastore\NoSuchObjectException(); - $xfer += $this->o4->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -7836,7 +10191,7 @@ class ThriftHiveMetastore_create_table_with_environment_context_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_table_with_environment_context_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_table_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -7847,16 +10202,6 @@ class ThriftHiveMetastore_create_table_with_environment_context_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->o4 !== null) { - $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); - $xfer += $this->o4->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -7864,12 +10209,13 @@ class ThriftHiveMetastore_create_table_with_environment_context_result { } -class ThriftHiveMetastore_drop_table_args { +class ThriftHiveMetastore_alter_table_with_environment_context_args { static $_TSPEC; public $dbname = null; - public $name = null; - public $deleteData = null; + public $tbl_name = null; + public $new_tbl = null; + public $environment_context = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -7879,12 +10225,18 @@ class ThriftHiveMetastore_drop_table_args { 'type' => TType::STRING, ), 2 => array( - 'var' => 'name', + 'var' => 'tbl_name', 'type' => TType::STRING, ), 3 => array( - 'var' => 'deleteData', - 'type' => TType::BOOL, + 'var' => 'new_tbl', + 'type' => TType::STRUCT, + 'class' => '\metastore\Table', + ), + 4 => array( + 'var' => 'environment_context', + 'type' => TType::STRUCT, + 'class' => '\metastore\EnvironmentContext', ), ); } @@ -7892,17 +10244,20 @@ class ThriftHiveMetastore_drop_table_args { if (isset($vals['dbname'])) { $this->dbname = $vals['dbname']; } - if (isset($vals['name'])) { - $this->name = $vals['name']; + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['deleteData'])) { - $this->deleteData = $vals['deleteData']; + if (isset($vals['new_tbl'])) { + $this->new_tbl = $vals['new_tbl']; + } + if (isset($vals['environment_context'])) { + $this->environment_context = $vals['environment_context']; } } } public function getName() { - return 'ThriftHiveMetastore_drop_table_args'; + return 'ThriftHiveMetastore_alter_table_with_environment_context_args'; } public function read($input) @@ -7929,14 +10284,23 @@ class ThriftHiveMetastore_drop_table_args { break; case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->name); + $xfer += $input->readString($this->tbl_name); } else { $xfer += $input->skip($ftype); } break; case 3: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->deleteData); + if ($ftype == TType::STRUCT) { + $this->new_tbl = new \metastore\Table(); + $xfer += $this->new_tbl->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRUCT) { + $this->environment_context = new \metastore\EnvironmentContext(); + $xfer += $this->environment_context->read($input); } else { $xfer += $input->skip($ftype); } @@ -7953,20 +10317,31 @@ class ThriftHiveMetastore_drop_table_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_table_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_table_with_environment_context_args'); if ($this->dbname !== null) { $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); $xfer += $output->writeString($this->dbname); $xfer += $output->writeFieldEnd(); } - if ($this->name !== null) { - $xfer += $output->writeFieldBegin('name', TType::STRING, 2); - $xfer += $output->writeString($this->name); + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->deleteData !== null) { - $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 3); - $xfer += $output->writeBool($this->deleteData); + if ($this->new_tbl !== null) { + if (!is_object($this->new_tbl)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('new_tbl', TType::STRUCT, 3); + $xfer += $this->new_tbl->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->environment_context !== null) { + if (!is_object($this->environment_context)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 4); + $xfer += $this->environment_context->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -7976,11 +10351,11 @@ class ThriftHiveMetastore_drop_table_args { } -class ThriftHiveMetastore_drop_table_result { +class ThriftHiveMetastore_alter_table_with_environment_context_result { static $_TSPEC; public $o1 = null; - public $o3 = null; + public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -7988,10 +10363,10 @@ class ThriftHiveMetastore_drop_table_result { 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\InvalidOperationException', ), 2 => array( - 'var' => 'o3', + 'var' => 'o2', 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), @@ -8001,14 +10376,14 @@ class ThriftHiveMetastore_drop_table_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; } } } public function getName() { - return 'ThriftHiveMetastore_drop_table_result'; + return 'ThriftHiveMetastore_alter_table_with_environment_context_result'; } public function read($input) @@ -8028,7 +10403,7 @@ class ThriftHiveMetastore_drop_table_result { { case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\InvalidOperationException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -8036,8 +10411,8 @@ class ThriftHiveMetastore_drop_table_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); - $xfer += $this->o3->read($input); + $this->o2 = new \metastore\MetaException(); + $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } @@ -8054,15 +10429,15 @@ class ThriftHiveMetastore_drop_table_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_table_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_table_with_environment_context_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 2); - $xfer += $this->o3->write($output); + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -8072,54 +10447,30 @@ class ThriftHiveMetastore_drop_table_result { } -class ThriftHiveMetastore_drop_table_with_environment_context_args { +class ThriftHiveMetastore_add_partition_args { static $_TSPEC; - public $dbname = null; - public $name = null; - public $deleteData = null; - public $environment_context = null; + public $new_part = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'dbname', - 'type' => TType::STRING, - ), - 2 => array( - 'var' => 'name', - 'type' => TType::STRING, - ), - 3 => array( - 'var' => 'deleteData', - 'type' => TType::BOOL, - ), - 4 => array( - 'var' => 'environment_context', + 'var' => 'new_part', 'type' => TType::STRUCT, - 'class' => '\metastore\EnvironmentContext', + 'class' => '\metastore\Partition', ), ); } if (is_array($vals)) { - if (isset($vals['dbname'])) { - $this->dbname = $vals['dbname']; - } - if (isset($vals['name'])) { - $this->name = $vals['name']; - } - if (isset($vals['deleteData'])) { - $this->deleteData = $vals['deleteData']; - } - if (isset($vals['environment_context'])) { - $this->environment_context = $vals['environment_context']; + if (isset($vals['new_part'])) { + $this->new_part = $vals['new_part']; } } } public function getName() { - return 'ThriftHiveMetastore_drop_table_with_environment_context_args'; + return 'ThriftHiveMetastore_add_partition_args'; } public function read($input) @@ -8138,68 +10489,32 @@ class ThriftHiveMetastore_drop_table_with_environment_context_args { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->dbname); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->deleteData); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: if ($ftype == TType::STRUCT) { - $this->environment_context = new \metastore\EnvironmentContext(); - $xfer += $this->environment_context->read($input); + $this->new_part = new \metastore\Partition(); + $xfer += $this->new_part->read($input); } else { $xfer += $input->skip($ftype); } - break; - default: - $xfer += $input->skip($ftype); - break; - } - $xfer += $input->readFieldEnd(); - } - $xfer += $input->readStructEnd(); - return $xfer; - } - - public function write($output) { - $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_table_with_environment_context_args'); - if ($this->dbname !== null) { - $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); - $xfer += $output->writeString($this->dbname); - $xfer += $output->writeFieldEnd(); - } - if ($this->name !== null) { - $xfer += $output->writeFieldBegin('name', TType::STRING, 2); - $xfer += $output->writeString($this->name); - $xfer += $output->writeFieldEnd(); - } - if ($this->deleteData !== null) { - $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 3); - $xfer += $output->writeBool($this->deleteData); - $xfer += $output->writeFieldEnd(); + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); } - if ($this->environment_context !== null) { - if (!is_object($this->environment_context)) { + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_partition_args'); + if ($this->new_part !== null) { + if (!is_object($this->new_part)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 4); - $xfer += $this->environment_context->write($output); + $xfer += $output->writeFieldBegin('new_part', TType::STRUCT, 1); + $xfer += $this->new_part->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -8209,21 +10524,33 @@ class ThriftHiveMetastore_drop_table_with_environment_context_args { } -class ThriftHiveMetastore_drop_table_with_environment_context_result { +class ThriftHiveMetastore_add_partition_result { static $_TSPEC; + public $success = null; public $o1 = null; + public $o2 = null; public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', + ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\InvalidObjectException', ), 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\AlreadyExistsException', + ), + 3 => array( 'var' => 'o3', 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', @@ -8231,9 +10558,15 @@ class ThriftHiveMetastore_drop_table_with_environment_context_result { ); } if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } if (isset($vals['o3'])) { $this->o3 = $vals['o3']; } @@ -8241,7 +10574,7 @@ class ThriftHiveMetastore_drop_table_with_environment_context_result { } public function getName() { - return 'ThriftHiveMetastore_drop_table_with_environment_context_result'; + return 'ThriftHiveMetastore_add_partition_result'; } public function read($input) @@ -8259,9 +10592,17 @@ class ThriftHiveMetastore_drop_table_with_environment_context_result { } switch ($fid) { + case 0: + if ($ftype == TType::STRUCT) { + $this->success = new \metastore\Partition(); + $xfer += $this->success->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\InvalidObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -8269,6 +10610,14 @@ class ThriftHiveMetastore_drop_table_with_environment_context_result { break; case 2: if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\AlreadyExistsException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRUCT) { $this->o3 = new \metastore\MetaException(); $xfer += $this->o3->read($input); } else { @@ -8287,14 +10636,27 @@ class ThriftHiveMetastore_drop_table_with_environment_context_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_table_with_environment_context_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_partition_result'); + if ($this->success !== null) { + if (!is_object($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); + $xfer += $output->writeFieldEnd(); + } if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 2); + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); $xfer += $this->o3->write($output); $xfer += $output->writeFieldEnd(); } @@ -8305,37 +10667,39 @@ class ThriftHiveMetastore_drop_table_with_environment_context_result { } -class ThriftHiveMetastore_get_tables_args { +class ThriftHiveMetastore_add_partition_with_environment_context_args { static $_TSPEC; - public $db_name = null; - public $pattern = null; + public $new_part = null; + public $environment_context = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', - 'type' => TType::STRING, + 'var' => 'new_part', + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', ), 2 => array( - 'var' => 'pattern', - 'type' => TType::STRING, + 'var' => 'environment_context', + 'type' => TType::STRUCT, + 'class' => '\metastore\EnvironmentContext', ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; + if (isset($vals['new_part'])) { + $this->new_part = $vals['new_part']; } - if (isset($vals['pattern'])) { - $this->pattern = $vals['pattern']; + if (isset($vals['environment_context'])) { + $this->environment_context = $vals['environment_context']; } } } public function getName() { - return 'ThriftHiveMetastore_get_tables_args'; + return 'ThriftHiveMetastore_add_partition_with_environment_context_args'; } public function read($input) @@ -8354,15 +10718,17 @@ class ThriftHiveMetastore_get_tables_args { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); + if ($ftype == TType::STRUCT) { + $this->new_part = new \metastore\Partition(); + $xfer += $this->new_part->read($input); } else { $xfer += $input->skip($ftype); } break; case 2: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->pattern); + if ($ftype == TType::STRUCT) { + $this->environment_context = new \metastore\EnvironmentContext(); + $xfer += $this->environment_context->read($input); } else { $xfer += $input->skip($ftype); } @@ -8379,15 +10745,21 @@ class ThriftHiveMetastore_get_tables_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_tables_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_partition_with_environment_context_args'); + if ($this->new_part !== null) { + if (!is_object($this->new_part)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('new_part', TType::STRUCT, 1); + $xfer += $this->new_part->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->pattern !== null) { - $xfer += $output->writeFieldBegin('pattern', TType::STRING, 2); - $xfer += $output->writeString($this->pattern); + if ($this->environment_context !== null) { + if (!is_object($this->environment_context)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 2); + $xfer += $this->environment_context->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -8397,26 +10769,35 @@ class ThriftHiveMetastore_get_tables_args { } -class ThriftHiveMetastore_get_tables_result { +class ThriftHiveMetastore_add_partition_with_environment_context_result { static $_TSPEC; public $success = null; public $o1 = null; + public $o2 = null; + public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidObjectException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\AlreadyExistsException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), ); @@ -8428,11 +10809,17 @@ class ThriftHiveMetastore_get_tables_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } } } public function getName() { - return 'ThriftHiveMetastore_get_tables_result'; + return 'ThriftHiveMetastore_add_partition_with_environment_context_result'; } public function read($input) @@ -8451,30 +10838,37 @@ class ThriftHiveMetastore_get_tables_result { switch ($fid) { case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size264 = 0; - $_etype267 = 0; - $xfer += $input->readListBegin($_etype267, $_size264); - for ($_i268 = 0; $_i268 < $_size264; ++$_i268) - { - $elem269 = null; - $xfer += $input->readString($elem269); - $this->success []= $elem269; - } - $xfer += $input->readListEnd(); + if ($ftype == TType::STRUCT) { + $this->success = new \metastore\Partition(); + $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\InvalidObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\AlreadyExistsException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\MetaException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -8487,22 +10881,13 @@ class ThriftHiveMetastore_get_tables_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_tables_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_partition_with_environment_context_result'); if ($this->success !== null) { - if (!is_array($this->success)) { + if (!is_object($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRING, count($this->success)); - { - foreach ($this->success as $iter270) - { - $xfer += $output->writeString($iter270); - } - } - $output->writeListEnd(); - } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -8510,6 +10895,16 @@ class ThriftHiveMetastore_get_tables_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -8517,29 +10912,34 @@ class ThriftHiveMetastore_get_tables_result { } -class ThriftHiveMetastore_get_all_tables_args { +class ThriftHiveMetastore_add_partitions_args { static $_TSPEC; - public $db_name = null; + public $new_parts = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', - 'type' => TType::STRING, + 'var' => 'new_parts', + 'type' => TType::LST, + 'etype' => TType::STRUCT, + 'elem' => array( + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', + ), ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; + if (isset($vals['new_parts'])) { + $this->new_parts = $vals['new_parts']; } } } public function getName() { - return 'ThriftHiveMetastore_get_all_tables_args'; + return 'ThriftHiveMetastore_add_partitions_args'; } public function read($input) @@ -8558,8 +10958,19 @@ class ThriftHiveMetastore_get_all_tables_args { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); + if ($ftype == TType::LST) { + $this->new_parts = array(); + $_size299 = 0; + $_etype302 = 0; + $xfer += $input->readListBegin($_etype302, $_size299); + for ($_i303 = 0; $_i303 < $_size299; ++$_i303) + { + $elem304 = null; + $elem304 = new \metastore\Partition(); + $xfer += $elem304->read($input); + $this->new_parts []= $elem304; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } @@ -8576,10 +10987,22 @@ class ThriftHiveMetastore_get_all_tables_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_all_tables_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_partitions_args'); + if ($this->new_parts !== null) { + if (!is_array($this->new_parts)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('new_parts', TType::LST, 1); + { + $output->writeListBegin(TType::STRUCT, count($this->new_parts)); + { + foreach ($this->new_parts as $iter305) + { + $xfer += $iter305->write($output); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -8589,26 +11012,34 @@ class ThriftHiveMetastore_get_all_tables_args { } -class ThriftHiveMetastore_get_all_tables_result { +class ThriftHiveMetastore_add_partitions_result { static $_TSPEC; public $success = null; public $o1 = null; + public $o2 = null; + public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), + 'type' => TType::I32, ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidObjectException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\AlreadyExistsException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), ); @@ -8620,11 +11051,17 @@ class ThriftHiveMetastore_get_all_tables_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } } } public function getName() { - return 'ThriftHiveMetastore_get_all_tables_result'; + return 'ThriftHiveMetastore_add_partitions_result'; } public function read($input) @@ -8643,30 +11080,36 @@ class ThriftHiveMetastore_get_all_tables_result { switch ($fid) { case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size271 = 0; - $_etype274 = 0; - $xfer += $input->readListBegin($_etype274, $_size271); - for ($_i275 = 0; $_i275 < $_size271; ++$_i275) - { - $elem276 = null; - $xfer += $input->readString($elem276); - $this->success []= $elem276; - } - $xfer += $input->readListEnd(); + if ($ftype == TType::I32) { + $xfer += $input->readI32($this->success); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\InvalidObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\AlreadyExistsException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\MetaException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -8679,22 +11122,10 @@ class ThriftHiveMetastore_get_all_tables_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_all_tables_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_partitions_result'); if ($this->success !== null) { - if (!is_array($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRING, count($this->success)); - { - foreach ($this->success as $iter277) - { - $xfer += $output->writeString($iter277); - } - } - $output->writeListEnd(); - } + $xfer += $output->writeFieldBegin('success', TType::I32, 0); + $xfer += $output->writeI32($this->success); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -8702,6 +11133,16 @@ class ThriftHiveMetastore_get_all_tables_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -8709,37 +11150,49 @@ class ThriftHiveMetastore_get_all_tables_result { } -class ThriftHiveMetastore_get_table_args { +class ThriftHiveMetastore_append_partition_args { static $_TSPEC; - public $dbname = null; + public $db_name = null; public $tbl_name = null; + public $part_vals = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'dbname', + 'var' => 'db_name', 'type' => TType::STRING, ), 2 => array( 'var' => 'tbl_name', 'type' => TType::STRING, ), + 3 => array( + 'var' => 'part_vals', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), + ), ); } if (is_array($vals)) { - if (isset($vals['dbname'])) { - $this->dbname = $vals['dbname']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; } if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } + if (isset($vals['part_vals'])) { + $this->part_vals = $vals['part_vals']; + } } } public function getName() { - return 'ThriftHiveMetastore_get_table_args'; + return 'ThriftHiveMetastore_append_partition_args'; } public function read($input) @@ -8759,7 +11212,7 @@ class ThriftHiveMetastore_get_table_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->dbname); + $xfer += $input->readString($this->db_name); } else { $xfer += $input->skip($ftype); } @@ -8771,6 +11224,23 @@ class ThriftHiveMetastore_get_table_args { $xfer += $input->skip($ftype); } break; + case 3: + if ($ftype == TType::LST) { + $this->part_vals = array(); + $_size306 = 0; + $_etype309 = 0; + $xfer += $input->readListBegin($_etype309, $_size306); + for ($_i310 = 0; $_i310 < $_size306; ++$_i310) + { + $elem311 = null; + $xfer += $input->readString($elem311); + $this->part_vals []= $elem311; + } + $xfer += $input->readListEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -8783,10 +11253,10 @@ class ThriftHiveMetastore_get_table_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_args'); - if ($this->dbname !== null) { - $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); - $xfer += $output->writeString($this->dbname); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); $xfer += $output->writeFieldEnd(); } if ($this->tbl_name !== null) { @@ -8794,6 +11264,23 @@ class ThriftHiveMetastore_get_table_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } + if ($this->part_vals !== null) { + if (!is_array($this->part_vals)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); + { + $output->writeListBegin(TType::STRING, count($this->part_vals)); + { + foreach ($this->part_vals as $iter312) + { + $xfer += $output->writeString($iter312); + } + } + $output->writeListEnd(); + } + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -8801,12 +11288,13 @@ class ThriftHiveMetastore_get_table_args { } -class ThriftHiveMetastore_get_table_result { +class ThriftHiveMetastore_append_partition_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; + public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -8814,17 +11302,22 @@ class ThriftHiveMetastore_get_table_result { 0 => array( 'var' => 'success', 'type' => TType::STRUCT, - 'class' => '\metastore\Table', + 'class' => '\metastore\Partition', ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\InvalidObjectException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\AlreadyExistsException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, + 'class' => '\metastore\MetaException', ), ); } @@ -8838,11 +11331,14 @@ class ThriftHiveMetastore_get_table_result { if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } } } public function getName() { - return 'ThriftHiveMetastore_get_table_result'; + return 'ThriftHiveMetastore_append_partition_result'; } public function read($input) @@ -8862,7 +11358,7 @@ class ThriftHiveMetastore_get_table_result { { case 0: if ($ftype == TType::STRUCT) { - $this->success = new \metastore\Table(); + $this->success = new \metastore\Partition(); $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); @@ -8870,7 +11366,7 @@ class ThriftHiveMetastore_get_table_result { break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\InvalidObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -8878,12 +11374,20 @@ class ThriftHiveMetastore_get_table_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\NoSuchObjectException(); + $this->o2 = new \metastore\AlreadyExistsException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\MetaException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -8896,7 +11400,7 @@ class ThriftHiveMetastore_get_table_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_result'); if ($this->success !== null) { if (!is_object($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); @@ -8915,6 +11419,11 @@ class ThriftHiveMetastore_get_table_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -8922,41 +11431,58 @@ class ThriftHiveMetastore_get_table_result { } -class ThriftHiveMetastore_get_table_objects_by_name_args { +class ThriftHiveMetastore_append_partition_with_environment_context_args { static $_TSPEC; - public $dbname = null; - public $tbl_names = null; + public $db_name = null; + public $tbl_name = null; + public $part_vals = null; + public $environment_context = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'dbname', + 'var' => 'db_name', 'type' => TType::STRING, ), 2 => array( - 'var' => 'tbl_names', + 'var' => 'tbl_name', + 'type' => TType::STRING, + ), + 3 => array( + 'var' => 'part_vals', 'type' => TType::LST, 'etype' => TType::STRING, 'elem' => array( 'type' => TType::STRING, ), ), + 4 => array( + 'var' => 'environment_context', + 'type' => TType::STRUCT, + 'class' => '\metastore\EnvironmentContext', + ), ); } if (is_array($vals)) { - if (isset($vals['dbname'])) { - $this->dbname = $vals['dbname']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; } - if (isset($vals['tbl_names'])) { - $this->tbl_names = $vals['tbl_names']; + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; + } + if (isset($vals['part_vals'])) { + $this->part_vals = $vals['part_vals']; + } + if (isset($vals['environment_context'])) { + $this->environment_context = $vals['environment_context']; } } } public function getName() { - return 'ThriftHiveMetastore_get_table_objects_by_name_args'; + return 'ThriftHiveMetastore_append_partition_with_environment_context_args'; } public function read($input) @@ -8976,28 +11502,43 @@ class ThriftHiveMetastore_get_table_objects_by_name_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->dbname); + $xfer += $input->readString($this->db_name); } else { $xfer += $input->skip($ftype); } break; case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->tbl_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: if ($ftype == TType::LST) { - $this->tbl_names = array(); - $_size278 = 0; - $_etype281 = 0; - $xfer += $input->readListBegin($_etype281, $_size278); - for ($_i282 = 0; $_i282 < $_size278; ++$_i282) + $this->part_vals = array(); + $_size313 = 0; + $_etype316 = 0; + $xfer += $input->readListBegin($_etype316, $_size313); + for ($_i317 = 0; $_i317 < $_size313; ++$_i317) { - $elem283 = null; - $xfer += $input->readString($elem283); - $this->tbl_names []= $elem283; + $elem318 = null; + $xfer += $input->readString($elem318); + $this->part_vals []= $elem318; } $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } break; + case 4: + if ($ftype == TType::STRUCT) { + $this->environment_context = new \metastore\EnvironmentContext(); + $xfer += $this->environment_context->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -9010,29 +11551,42 @@ class ThriftHiveMetastore_get_table_objects_by_name_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_objects_by_name_args'); - if ($this->dbname !== null) { - $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); - $xfer += $output->writeString($this->dbname); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_with_environment_context_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); $xfer += $output->writeFieldEnd(); } - if ($this->tbl_names !== null) { - if (!is_array($this->tbl_names)) { + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->part_vals !== null) { + if (!is_array($this->part_vals)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('tbl_names', TType::LST, 2); + $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); { - $output->writeListBegin(TType::STRING, count($this->tbl_names)); + $output->writeListBegin(TType::STRING, count($this->part_vals)); { - foreach ($this->tbl_names as $iter284) + foreach ($this->part_vals as $iter319) { - $xfer += $output->writeString($iter284); + $xfer += $output->writeString($iter319); } } $output->writeListEnd(); } $xfer += $output->writeFieldEnd(); } + if ($this->environment_context !== null) { + if (!is_object($this->environment_context)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 4); + $xfer += $this->environment_context->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -9040,7 +11594,7 @@ class ThriftHiveMetastore_get_table_objects_by_name_args { } -class ThriftHiveMetastore_get_table_objects_by_name_result { +class ThriftHiveMetastore_append_partition_with_environment_context_result { static $_TSPEC; public $success = null; @@ -9053,27 +11607,23 @@ class ThriftHiveMetastore_get_table_objects_by_name_result { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRUCT, - 'elem' => array( - 'type' => TType::STRUCT, - 'class' => '\metastore\Table', - ), + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\InvalidObjectException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidOperationException', + 'class' => '\metastore\AlreadyExistsException', ), 3 => array( 'var' => 'o3', 'type' => TType::STRUCT, - 'class' => '\metastore\UnknownDBException', + 'class' => '\metastore\MetaException', ), ); } @@ -9094,7 +11644,7 @@ class ThriftHiveMetastore_get_table_objects_by_name_result { } public function getName() { - return 'ThriftHiveMetastore_get_table_objects_by_name_result'; + return 'ThriftHiveMetastore_append_partition_with_environment_context_result'; } public function read($input) @@ -9113,26 +11663,16 @@ class ThriftHiveMetastore_get_table_objects_by_name_result { switch ($fid) { case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size285 = 0; - $_etype288 = 0; - $xfer += $input->readListBegin($_etype288, $_size285); - for ($_i289 = 0; $_i289 < $_size285; ++$_i289) - { - $elem290 = null; - $elem290 = new \metastore\Table(); - $xfer += $elem290->read($input); - $this->success []= $elem290; - } - $xfer += $input->readListEnd(); + if ($ftype == TType::STRUCT) { + $this->success = new \metastore\Partition(); + $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\InvalidObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -9140,7 +11680,7 @@ class ThriftHiveMetastore_get_table_objects_by_name_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\InvalidOperationException(); + $this->o2 = new \metastore\AlreadyExistsException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -9148,7 +11688,7 @@ class ThriftHiveMetastore_get_table_objects_by_name_result { break; case 3: if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\UnknownDBException(); + $this->o3 = new \metastore\MetaException(); $xfer += $this->o3->read($input); } else { $xfer += $input->skip($ftype); @@ -9166,22 +11706,13 @@ class ThriftHiveMetastore_get_table_objects_by_name_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_objects_by_name_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_with_environment_context_result'); if ($this->success !== null) { - if (!is_array($this->success)) { + if (!is_object($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRUCT, count($this->success)); - { - foreach ($this->success as $iter291) - { - $xfer += $iter291->write($output); - } - } - $output->writeListEnd(); - } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -9206,45 +11737,45 @@ class ThriftHiveMetastore_get_table_objects_by_name_result { } -class ThriftHiveMetastore_get_table_names_by_filter_args { +class ThriftHiveMetastore_append_partition_by_name_args { static $_TSPEC; - public $dbname = null; - public $filter = null; - public $max_tables = -1; + public $db_name = null; + public $tbl_name = null; + public $part_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'dbname', + 'var' => 'db_name', 'type' => TType::STRING, ), 2 => array( - 'var' => 'filter', + 'var' => 'tbl_name', 'type' => TType::STRING, ), 3 => array( - 'var' => 'max_tables', - 'type' => TType::I16, + 'var' => 'part_name', + 'type' => TType::STRING, ), ); } if (is_array($vals)) { - if (isset($vals['dbname'])) { - $this->dbname = $vals['dbname']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; } - if (isset($vals['filter'])) { - $this->filter = $vals['filter']; + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['max_tables'])) { - $this->max_tables = $vals['max_tables']; + if (isset($vals['part_name'])) { + $this->part_name = $vals['part_name']; } } } public function getName() { - return 'ThriftHiveMetastore_get_table_names_by_filter_args'; + return 'ThriftHiveMetastore_append_partition_by_name_args'; } public function read($input) @@ -9264,21 +11795,21 @@ class ThriftHiveMetastore_get_table_names_by_filter_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->dbname); + $xfer += $input->readString($this->db_name); } else { $xfer += $input->skip($ftype); } break; case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->filter); + $xfer += $input->readString($this->tbl_name); } else { $xfer += $input->skip($ftype); } break; case 3: - if ($ftype == TType::I16) { - $xfer += $input->readI16($this->max_tables); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->part_name); } else { $xfer += $input->skip($ftype); } @@ -9295,20 +11826,20 @@ class ThriftHiveMetastore_get_table_names_by_filter_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_names_by_filter_args'); - if ($this->dbname !== null) { - $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); - $xfer += $output->writeString($this->dbname); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_by_name_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); $xfer += $output->writeFieldEnd(); } - if ($this->filter !== null) { - $xfer += $output->writeFieldBegin('filter', TType::STRING, 2); - $xfer += $output->writeString($this->filter); + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->max_tables !== null) { - $xfer += $output->writeFieldBegin('max_tables', TType::I16, 3); - $xfer += $output->writeI16($this->max_tables); + if ($this->part_name !== null) { + $xfer += $output->writeFieldBegin('part_name', TType::STRING, 3); + $xfer += $output->writeString($this->part_name); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -9318,7 +11849,7 @@ class ThriftHiveMetastore_get_table_names_by_filter_args { } -class ThriftHiveMetastore_get_table_names_by_filter_result { +class ThriftHiveMetastore_append_partition_by_name_result { static $_TSPEC; public $success = null; @@ -9331,26 +11862,23 @@ class ThriftHiveMetastore_get_table_names_by_filter_result { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\InvalidObjectException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidOperationException', + 'class' => '\metastore\AlreadyExistsException', ), 3 => array( 'var' => 'o3', 'type' => TType::STRUCT, - 'class' => '\metastore\UnknownDBException', + 'class' => '\metastore\MetaException', ), ); } @@ -9371,7 +11899,7 @@ class ThriftHiveMetastore_get_table_names_by_filter_result { } public function getName() { - return 'ThriftHiveMetastore_get_table_names_by_filter_result'; + return 'ThriftHiveMetastore_append_partition_by_name_result'; } public function read($input) @@ -9390,25 +11918,16 @@ class ThriftHiveMetastore_get_table_names_by_filter_result { switch ($fid) { case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size292 = 0; - $_etype295 = 0; - $xfer += $input->readListBegin($_etype295, $_size292); - for ($_i296 = 0; $_i296 < $_size292; ++$_i296) - { - $elem297 = null; - $xfer += $input->readString($elem297); - $this->success []= $elem297; - } - $xfer += $input->readListEnd(); + if ($ftype == TType::STRUCT) { + $this->success = new \metastore\Partition(); + $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\InvalidObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -9416,7 +11935,7 @@ class ThriftHiveMetastore_get_table_names_by_filter_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\InvalidOperationException(); + $this->o2 = new \metastore\AlreadyExistsException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -9424,7 +11943,7 @@ class ThriftHiveMetastore_get_table_names_by_filter_result { break; case 3: if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\UnknownDBException(); + $this->o3 = new \metastore\MetaException(); $xfer += $this->o3->read($input); } else { $xfer += $input->skip($ftype); @@ -9442,22 +11961,13 @@ class ThriftHiveMetastore_get_table_names_by_filter_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_names_by_filter_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_by_name_result'); if ($this->success !== null) { - if (!is_array($this->success)) { + if (!is_object($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRING, count($this->success)); - { - foreach ($this->success as $iter298) - { - $xfer += $output->writeString($iter298); - } - } - $output->writeListEnd(); - } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -9482,18 +11992,19 @@ class ThriftHiveMetastore_get_table_names_by_filter_result { } -class ThriftHiveMetastore_alter_table_args { +class ThriftHiveMetastore_append_partition_by_name_with_environment_context_args { static $_TSPEC; - public $dbname = null; + public $db_name = null; public $tbl_name = null; - public $new_tbl = null; + public $part_name = null; + public $environment_context = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'dbname', + 'var' => 'db_name', 'type' => TType::STRING, ), 2 => array( @@ -9501,27 +12012,34 @@ class ThriftHiveMetastore_alter_table_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'new_tbl', + 'var' => 'part_name', + 'type' => TType::STRING, + ), + 4 => array( + 'var' => 'environment_context', 'type' => TType::STRUCT, - 'class' => '\metastore\Table', + 'class' => '\metastore\EnvironmentContext', ), ); } if (is_array($vals)) { - if (isset($vals['dbname'])) { - $this->dbname = $vals['dbname']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; } if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['new_tbl'])) { - $this->new_tbl = $vals['new_tbl']; + if (isset($vals['part_name'])) { + $this->part_name = $vals['part_name']; + } + if (isset($vals['environment_context'])) { + $this->environment_context = $vals['environment_context']; } } } public function getName() { - return 'ThriftHiveMetastore_alter_table_args'; + return 'ThriftHiveMetastore_append_partition_by_name_with_environment_context_args'; } public function read($input) @@ -9541,7 +12059,7 @@ class ThriftHiveMetastore_alter_table_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->dbname); + $xfer += $input->readString($this->db_name); } else { $xfer += $input->skip($ftype); } @@ -9554,9 +12072,16 @@ class ThriftHiveMetastore_alter_table_args { } break; case 3: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->part_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: if ($ftype == TType::STRUCT) { - $this->new_tbl = new \metastore\Table(); - $xfer += $this->new_tbl->read($input); + $this->environment_context = new \metastore\EnvironmentContext(); + $xfer += $this->environment_context->read($input); } else { $xfer += $input->skip($ftype); } @@ -9573,10 +12098,10 @@ class ThriftHiveMetastore_alter_table_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_table_args'); - if ($this->dbname !== null) { - $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); - $xfer += $output->writeString($this->dbname); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_by_name_with_environment_context_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); $xfer += $output->writeFieldEnd(); } if ($this->tbl_name !== null) { @@ -9584,12 +12109,17 @@ class ThriftHiveMetastore_alter_table_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->new_tbl !== null) { - if (!is_object($this->new_tbl)) { + if ($this->part_name !== null) { + $xfer += $output->writeFieldBegin('part_name', TType::STRING, 3); + $xfer += $output->writeString($this->part_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->environment_context !== null) { + if (!is_object($this->environment_context)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('new_tbl', TType::STRUCT, 3); - $xfer += $this->new_tbl->write($output); + $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 4); + $xfer += $this->environment_context->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -9599,39 +12129,57 @@ class ThriftHiveMetastore_alter_table_args { } -class ThriftHiveMetastore_alter_table_result { +class ThriftHiveMetastore_append_partition_by_name_with_environment_context_result { static $_TSPEC; + public $success = null; public $o1 = null; public $o2 = null; + public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', + ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidOperationException', + 'class' => '\metastore\InvalidObjectException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, + 'class' => '\metastore\AlreadyExistsException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), ); } if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } } } public function getName() { - return 'ThriftHiveMetastore_alter_table_result'; + return 'ThriftHiveMetastore_append_partition_by_name_with_environment_context_result'; } public function read($input) @@ -9649,9 +12197,17 @@ class ThriftHiveMetastore_alter_table_result { } switch ($fid) { + case 0: + if ($ftype == TType::STRUCT) { + $this->success = new \metastore\Partition(); + $xfer += $this->success->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidOperationException(); + $this->o1 = new \metastore\InvalidObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -9659,12 +12215,20 @@ class ThriftHiveMetastore_alter_table_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); + $this->o2 = new \metastore\AlreadyExistsException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\MetaException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -9677,7 +12241,15 @@ class ThriftHiveMetastore_alter_table_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_table_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_by_name_with_environment_context_result'); + if ($this->success !== null) { + if (!is_object($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); + $xfer += $output->writeFieldEnd(); + } if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -9688,6 +12260,11 @@ class ThriftHiveMetastore_alter_table_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -9695,19 +12272,19 @@ class ThriftHiveMetastore_alter_table_result { } -class ThriftHiveMetastore_alter_table_with_environment_context_args { +class ThriftHiveMetastore_drop_partition_args { static $_TSPEC; - public $dbname = null; + public $db_name = null; public $tbl_name = null; - public $new_tbl = null; - public $environment_context = null; + public $part_vals = null; + public $deleteData = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'dbname', + 'var' => 'db_name', 'type' => TType::STRING, ), 2 => array( @@ -9715,35 +12292,37 @@ class ThriftHiveMetastore_alter_table_with_environment_context_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'new_tbl', - 'type' => TType::STRUCT, - 'class' => '\metastore\Table', + 'var' => 'part_vals', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), ), 4 => array( - 'var' => 'environment_context', - 'type' => TType::STRUCT, - 'class' => '\metastore\EnvironmentContext', + 'var' => 'deleteData', + 'type' => TType::BOOL, ), ); } if (is_array($vals)) { - if (isset($vals['dbname'])) { - $this->dbname = $vals['dbname']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; } if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['new_tbl'])) { - $this->new_tbl = $vals['new_tbl']; + if (isset($vals['part_vals'])) { + $this->part_vals = $vals['part_vals']; } - if (isset($vals['environment_context'])) { - $this->environment_context = $vals['environment_context']; + if (isset($vals['deleteData'])) { + $this->deleteData = $vals['deleteData']; } } } public function getName() { - return 'ThriftHiveMetastore_alter_table_with_environment_context_args'; + return 'ThriftHiveMetastore_drop_partition_args'; } public function read($input) @@ -9763,7 +12342,7 @@ class ThriftHiveMetastore_alter_table_with_environment_context_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->dbname); + $xfer += $input->readString($this->db_name); } else { $xfer += $input->skip($ftype); } @@ -9776,17 +12355,25 @@ class ThriftHiveMetastore_alter_table_with_environment_context_args { } break; case 3: - if ($ftype == TType::STRUCT) { - $this->new_tbl = new \metastore\Table(); - $xfer += $this->new_tbl->read($input); + if ($ftype == TType::LST) { + $this->part_vals = array(); + $_size320 = 0; + $_etype323 = 0; + $xfer += $input->readListBegin($_etype323, $_size320); + for ($_i324 = 0; $_i324 < $_size320; ++$_i324) + { + $elem325 = null; + $xfer += $input->readString($elem325); + $this->part_vals []= $elem325; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } break; case 4: - if ($ftype == TType::STRUCT) { - $this->environment_context = new \metastore\EnvironmentContext(); - $xfer += $this->environment_context->read($input); + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->deleteData); } else { $xfer += $input->skip($ftype); } @@ -9803,10 +12390,10 @@ class ThriftHiveMetastore_alter_table_with_environment_context_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_table_with_environment_context_args'); - if ($this->dbname !== null) { - $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); - $xfer += $output->writeString($this->dbname); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); $xfer += $output->writeFieldEnd(); } if ($this->tbl_name !== null) { @@ -9814,20 +12401,26 @@ class ThriftHiveMetastore_alter_table_with_environment_context_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->new_tbl !== null) { - if (!is_object($this->new_tbl)) { + if ($this->part_vals !== null) { + if (!is_array($this->part_vals)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('new_tbl', TType::STRUCT, 3); - $xfer += $this->new_tbl->write($output); + $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); + { + $output->writeListBegin(TType::STRING, count($this->part_vals)); + { + foreach ($this->part_vals as $iter326) + { + $xfer += $output->writeString($iter326); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } - if ($this->environment_context !== null) { - if (!is_object($this->environment_context)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 4); - $xfer += $this->environment_context->write($output); + if ($this->deleteData !== null) { + $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 4); + $xfer += $output->writeBool($this->deleteData); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -9837,19 +12430,24 @@ class ThriftHiveMetastore_alter_table_with_environment_context_args { } -class ThriftHiveMetastore_alter_table_with_environment_context_result { +class ThriftHiveMetastore_drop_partition_result { static $_TSPEC; + public $success = null; public $o1 = null; public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::BOOL, + ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidOperationException', + 'class' => '\metastore\NoSuchObjectException', ), 2 => array( 'var' => 'o2', @@ -9859,6 +12457,9 @@ class ThriftHiveMetastore_alter_table_with_environment_context_result { ); } if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } @@ -9869,7 +12470,7 @@ class ThriftHiveMetastore_alter_table_with_environment_context_result { } public function getName() { - return 'ThriftHiveMetastore_alter_table_with_environment_context_result'; + return 'ThriftHiveMetastore_drop_partition_result'; } public function read($input) @@ -9887,9 +12488,16 @@ class ThriftHiveMetastore_alter_table_with_environment_context_result { } switch ($fid) { + case 0: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->success); + } else { + $xfer += $input->skip($ftype); + } + break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidOperationException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -9915,7 +12523,12 @@ class ThriftHiveMetastore_alter_table_with_environment_context_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_table_with_environment_context_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_result'); + if ($this->success !== null) { + $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); + $xfer += $output->writeBool($this->success); + $xfer += $output->writeFieldEnd(); + } if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -9933,30 +12546,66 @@ class ThriftHiveMetastore_alter_table_with_environment_context_result { } -class ThriftHiveMetastore_add_partition_args { +class ThriftHiveMetastore_drop_partition_with_environment_context_args { static $_TSPEC; - public $new_part = null; + public $db_name = null; + public $tbl_name = null; + public $part_vals = null; + public $deleteData = null; + public $environment_context = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'new_part', + 'var' => 'db_name', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'tbl_name', + 'type' => TType::STRING, + ), + 3 => array( + 'var' => 'part_vals', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), + ), + 4 => array( + 'var' => 'deleteData', + 'type' => TType::BOOL, + ), + 5 => array( + 'var' => 'environment_context', 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', + 'class' => '\metastore\EnvironmentContext', ), ); } if (is_array($vals)) { - if (isset($vals['new_part'])) { - $this->new_part = $vals['new_part']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; + } + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; + } + if (isset($vals['part_vals'])) { + $this->part_vals = $vals['part_vals']; + } + if (isset($vals['deleteData'])) { + $this->deleteData = $vals['deleteData']; + } + if (isset($vals['environment_context'])) { + $this->environment_context = $vals['environment_context']; } } } public function getName() { - return 'ThriftHiveMetastore_add_partition_args'; + return 'ThriftHiveMetastore_drop_partition_with_environment_context_args'; } public function read($input) @@ -9975,9 +12624,47 @@ class ThriftHiveMetastore_add_partition_args { switch ($fid) { case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->db_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->tbl_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::LST) { + $this->part_vals = array(); + $_size327 = 0; + $_etype330 = 0; + $xfer += $input->readListBegin($_etype330, $_size327); + for ($_i331 = 0; $_i331 < $_size327; ++$_i331) + { + $elem332 = null; + $xfer += $input->readString($elem332); + $this->part_vals []= $elem332; + } + $xfer += $input->readListEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->deleteData); + } else { + $xfer += $input->skip($ftype); + } + break; + case 5: if ($ftype == TType::STRUCT) { - $this->new_part = new \metastore\Partition(); - $xfer += $this->new_part->read($input); + $this->environment_context = new \metastore\EnvironmentContext(); + $xfer += $this->environment_context->read($input); } else { $xfer += $input->skip($ftype); } @@ -9994,13 +12681,45 @@ class ThriftHiveMetastore_add_partition_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_partition_args'); - if ($this->new_part !== null) { - if (!is_object($this->new_part)) { + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_with_environment_context_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->part_vals !== null) { + if (!is_array($this->part_vals)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('new_part', TType::STRUCT, 1); - $xfer += $this->new_part->write($output); + $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); + { + $output->writeListBegin(TType::STRING, count($this->part_vals)); + { + foreach ($this->part_vals as $iter333) + { + $xfer += $output->writeString($iter333); + } + } + $output->writeListEnd(); + } + $xfer += $output->writeFieldEnd(); + } + if ($this->deleteData !== null) { + $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 4); + $xfer += $output->writeBool($this->deleteData); + $xfer += $output->writeFieldEnd(); + } + if ($this->environment_context !== null) { + if (!is_object($this->environment_context)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 5); + $xfer += $this->environment_context->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -10010,35 +12729,28 @@ class ThriftHiveMetastore_add_partition_args { } -class ThriftHiveMetastore_add_partition_result { +class ThriftHiveMetastore_drop_partition_with_environment_context_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; - public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', + 'type' => TType::BOOL, ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', + 'class' => '\metastore\NoSuchObjectException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\AlreadyExistsException', - ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), ); @@ -10053,14 +12765,11 @@ class ThriftHiveMetastore_add_partition_result { if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } } } public function getName() { - return 'ThriftHiveMetastore_add_partition_result'; + return 'ThriftHiveMetastore_drop_partition_with_environment_context_result'; } public function read($input) @@ -10079,16 +12788,15 @@ class ThriftHiveMetastore_add_partition_result { switch ($fid) { case 0: - if ($ftype == TType::STRUCT) { - $this->success = new \metastore\Partition(); - $xfer += $this->success->read($input); + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->success); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidObjectException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -10096,20 +12804,12 @@ class ThriftHiveMetastore_add_partition_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\AlreadyExistsException(); + $this->o2 = new \metastore\MetaException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); - $xfer += $this->o3->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -10122,13 +12822,10 @@ class ThriftHiveMetastore_add_partition_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_partition_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_with_environment_context_result'); if ($this->success !== null) { - if (!is_object($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); - $xfer += $this->success->write($output); + $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); + $xfer += $output->writeBool($this->success); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -10141,11 +12838,6 @@ class ThriftHiveMetastore_add_partition_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -10153,39 +12845,53 @@ class ThriftHiveMetastore_add_partition_result { } -class ThriftHiveMetastore_add_partition_with_environment_context_args { +class ThriftHiveMetastore_drop_partition_by_name_args { static $_TSPEC; - public $new_part = null; - public $environment_context = null; + public $db_name = null; + public $tbl_name = null; + public $part_name = null; + public $deleteData = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'new_part', - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', + 'var' => 'db_name', + 'type' => TType::STRING, ), 2 => array( - 'var' => 'environment_context', - 'type' => TType::STRUCT, - 'class' => '\metastore\EnvironmentContext', + 'var' => 'tbl_name', + 'type' => TType::STRING, + ), + 3 => array( + 'var' => 'part_name', + 'type' => TType::STRING, + ), + 4 => array( + 'var' => 'deleteData', + 'type' => TType::BOOL, ), ); } if (is_array($vals)) { - if (isset($vals['new_part'])) { - $this->new_part = $vals['new_part']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; } - if (isset($vals['environment_context'])) { - $this->environment_context = $vals['environment_context']; + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; + } + if (isset($vals['part_name'])) { + $this->part_name = $vals['part_name']; + } + if (isset($vals['deleteData'])) { + $this->deleteData = $vals['deleteData']; } } } public function getName() { - return 'ThriftHiveMetastore_add_partition_with_environment_context_args'; + return 'ThriftHiveMetastore_drop_partition_by_name_args'; } public function read($input) @@ -10204,17 +12910,29 @@ class ThriftHiveMetastore_add_partition_with_environment_context_args { switch ($fid) { case 1: - if ($ftype == TType::STRUCT) { - $this->new_part = new \metastore\Partition(); - $xfer += $this->new_part->read($input); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->db_name); } else { $xfer += $input->skip($ftype); } break; case 2: - if ($ftype == TType::STRUCT) { - $this->environment_context = new \metastore\EnvironmentContext(); - $xfer += $this->environment_context->read($input); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->tbl_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->part_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->deleteData); } else { $xfer += $input->skip($ftype); } @@ -10231,21 +12949,25 @@ class ThriftHiveMetastore_add_partition_with_environment_context_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_partition_with_environment_context_args'); - if ($this->new_part !== null) { - if (!is_object($this->new_part)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('new_part', TType::STRUCT, 1); - $xfer += $this->new_part->write($output); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_by_name_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); $xfer += $output->writeFieldEnd(); } - if ($this->environment_context !== null) { - if (!is_object($this->environment_context)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 2); - $xfer += $this->environment_context->write($output); + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->part_name !== null) { + $xfer += $output->writeFieldBegin('part_name', TType::STRING, 3); + $xfer += $output->writeString($this->part_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->deleteData !== null) { + $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 4); + $xfer += $output->writeBool($this->deleteData); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -10255,35 +12977,28 @@ class ThriftHiveMetastore_add_partition_with_environment_context_args { } -class ThriftHiveMetastore_add_partition_with_environment_context_result { +class ThriftHiveMetastore_drop_partition_by_name_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; - public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', + 'type' => TType::BOOL, ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', + 'class' => '\metastore\NoSuchObjectException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\AlreadyExistsException', - ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), ); @@ -10298,14 +13013,11 @@ class ThriftHiveMetastore_add_partition_with_environment_context_result { if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } } } public function getName() { - return 'ThriftHiveMetastore_add_partition_with_environment_context_result'; + return 'ThriftHiveMetastore_drop_partition_by_name_result'; } public function read($input) @@ -10324,16 +13036,15 @@ class ThriftHiveMetastore_add_partition_with_environment_context_result { switch ($fid) { case 0: - if ($ftype == TType::STRUCT) { - $this->success = new \metastore\Partition(); - $xfer += $this->success->read($input); + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->success); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidObjectException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -10341,20 +13052,12 @@ class ThriftHiveMetastore_add_partition_with_environment_context_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\AlreadyExistsException(); + $this->o2 = new \metastore\MetaException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); - $xfer += $this->o3->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -10367,13 +13070,10 @@ class ThriftHiveMetastore_add_partition_with_environment_context_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_partition_with_environment_context_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_by_name_result'); if ($this->success !== null) { - if (!is_object($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); - $xfer += $this->success->write($output); + $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); + $xfer += $output->writeBool($this->success); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -10386,11 +13086,6 @@ class ThriftHiveMetastore_add_partition_with_environment_context_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -10398,34 +13093,62 @@ class ThriftHiveMetastore_add_partition_with_environment_context_result { } -class ThriftHiveMetastore_add_partitions_args { +class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args { static $_TSPEC; - public $new_parts = null; + public $db_name = null; + public $tbl_name = null; + public $part_name = null; + public $deleteData = null; + public $environment_context = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'new_parts', - 'type' => TType::LST, - 'etype' => TType::STRUCT, - 'elem' => array( - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', - ), + 'var' => 'db_name', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'tbl_name', + 'type' => TType::STRING, + ), + 3 => array( + 'var' => 'part_name', + 'type' => TType::STRING, + ), + 4 => array( + 'var' => 'deleteData', + 'type' => TType::BOOL, + ), + 5 => array( + 'var' => 'environment_context', + 'type' => TType::STRUCT, + 'class' => '\metastore\EnvironmentContext', ), ); } if (is_array($vals)) { - if (isset($vals['new_parts'])) { - $this->new_parts = $vals['new_parts']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; + } + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; + } + if (isset($vals['part_name'])) { + $this->part_name = $vals['part_name']; + } + if (isset($vals['deleteData'])) { + $this->deleteData = $vals['deleteData']; + } + if (isset($vals['environment_context'])) { + $this->environment_context = $vals['environment_context']; } } } public function getName() { - return 'ThriftHiveMetastore_add_partitions_args'; + return 'ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args'; } public function read($input) @@ -10444,19 +13167,37 @@ class ThriftHiveMetastore_add_partitions_args { switch ($fid) { case 1: - if ($ftype == TType::LST) { - $this->new_parts = array(); - $_size299 = 0; - $_etype302 = 0; - $xfer += $input->readListBegin($_etype302, $_size299); - for ($_i303 = 0; $_i303 < $_size299; ++$_i303) - { - $elem304 = null; - $elem304 = new \metastore\Partition(); - $xfer += $elem304->read($input); - $this->new_parts []= $elem304; - } - $xfer += $input->readListEnd(); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->db_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->tbl_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->part_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->deleteData); + } else { + $xfer += $input->skip($ftype); + } + break; + case 5: + if ($ftype == TType::STRUCT) { + $this->environment_context = new \metastore\EnvironmentContext(); + $xfer += $this->environment_context->read($input); } else { $xfer += $input->skip($ftype); } @@ -10473,22 +13214,33 @@ class ThriftHiveMetastore_add_partitions_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_partitions_args'); - if ($this->new_parts !== null) { - if (!is_array($this->new_parts)) { + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->part_name !== null) { + $xfer += $output->writeFieldBegin('part_name', TType::STRING, 3); + $xfer += $output->writeString($this->part_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->deleteData !== null) { + $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 4); + $xfer += $output->writeBool($this->deleteData); + $xfer += $output->writeFieldEnd(); + } + if ($this->environment_context !== null) { + if (!is_object($this->environment_context)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('new_parts', TType::LST, 1); - { - $output->writeListBegin(TType::STRUCT, count($this->new_parts)); - { - foreach ($this->new_parts as $iter305) - { - $xfer += $iter305->write($output); - } - } - $output->writeListEnd(); - } + $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 5); + $xfer += $this->environment_context->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -10498,34 +13250,28 @@ class ThriftHiveMetastore_add_partitions_args { } -class ThriftHiveMetastore_add_partitions_result { +class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; - public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::I32, + 'type' => TType::BOOL, ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', + 'class' => '\metastore\NoSuchObjectException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\AlreadyExistsException', - ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), ); @@ -10540,14 +13286,11 @@ class ThriftHiveMetastore_add_partitions_result { if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } } } public function getName() { - return 'ThriftHiveMetastore_add_partitions_result'; + return 'ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result'; } public function read($input) @@ -10566,15 +13309,15 @@ class ThriftHiveMetastore_add_partitions_result { switch ($fid) { case 0: - if ($ftype == TType::I32) { - $xfer += $input->readI32($this->success); + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->success); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidObjectException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -10582,20 +13325,12 @@ class ThriftHiveMetastore_add_partitions_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\AlreadyExistsException(); + $this->o2 = new \metastore\MetaException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); - $xfer += $this->o3->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -10608,10 +13343,10 @@ class ThriftHiveMetastore_add_partitions_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_partitions_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result'); if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::I32, 0); - $xfer += $output->writeI32($this->success); + $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); + $xfer += $output->writeBool($this->success); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -10624,11 +13359,6 @@ class ThriftHiveMetastore_add_partitions_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -10636,7 +13366,7 @@ class ThriftHiveMetastore_add_partitions_result { } -class ThriftHiveMetastore_append_partition_args { +class ThriftHiveMetastore_get_partition_args { static $_TSPEC; public $db_name = null; @@ -10678,7 +13408,7 @@ class ThriftHiveMetastore_append_partition_args { } public function getName() { - return 'ThriftHiveMetastore_append_partition_args'; + return 'ThriftHiveMetastore_get_partition_args'; } public function read($input) @@ -10713,14 +13443,14 @@ class ThriftHiveMetastore_append_partition_args { case 3: if ($ftype == TType::LST) { $this->part_vals = array(); - $_size306 = 0; - $_etype309 = 0; - $xfer += $input->readListBegin($_etype309, $_size306); - for ($_i310 = 0; $_i310 < $_size306; ++$_i310) + $_size334 = 0; + $_etype337 = 0; + $xfer += $input->readListBegin($_etype337, $_size334); + for ($_i338 = 0; $_i338 < $_size334; ++$_i338) { - $elem311 = null; - $xfer += $input->readString($elem311); - $this->part_vals []= $elem311; + $elem339 = null; + $xfer += $input->readString($elem339); + $this->part_vals []= $elem339; } $xfer += $input->readListEnd(); } else { @@ -10739,7 +13469,7 @@ class ThriftHiveMetastore_append_partition_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -10758,9 +13488,9 @@ class ThriftHiveMetastore_append_partition_args { { $output->writeListBegin(TType::STRING, count($this->part_vals)); { - foreach ($this->part_vals as $iter312) + foreach ($this->part_vals as $iter340) { - $xfer += $output->writeString($iter312); + $xfer += $output->writeString($iter340); } } $output->writeListEnd(); @@ -10774,13 +13504,12 @@ class ThriftHiveMetastore_append_partition_args { } -class ThriftHiveMetastore_append_partition_result { +class ThriftHiveMetastore_get_partition_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; - public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -10793,17 +13522,12 @@ class ThriftHiveMetastore_append_partition_result { 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', + 'class' => '\metastore\MetaException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\AlreadyExistsException', - ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchObjectException', ), ); } @@ -10817,14 +13541,11 @@ class ThriftHiveMetastore_append_partition_result { if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } } } public function getName() { - return 'ThriftHiveMetastore_append_partition_result'; + return 'ThriftHiveMetastore_get_partition_result'; } public function read($input) @@ -10852,7 +13573,7 @@ class ThriftHiveMetastore_append_partition_result { break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidObjectException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -10860,20 +13581,12 @@ class ThriftHiveMetastore_append_partition_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\AlreadyExistsException(); + $this->o2 = new \metastore\NoSuchObjectException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); - $xfer += $this->o3->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -10886,7 +13599,7 @@ class ThriftHiveMetastore_append_partition_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_result'); if ($this->success !== null) { if (!is_object($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); @@ -10905,11 +13618,6 @@ class ThriftHiveMetastore_append_partition_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -10917,58 +13625,69 @@ class ThriftHiveMetastore_append_partition_result { } -class ThriftHiveMetastore_append_partition_with_environment_context_args { +class ThriftHiveMetastore_exchange_partition_args { static $_TSPEC; - public $db_name = null; - public $tbl_name = null; - public $part_vals = null; - public $environment_context = null; + public $partitionSpecs = null; + public $source_db = null; + public $source_table_name = null; + public $dest_db = null; + public $dest_table_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', - 'type' => TType::STRING, + 'var' => 'partitionSpecs', + 'type' => TType::MAP, + 'ktype' => TType::STRING, + 'vtype' => TType::STRING, + 'key' => array( + 'type' => TType::STRING, + ), + 'val' => array( + 'type' => TType::STRING, + ), ), 2 => array( - 'var' => 'tbl_name', + 'var' => 'source_db', 'type' => TType::STRING, ), 3 => array( - 'var' => 'part_vals', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), + 'var' => 'source_table_name', + 'type' => TType::STRING, ), 4 => array( - 'var' => 'environment_context', - 'type' => TType::STRUCT, - 'class' => '\metastore\EnvironmentContext', + 'var' => 'dest_db', + 'type' => TType::STRING, + ), + 5 => array( + 'var' => 'dest_table_name', + 'type' => TType::STRING, ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; + if (isset($vals['partitionSpecs'])) { + $this->partitionSpecs = $vals['partitionSpecs']; } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; + if (isset($vals['source_db'])) { + $this->source_db = $vals['source_db']; } - if (isset($vals['part_vals'])) { - $this->part_vals = $vals['part_vals']; + if (isset($vals['source_table_name'])) { + $this->source_table_name = $vals['source_table_name']; } - if (isset($vals['environment_context'])) { - $this->environment_context = $vals['environment_context']; + if (isset($vals['dest_db'])) { + $this->dest_db = $vals['dest_db']; + } + if (isset($vals['dest_table_name'])) { + $this->dest_table_name = $vals['dest_table_name']; } } } public function getName() { - return 'ThriftHiveMetastore_append_partition_with_environment_context_args'; + return 'ThriftHiveMetastore_exchange_partition_args'; } public function read($input) @@ -10987,40 +13706,49 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); + if ($ftype == TType::MAP) { + $this->partitionSpecs = array(); + $_size341 = 0; + $_ktype342 = 0; + $_vtype343 = 0; + $xfer += $input->readMapBegin($_ktype342, $_vtype343, $_size341); + for ($_i345 = 0; $_i345 < $_size341; ++$_i345) + { + $key346 = ''; + $val347 = ''; + $xfer += $input->readString($key346); + $xfer += $input->readString($val347); + $this->partitionSpecs[$key346] = $val347; + } + $xfer += $input->readMapEnd(); } else { $xfer += $input->skip($ftype); } break; case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); + $xfer += $input->readString($this->source_db); } else { $xfer += $input->skip($ftype); } break; case 3: - if ($ftype == TType::LST) { - $this->part_vals = array(); - $_size313 = 0; - $_etype316 = 0; - $xfer += $input->readListBegin($_etype316, $_size313); - for ($_i317 = 0; $_i317 < $_size313; ++$_i317) - { - $elem318 = null; - $xfer += $input->readString($elem318); - $this->part_vals []= $elem318; - } - $xfer += $input->readListEnd(); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->source_table_name); } else { $xfer += $input->skip($ftype); } break; case 4: - if ($ftype == TType::STRUCT) { - $this->environment_context = new \metastore\EnvironmentContext(); - $xfer += $this->environment_context->read($input); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->dest_db); + } else { + $xfer += $input->skip($ftype); + } + break; + case 5: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->dest_table_name); } else { $xfer += $input->skip($ftype); } @@ -11037,40 +13765,43 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_with_environment_context_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->part_vals !== null) { - if (!is_array($this->part_vals)) { + $xfer += $output->writeStructBegin('ThriftHiveMetastore_exchange_partition_args'); + if ($this->partitionSpecs !== null) { + if (!is_array($this->partitionSpecs)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); + $xfer += $output->writeFieldBegin('partitionSpecs', TType::MAP, 1); { - $output->writeListBegin(TType::STRING, count($this->part_vals)); + $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs)); { - foreach ($this->part_vals as $iter319) + foreach ($this->partitionSpecs as $kiter348 => $viter349) { - $xfer += $output->writeString($iter319); + $xfer += $output->writeString($kiter348); + $xfer += $output->writeString($viter349); } } - $output->writeListEnd(); + $output->writeMapEnd(); } $xfer += $output->writeFieldEnd(); } - if ($this->environment_context !== null) { - if (!is_object($this->environment_context)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 4); - $xfer += $this->environment_context->write($output); + if ($this->source_db !== null) { + $xfer += $output->writeFieldBegin('source_db', TType::STRING, 2); + $xfer += $output->writeString($this->source_db); + $xfer += $output->writeFieldEnd(); + } + if ($this->source_table_name !== null) { + $xfer += $output->writeFieldBegin('source_table_name', TType::STRING, 3); + $xfer += $output->writeString($this->source_table_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->dest_db !== null) { + $xfer += $output->writeFieldBegin('dest_db', TType::STRING, 4); + $xfer += $output->writeString($this->dest_db); + $xfer += $output->writeFieldEnd(); + } + if ($this->dest_table_name !== null) { + $xfer += $output->writeFieldBegin('dest_table_name', TType::STRING, 5); + $xfer += $output->writeString($this->dest_table_name); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -11080,13 +13811,14 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args { } -class ThriftHiveMetastore_append_partition_with_environment_context_result { +class ThriftHiveMetastore_exchange_partition_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; public $o3 = null; + public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -11099,17 +13831,22 @@ class ThriftHiveMetastore_append_partition_with_environment_context_result { 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', + 'class' => '\metastore\MetaException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\AlreadyExistsException', + 'class' => '\metastore\NoSuchObjectException', ), 3 => array( 'var' => 'o3', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\InvalidObjectException', + ), + 4 => array( + 'var' => 'o4', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidInputException', ), ); } @@ -11126,11 +13863,14 @@ class ThriftHiveMetastore_append_partition_with_environment_context_result { if (isset($vals['o3'])) { $this->o3 = $vals['o3']; } + if (isset($vals['o4'])) { + $this->o4 = $vals['o4']; + } } } public function getName() { - return 'ThriftHiveMetastore_append_partition_with_environment_context_result'; + return 'ThriftHiveMetastore_exchange_partition_result'; } public function read($input) @@ -11158,7 +13898,7 @@ class ThriftHiveMetastore_append_partition_with_environment_context_result { break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidObjectException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -11166,7 +13906,7 @@ class ThriftHiveMetastore_append_partition_with_environment_context_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\AlreadyExistsException(); + $this->o2 = new \metastore\NoSuchObjectException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -11174,12 +13914,20 @@ class ThriftHiveMetastore_append_partition_with_environment_context_result { break; case 3: if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); + $this->o3 = new \metastore\InvalidObjectException(); $xfer += $this->o3->read($input); } else { $xfer += $input->skip($ftype); } break; + case 4: + if ($ftype == TType::STRUCT) { + $this->o4 = new \metastore\InvalidInputException(); + $xfer += $this->o4->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -11192,7 +13940,7 @@ class ThriftHiveMetastore_append_partition_with_environment_context_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_with_environment_context_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_exchange_partition_result'); if ($this->success !== null) { if (!is_object($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); @@ -11216,6 +13964,11 @@ class ThriftHiveMetastore_append_partition_with_environment_context_result { $xfer += $this->o3->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o4 !== null) { + $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); + $xfer += $this->o4->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -11223,12 +13976,14 @@ class ThriftHiveMetastore_append_partition_with_environment_context_result { } -class ThriftHiveMetastore_append_partition_by_name_args { +class ThriftHiveMetastore_get_partition_with_auth_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; - public $part_name = null; + public $part_vals = null; + public $user_name = null; + public $group_names = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -11242,9 +13997,25 @@ class ThriftHiveMetastore_append_partition_by_name_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'part_name', + 'var' => 'part_vals', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), + ), + 4 => array( + 'var' => 'user_name', 'type' => TType::STRING, ), + 5 => array( + 'var' => 'group_names', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), + ), ); } if (is_array($vals)) { @@ -11254,14 +14025,20 @@ class ThriftHiveMetastore_append_partition_by_name_args { if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['part_name'])) { - $this->part_name = $vals['part_name']; + if (isset($vals['part_vals'])) { + $this->part_vals = $vals['part_vals']; + } + if (isset($vals['user_name'])) { + $this->user_name = $vals['user_name']; + } + if (isset($vals['group_names'])) { + $this->group_names = $vals['group_names']; } } } public function getName() { - return 'ThriftHiveMetastore_append_partition_by_name_args'; + return 'ThriftHiveMetastore_get_partition_with_auth_args'; } public function read($input) @@ -11294,8 +14071,42 @@ class ThriftHiveMetastore_append_partition_by_name_args { } break; case 3: + if ($ftype == TType::LST) { + $this->part_vals = array(); + $_size350 = 0; + $_etype353 = 0; + $xfer += $input->readListBegin($_etype353, $_size350); + for ($_i354 = 0; $_i354 < $_size350; ++$_i354) + { + $elem355 = null; + $xfer += $input->readString($elem355); + $this->part_vals []= $elem355; + } + $xfer += $input->readListEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->part_name); + $xfer += $input->readString($this->user_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 5: + if ($ftype == TType::LST) { + $this->group_names = array(); + $_size356 = 0; + $_etype359 = 0; + $xfer += $input->readListBegin($_etype359, $_size356); + for ($_i360 = 0; $_i360 < $_size356; ++$_i360) + { + $elem361 = null; + $xfer += $input->readString($elem361); + $this->group_names []= $elem361; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } @@ -11312,7 +14123,7 @@ class ThriftHiveMetastore_append_partition_by_name_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_by_name_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_with_auth_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -11323,9 +14134,43 @@ class ThriftHiveMetastore_append_partition_by_name_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->part_name !== null) { - $xfer += $output->writeFieldBegin('part_name', TType::STRING, 3); - $xfer += $output->writeString($this->part_name); + if ($this->part_vals !== null) { + if (!is_array($this->part_vals)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); + { + $output->writeListBegin(TType::STRING, count($this->part_vals)); + { + foreach ($this->part_vals as $iter362) + { + $xfer += $output->writeString($iter362); + } + } + $output->writeListEnd(); + } + $xfer += $output->writeFieldEnd(); + } + if ($this->user_name !== null) { + $xfer += $output->writeFieldBegin('user_name', TType::STRING, 4); + $xfer += $output->writeString($this->user_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->group_names !== null) { + if (!is_array($this->group_names)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('group_names', TType::LST, 5); + { + $output->writeListBegin(TType::STRING, count($this->group_names)); + { + foreach ($this->group_names as $iter363) + { + $xfer += $output->writeString($iter363); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -11335,13 +14180,12 @@ class ThriftHiveMetastore_append_partition_by_name_args { } -class ThriftHiveMetastore_append_partition_by_name_result { +class ThriftHiveMetastore_get_partition_with_auth_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; - public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -11354,17 +14198,12 @@ class ThriftHiveMetastore_append_partition_by_name_result { 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', + 'class' => '\metastore\MetaException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\AlreadyExistsException', - ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchObjectException', ), ); } @@ -11378,14 +14217,11 @@ class ThriftHiveMetastore_append_partition_by_name_result { if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } } } public function getName() { - return 'ThriftHiveMetastore_append_partition_by_name_result'; + return 'ThriftHiveMetastore_get_partition_with_auth_result'; } public function read($input) @@ -11413,7 +14249,7 @@ class ThriftHiveMetastore_append_partition_by_name_result { break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidObjectException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -11421,20 +14257,12 @@ class ThriftHiveMetastore_append_partition_by_name_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\AlreadyExistsException(); + $this->o2 = new \metastore\NoSuchObjectException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); - $xfer += $this->o3->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -11447,7 +14275,7 @@ class ThriftHiveMetastore_append_partition_by_name_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_by_name_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_with_auth_result'); if ($this->success !== null) { if (!is_object($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); @@ -11466,11 +14294,6 @@ class ThriftHiveMetastore_append_partition_by_name_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -11478,13 +14301,12 @@ class ThriftHiveMetastore_append_partition_by_name_result { } -class ThriftHiveMetastore_append_partition_by_name_with_environment_context_args { +class ThriftHiveMetastore_get_partition_by_name_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; public $part_name = null; - public $environment_context = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -11501,11 +14323,6 @@ class ThriftHiveMetastore_append_partition_by_name_with_environment_context_args 'var' => 'part_name', 'type' => TType::STRING, ), - 4 => array( - 'var' => 'environment_context', - 'type' => TType::STRUCT, - 'class' => '\metastore\EnvironmentContext', - ), ); } if (is_array($vals)) { @@ -11518,14 +14335,11 @@ class ThriftHiveMetastore_append_partition_by_name_with_environment_context_args if (isset($vals['part_name'])) { $this->part_name = $vals['part_name']; } - if (isset($vals['environment_context'])) { - $this->environment_context = $vals['environment_context']; - } } } public function getName() { - return 'ThriftHiveMetastore_append_partition_by_name_with_environment_context_args'; + return 'ThriftHiveMetastore_get_partition_by_name_args'; } public function read($input) @@ -11564,14 +14378,6 @@ class ThriftHiveMetastore_append_partition_by_name_with_environment_context_args $xfer += $input->skip($ftype); } break; - case 4: - if ($ftype == TType::STRUCT) { - $this->environment_context = new \metastore\EnvironmentContext(); - $xfer += $this->environment_context->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -11584,7 +14390,7 @@ class ThriftHiveMetastore_append_partition_by_name_with_environment_context_args public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_by_name_with_environment_context_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_by_name_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -11600,14 +14406,6 @@ class ThriftHiveMetastore_append_partition_by_name_with_environment_context_args $xfer += $output->writeString($this->part_name); $xfer += $output->writeFieldEnd(); } - if ($this->environment_context !== null) { - if (!is_object($this->environment_context)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 4); - $xfer += $this->environment_context->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -11615,13 +14413,12 @@ class ThriftHiveMetastore_append_partition_by_name_with_environment_context_args } -class ThriftHiveMetastore_append_partition_by_name_with_environment_context_result { +class ThriftHiveMetastore_get_partition_by_name_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; - public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -11634,17 +14431,12 @@ class ThriftHiveMetastore_append_partition_by_name_with_environment_context_resu 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', + 'class' => '\metastore\MetaException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\AlreadyExistsException', - ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchObjectException', ), ); } @@ -11658,14 +14450,11 @@ class ThriftHiveMetastore_append_partition_by_name_with_environment_context_resu if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } } } public function getName() { - return 'ThriftHiveMetastore_append_partition_by_name_with_environment_context_result'; + return 'ThriftHiveMetastore_get_partition_by_name_result'; } public function read($input) @@ -11693,7 +14482,7 @@ class ThriftHiveMetastore_append_partition_by_name_with_environment_context_resu break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidObjectException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -11701,20 +14490,12 @@ class ThriftHiveMetastore_append_partition_by_name_with_environment_context_resu break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\AlreadyExistsException(); + $this->o2 = new \metastore\NoSuchObjectException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); - $xfer += $this->o3->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -11727,7 +14508,7 @@ class ThriftHiveMetastore_append_partition_by_name_with_environment_context_resu public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_by_name_with_environment_context_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_by_name_result'); if ($this->success !== null) { if (!is_object($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); @@ -11746,11 +14527,6 @@ class ThriftHiveMetastore_append_partition_by_name_with_environment_context_resu $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -11758,13 +14534,12 @@ class ThriftHiveMetastore_append_partition_by_name_with_environment_context_resu } -class ThriftHiveMetastore_drop_partition_args { +class ThriftHiveMetastore_get_partitions_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; - public $part_vals = null; - public $deleteData = null; + public $max_parts = -1; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -11778,16 +14553,8 @@ class ThriftHiveMetastore_drop_partition_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'part_vals', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), - ), - 4 => array( - 'var' => 'deleteData', - 'type' => TType::BOOL, + 'var' => 'max_parts', + 'type' => TType::I16, ), ); } @@ -11798,17 +14565,14 @@ class ThriftHiveMetastore_drop_partition_args { if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['part_vals'])) { - $this->part_vals = $vals['part_vals']; - } - if (isset($vals['deleteData'])) { - $this->deleteData = $vals['deleteData']; + if (isset($vals['max_parts'])) { + $this->max_parts = $vals['max_parts']; } } } public function getName() { - return 'ThriftHiveMetastore_drop_partition_args'; + return 'ThriftHiveMetastore_get_partitions_args'; } public function read($input) @@ -11841,25 +14605,8 @@ class ThriftHiveMetastore_drop_partition_args { } break; case 3: - if ($ftype == TType::LST) { - $this->part_vals = array(); - $_size320 = 0; - $_etype323 = 0; - $xfer += $input->readListBegin($_etype323, $_size320); - for ($_i324 = 0; $_i324 < $_size320; ++$_i324) - { - $elem325 = null; - $xfer += $input->readString($elem325); - $this->part_vals []= $elem325; - } - $xfer += $input->readListEnd(); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->deleteData); + if ($ftype == TType::I16) { + $xfer += $input->readI16($this->max_parts); } else { $xfer += $input->skip($ftype); } @@ -11876,37 +14623,20 @@ class ThriftHiveMetastore_drop_partition_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); $xfer += $output->writeFieldEnd(); - } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->part_vals !== null) { - if (!is_array($this->part_vals)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); - { - $output->writeListBegin(TType::STRING, count($this->part_vals)); - { - foreach ($this->part_vals as $iter326) - { - $xfer += $output->writeString($iter326); - } - } - $output->writeListEnd(); - } + } + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->deleteData !== null) { - $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 4); - $xfer += $output->writeBool($this->deleteData); + if ($this->max_parts !== null) { + $xfer += $output->writeFieldBegin('max_parts', TType::I16, 3); + $xfer += $output->writeI16($this->max_parts); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -11916,7 +14646,7 @@ class ThriftHiveMetastore_drop_partition_args { } -class ThriftHiveMetastore_drop_partition_result { +class ThriftHiveMetastore_get_partitions_result { static $_TSPEC; public $success = null; @@ -11928,7 +14658,12 @@ class ThriftHiveMetastore_drop_partition_result { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::BOOL, + 'type' => TType::LST, + 'etype' => TType::STRUCT, + 'elem' => array( + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', + ), ), 1 => array( 'var' => 'o1', @@ -11956,7 +14691,7 @@ class ThriftHiveMetastore_drop_partition_result { } public function getName() { - return 'ThriftHiveMetastore_drop_partition_result'; + return 'ThriftHiveMetastore_get_partitions_result'; } public function read($input) @@ -11975,8 +14710,19 @@ class ThriftHiveMetastore_drop_partition_result { switch ($fid) { case 0: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->success); + if ($ftype == TType::LST) { + $this->success = array(); + $_size364 = 0; + $_etype367 = 0; + $xfer += $input->readListBegin($_etype367, $_size364); + for ($_i368 = 0; $_i368 < $_size364; ++$_i368) + { + $elem369 = null; + $elem369 = new \metastore\Partition(); + $xfer += $elem369->read($input); + $this->success []= $elem369; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } @@ -12009,10 +14755,22 @@ class ThriftHiveMetastore_drop_partition_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_result'); if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); - $xfer += $output->writeBool($this->success); + if (!is_array($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRUCT, count($this->success)); + { + foreach ($this->success as $iter370) + { + $xfer += $iter370->write($output); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -12032,14 +14790,14 @@ class ThriftHiveMetastore_drop_partition_result { } -class ThriftHiveMetastore_drop_partition_with_environment_context_args { +class ThriftHiveMetastore_get_partitions_with_auth_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; - public $part_vals = null; - public $deleteData = null; - public $environment_context = null; + public $max_parts = -1; + public $user_name = null; + public $group_names = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -12053,22 +14811,21 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'part_vals', + 'var' => 'max_parts', + 'type' => TType::I16, + ), + 4 => array( + 'var' => 'user_name', + 'type' => TType::STRING, + ), + 5 => array( + 'var' => 'group_names', 'type' => TType::LST, 'etype' => TType::STRING, 'elem' => array( 'type' => TType::STRING, ), ), - 4 => array( - 'var' => 'deleteData', - 'type' => TType::BOOL, - ), - 5 => array( - 'var' => 'environment_context', - 'type' => TType::STRUCT, - 'class' => '\metastore\EnvironmentContext', - ), ); } if (is_array($vals)) { @@ -12078,20 +14835,20 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args { if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['part_vals'])) { - $this->part_vals = $vals['part_vals']; + if (isset($vals['max_parts'])) { + $this->max_parts = $vals['max_parts']; } - if (isset($vals['deleteData'])) { - $this->deleteData = $vals['deleteData']; + if (isset($vals['user_name'])) { + $this->user_name = $vals['user_name']; } - if (isset($vals['environment_context'])) { - $this->environment_context = $vals['environment_context']; + if (isset($vals['group_names'])) { + $this->group_names = $vals['group_names']; } } } public function getName() { - return 'ThriftHiveMetastore_drop_partition_with_environment_context_args'; + return 'ThriftHiveMetastore_get_partitions_with_auth_args'; } public function read($input) @@ -12124,33 +14881,32 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args { } break; case 3: - if ($ftype == TType::LST) { - $this->part_vals = array(); - $_size327 = 0; - $_etype330 = 0; - $xfer += $input->readListBegin($_etype330, $_size327); - for ($_i331 = 0; $_i331 < $_size327; ++$_i331) - { - $elem332 = null; - $xfer += $input->readString($elem332); - $this->part_vals []= $elem332; - } - $xfer += $input->readListEnd(); + if ($ftype == TType::I16) { + $xfer += $input->readI16($this->max_parts); } else { $xfer += $input->skip($ftype); } break; case 4: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->deleteData); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->user_name); } else { $xfer += $input->skip($ftype); } break; case 5: - if ($ftype == TType::STRUCT) { - $this->environment_context = new \metastore\EnvironmentContext(); - $xfer += $this->environment_context->read($input); + if ($ftype == TType::LST) { + $this->group_names = array(); + $_size371 = 0; + $_etype374 = 0; + $xfer += $input->readListBegin($_etype374, $_size371); + for ($_i375 = 0; $_i375 < $_size371; ++$_i375) + { + $elem376 = null; + $xfer += $input->readString($elem376); + $this->group_names []= $elem376; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } @@ -12167,7 +14923,7 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_with_environment_context_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_with_auth_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -12178,36 +14934,33 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->part_vals !== null) { - if (!is_array($this->part_vals)) { + if ($this->max_parts !== null) { + $xfer += $output->writeFieldBegin('max_parts', TType::I16, 3); + $xfer += $output->writeI16($this->max_parts); + $xfer += $output->writeFieldEnd(); + } + if ($this->user_name !== null) { + $xfer += $output->writeFieldBegin('user_name', TType::STRING, 4); + $xfer += $output->writeString($this->user_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->group_names !== null) { + if (!is_array($this->group_names)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); + $xfer += $output->writeFieldBegin('group_names', TType::LST, 5); { - $output->writeListBegin(TType::STRING, count($this->part_vals)); + $output->writeListBegin(TType::STRING, count($this->group_names)); { - foreach ($this->part_vals as $iter333) + foreach ($this->group_names as $iter377) { - $xfer += $output->writeString($iter333); + $xfer += $output->writeString($iter377); } } $output->writeListEnd(); } $xfer += $output->writeFieldEnd(); } - if ($this->deleteData !== null) { - $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 4); - $xfer += $output->writeBool($this->deleteData); - $xfer += $output->writeFieldEnd(); - } - if ($this->environment_context !== null) { - if (!is_object($this->environment_context)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 5); - $xfer += $this->environment_context->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -12215,7 +14968,7 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args { } -class ThriftHiveMetastore_drop_partition_with_environment_context_result { +class ThriftHiveMetastore_get_partitions_with_auth_result { static $_TSPEC; public $success = null; @@ -12227,7 +14980,12 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_result { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::BOOL, + 'type' => TType::LST, + 'etype' => TType::STRUCT, + 'elem' => array( + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', + ), ), 1 => array( 'var' => 'o1', @@ -12255,7 +15013,7 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_result { } public function getName() { - return 'ThriftHiveMetastore_drop_partition_with_environment_context_result'; + return 'ThriftHiveMetastore_get_partitions_with_auth_result'; } public function read($input) @@ -12274,8 +15032,19 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_result { switch ($fid) { case 0: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->success); + if ($ftype == TType::LST) { + $this->success = array(); + $_size378 = 0; + $_etype381 = 0; + $xfer += $input->readListBegin($_etype381, $_size378); + for ($_i382 = 0; $_i382 < $_size378; ++$_i382) + { + $elem383 = null; + $elem383 = new \metastore\Partition(); + $xfer += $elem383->read($input); + $this->success []= $elem383; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } @@ -12308,10 +15077,22 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_with_environment_context_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_with_auth_result'); if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); - $xfer += $output->writeBool($this->success); + if (!is_array($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRUCT, count($this->success)); + { + foreach ($this->success as $iter384) + { + $xfer += $iter384->write($output); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -12331,13 +15112,12 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_result { } -class ThriftHiveMetastore_drop_partition_by_name_args { +class ThriftHiveMetastore_get_partition_names_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; - public $part_name = null; - public $deleteData = null; + public $max_parts = -1; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -12351,12 +15131,8 @@ class ThriftHiveMetastore_drop_partition_by_name_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'part_name', - 'type' => TType::STRING, - ), - 4 => array( - 'var' => 'deleteData', - 'type' => TType::BOOL, + 'var' => 'max_parts', + 'type' => TType::I16, ), ); } @@ -12367,17 +15143,14 @@ class ThriftHiveMetastore_drop_partition_by_name_args { if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['part_name'])) { - $this->part_name = $vals['part_name']; - } - if (isset($vals['deleteData'])) { - $this->deleteData = $vals['deleteData']; + if (isset($vals['max_parts'])) { + $this->max_parts = $vals['max_parts']; } } } public function getName() { - return 'ThriftHiveMetastore_drop_partition_by_name_args'; + return 'ThriftHiveMetastore_get_partition_names_args'; } public function read($input) @@ -12410,15 +15183,8 @@ class ThriftHiveMetastore_drop_partition_by_name_args { } break; case 3: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->part_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->deleteData); + if ($ftype == TType::I16) { + $xfer += $input->readI16($this->max_parts); } else { $xfer += $input->skip($ftype); } @@ -12435,7 +15201,7 @@ class ThriftHiveMetastore_drop_partition_by_name_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_by_name_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_names_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -12446,14 +15212,9 @@ class ThriftHiveMetastore_drop_partition_by_name_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->part_name !== null) { - $xfer += $output->writeFieldBegin('part_name', TType::STRING, 3); - $xfer += $output->writeString($this->part_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->deleteData !== null) { - $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 4); - $xfer += $output->writeBool($this->deleteData); + if ($this->max_parts !== null) { + $xfer += $output->writeFieldBegin('max_parts', TType::I16, 3); + $xfer += $output->writeI16($this->max_parts); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -12463,11 +15224,10 @@ class ThriftHiveMetastore_drop_partition_by_name_args { } -class ThriftHiveMetastore_drop_partition_by_name_result { +class ThriftHiveMetastore_get_partition_names_result { static $_TSPEC; public $success = null; - public $o1 = null; public $o2 = null; public function __construct($vals=null) { @@ -12475,14 +15235,13 @@ class ThriftHiveMetastore_drop_partition_by_name_result { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::BOOL, + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), ), 1 => array( - 'var' => 'o1', - 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', - ), - 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', @@ -12493,9 +15252,6 @@ class ThriftHiveMetastore_drop_partition_by_name_result { if (isset($vals['success'])) { $this->success = $vals['success']; } - if (isset($vals['o1'])) { - $this->o1 = $vals['o1']; - } if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } @@ -12503,7 +15259,7 @@ class ThriftHiveMetastore_drop_partition_by_name_result { } public function getName() { - return 'ThriftHiveMetastore_drop_partition_by_name_result'; + return 'ThriftHiveMetastore_get_partition_names_result'; } public function read($input) @@ -12522,22 +15278,24 @@ class ThriftHiveMetastore_drop_partition_by_name_result { switch ($fid) { case 0: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->success); + if ($ftype == TType::LST) { + $this->success = array(); + $_size385 = 0; + $_etype388 = 0; + $xfer += $input->readListBegin($_etype388, $_size385); + for ($_i389 = 0; $_i389 < $_size385; ++$_i389) + { + $elem390 = null; + $xfer += $input->readString($elem390); + $this->success []= $elem390; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); - $xfer += $this->o1->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRUCT) { $this->o2 = new \metastore\MetaException(); $xfer += $this->o2->read($input); } else { @@ -12556,19 +15314,26 @@ class ThriftHiveMetastore_drop_partition_by_name_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_by_name_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_names_result'); if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); - $xfer += $output->writeBool($this->success); - $xfer += $output->writeFieldEnd(); - } - if ($this->o1 !== null) { - $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); - $xfer += $this->o1->write($output); + if (!is_array($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRING, count($this->success)); + { + foreach ($this->success as $iter391) + { + $xfer += $output->writeString($iter391); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 1); $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } @@ -12579,14 +15344,13 @@ class ThriftHiveMetastore_drop_partition_by_name_result { } -class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args { +class ThriftHiveMetastore_get_partitions_ps_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; - public $part_name = null; - public $deleteData = null; - public $environment_context = null; + public $part_vals = null; + public $max_parts = -1; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -12600,17 +15364,16 @@ class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'part_name', - 'type' => TType::STRING, + 'var' => 'part_vals', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), ), 4 => array( - 'var' => 'deleteData', - 'type' => TType::BOOL, - ), - 5 => array( - 'var' => 'environment_context', - 'type' => TType::STRUCT, - 'class' => '\metastore\EnvironmentContext', + 'var' => 'max_parts', + 'type' => TType::I16, ), ); } @@ -12621,20 +15384,17 @@ class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args { if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['part_name'])) { - $this->part_name = $vals['part_name']; - } - if (isset($vals['deleteData'])) { - $this->deleteData = $vals['deleteData']; + if (isset($vals['part_vals'])) { + $this->part_vals = $vals['part_vals']; } - if (isset($vals['environment_context'])) { - $this->environment_context = $vals['environment_context']; + if (isset($vals['max_parts'])) { + $this->max_parts = $vals['max_parts']; } } } public function getName() { - return 'ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args'; + return 'ThriftHiveMetastore_get_partitions_ps_args'; } public function read($input) @@ -12667,23 +15427,25 @@ class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args { } break; case 3: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->part_name); + if ($ftype == TType::LST) { + $this->part_vals = array(); + $_size392 = 0; + $_etype395 = 0; + $xfer += $input->readListBegin($_etype395, $_size392); + for ($_i396 = 0; $_i396 < $_size392; ++$_i396) + { + $elem397 = null; + $xfer += $input->readString($elem397); + $this->part_vals []= $elem397; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } break; case 4: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->deleteData); - } else { - $xfer += $input->skip($ftype); - } - break; - case 5: - if ($ftype == TType::STRUCT) { - $this->environment_context = new \metastore\EnvironmentContext(); - $xfer += $this->environment_context->read($input); + if ($ftype == TType::I16) { + $xfer += $input->readI16($this->max_parts); } else { $xfer += $input->skip($ftype); } @@ -12700,7 +15462,7 @@ class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_ps_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -12711,22 +15473,26 @@ class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->part_name !== null) { - $xfer += $output->writeFieldBegin('part_name', TType::STRING, 3); - $xfer += $output->writeString($this->part_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->deleteData !== null) { - $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 4); - $xfer += $output->writeBool($this->deleteData); - $xfer += $output->writeFieldEnd(); - } - if ($this->environment_context !== null) { - if (!is_object($this->environment_context)) { + if ($this->part_vals !== null) { + if (!is_array($this->part_vals)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 5); - $xfer += $this->environment_context->write($output); + $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); + { + $output->writeListBegin(TType::STRING, count($this->part_vals)); + { + foreach ($this->part_vals as $iter398) + { + $xfer += $output->writeString($iter398); + } + } + $output->writeListEnd(); + } + $xfer += $output->writeFieldEnd(); + } + if ($this->max_parts !== null) { + $xfer += $output->writeFieldBegin('max_parts', TType::I16, 4); + $xfer += $output->writeI16($this->max_parts); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -12736,7 +15502,7 @@ class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args { } -class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result { +class ThriftHiveMetastore_get_partitions_ps_result { static $_TSPEC; public $success = null; @@ -12748,17 +15514,22 @@ class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::BOOL, + 'type' => TType::LST, + 'etype' => TType::STRUCT, + 'elem' => array( + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', + ), ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\MetaException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchObjectException', ), ); } @@ -12776,7 +15547,7 @@ class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result } public function getName() { - return 'ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result'; + return 'ThriftHiveMetastore_get_partitions_ps_result'; } public function read($input) @@ -12795,15 +15566,26 @@ class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result switch ($fid) { case 0: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->success); + if ($ftype == TType::LST) { + $this->success = array(); + $_size399 = 0; + $_etype402 = 0; + $xfer += $input->readListBegin($_etype402, $_size399); + for ($_i403 = 0; $_i403 < $_size399; ++$_i403) + { + $elem404 = null; + $elem404 = new \metastore\Partition(); + $xfer += $elem404->read($input); + $this->success []= $elem404; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -12811,7 +15593,7 @@ class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); + $this->o2 = new \metastore\NoSuchObjectException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -12829,10 +15611,22 @@ class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_ps_result'); if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); - $xfer += $output->writeBool($this->success); + if (!is_array($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRUCT, count($this->success)); + { + foreach ($this->success as $iter405) + { + $xfer += $iter405->write($output); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -12852,12 +15646,15 @@ class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result } -class ThriftHiveMetastore_get_partition_args { +class ThriftHiveMetastore_get_partitions_ps_with_auth_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; public $part_vals = null; + public $max_parts = -1; + public $user_name = null; + public $group_names = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -12878,6 +15675,22 @@ class ThriftHiveMetastore_get_partition_args { 'type' => TType::STRING, ), ), + 4 => array( + 'var' => 'max_parts', + 'type' => TType::I16, + ), + 5 => array( + 'var' => 'user_name', + 'type' => TType::STRING, + ), + 6 => array( + 'var' => 'group_names', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), + ), ); } if (is_array($vals)) { @@ -12890,11 +15703,20 @@ class ThriftHiveMetastore_get_partition_args { if (isset($vals['part_vals'])) { $this->part_vals = $vals['part_vals']; } + if (isset($vals['max_parts'])) { + $this->max_parts = $vals['max_parts']; + } + if (isset($vals['user_name'])) { + $this->user_name = $vals['user_name']; + } + if (isset($vals['group_names'])) { + $this->group_names = $vals['group_names']; + } } } public function getName() { - return 'ThriftHiveMetastore_get_partition_args'; + return 'ThriftHiveMetastore_get_partitions_ps_with_auth_args'; } public function read($input) @@ -12919,24 +15741,55 @@ class ThriftHiveMetastore_get_partition_args { $xfer += $input->skip($ftype); } break; - case 2: + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->tbl_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::LST) { + $this->part_vals = array(); + $_size406 = 0; + $_etype409 = 0; + $xfer += $input->readListBegin($_etype409, $_size406); + for ($_i410 = 0; $_i410 < $_size406; ++$_i410) + { + $elem411 = null; + $xfer += $input->readString($elem411); + $this->part_vals []= $elem411; + } + $xfer += $input->readListEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::I16) { + $xfer += $input->readI16($this->max_parts); + } else { + $xfer += $input->skip($ftype); + } + break; + case 5: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); + $xfer += $input->readString($this->user_name); } else { $xfer += $input->skip($ftype); } break; - case 3: + case 6: if ($ftype == TType::LST) { - $this->part_vals = array(); - $_size334 = 0; - $_etype337 = 0; - $xfer += $input->readListBegin($_etype337, $_size334); - for ($_i338 = 0; $_i338 < $_size334; ++$_i338) + $this->group_names = array(); + $_size412 = 0; + $_etype415 = 0; + $xfer += $input->readListBegin($_etype415, $_size412); + for ($_i416 = 0; $_i416 < $_size412; ++$_i416) { - $elem339 = null; - $xfer += $input->readString($elem339); - $this->part_vals []= $elem339; + $elem417 = null; + $xfer += $input->readString($elem417); + $this->group_names []= $elem417; } $xfer += $input->readListEnd(); } else { @@ -12955,7 +15808,7 @@ class ThriftHiveMetastore_get_partition_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_ps_with_auth_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -12974,9 +15827,36 @@ class ThriftHiveMetastore_get_partition_args { { $output->writeListBegin(TType::STRING, count($this->part_vals)); { - foreach ($this->part_vals as $iter340) + foreach ($this->part_vals as $iter418) { - $xfer += $output->writeString($iter340); + $xfer += $output->writeString($iter418); + } + } + $output->writeListEnd(); + } + $xfer += $output->writeFieldEnd(); + } + if ($this->max_parts !== null) { + $xfer += $output->writeFieldBegin('max_parts', TType::I16, 4); + $xfer += $output->writeI16($this->max_parts); + $xfer += $output->writeFieldEnd(); + } + if ($this->user_name !== null) { + $xfer += $output->writeFieldBegin('user_name', TType::STRING, 5); + $xfer += $output->writeString($this->user_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->group_names !== null) { + if (!is_array($this->group_names)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('group_names', TType::LST, 6); + { + $output->writeListBegin(TType::STRING, count($this->group_names)); + { + foreach ($this->group_names as $iter419) + { + $xfer += $output->writeString($iter419); } } $output->writeListEnd(); @@ -12990,7 +15870,7 @@ class ThriftHiveMetastore_get_partition_args { } -class ThriftHiveMetastore_get_partition_result { +class ThriftHiveMetastore_get_partitions_ps_with_auth_result { static $_TSPEC; public $success = null; @@ -13002,18 +15882,22 @@ class ThriftHiveMetastore_get_partition_result { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', + 'type' => TType::LST, + 'etype' => TType::STRUCT, + 'elem' => array( + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', + ), ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchObjectException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\MetaException', ), ); } @@ -13031,7 +15915,7 @@ class ThriftHiveMetastore_get_partition_result { } public function getName() { - return 'ThriftHiveMetastore_get_partition_result'; + return 'ThriftHiveMetastore_get_partitions_ps_with_auth_result'; } public function read($input) @@ -13050,16 +15934,26 @@ class ThriftHiveMetastore_get_partition_result { switch ($fid) { case 0: - if ($ftype == TType::STRUCT) { - $this->success = new \metastore\Partition(); - $xfer += $this->success->read($input); + if ($ftype == TType::LST) { + $this->success = array(); + $_size420 = 0; + $_etype423 = 0; + $xfer += $input->readListBegin($_etype423, $_size420); + for ($_i424 = 0; $_i424 < $_size420; ++$_i424) + { + $elem425 = null; + $elem425 = new \metastore\Partition(); + $xfer += $elem425->read($input); + $this->success []= $elem425; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -13067,7 +15961,7 @@ class ThriftHiveMetastore_get_partition_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\NoSuchObjectException(); + $this->o2 = new \metastore\MetaException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -13085,13 +15979,22 @@ class ThriftHiveMetastore_get_partition_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_ps_with_auth_result'); if ($this->success !== null) { - if (!is_object($this->success)) { + if (!is_array($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); - $xfer += $this->success->write($output); + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRUCT, count($this->success)); + { + foreach ($this->success as $iter426) + { + $xfer += $iter426->write($output); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -13111,69 +16014,57 @@ class ThriftHiveMetastore_get_partition_result { } -class ThriftHiveMetastore_exchange_partition_args { +class ThriftHiveMetastore_get_partition_names_ps_args { static $_TSPEC; - public $partitionSpecs = null; - public $source_db = null; - public $source_table_name = null; - public $dest_db = null; - public $dest_table_name = null; + public $db_name = null; + public $tbl_name = null; + public $part_vals = null; + public $max_parts = -1; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'partitionSpecs', - 'type' => TType::MAP, - 'ktype' => TType::STRING, - 'vtype' => TType::STRING, - 'key' => array( - 'type' => TType::STRING, - ), - 'val' => array( - 'type' => TType::STRING, - ), + 'var' => 'db_name', + 'type' => TType::STRING, ), 2 => array( - 'var' => 'source_db', + 'var' => 'tbl_name', 'type' => TType::STRING, ), 3 => array( - 'var' => 'source_table_name', - 'type' => TType::STRING, + 'var' => 'part_vals', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), ), 4 => array( - 'var' => 'dest_db', - 'type' => TType::STRING, - ), - 5 => array( - 'var' => 'dest_table_name', - 'type' => TType::STRING, + 'var' => 'max_parts', + 'type' => TType::I16, ), ); } if (is_array($vals)) { - if (isset($vals['partitionSpecs'])) { - $this->partitionSpecs = $vals['partitionSpecs']; - } - if (isset($vals['source_db'])) { - $this->source_db = $vals['source_db']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; } - if (isset($vals['source_table_name'])) { - $this->source_table_name = $vals['source_table_name']; + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['dest_db'])) { - $this->dest_db = $vals['dest_db']; + if (isset($vals['part_vals'])) { + $this->part_vals = $vals['part_vals']; } - if (isset($vals['dest_table_name'])) { - $this->dest_table_name = $vals['dest_table_name']; + if (isset($vals['max_parts'])) { + $this->max_parts = $vals['max_parts']; } } } public function getName() { - return 'ThriftHiveMetastore_exchange_partition_args'; + return 'ThriftHiveMetastore_get_partition_names_ps_args'; } public function read($input) @@ -13192,49 +16083,39 @@ class ThriftHiveMetastore_exchange_partition_args { switch ($fid) { case 1: - if ($ftype == TType::MAP) { - $this->partitionSpecs = array(); - $_size341 = 0; - $_ktype342 = 0; - $_vtype343 = 0; - $xfer += $input->readMapBegin($_ktype342, $_vtype343, $_size341); - for ($_i345 = 0; $_i345 < $_size341; ++$_i345) - { - $key346 = ''; - $val347 = ''; - $xfer += $input->readString($key346); - $xfer += $input->readString($val347); - $this->partitionSpecs[$key346] = $val347; - } - $xfer += $input->readMapEnd(); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->db_name); } else { $xfer += $input->skip($ftype); } break; case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->source_db); + $xfer += $input->readString($this->tbl_name); } else { $xfer += $input->skip($ftype); } break; case 3: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->source_table_name); + if ($ftype == TType::LST) { + $this->part_vals = array(); + $_size427 = 0; + $_etype430 = 0; + $xfer += $input->readListBegin($_etype430, $_size427); + for ($_i431 = 0; $_i431 < $_size427; ++$_i431) + { + $elem432 = null; + $xfer += $input->readString($elem432); + $this->part_vals []= $elem432; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } break; case 4: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->dest_db); - } else { - $xfer += $input->skip($ftype); - } - break; - case 5: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->dest_table_name); + if ($ftype == TType::I16) { + $xfer += $input->readI16($this->max_parts); } else { $xfer += $input->skip($ftype); } @@ -13251,43 +16132,37 @@ class ThriftHiveMetastore_exchange_partition_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_exchange_partition_args'); - if ($this->partitionSpecs !== null) { - if (!is_array($this->partitionSpecs)) { + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_names_ps_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->part_vals !== null) { + if (!is_array($this->part_vals)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('partitionSpecs', TType::MAP, 1); + $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); { - $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs)); + $output->writeListBegin(TType::STRING, count($this->part_vals)); { - foreach ($this->partitionSpecs as $kiter348 => $viter349) + foreach ($this->part_vals as $iter433) { - $xfer += $output->writeString($kiter348); - $xfer += $output->writeString($viter349); + $xfer += $output->writeString($iter433); } } - $output->writeMapEnd(); + $output->writeListEnd(); } $xfer += $output->writeFieldEnd(); } - if ($this->source_db !== null) { - $xfer += $output->writeFieldBegin('source_db', TType::STRING, 2); - $xfer += $output->writeString($this->source_db); - $xfer += $output->writeFieldEnd(); - } - if ($this->source_table_name !== null) { - $xfer += $output->writeFieldBegin('source_table_name', TType::STRING, 3); - $xfer += $output->writeString($this->source_table_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->dest_db !== null) { - $xfer += $output->writeFieldBegin('dest_db', TType::STRING, 4); - $xfer += $output->writeString($this->dest_db); - $xfer += $output->writeFieldEnd(); - } - if ($this->dest_table_name !== null) { - $xfer += $output->writeFieldBegin('dest_table_name', TType::STRING, 5); - $xfer += $output->writeString($this->dest_table_name); + if ($this->max_parts !== null) { + $xfer += $output->writeFieldBegin('max_parts', TType::I16, 4); + $xfer += $output->writeI16($this->max_parts); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -13297,22 +16172,23 @@ class ThriftHiveMetastore_exchange_partition_args { } -class ThriftHiveMetastore_exchange_partition_result { +class ThriftHiveMetastore_get_partition_names_ps_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; - public $o3 = null; - public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), ), 1 => array( 'var' => 'o1', @@ -13324,16 +16200,6 @@ class ThriftHiveMetastore_exchange_partition_result { 'type' => TType::STRUCT, 'class' => '\metastore\NoSuchObjectException', ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', - ), - 4 => array( - 'var' => 'o4', - 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidInputException', - ), ); } if (is_array($vals)) { @@ -13346,17 +16212,11 @@ class ThriftHiveMetastore_exchange_partition_result { if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } - if (isset($vals['o4'])) { - $this->o4 = $vals['o4']; - } } } public function getName() { - return 'ThriftHiveMetastore_exchange_partition_result'; + return 'ThriftHiveMetastore_get_partition_names_ps_result'; } public function read($input) @@ -13375,9 +16235,18 @@ class ThriftHiveMetastore_exchange_partition_result { switch ($fid) { case 0: - if ($ftype == TType::STRUCT) { - $this->success = new \metastore\Partition(); - $xfer += $this->success->read($input); + if ($ftype == TType::LST) { + $this->success = array(); + $_size434 = 0; + $_etype437 = 0; + $xfer += $input->readListBegin($_etype437, $_size434); + for ($_i438 = 0; $_i438 < $_size434; ++$_i438) + { + $elem439 = null; + $xfer += $input->readString($elem439); + $this->success []= $elem439; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } @@ -13398,22 +16267,6 @@ class ThriftHiveMetastore_exchange_partition_result { $xfer += $input->skip($ftype); } break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\InvalidObjectException(); - $xfer += $this->o3->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::STRUCT) { - $this->o4 = new \metastore\InvalidInputException(); - $xfer += $this->o4->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -13426,13 +16279,22 @@ class ThriftHiveMetastore_exchange_partition_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_exchange_partition_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_names_ps_result'); if ($this->success !== null) { - if (!is_object($this->success)) { + if (!is_array($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); - $xfer += $this->success->write($output); + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRING, count($this->success)); + { + foreach ($this->success as $iter440) + { + $xfer += $output->writeString($iter440); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -13445,16 +16307,6 @@ class ThriftHiveMetastore_exchange_partition_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->o4 !== null) { - $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); - $xfer += $this->o4->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -13462,14 +16314,13 @@ class ThriftHiveMetastore_exchange_partition_result { } -class ThriftHiveMetastore_get_partition_with_auth_args { +class ThriftHiveMetastore_get_partitions_by_filter_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; - public $part_vals = null; - public $user_name = null; - public $group_names = null; + public $filter = null; + public $max_parts = -1; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -13483,24 +16334,12 @@ class ThriftHiveMetastore_get_partition_with_auth_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'part_vals', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), - ), - 4 => array( - 'var' => 'user_name', + 'var' => 'filter', 'type' => TType::STRING, ), - 5 => array( - 'var' => 'group_names', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), + 4 => array( + 'var' => 'max_parts', + 'type' => TType::I16, ), ); } @@ -13511,20 +16350,17 @@ class ThriftHiveMetastore_get_partition_with_auth_args { if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['part_vals'])) { - $this->part_vals = $vals['part_vals']; - } - if (isset($vals['user_name'])) { - $this->user_name = $vals['user_name']; + if (isset($vals['filter'])) { + $this->filter = $vals['filter']; } - if (isset($vals['group_names'])) { - $this->group_names = $vals['group_names']; + if (isset($vals['max_parts'])) { + $this->max_parts = $vals['max_parts']; } } } public function getName() { - return 'ThriftHiveMetastore_get_partition_with_auth_args'; + return 'ThriftHiveMetastore_get_partitions_by_filter_args'; } public function read($input) @@ -13557,42 +16393,15 @@ class ThriftHiveMetastore_get_partition_with_auth_args { } break; case 3: - if ($ftype == TType::LST) { - $this->part_vals = array(); - $_size350 = 0; - $_etype353 = 0; - $xfer += $input->readListBegin($_etype353, $_size350); - for ($_i354 = 0; $_i354 < $_size350; ++$_i354) - { - $elem355 = null; - $xfer += $input->readString($elem355); - $this->part_vals []= $elem355; - } - $xfer += $input->readListEnd(); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->user_name); + $xfer += $input->readString($this->filter); } else { $xfer += $input->skip($ftype); } break; - case 5: - if ($ftype == TType::LST) { - $this->group_names = array(); - $_size356 = 0; - $_etype359 = 0; - $xfer += $input->readListBegin($_etype359, $_size356); - for ($_i360 = 0; $_i360 < $_size356; ++$_i360) - { - $elem361 = null; - $xfer += $input->readString($elem361); - $this->group_names []= $elem361; - } - $xfer += $input->readListEnd(); + case 4: + if ($ftype == TType::I16) { + $xfer += $input->readI16($this->max_parts); } else { $xfer += $input->skip($ftype); } @@ -13609,7 +16418,7 @@ class ThriftHiveMetastore_get_partition_with_auth_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_with_auth_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_by_filter_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -13620,43 +16429,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->part_vals !== null) { - if (!is_array($this->part_vals)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); - { - $output->writeListBegin(TType::STRING, count($this->part_vals)); - { - foreach ($this->part_vals as $iter362) - { - $xfer += $output->writeString($iter362); - } - } - $output->writeListEnd(); - } - $xfer += $output->writeFieldEnd(); - } - if ($this->user_name !== null) { - $xfer += $output->writeFieldBegin('user_name', TType::STRING, 4); - $xfer += $output->writeString($this->user_name); + if ($this->filter !== null) { + $xfer += $output->writeFieldBegin('filter', TType::STRING, 3); + $xfer += $output->writeString($this->filter); $xfer += $output->writeFieldEnd(); } - if ($this->group_names !== null) { - if (!is_array($this->group_names)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('group_names', TType::LST, 5); - { - $output->writeListBegin(TType::STRING, count($this->group_names)); - { - foreach ($this->group_names as $iter363) - { - $xfer += $output->writeString($iter363); - } - } - $output->writeListEnd(); - } + if ($this->max_parts !== null) { + $xfer += $output->writeFieldBegin('max_parts', TType::I16, 4); + $xfer += $output->writeI16($this->max_parts); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -13666,7 +16446,7 @@ class ThriftHiveMetastore_get_partition_with_auth_args { } -class ThriftHiveMetastore_get_partition_with_auth_result { +class ThriftHiveMetastore_get_partitions_by_filter_result { static $_TSPEC; public $success = null; @@ -13678,8 +16458,12 @@ class ThriftHiveMetastore_get_partition_with_auth_result { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', + 'type' => TType::LST, + 'etype' => TType::STRUCT, + 'elem' => array( + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', + ), ), 1 => array( 'var' => 'o1', @@ -13707,7 +16491,7 @@ class ThriftHiveMetastore_get_partition_with_auth_result { } public function getName() { - return 'ThriftHiveMetastore_get_partition_with_auth_result'; + return 'ThriftHiveMetastore_get_partitions_by_filter_result'; } public function read($input) @@ -13726,9 +16510,19 @@ class ThriftHiveMetastore_get_partition_with_auth_result { switch ($fid) { case 0: - if ($ftype == TType::STRUCT) { - $this->success = new \metastore\Partition(); - $xfer += $this->success->read($input); + if ($ftype == TType::LST) { + $this->success = array(); + $_size441 = 0; + $_etype444 = 0; + $xfer += $input->readListBegin($_etype444, $_size441); + for ($_i445 = 0; $_i445 < $_size441; ++$_i445) + { + $elem446 = null; + $elem446 = new \metastore\Partition(); + $xfer += $elem446->read($input); + $this->success []= $elem446; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } @@ -13761,13 +16555,22 @@ class ThriftHiveMetastore_get_partition_with_auth_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_with_auth_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_by_filter_result'); if ($this->success !== null) { - if (!is_object($this->success)) { + if (!is_array($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); - $xfer += $this->success->write($output); + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRUCT, count($this->success)); + { + foreach ($this->success as $iter447) + { + $xfer += $iter447->write($output); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -13787,12 +16590,12 @@ class ThriftHiveMetastore_get_partition_with_auth_result { } -class ThriftHiveMetastore_get_partition_by_name_args { +class ThriftHiveMetastore_get_partitions_by_names_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; - public $part_name = null; + public $names = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -13806,8 +16609,12 @@ class ThriftHiveMetastore_get_partition_by_name_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'part_name', - 'type' => TType::STRING, + 'var' => 'names', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), ), ); } @@ -13818,14 +16625,14 @@ class ThriftHiveMetastore_get_partition_by_name_args { if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['part_name'])) { - $this->part_name = $vals['part_name']; + if (isset($vals['names'])) { + $this->names = $vals['names']; } } } public function getName() { - return 'ThriftHiveMetastore_get_partition_by_name_args'; + return 'ThriftHiveMetastore_get_partitions_by_names_args'; } public function read($input) @@ -13858,8 +16665,18 @@ class ThriftHiveMetastore_get_partition_by_name_args { } break; case 3: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->part_name); + if ($ftype == TType::LST) { + $this->names = array(); + $_size448 = 0; + $_etype451 = 0; + $xfer += $input->readListBegin($_etype451, $_size448); + for ($_i452 = 0; $_i452 < $_size448; ++$_i452) + { + $elem453 = null; + $xfer += $input->readString($elem453); + $this->names []= $elem453; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } @@ -13876,7 +16693,7 @@ class ThriftHiveMetastore_get_partition_by_name_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_by_name_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_by_names_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -13887,9 +16704,21 @@ class ThriftHiveMetastore_get_partition_by_name_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->part_name !== null) { - $xfer += $output->writeFieldBegin('part_name', TType::STRING, 3); - $xfer += $output->writeString($this->part_name); + if ($this->names !== null) { + if (!is_array($this->names)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('names', TType::LST, 3); + { + $output->writeListBegin(TType::STRING, count($this->names)); + { + foreach ($this->names as $iter454) + { + $xfer += $output->writeString($iter454); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -13899,7 +16728,7 @@ class ThriftHiveMetastore_get_partition_by_name_args { } -class ThriftHiveMetastore_get_partition_by_name_result { +class ThriftHiveMetastore_get_partitions_by_names_result { static $_TSPEC; public $success = null; @@ -13911,8 +16740,12 @@ class ThriftHiveMetastore_get_partition_by_name_result { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', + 'type' => TType::LST, + 'etype' => TType::STRUCT, + 'elem' => array( + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', + ), ), 1 => array( 'var' => 'o1', @@ -13940,7 +16773,7 @@ class ThriftHiveMetastore_get_partition_by_name_result { } public function getName() { - return 'ThriftHiveMetastore_get_partition_by_name_result'; + return 'ThriftHiveMetastore_get_partitions_by_names_result'; } public function read($input) @@ -13959,9 +16792,19 @@ class ThriftHiveMetastore_get_partition_by_name_result { switch ($fid) { case 0: - if ($ftype == TType::STRUCT) { - $this->success = new \metastore\Partition(); - $xfer += $this->success->read($input); + if ($ftype == TType::LST) { + $this->success = array(); + $_size455 = 0; + $_etype458 = 0; + $xfer += $input->readListBegin($_etype458, $_size455); + for ($_i459 = 0; $_i459 < $_size455; ++$_i459) + { + $elem460 = null; + $elem460 = new \metastore\Partition(); + $xfer += $elem460->read($input); + $this->success []= $elem460; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } @@ -13994,13 +16837,22 @@ class ThriftHiveMetastore_get_partition_by_name_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_by_name_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_by_names_result'); if ($this->success !== null) { - if (!is_object($this->success)) { + if (!is_array($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); - $xfer += $this->success->write($output); + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRUCT, count($this->success)); + { + foreach ($this->success as $iter461) + { + $xfer += $iter461->write($output); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -14020,12 +16872,12 @@ class ThriftHiveMetastore_get_partition_by_name_result { } -class ThriftHiveMetastore_get_partitions_args { +class ThriftHiveMetastore_alter_partition_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; - public $max_parts = -1; + public $new_part = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -14039,8 +16891,9 @@ class ThriftHiveMetastore_get_partitions_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'max_parts', - 'type' => TType::I16, + 'var' => 'new_part', + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', ), ); } @@ -14051,14 +16904,14 @@ class ThriftHiveMetastore_get_partitions_args { if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['max_parts'])) { - $this->max_parts = $vals['max_parts']; + if (isset($vals['new_part'])) { + $this->new_part = $vals['new_part']; } } } public function getName() { - return 'ThriftHiveMetastore_get_partitions_args'; + return 'ThriftHiveMetastore_alter_partition_args'; } public function read($input) @@ -14091,8 +16944,9 @@ class ThriftHiveMetastore_get_partitions_args { } break; case 3: - if ($ftype == TType::I16) { - $xfer += $input->readI16($this->max_parts); + if ($ftype == TType::STRUCT) { + $this->new_part = new \metastore\Partition(); + $xfer += $this->new_part->read($input); } else { $xfer += $input->skip($ftype); } @@ -14109,7 +16963,7 @@ class ThriftHiveMetastore_get_partitions_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_partition_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -14120,9 +16974,12 @@ class ThriftHiveMetastore_get_partitions_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->max_parts !== null) { - $xfer += $output->writeFieldBegin('max_parts', TType::I16, 3); - $xfer += $output->writeI16($this->max_parts); + if ($this->new_part !== null) { + if (!is_object($this->new_part)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('new_part', TType::STRUCT, 3); + $xfer += $this->new_part->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -14132,29 +16989,19 @@ class ThriftHiveMetastore_get_partitions_args { } -class ThriftHiveMetastore_get_partitions_result { +class ThriftHiveMetastore_alter_partition_result { static $_TSPEC; - public $success = null; public $o1 = null; public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRUCT, - 'elem' => array( - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', - ), - ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\InvalidOperationException', ), 2 => array( 'var' => 'o2', @@ -14164,9 +17011,6 @@ class ThriftHiveMetastore_get_partitions_result { ); } if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; - } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } @@ -14177,7 +17021,7 @@ class ThriftHiveMetastore_get_partitions_result { } public function getName() { - return 'ThriftHiveMetastore_get_partitions_result'; + return 'ThriftHiveMetastore_alter_partition_result'; } public function read($input) @@ -14195,27 +17039,9 @@ class ThriftHiveMetastore_get_partitions_result { } switch ($fid) { - case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size364 = 0; - $_etype367 = 0; - $xfer += $input->readListBegin($_etype367, $_size364); - for ($_i368 = 0; $_i368 < $_size364; ++$_i368) - { - $elem369 = null; - $elem369 = new \metastore\Partition(); - $xfer += $elem369->read($input); - $this->success []= $elem369; - } - $xfer += $input->readListEnd(); - } else { - $xfer += $input->skip($ftype); - } - break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\InvalidOperationException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -14241,24 +17067,7 @@ class ThriftHiveMetastore_get_partitions_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_result'); - if ($this->success !== null) { - if (!is_array($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRUCT, count($this->success)); - { - foreach ($this->success as $iter370) - { - $xfer += $iter370->write($output); - } - } - $output->writeListEnd(); - } - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_partition_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -14276,14 +17085,12 @@ class ThriftHiveMetastore_get_partitions_result { } -class ThriftHiveMetastore_get_partitions_with_auth_args { +class ThriftHiveMetastore_alter_partitions_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; - public $max_parts = -1; - public $user_name = null; - public $group_names = null; + public $new_parts = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -14297,19 +17104,12 @@ class ThriftHiveMetastore_get_partitions_with_auth_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'max_parts', - 'type' => TType::I16, - ), - 4 => array( - 'var' => 'user_name', - 'type' => TType::STRING, - ), - 5 => array( - 'var' => 'group_names', + 'var' => 'new_parts', 'type' => TType::LST, - 'etype' => TType::STRING, + 'etype' => TType::STRUCT, 'elem' => array( - 'type' => TType::STRING, + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', ), ), ); @@ -14321,20 +17121,14 @@ class ThriftHiveMetastore_get_partitions_with_auth_args { if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['max_parts'])) { - $this->max_parts = $vals['max_parts']; - } - if (isset($vals['user_name'])) { - $this->user_name = $vals['user_name']; - } - if (isset($vals['group_names'])) { - $this->group_names = $vals['group_names']; + if (isset($vals['new_parts'])) { + $this->new_parts = $vals['new_parts']; } } } public function getName() { - return 'ThriftHiveMetastore_get_partitions_with_auth_args'; + return 'ThriftHiveMetastore_alter_partitions_args'; } public function read($input) @@ -14353,44 +17147,31 @@ class ThriftHiveMetastore_get_partitions_with_auth_args { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::I16) { - $xfer += $input->readI16($this->max_parts); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->db_name); } else { $xfer += $input->skip($ftype); } break; - case 4: + case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->user_name); + $xfer += $input->readString($this->tbl_name); } else { $xfer += $input->skip($ftype); } break; - case 5: + case 3: if ($ftype == TType::LST) { - $this->group_names = array(); - $_size371 = 0; - $_etype374 = 0; - $xfer += $input->readListBegin($_etype374, $_size371); - for ($_i375 = 0; $_i375 < $_size371; ++$_i375) + $this->new_parts = array(); + $_size462 = 0; + $_etype465 = 0; + $xfer += $input->readListBegin($_etype465, $_size462); + for ($_i466 = 0; $_i466 < $_size462; ++$_i466) { - $elem376 = null; - $xfer += $input->readString($elem376); - $this->group_names []= $elem376; + $elem467 = null; + $elem467 = new \metastore\Partition(); + $xfer += $elem467->read($input); + $this->new_parts []= $elem467; } $xfer += $input->readListEnd(); } else { @@ -14409,7 +17190,7 @@ class ThriftHiveMetastore_get_partitions_with_auth_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_with_auth_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_partitions_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -14420,27 +17201,17 @@ class ThriftHiveMetastore_get_partitions_with_auth_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->max_parts !== null) { - $xfer += $output->writeFieldBegin('max_parts', TType::I16, 3); - $xfer += $output->writeI16($this->max_parts); - $xfer += $output->writeFieldEnd(); - } - if ($this->user_name !== null) { - $xfer += $output->writeFieldBegin('user_name', TType::STRING, 4); - $xfer += $output->writeString($this->user_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->group_names !== null) { - if (!is_array($this->group_names)) { + if ($this->new_parts !== null) { + if (!is_array($this->new_parts)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('group_names', TType::LST, 5); + $xfer += $output->writeFieldBegin('new_parts', TType::LST, 3); { - $output->writeListBegin(TType::STRING, count($this->group_names)); + $output->writeListBegin(TType::STRUCT, count($this->new_parts)); { - foreach ($this->group_names as $iter377) + foreach ($this->new_parts as $iter468) { - $xfer += $output->writeString($iter377); + $xfer += $iter468->write($output); } } $output->writeListEnd(); @@ -14454,29 +17225,19 @@ class ThriftHiveMetastore_get_partitions_with_auth_args { } -class ThriftHiveMetastore_get_partitions_with_auth_result { +class ThriftHiveMetastore_alter_partitions_result { static $_TSPEC; - public $success = null; public $o1 = null; public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRUCT, - 'elem' => array( - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', - ), - ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\InvalidOperationException', ), 2 => array( 'var' => 'o2', @@ -14486,9 +17247,6 @@ class ThriftHiveMetastore_get_partitions_with_auth_result { ); } if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; - } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } @@ -14499,7 +17257,7 @@ class ThriftHiveMetastore_get_partitions_with_auth_result { } public function getName() { - return 'ThriftHiveMetastore_get_partitions_with_auth_result'; + return 'ThriftHiveMetastore_alter_partitions_result'; } public function read($input) @@ -14517,27 +17275,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_result { } switch ($fid) { - case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size378 = 0; - $_etype381 = 0; - $xfer += $input->readListBegin($_etype381, $_size378); - for ($_i382 = 0; $_i382 < $_size378; ++$_i382) - { - $elem383 = null; - $elem383 = new \metastore\Partition(); - $xfer += $elem383->read($input); - $this->success []= $elem383; - } - $xfer += $input->readListEnd(); - } else { - $xfer += $input->skip($ftype); - } - break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\InvalidOperationException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -14563,24 +17303,7 @@ class ThriftHiveMetastore_get_partitions_with_auth_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_with_auth_result'); - if ($this->success !== null) { - if (!is_array($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRUCT, count($this->success)); - { - foreach ($this->success as $iter384) - { - $xfer += $iter384->write($output); - } - } - $output->writeListEnd(); - } - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_partitions_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -14598,12 +17321,13 @@ class ThriftHiveMetastore_get_partitions_with_auth_result { } -class ThriftHiveMetastore_get_partition_names_args { +class ThriftHiveMetastore_alter_partition_with_environment_context_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; - public $max_parts = -1; + public $new_part = null; + public $environment_context = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -14617,8 +17341,14 @@ class ThriftHiveMetastore_get_partition_names_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'max_parts', - 'type' => TType::I16, + 'var' => 'new_part', + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', + ), + 4 => array( + 'var' => 'environment_context', + 'type' => TType::STRUCT, + 'class' => '\metastore\EnvironmentContext', ), ); } @@ -14629,14 +17359,17 @@ class ThriftHiveMetastore_get_partition_names_args { if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['max_parts'])) { - $this->max_parts = $vals['max_parts']; + if (isset($vals['new_part'])) { + $this->new_part = $vals['new_part']; + } + if (isset($vals['environment_context'])) { + $this->environment_context = $vals['environment_context']; } } } public function getName() { - return 'ThriftHiveMetastore_get_partition_names_args'; + return 'ThriftHiveMetastore_alter_partition_with_environment_context_args'; } public function read($input) @@ -14669,8 +17402,17 @@ class ThriftHiveMetastore_get_partition_names_args { } break; case 3: - if ($ftype == TType::I16) { - $xfer += $input->readI16($this->max_parts); + if ($ftype == TType::STRUCT) { + $this->new_part = new \metastore\Partition(); + $xfer += $this->new_part->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRUCT) { + $this->environment_context = new \metastore\EnvironmentContext(); + $xfer += $this->environment_context->read($input); } else { $xfer += $input->skip($ftype); } @@ -14687,7 +17429,7 @@ class ThriftHiveMetastore_get_partition_names_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_names_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_partition_with_environment_context_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -14698,9 +17440,20 @@ class ThriftHiveMetastore_get_partition_names_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->max_parts !== null) { - $xfer += $output->writeFieldBegin('max_parts', TType::I16, 3); - $xfer += $output->writeI16($this->max_parts); + if ($this->new_part !== null) { + if (!is_object($this->new_part)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('new_part', TType::STRUCT, 3); + $xfer += $this->new_part->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->environment_context !== null) { + if (!is_object($this->environment_context)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 4); + $xfer += $this->environment_context->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -14710,24 +17463,21 @@ class ThriftHiveMetastore_get_partition_names_args { } -class ThriftHiveMetastore_get_partition_names_result { +class ThriftHiveMetastore_alter_partition_with_environment_context_result { static $_TSPEC; - public $success = null; + public $o1 = null; public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), - ), 1 => array( + 'var' => 'o1', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidOperationException', + ), + 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', @@ -14735,8 +17485,8 @@ class ThriftHiveMetastore_get_partition_names_result { ); } if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; + if (isset($vals['o1'])) { + $this->o1 = $vals['o1']; } if (isset($vals['o2'])) { $this->o2 = $vals['o2']; @@ -14745,7 +17495,7 @@ class ThriftHiveMetastore_get_partition_names_result { } public function getName() { - return 'ThriftHiveMetastore_get_partition_names_result'; + return 'ThriftHiveMetastore_alter_partition_with_environment_context_result'; } public function read($input) @@ -14763,24 +17513,15 @@ class ThriftHiveMetastore_get_partition_names_result { } switch ($fid) { - case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size385 = 0; - $_etype388 = 0; - $xfer += $input->readListBegin($_etype388, $_size385); - for ($_i389 = 0; $_i389 < $_size385; ++$_i389) - { - $elem390 = null; - $xfer += $input->readString($elem390); - $this->success []= $elem390; - } - $xfer += $input->readListEnd(); + case 1: + if ($ftype == TType::STRUCT) { + $this->o1 = new \metastore\InvalidOperationException(); + $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; - case 1: + case 2: if ($ftype == TType::STRUCT) { $this->o2 = new \metastore\MetaException(); $xfer += $this->o2->read($input); @@ -14800,26 +17541,14 @@ class ThriftHiveMetastore_get_partition_names_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_names_result'); - if ($this->success !== null) { - if (!is_array($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRING, count($this->success)); - { - foreach ($this->success as $iter391) - { - $xfer += $output->writeString($iter391); - } - } - $output->writeListEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_partition_with_environment_context_result'); + if ($this->o1 !== null) { + $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); + $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 1); + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } @@ -14830,13 +17559,13 @@ class ThriftHiveMetastore_get_partition_names_result { } -class ThriftHiveMetastore_get_partitions_ps_args { +class ThriftHiveMetastore_rename_partition_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; public $part_vals = null; - public $max_parts = -1; + public $new_part = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -14858,8 +17587,9 @@ class ThriftHiveMetastore_get_partitions_ps_args { ), ), 4 => array( - 'var' => 'max_parts', - 'type' => TType::I16, + 'var' => 'new_part', + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', ), ); } @@ -14873,14 +17603,14 @@ class ThriftHiveMetastore_get_partitions_ps_args { if (isset($vals['part_vals'])) { $this->part_vals = $vals['part_vals']; } - if (isset($vals['max_parts'])) { - $this->max_parts = $vals['max_parts']; + if (isset($vals['new_part'])) { + $this->new_part = $vals['new_part']; } } } public function getName() { - return 'ThriftHiveMetastore_get_partitions_ps_args'; + return 'ThriftHiveMetastore_rename_partition_args'; } public function read($input) @@ -14915,14 +17645,14 @@ class ThriftHiveMetastore_get_partitions_ps_args { case 3: if ($ftype == TType::LST) { $this->part_vals = array(); - $_size392 = 0; - $_etype395 = 0; - $xfer += $input->readListBegin($_etype395, $_size392); - for ($_i396 = 0; $_i396 < $_size392; ++$_i396) + $_size469 = 0; + $_etype472 = 0; + $xfer += $input->readListBegin($_etype472, $_size469); + for ($_i473 = 0; $_i473 < $_size469; ++$_i473) { - $elem397 = null; - $xfer += $input->readString($elem397); - $this->part_vals []= $elem397; + $elem474 = null; + $xfer += $input->readString($elem474); + $this->part_vals []= $elem474; } $xfer += $input->readListEnd(); } else { @@ -14930,8 +17660,9 @@ class ThriftHiveMetastore_get_partitions_ps_args { } break; case 4: - if ($ftype == TType::I16) { - $xfer += $input->readI16($this->max_parts); + if ($ftype == TType::STRUCT) { + $this->new_part = new \metastore\Partition(); + $xfer += $this->new_part->read($input); } else { $xfer += $input->skip($ftype); } @@ -14948,7 +17679,7 @@ class ThriftHiveMetastore_get_partitions_ps_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_ps_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_rename_partition_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -14967,18 +17698,21 @@ class ThriftHiveMetastore_get_partitions_ps_args { { $output->writeListBegin(TType::STRING, count($this->part_vals)); { - foreach ($this->part_vals as $iter398) + foreach ($this->part_vals as $iter475) { - $xfer += $output->writeString($iter398); + $xfer += $output->writeString($iter475); } } $output->writeListEnd(); } $xfer += $output->writeFieldEnd(); } - if ($this->max_parts !== null) { - $xfer += $output->writeFieldBegin('max_parts', TType::I16, 4); - $xfer += $output->writeI16($this->max_parts); + if ($this->new_part !== null) { + if (!is_object($this->new_part)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('new_part', TType::STRUCT, 4); + $xfer += $this->new_part->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -14988,41 +17722,28 @@ class ThriftHiveMetastore_get_partitions_ps_args { } -class ThriftHiveMetastore_get_partitions_ps_result { +class ThriftHiveMetastore_rename_partition_result { static $_TSPEC; - public $success = null; public $o1 = null; public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRUCT, - 'elem' => array( - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', - ), - ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\InvalidOperationException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\MetaException', ), ); } if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; - } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } @@ -15033,7 +17754,7 @@ class ThriftHiveMetastore_get_partitions_ps_result { } public function getName() { - return 'ThriftHiveMetastore_get_partitions_ps_result'; + return 'ThriftHiveMetastore_rename_partition_result'; } public function read($input) @@ -15051,27 +17772,9 @@ class ThriftHiveMetastore_get_partitions_ps_result { } switch ($fid) { - case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size399 = 0; - $_etype402 = 0; - $xfer += $input->readListBegin($_etype402, $_size399); - for ($_i403 = 0; $_i403 < $_size399; ++$_i403) - { - $elem404 = null; - $elem404 = new \metastore\Partition(); - $xfer += $elem404->read($input); - $this->success []= $elem404; - } - $xfer += $input->readListEnd(); - } else { - $xfer += $input->skip($ftype); - } - break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\InvalidOperationException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -15079,7 +17782,7 @@ class ThriftHiveMetastore_get_partitions_ps_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\NoSuchObjectException(); + $this->o2 = new \metastore\MetaException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -15097,24 +17800,7 @@ class ThriftHiveMetastore_get_partitions_ps_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_ps_result'); - if ($this->success !== null) { - if (!is_array($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRUCT, count($this->success)); - { - foreach ($this->success as $iter405) - { - $xfer += $iter405->write($output); - } - } - $output->writeListEnd(); - } - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_rename_partition_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -15132,28 +17818,16 @@ class ThriftHiveMetastore_get_partitions_ps_result { } -class ThriftHiveMetastore_get_partitions_ps_with_auth_args { +class ThriftHiveMetastore_partition_name_has_valid_characters_args { static $_TSPEC; - public $db_name = null; - public $tbl_name = null; public $part_vals = null; - public $max_parts = -1; - public $user_name = null; - public $group_names = null; + public $throw_exception = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', - 'type' => TType::STRING, - ), - 2 => array( - 'var' => 'tbl_name', - 'type' => TType::STRING, - ), - 3 => array( 'var' => 'part_vals', 'type' => TType::LST, 'etype' => TType::STRING, @@ -15161,48 +17835,24 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args { 'type' => TType::STRING, ), ), - 4 => array( - 'var' => 'max_parts', - 'type' => TType::I16, - ), - 5 => array( - 'var' => 'user_name', - 'type' => TType::STRING, - ), - 6 => array( - 'var' => 'group_names', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), + 2 => array( + 'var' => 'throw_exception', + 'type' => TType::BOOL, ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; - } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; - } if (isset($vals['part_vals'])) { $this->part_vals = $vals['part_vals']; } - if (isset($vals['max_parts'])) { - $this->max_parts = $vals['max_parts']; - } - if (isset($vals['user_name'])) { - $this->user_name = $vals['user_name']; - } - if (isset($vals['group_names'])) { - $this->group_names = $vals['group_names']; + if (isset($vals['throw_exception'])) { + $this->throw_exception = $vals['throw_exception']; } } } public function getName() { - return 'ThriftHiveMetastore_get_partitions_ps_with_auth_args'; + return 'ThriftHiveMetastore_partition_name_has_valid_characters_args'; } public function read($input) @@ -15221,63 +17871,25 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: if ($ftype == TType::LST) { $this->part_vals = array(); - $_size406 = 0; - $_etype409 = 0; - $xfer += $input->readListBegin($_etype409, $_size406); - for ($_i410 = 0; $_i410 < $_size406; ++$_i410) + $_size476 = 0; + $_etype479 = 0; + $xfer += $input->readListBegin($_etype479, $_size476); + for ($_i480 = 0; $_i480 < $_size476; ++$_i480) { - $elem411 = null; - $xfer += $input->readString($elem411); - $this->part_vals []= $elem411; + $elem481 = null; + $xfer += $input->readString($elem481); + $this->part_vals []= $elem481; } $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } break; - case 4: - if ($ftype == TType::I16) { - $xfer += $input->readI16($this->max_parts); - } else { - $xfer += $input->skip($ftype); - } - break; - case 5: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->user_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 6: - if ($ftype == TType::LST) { - $this->group_names = array(); - $_size412 = 0; - $_etype415 = 0; - $xfer += $input->readListBegin($_etype415, $_size412); - for ($_i416 = 0; $_i416 < $_size412; ++$_i416) - { - $elem417 = null; - $xfer += $input->readString($elem417); - $this->group_names []= $elem417; - } - $xfer += $input->readListEnd(); + case 2: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->throw_exception); } else { $xfer += $input->skip($ftype); } @@ -15294,59 +17906,27 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_ps_with_auth_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_partition_name_has_valid_characters_args'); if ($this->part_vals !== null) { if (!is_array($this->part_vals)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); + $xfer += $output->writeFieldBegin('part_vals', TType::LST, 1); { $output->writeListBegin(TType::STRING, count($this->part_vals)); { - foreach ($this->part_vals as $iter418) + foreach ($this->part_vals as $iter482) { - $xfer += $output->writeString($iter418); + $xfer += $output->writeString($iter482); } } $output->writeListEnd(); } $xfer += $output->writeFieldEnd(); } - if ($this->max_parts !== null) { - $xfer += $output->writeFieldBegin('max_parts', TType::I16, 4); - $xfer += $output->writeI16($this->max_parts); - $xfer += $output->writeFieldEnd(); - } - if ($this->user_name !== null) { - $xfer += $output->writeFieldBegin('user_name', TType::STRING, 5); - $xfer += $output->writeString($this->user_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->group_names !== null) { - if (!is_array($this->group_names)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('group_names', TType::LST, 6); - { - $output->writeListBegin(TType::STRING, count($this->group_names)); - { - foreach ($this->group_names as $iter419) - { - $xfer += $output->writeString($iter419); - } - } - $output->writeListEnd(); - } + if ($this->throw_exception !== null) { + $xfer += $output->writeFieldBegin('throw_exception', TType::BOOL, 2); + $xfer += $output->writeBool($this->throw_exception); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -15356,33 +17936,22 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args { } -class ThriftHiveMetastore_get_partitions_ps_with_auth_result { +class ThriftHiveMetastore_partition_name_has_valid_characters_result { static $_TSPEC; public $success = null; public $o1 = null; - public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRUCT, - 'elem' => array( - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', - ), + 'type' => TType::BOOL, ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', - ), - 2 => array( - 'var' => 'o2', - 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), ); @@ -15394,14 +17963,11 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } - if (isset($vals['o2'])) { - $this->o2 = $vals['o2']; - } } } public function getName() { - return 'ThriftHiveMetastore_get_partitions_ps_with_auth_result'; + return 'ThriftHiveMetastore_partition_name_has_valid_characters_result'; } public function read($input) @@ -15420,39 +17986,20 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result { switch ($fid) { case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size420 = 0; - $_etype423 = 0; - $xfer += $input->readListBegin($_etype423, $_size420); - for ($_i424 = 0; $_i424 < $_size420; ++$_i424) - { - $elem425 = null; - $elem425 = new \metastore\Partition(); - $xfer += $elem425->read($input); - $this->success []= $elem425; - } - $xfer += $input->readListEnd(); + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->success); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; - case 2: - if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); - $xfer += $this->o2->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -15465,22 +18012,10 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_ps_with_auth_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_partition_name_has_valid_characters_result'); if ($this->success !== null) { - if (!is_array($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRUCT, count($this->success)); - { - foreach ($this->success as $iter426) - { - $xfer += $iter426->write($output); - } - } - $output->writeListEnd(); - } + $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); + $xfer += $output->writeBool($this->success); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -15488,11 +18023,6 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); - $xfer += $this->o2->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -15500,57 +18030,37 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result { } -class ThriftHiveMetastore_get_partition_names_ps_args { +class ThriftHiveMetastore_get_config_value_args { static $_TSPEC; - public $db_name = null; - public $tbl_name = null; - public $part_vals = null; - public $max_parts = -1; + public $name = null; + public $defaultValue = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', + 'var' => 'name', 'type' => TType::STRING, ), 2 => array( - 'var' => 'tbl_name', + 'var' => 'defaultValue', 'type' => TType::STRING, ), - 3 => array( - 'var' => 'part_vals', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), - ), - 4 => array( - 'var' => 'max_parts', - 'type' => TType::I16, - ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; - } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; - } - if (isset($vals['part_vals'])) { - $this->part_vals = $vals['part_vals']; + if (isset($vals['name'])) { + $this->name = $vals['name']; } - if (isset($vals['max_parts'])) { - $this->max_parts = $vals['max_parts']; + if (isset($vals['defaultValue'])) { + $this->defaultValue = $vals['defaultValue']; } } } public function getName() { - return 'ThriftHiveMetastore_get_partition_names_ps_args'; + return 'ThriftHiveMetastore_get_config_value_args'; } public function read($input) @@ -15570,38 +18080,14 @@ class ThriftHiveMetastore_get_partition_names_ps_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); + $xfer += $input->readString($this->name); } else { $xfer += $input->skip($ftype); } break; case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::LST) { - $this->part_vals = array(); - $_size427 = 0; - $_etype430 = 0; - $xfer += $input->readListBegin($_etype430, $_size427); - for ($_i431 = 0; $_i431 < $_size427; ++$_i431) - { - $elem432 = null; - $xfer += $input->readString($elem432); - $this->part_vals []= $elem432; - } - $xfer += $input->readListEnd(); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::I16) { - $xfer += $input->readI16($this->max_parts); + $xfer += $input->readString($this->defaultValue); } else { $xfer += $input->skip($ftype); } @@ -15618,37 +18104,15 @@ class ThriftHiveMetastore_get_partition_names_ps_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_names_ps_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->part_vals !== null) { - if (!is_array($this->part_vals)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); - { - $output->writeListBegin(TType::STRING, count($this->part_vals)); - { - foreach ($this->part_vals as $iter433) - { - $xfer += $output->writeString($iter433); - } - } - $output->writeListEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_config_value_args'); + if ($this->name !== null) { + $xfer += $output->writeFieldBegin('name', TType::STRING, 1); + $xfer += $output->writeString($this->name); $xfer += $output->writeFieldEnd(); } - if ($this->max_parts !== null) { - $xfer += $output->writeFieldBegin('max_parts', TType::I16, 4); - $xfer += $output->writeI16($this->max_parts); + if ($this->defaultValue !== null) { + $xfer += $output->writeFieldBegin('defaultValue', TType::STRING, 2); + $xfer += $output->writeString($this->defaultValue); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -15658,33 +18122,23 @@ class ThriftHiveMetastore_get_partition_names_ps_args { } -class ThriftHiveMetastore_get_partition_names_ps_result { +class ThriftHiveMetastore_get_config_value_result { static $_TSPEC; public $success = null; public $o1 = null; - public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), + 'type' => TType::STRING, ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', - ), - 2 => array( - 'var' => 'o2', - 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\ConfigValSecurityException', ), ); } @@ -15695,14 +18149,11 @@ class ThriftHiveMetastore_get_partition_names_ps_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } - if (isset($vals['o2'])) { - $this->o2 = $vals['o2']; - } } } public function getName() { - return 'ThriftHiveMetastore_get_partition_names_ps_result'; + return 'ThriftHiveMetastore_get_config_value_result'; } public function read($input) @@ -15721,38 +18172,20 @@ class ThriftHiveMetastore_get_partition_names_ps_result { switch ($fid) { case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size434 = 0; - $_etype437 = 0; - $xfer += $input->readListBegin($_etype437, $_size434); - for ($_i438 = 0; $_i438 < $_size434; ++$_i438) - { - $elem439 = null; - $xfer += $input->readString($elem439); - $this->success []= $elem439; - } - $xfer += $input->readListEnd(); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->success); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\ConfigValSecurityException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; - case 2: - if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\NoSuchObjectException(); - $xfer += $this->o2->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -15765,22 +18198,10 @@ class ThriftHiveMetastore_get_partition_names_ps_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_names_ps_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_config_value_result'); if ($this->success !== null) { - if (!is_array($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRING, count($this->success)); - { - foreach ($this->success as $iter440) - { - $xfer += $output->writeString($iter440); - } - } - $output->writeListEnd(); - } + $xfer += $output->writeFieldBegin('success', TType::STRING, 0); + $xfer += $output->writeString($this->success); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -15788,11 +18209,6 @@ class ThriftHiveMetastore_get_partition_names_ps_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); - $xfer += $this->o2->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -15800,53 +18216,29 @@ class ThriftHiveMetastore_get_partition_names_ps_result { } -class ThriftHiveMetastore_get_partitions_by_filter_args { +class ThriftHiveMetastore_partition_name_to_vals_args { static $_TSPEC; - public $db_name = null; - public $tbl_name = null; - public $filter = null; - public $max_parts = -1; + public $part_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', - 'type' => TType::STRING, - ), - 2 => array( - 'var' => 'tbl_name', - 'type' => TType::STRING, - ), - 3 => array( - 'var' => 'filter', + 'var' => 'part_name', 'type' => TType::STRING, ), - 4 => array( - 'var' => 'max_parts', - 'type' => TType::I16, - ), ); - } - if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; - } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; - } - if (isset($vals['filter'])) { - $this->filter = $vals['filter']; - } - if (isset($vals['max_parts'])) { - $this->max_parts = $vals['max_parts']; + } + if (is_array($vals)) { + if (isset($vals['part_name'])) { + $this->part_name = $vals['part_name']; } } } public function getName() { - return 'ThriftHiveMetastore_get_partitions_by_filter_args'; + return 'ThriftHiveMetastore_partition_name_to_vals_args'; } public function read($input) @@ -15866,28 +18258,7 @@ class ThriftHiveMetastore_get_partitions_by_filter_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->filter); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::I16) { - $xfer += $input->readI16($this->max_parts); + $xfer += $input->readString($this->part_name); } else { $xfer += $input->skip($ftype); } @@ -15904,25 +18275,10 @@ class ThriftHiveMetastore_get_partitions_by_filter_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_by_filter_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->filter !== null) { - $xfer += $output->writeFieldBegin('filter', TType::STRING, 3); - $xfer += $output->writeString($this->filter); - $xfer += $output->writeFieldEnd(); - } - if ($this->max_parts !== null) { - $xfer += $output->writeFieldBegin('max_parts', TType::I16, 4); - $xfer += $output->writeI16($this->max_parts); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_partition_name_to_vals_args'); + if ($this->part_name !== null) { + $xfer += $output->writeFieldBegin('part_name', TType::STRING, 1); + $xfer += $output->writeString($this->part_name); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -15932,12 +18288,11 @@ class ThriftHiveMetastore_get_partitions_by_filter_args { } -class ThriftHiveMetastore_get_partitions_by_filter_result { +class ThriftHiveMetastore_partition_name_to_vals_result { static $_TSPEC; public $success = null; public $o1 = null; - public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -15945,10 +18300,9 @@ class ThriftHiveMetastore_get_partitions_by_filter_result { 0 => array( 'var' => 'success', 'type' => TType::LST, - 'etype' => TType::STRUCT, + 'etype' => TType::STRING, 'elem' => array( - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', + 'type' => TType::STRING, ), ), 1 => array( @@ -15956,11 +18310,6 @@ class ThriftHiveMetastore_get_partitions_by_filter_result { 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), - 2 => array( - 'var' => 'o2', - 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', - ), ); } if (is_array($vals)) { @@ -15970,14 +18319,11 @@ class ThriftHiveMetastore_get_partitions_by_filter_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } - if (isset($vals['o2'])) { - $this->o2 = $vals['o2']; - } } } public function getName() { - return 'ThriftHiveMetastore_get_partitions_by_filter_result'; + return 'ThriftHiveMetastore_partition_name_to_vals_result'; } public function read($input) @@ -15998,15 +18344,14 @@ class ThriftHiveMetastore_get_partitions_by_filter_result { case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size441 = 0; - $_etype444 = 0; - $xfer += $input->readListBegin($_etype444, $_size441); - for ($_i445 = 0; $_i445 < $_size441; ++$_i445) + $_size483 = 0; + $_etype486 = 0; + $xfer += $input->readListBegin($_etype486, $_size483); + for ($_i487 = 0; $_i487 < $_size483; ++$_i487) { - $elem446 = null; - $elem446 = new \metastore\Partition(); - $xfer += $elem446->read($input); - $this->success []= $elem446; + $elem488 = null; + $xfer += $input->readString($elem488); + $this->success []= $elem488; } $xfer += $input->readListEnd(); } else { @@ -16021,14 +18366,6 @@ class ThriftHiveMetastore_get_partitions_by_filter_result { $xfer += $input->skip($ftype); } break; - case 2: - if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\NoSuchObjectException(); - $xfer += $this->o2->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -16041,18 +18378,18 @@ class ThriftHiveMetastore_get_partitions_by_filter_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_by_filter_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_partition_name_to_vals_result'); if ($this->success !== null) { if (!is_array($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } $xfer += $output->writeFieldBegin('success', TType::LST, 0); { - $output->writeListBegin(TType::STRUCT, count($this->success)); + $output->writeListBegin(TType::STRING, count($this->success)); { - foreach ($this->success as $iter447) + foreach ($this->success as $iter489) { - $xfer += $iter447->write($output); + $xfer += $output->writeString($iter489); } } $output->writeListEnd(); @@ -16064,11 +18401,6 @@ class ThriftHiveMetastore_get_partitions_by_filter_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); - $xfer += $this->o2->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -16076,49 +18408,29 @@ class ThriftHiveMetastore_get_partitions_by_filter_result { } -class ThriftHiveMetastore_get_partitions_by_names_args { +class ThriftHiveMetastore_partition_name_to_spec_args { static $_TSPEC; - public $db_name = null; - public $tbl_name = null; - public $names = null; + public $part_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', - 'type' => TType::STRING, - ), - 2 => array( - 'var' => 'tbl_name', + 'var' => 'part_name', 'type' => TType::STRING, ), - 3 => array( - 'var' => 'names', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), - ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; - } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; - } - if (isset($vals['names'])) { - $this->names = $vals['names']; + if (isset($vals['part_name'])) { + $this->part_name = $vals['part_name']; } } } public function getName() { - return 'ThriftHiveMetastore_get_partitions_by_names_args'; + return 'ThriftHiveMetastore_partition_name_to_spec_args'; } public function read($input) @@ -16138,31 +18450,7 @@ class ThriftHiveMetastore_get_partitions_by_names_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::LST) { - $this->names = array(); - $_size448 = 0; - $_etype451 = 0; - $xfer += $input->readListBegin($_etype451, $_size448); - for ($_i452 = 0; $_i452 < $_size448; ++$_i452) - { - $elem453 = null; - $xfer += $input->readString($elem453); - $this->names []= $elem453; - } - $xfer += $input->readListEnd(); + $xfer += $input->readString($this->part_name); } else { $xfer += $input->skip($ftype); } @@ -16179,32 +18467,10 @@ class ThriftHiveMetastore_get_partitions_by_names_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_by_names_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->names !== null) { - if (!is_array($this->names)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('names', TType::LST, 3); - { - $output->writeListBegin(TType::STRING, count($this->names)); - { - foreach ($this->names as $iter454) - { - $xfer += $output->writeString($iter454); - } - } - $output->writeListEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_partition_name_to_spec_args'); + if ($this->part_name !== null) { + $xfer += $output->writeFieldBegin('part_name', TType::STRING, 1); + $xfer += $output->writeString($this->part_name); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -16214,23 +18480,25 @@ class ThriftHiveMetastore_get_partitions_by_names_args { } -class ThriftHiveMetastore_get_partitions_by_names_result { +class ThriftHiveMetastore_partition_name_to_spec_result { static $_TSPEC; public $success = null; public $o1 = null; - public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRUCT, - 'elem' => array( - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', + 'type' => TType::MAP, + 'ktype' => TType::STRING, + 'vtype' => TType::STRING, + 'key' => array( + 'type' => TType::STRING, + ), + 'val' => array( + 'type' => TType::STRING, ), ), 1 => array( @@ -16238,11 +18506,6 @@ class ThriftHiveMetastore_get_partitions_by_names_result { 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), - 2 => array( - 'var' => 'o2', - 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', - ), ); } if (is_array($vals)) { @@ -16252,14 +18515,11 @@ class ThriftHiveMetastore_get_partitions_by_names_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } - if (isset($vals['o2'])) { - $this->o2 = $vals['o2']; - } } } public function getName() { - return 'ThriftHiveMetastore_get_partitions_by_names_result'; + return 'ThriftHiveMetastore_partition_name_to_spec_result'; } public function read($input) @@ -16278,19 +18538,21 @@ class ThriftHiveMetastore_get_partitions_by_names_result { switch ($fid) { case 0: - if ($ftype == TType::LST) { + if ($ftype == TType::MAP) { $this->success = array(); - $_size455 = 0; - $_etype458 = 0; - $xfer += $input->readListBegin($_etype458, $_size455); - for ($_i459 = 0; $_i459 < $_size455; ++$_i459) + $_size490 = 0; + $_ktype491 = 0; + $_vtype492 = 0; + $xfer += $input->readMapBegin($_ktype491, $_vtype492, $_size490); + for ($_i494 = 0; $_i494 < $_size490; ++$_i494) { - $elem460 = null; - $elem460 = new \metastore\Partition(); - $xfer += $elem460->read($input); - $this->success []= $elem460; + $key495 = ''; + $val496 = ''; + $xfer += $input->readString($key495); + $xfer += $input->readString($val496); + $this->success[$key495] = $val496; } - $xfer += $input->readListEnd(); + $xfer += $input->readMapEnd(); } else { $xfer += $input->skip($ftype); } @@ -16303,14 +18565,6 @@ class ThriftHiveMetastore_get_partitions_by_names_result { $xfer += $input->skip($ftype); } break; - case 2: - if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\NoSuchObjectException(); - $xfer += $this->o2->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -16323,21 +18577,22 @@ class ThriftHiveMetastore_get_partitions_by_names_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_by_names_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_partition_name_to_spec_result'); if ($this->success !== null) { if (!is_array($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); + $xfer += $output->writeFieldBegin('success', TType::MAP, 0); { - $output->writeListBegin(TType::STRUCT, count($this->success)); + $output->writeMapBegin(TType::STRING, TType::STRING, count($this->success)); { - foreach ($this->success as $iter461) + foreach ($this->success as $kiter497 => $viter498) { - $xfer += $iter461->write($output); + $xfer += $output->writeString($kiter497); + $xfer += $output->writeString($viter498); } } - $output->writeListEnd(); + $output->writeMapEnd(); } $xfer += $output->writeFieldEnd(); } @@ -16346,11 +18601,6 @@ class ThriftHiveMetastore_get_partitions_by_names_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); - $xfer += $this->o2->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -16358,12 +18608,13 @@ class ThriftHiveMetastore_get_partitions_by_names_result { } -class ThriftHiveMetastore_alter_partition_args { +class ThriftHiveMetastore_markPartitionForEvent_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; - public $new_part = null; + public $part_vals = null; + public $eventType = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -16377,9 +18628,20 @@ class ThriftHiveMetastore_alter_partition_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'new_part', - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', + 'var' => 'part_vals', + 'type' => TType::MAP, + 'ktype' => TType::STRING, + 'vtype' => TType::STRING, + 'key' => array( + 'type' => TType::STRING, + ), + 'val' => array( + 'type' => TType::STRING, + ), + ), + 4 => array( + 'var' => 'eventType', + 'type' => TType::I32, ), ); } @@ -16390,14 +18652,17 @@ class ThriftHiveMetastore_alter_partition_args { if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['new_part'])) { - $this->new_part = $vals['new_part']; + if (isset($vals['part_vals'])) { + $this->part_vals = $vals['part_vals']; + } + if (isset($vals['eventType'])) { + $this->eventType = $vals['eventType']; } } } public function getName() { - return 'ThriftHiveMetastore_alter_partition_args'; + return 'ThriftHiveMetastore_markPartitionForEvent_args'; } public function read($input) @@ -16430,9 +18695,28 @@ class ThriftHiveMetastore_alter_partition_args { } break; case 3: - if ($ftype == TType::STRUCT) { - $this->new_part = new \metastore\Partition(); - $xfer += $this->new_part->read($input); + if ($ftype == TType::MAP) { + $this->part_vals = array(); + $_size499 = 0; + $_ktype500 = 0; + $_vtype501 = 0; + $xfer += $input->readMapBegin($_ktype500, $_vtype501, $_size499); + for ($_i503 = 0; $_i503 < $_size499; ++$_i503) + { + $key504 = ''; + $val505 = ''; + $xfer += $input->readString($key504); + $xfer += $input->readString($val505); + $this->part_vals[$key504] = $val505; + } + $xfer += $input->readMapEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::I32) { + $xfer += $input->readI32($this->eventType); } else { $xfer += $input->skip($ftype); } @@ -16449,7 +18733,7 @@ class ThriftHiveMetastore_alter_partition_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_partition_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_markPartitionForEvent_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -16460,12 +18744,27 @@ class ThriftHiveMetastore_alter_partition_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->new_part !== null) { - if (!is_object($this->new_part)) { + if ($this->part_vals !== null) { + if (!is_array($this->part_vals)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('new_part', TType::STRUCT, 3); - $xfer += $this->new_part->write($output); + $xfer += $output->writeFieldBegin('part_vals', TType::MAP, 3); + { + $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals)); + { + foreach ($this->part_vals as $kiter506 => $viter507) + { + $xfer += $output->writeString($kiter506); + $xfer += $output->writeString($viter507); + } + } + $output->writeMapEnd(); + } + $xfer += $output->writeFieldEnd(); + } + if ($this->eventType !== null) { + $xfer += $output->writeFieldBegin('eventType', TType::I32, 4); + $xfer += $output->writeI32($this->eventType); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -16475,11 +18774,15 @@ class ThriftHiveMetastore_alter_partition_args { } -class ThriftHiveMetastore_alter_partition_result { +class ThriftHiveMetastore_markPartitionForEvent_result { static $_TSPEC; public $o1 = null; public $o2 = null; + public $o3 = null; + public $o4 = null; + public $o5 = null; + public $o6 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -16487,12 +18790,32 @@ class ThriftHiveMetastore_alter_partition_result { 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidOperationException', + 'class' => '\metastore\MetaException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchObjectException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, + 'class' => '\metastore\UnknownDBException', + ), + 4 => array( + 'var' => 'o4', + 'type' => TType::STRUCT, + 'class' => '\metastore\UnknownTableException', + ), + 5 => array( + 'var' => 'o5', + 'type' => TType::STRUCT, + 'class' => '\metastore\UnknownPartitionException', + ), + 6 => array( + 'var' => 'o6', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidPartitionException', ), ); } @@ -16503,11 +18826,23 @@ class ThriftHiveMetastore_alter_partition_result { if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } + if (isset($vals['o4'])) { + $this->o4 = $vals['o4']; + } + if (isset($vals['o5'])) { + $this->o5 = $vals['o5']; + } + if (isset($vals['o6'])) { + $this->o6 = $vals['o6']; + } } } public function getName() { - return 'ThriftHiveMetastore_alter_partition_result'; + return 'ThriftHiveMetastore_markPartitionForEvent_result'; } public function read($input) @@ -16527,7 +18862,7 @@ class ThriftHiveMetastore_alter_partition_result { { case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidOperationException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -16535,12 +18870,44 @@ class ThriftHiveMetastore_alter_partition_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); + $this->o2 = new \metastore\NoSuchObjectException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\UnknownDBException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRUCT) { + $this->o4 = new \metastore\UnknownTableException(); + $xfer += $this->o4->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 5: + if ($ftype == TType::STRUCT) { + $this->o5 = new \metastore\UnknownPartitionException(); + $xfer += $this->o5->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 6: + if ($ftype == TType::STRUCT) { + $this->o6 = new \metastore\InvalidPartitionException(); + $xfer += $this->o6->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -16553,7 +18920,7 @@ class ThriftHiveMetastore_alter_partition_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_partition_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_markPartitionForEvent_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -16564,6 +18931,26 @@ class ThriftHiveMetastore_alter_partition_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o4 !== null) { + $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); + $xfer += $this->o4->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o5 !== null) { + $xfer += $output->writeFieldBegin('o5', TType::STRUCT, 5); + $xfer += $this->o5->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o6 !== null) { + $xfer += $output->writeFieldBegin('o6', TType::STRUCT, 6); + $xfer += $this->o6->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -16571,12 +18958,13 @@ class ThriftHiveMetastore_alter_partition_result { } -class ThriftHiveMetastore_alter_partitions_args { +class ThriftHiveMetastore_isPartitionMarkedForEvent_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; - public $new_parts = null; + public $part_vals = null; + public $eventType = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -16590,14 +18978,21 @@ class ThriftHiveMetastore_alter_partitions_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'new_parts', - 'type' => TType::LST, - 'etype' => TType::STRUCT, - 'elem' => array( - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', + 'var' => 'part_vals', + 'type' => TType::MAP, + 'ktype' => TType::STRING, + 'vtype' => TType::STRING, + 'key' => array( + 'type' => TType::STRING, + ), + 'val' => array( + 'type' => TType::STRING, ), ), + 4 => array( + 'var' => 'eventType', + 'type' => TType::I32, + ), ); } if (is_array($vals)) { @@ -16607,14 +19002,17 @@ class ThriftHiveMetastore_alter_partitions_args { if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['new_parts'])) { - $this->new_parts = $vals['new_parts']; + if (isset($vals['part_vals'])) { + $this->part_vals = $vals['part_vals']; + } + if (isset($vals['eventType'])) { + $this->eventType = $vals['eventType']; } } } public function getName() { - return 'ThriftHiveMetastore_alter_partitions_args'; + return 'ThriftHiveMetastore_isPartitionMarkedForEvent_args'; } public function read($input) @@ -16647,19 +19045,28 @@ class ThriftHiveMetastore_alter_partitions_args { } break; case 3: - if ($ftype == TType::LST) { - $this->new_parts = array(); - $_size462 = 0; - $_etype465 = 0; - $xfer += $input->readListBegin($_etype465, $_size462); - for ($_i466 = 0; $_i466 < $_size462; ++$_i466) + if ($ftype == TType::MAP) { + $this->part_vals = array(); + $_size508 = 0; + $_ktype509 = 0; + $_vtype510 = 0; + $xfer += $input->readMapBegin($_ktype509, $_vtype510, $_size508); + for ($_i512 = 0; $_i512 < $_size508; ++$_i512) { - $elem467 = null; - $elem467 = new \metastore\Partition(); - $xfer += $elem467->read($input); - $this->new_parts []= $elem467; + $key513 = ''; + $val514 = ''; + $xfer += $input->readString($key513); + $xfer += $input->readString($val514); + $this->part_vals[$key513] = $val514; } - $xfer += $input->readListEnd(); + $xfer += $input->readMapEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::I32) { + $xfer += $input->readI32($this->eventType); } else { $xfer += $input->skip($ftype); } @@ -16676,7 +19083,7 @@ class ThriftHiveMetastore_alter_partitions_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_partitions_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_isPartitionMarkedForEvent_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -16687,23 +19094,29 @@ class ThriftHiveMetastore_alter_partitions_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->new_parts !== null) { - if (!is_array($this->new_parts)) { + if ($this->part_vals !== null) { + if (!is_array($this->part_vals)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('new_parts', TType::LST, 3); + $xfer += $output->writeFieldBegin('part_vals', TType::MAP, 3); { - $output->writeListBegin(TType::STRUCT, count($this->new_parts)); + $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals)); { - foreach ($this->new_parts as $iter468) + foreach ($this->part_vals as $kiter515 => $viter516) { - $xfer += $iter468->write($output); + $xfer += $output->writeString($kiter515); + $xfer += $output->writeString($viter516); } } - $output->writeListEnd(); + $output->writeMapEnd(); } $xfer += $output->writeFieldEnd(); } + if ($this->eventType !== null) { + $xfer += $output->writeFieldBegin('eventType', TType::I32, 4); + $xfer += $output->writeI32($this->eventType); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -16711,39 +19124,83 @@ class ThriftHiveMetastore_alter_partitions_args { } -class ThriftHiveMetastore_alter_partitions_result { +class ThriftHiveMetastore_isPartitionMarkedForEvent_result { static $_TSPEC; + public $success = null; public $o1 = null; public $o2 = null; + public $o3 = null; + public $o4 = null; + public $o5 = null; + public $o6 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::BOOL, + ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidOperationException', + 'class' => '\metastore\MetaException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchObjectException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, + 'class' => '\metastore\UnknownDBException', + ), + 4 => array( + 'var' => 'o4', + 'type' => TType::STRUCT, + 'class' => '\metastore\UnknownTableException', + ), + 5 => array( + 'var' => 'o5', + 'type' => TType::STRUCT, + 'class' => '\metastore\UnknownPartitionException', + ), + 6 => array( + 'var' => 'o6', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidPartitionException', ), ); } if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } + if (isset($vals['o4'])) { + $this->o4 = $vals['o4']; + } + if (isset($vals['o5'])) { + $this->o5 = $vals['o5']; + } + if (isset($vals['o6'])) { + $this->o6 = $vals['o6']; + } } } public function getName() { - return 'ThriftHiveMetastore_alter_partitions_result'; + return 'ThriftHiveMetastore_isPartitionMarkedForEvent_result'; } public function read($input) @@ -16761,9 +19218,16 @@ class ThriftHiveMetastore_alter_partitions_result { } switch ($fid) { + case 0: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->success); + } else { + $xfer += $input->skip($ftype); + } + break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidOperationException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -16771,12 +19235,44 @@ class ThriftHiveMetastore_alter_partitions_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); + $this->o2 = new \metastore\NoSuchObjectException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\UnknownDBException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRUCT) { + $this->o4 = new \metastore\UnknownTableException(); + $xfer += $this->o4->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 5: + if ($ftype == TType::STRUCT) { + $this->o5 = new \metastore\UnknownPartitionException(); + $xfer += $this->o5->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 6: + if ($ftype == TType::STRUCT) { + $this->o6 = new \metastore\InvalidPartitionException(); + $xfer += $this->o6->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -16789,7 +19285,12 @@ class ThriftHiveMetastore_alter_partitions_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_partitions_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_isPartitionMarkedForEvent_result'); + if ($this->success !== null) { + $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); + $xfer += $output->writeBool($this->success); + $xfer += $output->writeFieldEnd(); + } if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -16800,6 +19301,26 @@ class ThriftHiveMetastore_alter_partitions_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o4 !== null) { + $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); + $xfer += $this->o4->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o5 !== null) { + $xfer += $output->writeFieldBegin('o5', TType::STRUCT, 5); + $xfer += $this->o5->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o6 !== null) { + $xfer += $output->writeFieldBegin('o6', TType::STRUCT, 6); + $xfer += $this->o6->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -16807,55 +19328,39 @@ class ThriftHiveMetastore_alter_partitions_result { } -class ThriftHiveMetastore_alter_partition_with_environment_context_args { +class ThriftHiveMetastore_add_index_args { static $_TSPEC; - public $db_name = null; - public $tbl_name = null; - public $new_part = null; - public $environment_context = null; + public $new_index = null; + public $index_table = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', - 'type' => TType::STRING, - ), - 2 => array( - 'var' => 'tbl_name', - 'type' => TType::STRING, - ), - 3 => array( - 'var' => 'new_part', + 'var' => 'new_index', 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', + 'class' => '\metastore\Index', ), - 4 => array( - 'var' => 'environment_context', + 2 => array( + 'var' => 'index_table', 'type' => TType::STRUCT, - 'class' => '\metastore\EnvironmentContext', + 'class' => '\metastore\Table', ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; - } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; - } - if (isset($vals['new_part'])) { - $this->new_part = $vals['new_part']; + if (isset($vals['new_index'])) { + $this->new_index = $vals['new_index']; } - if (isset($vals['environment_context'])) { - $this->environment_context = $vals['environment_context']; + if (isset($vals['index_table'])) { + $this->index_table = $vals['index_table']; } } } public function getName() { - return 'ThriftHiveMetastore_alter_partition_with_environment_context_args'; + return 'ThriftHiveMetastore_add_index_args'; } public function read($input) @@ -16874,31 +19379,17 @@ class ThriftHiveMetastore_alter_partition_with_environment_context_args { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: if ($ftype == TType::STRUCT) { - $this->new_part = new \metastore\Partition(); - $xfer += $this->new_part->read($input); + $this->new_index = new \metastore\Index(); + $xfer += $this->new_index->read($input); } else { $xfer += $input->skip($ftype); } break; - case 4: + case 2: if ($ftype == TType::STRUCT) { - $this->environment_context = new \metastore\EnvironmentContext(); - $xfer += $this->environment_context->read($input); + $this->index_table = new \metastore\Table(); + $xfer += $this->index_table->read($input); } else { $xfer += $input->skip($ftype); } @@ -16915,31 +19406,21 @@ class ThriftHiveMetastore_alter_partition_with_environment_context_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_partition_with_environment_context_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->new_part !== null) { - if (!is_object($this->new_part)) { + $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_index_args'); + if ($this->new_index !== null) { + if (!is_object($this->new_index)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('new_part', TType::STRUCT, 3); - $xfer += $this->new_part->write($output); + $xfer += $output->writeFieldBegin('new_index', TType::STRUCT, 1); + $xfer += $this->new_index->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->environment_context !== null) { - if (!is_object($this->environment_context)) { + if ($this->index_table !== null) { + if (!is_object($this->index_table)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 4); - $xfer += $this->environment_context->write($output); + $xfer += $output->writeFieldBegin('index_table', TType::STRUCT, 2); + $xfer += $this->index_table->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -16949,39 +19430,57 @@ class ThriftHiveMetastore_alter_partition_with_environment_context_args { } -class ThriftHiveMetastore_alter_partition_with_environment_context_result { +class ThriftHiveMetastore_add_index_result { static $_TSPEC; + public $success = null; public $o1 = null; public $o2 = null; + public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::STRUCT, + 'class' => '\metastore\Index', + ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidOperationException', + 'class' => '\metastore\InvalidObjectException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, + 'class' => '\metastore\AlreadyExistsException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), ); } if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } } } public function getName() { - return 'ThriftHiveMetastore_alter_partition_with_environment_context_result'; + return 'ThriftHiveMetastore_add_index_result'; } public function read($input) @@ -16999,9 +19498,17 @@ class ThriftHiveMetastore_alter_partition_with_environment_context_result { } switch ($fid) { + case 0: + if ($ftype == TType::STRUCT) { + $this->success = new \metastore\Index(); + $xfer += $this->success->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidOperationException(); + $this->o1 = new \metastore\InvalidObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -17009,12 +19516,20 @@ class ThriftHiveMetastore_alter_partition_with_environment_context_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); + $this->o2 = new \metastore\AlreadyExistsException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\MetaException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -17027,7 +19542,15 @@ class ThriftHiveMetastore_alter_partition_with_environment_context_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_partition_with_environment_context_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_index_result'); + if ($this->success !== null) { + if (!is_object($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); + $xfer += $output->writeFieldEnd(); + } if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -17038,6 +19561,11 @@ class ThriftHiveMetastore_alter_partition_with_environment_context_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -17045,58 +19573,54 @@ class ThriftHiveMetastore_alter_partition_with_environment_context_result { } -class ThriftHiveMetastore_rename_partition_args { +class ThriftHiveMetastore_alter_index_args { static $_TSPEC; - public $db_name = null; - public $tbl_name = null; - public $part_vals = null; - public $new_part = null; + public $dbname = null; + public $base_tbl_name = null; + public $idx_name = null; + public $new_idx = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', + 'var' => 'dbname', 'type' => TType::STRING, ), 2 => array( - 'var' => 'tbl_name', + 'var' => 'base_tbl_name', 'type' => TType::STRING, ), 3 => array( - 'var' => 'part_vals', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), + 'var' => 'idx_name', + 'type' => TType::STRING, ), 4 => array( - 'var' => 'new_part', + 'var' => 'new_idx', 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', + 'class' => '\metastore\Index', ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; + if (isset($vals['dbname'])) { + $this->dbname = $vals['dbname']; } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; + if (isset($vals['base_tbl_name'])) { + $this->base_tbl_name = $vals['base_tbl_name']; } - if (isset($vals['part_vals'])) { - $this->part_vals = $vals['part_vals']; + if (isset($vals['idx_name'])) { + $this->idx_name = $vals['idx_name']; } - if (isset($vals['new_part'])) { - $this->new_part = $vals['new_part']; + if (isset($vals['new_idx'])) { + $this->new_idx = $vals['new_idx']; } } } public function getName() { - return 'ThriftHiveMetastore_rename_partition_args'; + return 'ThriftHiveMetastore_alter_index_args'; } public function read($input) @@ -17116,39 +19640,29 @@ class ThriftHiveMetastore_rename_partition_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); + $xfer += $input->readString($this->dbname); } else { $xfer += $input->skip($ftype); } break; case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); + $xfer += $input->readString($this->base_tbl_name); } else { $xfer += $input->skip($ftype); } break; case 3: - if ($ftype == TType::LST) { - $this->part_vals = array(); - $_size469 = 0; - $_etype472 = 0; - $xfer += $input->readListBegin($_etype472, $_size469); - for ($_i473 = 0; $_i473 < $_size469; ++$_i473) - { - $elem474 = null; - $xfer += $input->readString($elem474); - $this->part_vals []= $elem474; - } - $xfer += $input->readListEnd(); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->idx_name); } else { $xfer += $input->skip($ftype); } break; case 4: if ($ftype == TType::STRUCT) { - $this->new_part = new \metastore\Partition(); - $xfer += $this->new_part->read($input); + $this->new_idx = new \metastore\Index(); + $xfer += $this->new_idx->read($input); } else { $xfer += $input->skip($ftype); } @@ -17165,40 +19679,28 @@ class ThriftHiveMetastore_rename_partition_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_rename_partition_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_index_args'); + if ($this->dbname !== null) { + $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); + $xfer += $output->writeString($this->dbname); $xfer += $output->writeFieldEnd(); } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); + if ($this->base_tbl_name !== null) { + $xfer += $output->writeFieldBegin('base_tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->base_tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->part_vals !== null) { - if (!is_array($this->part_vals)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); - { - $output->writeListBegin(TType::STRING, count($this->part_vals)); - { - foreach ($this->part_vals as $iter475) - { - $xfer += $output->writeString($iter475); - } - } - $output->writeListEnd(); - } + if ($this->idx_name !== null) { + $xfer += $output->writeFieldBegin('idx_name', TType::STRING, 3); + $xfer += $output->writeString($this->idx_name); $xfer += $output->writeFieldEnd(); } - if ($this->new_part !== null) { - if (!is_object($this->new_part)) { + if ($this->new_idx !== null) { + if (!is_object($this->new_idx)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('new_part', TType::STRUCT, 4); - $xfer += $this->new_part->write($output); + $xfer += $output->writeFieldBegin('new_idx', TType::STRUCT, 4); + $xfer += $this->new_idx->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -17208,7 +19710,7 @@ class ThriftHiveMetastore_rename_partition_args { } -class ThriftHiveMetastore_rename_partition_result { +class ThriftHiveMetastore_alter_index_result { static $_TSPEC; public $o1 = null; @@ -17240,7 +19742,7 @@ class ThriftHiveMetastore_rename_partition_result { } public function getName() { - return 'ThriftHiveMetastore_rename_partition_result'; + return 'ThriftHiveMetastore_alter_index_result'; } public function read($input) @@ -17286,7 +19788,7 @@ class ThriftHiveMetastore_rename_partition_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_rename_partition_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_index_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -17304,41 +19806,53 @@ class ThriftHiveMetastore_rename_partition_result { } -class ThriftHiveMetastore_partition_name_has_valid_characters_args { +class ThriftHiveMetastore_drop_index_by_name_args { static $_TSPEC; - public $part_vals = null; - public $throw_exception = null; + public $db_name = null; + public $tbl_name = null; + public $index_name = null; + public $deleteData = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'part_vals', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), + 'var' => 'db_name', + 'type' => TType::STRING, ), 2 => array( - 'var' => 'throw_exception', + 'var' => 'tbl_name', + 'type' => TType::STRING, + ), + 3 => array( + 'var' => 'index_name', + 'type' => TType::STRING, + ), + 4 => array( + 'var' => 'deleteData', 'type' => TType::BOOL, ), ); } if (is_array($vals)) { - if (isset($vals['part_vals'])) { - $this->part_vals = $vals['part_vals']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; } - if (isset($vals['throw_exception'])) { - $this->throw_exception = $vals['throw_exception']; + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; + } + if (isset($vals['index_name'])) { + $this->index_name = $vals['index_name']; + } + if (isset($vals['deleteData'])) { + $this->deleteData = $vals['deleteData']; } } } public function getName() { - return 'ThriftHiveMetastore_partition_name_has_valid_characters_args'; + return 'ThriftHiveMetastore_drop_index_by_name_args'; } public function read($input) @@ -17357,25 +19871,29 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args { switch ($fid) { case 1: - if ($ftype == TType::LST) { - $this->part_vals = array(); - $_size476 = 0; - $_etype479 = 0; - $xfer += $input->readListBegin($_etype479, $_size476); - for ($_i480 = 0; $_i480 < $_size476; ++$_i480) - { - $elem481 = null; - $xfer += $input->readString($elem481); - $this->part_vals []= $elem481; - } - $xfer += $input->readListEnd(); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->db_name); } else { $xfer += $input->skip($ftype); } break; case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->tbl_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->index_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->throw_exception); + $xfer += $input->readBool($this->deleteData); } else { $xfer += $input->skip($ftype); } @@ -17392,27 +19910,25 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_partition_name_has_valid_characters_args'); - if ($this->part_vals !== null) { - if (!is_array($this->part_vals)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('part_vals', TType::LST, 1); - { - $output->writeListBegin(TType::STRING, count($this->part_vals)); - { - foreach ($this->part_vals as $iter482) - { - $xfer += $output->writeString($iter482); - } - } - $output->writeListEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_index_by_name_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->index_name !== null) { + $xfer += $output->writeFieldBegin('index_name', TType::STRING, 3); + $xfer += $output->writeString($this->index_name); $xfer += $output->writeFieldEnd(); } - if ($this->throw_exception !== null) { - $xfer += $output->writeFieldBegin('throw_exception', TType::BOOL, 2); - $xfer += $output->writeBool($this->throw_exception); + if ($this->deleteData !== null) { + $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 4); + $xfer += $output->writeBool($this->deleteData); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -17422,11 +19938,12 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args { } -class ThriftHiveMetastore_partition_name_has_valid_characters_result { +class ThriftHiveMetastore_drop_index_by_name_result { static $_TSPEC; public $success = null; public $o1 = null; + public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -17438,6 +19955,11 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_result { 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, + 'class' => '\metastore\NoSuchObjectException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), ); @@ -17449,11 +19971,14 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } } } public function getName() { - return 'ThriftHiveMetastore_partition_name_has_valid_characters_result'; + return 'ThriftHiveMetastore_drop_index_by_name_result'; } public function read($input) @@ -17480,12 +20005,20 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_result { break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\MetaException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -17498,7 +20031,7 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_partition_name_has_valid_characters_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_index_by_name_result'); if ($this->success !== null) { $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); $xfer += $output->writeBool($this->success); @@ -17509,6 +20042,11 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -17516,37 +20054,45 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_result { } -class ThriftHiveMetastore_get_config_value_args { +class ThriftHiveMetastore_get_index_by_name_args { static $_TSPEC; - public $name = null; - public $defaultValue = null; + public $db_name = null; + public $tbl_name = null; + public $index_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'name', + 'var' => 'db_name', 'type' => TType::STRING, ), 2 => array( - 'var' => 'defaultValue', + 'var' => 'tbl_name', + 'type' => TType::STRING, + ), + 3 => array( + 'var' => 'index_name', 'type' => TType::STRING, ), ); } if (is_array($vals)) { - if (isset($vals['name'])) { - $this->name = $vals['name']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; } - if (isset($vals['defaultValue'])) { - $this->defaultValue = $vals['defaultValue']; + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; + } + if (isset($vals['index_name'])) { + $this->index_name = $vals['index_name']; } } } public function getName() { - return 'ThriftHiveMetastore_get_config_value_args'; + return 'ThriftHiveMetastore_get_index_by_name_args'; } public function read($input) @@ -17566,14 +20112,21 @@ class ThriftHiveMetastore_get_config_value_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->name); + $xfer += $input->readString($this->db_name); } else { $xfer += $input->skip($ftype); } break; case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->defaultValue); + $xfer += $input->readString($this->tbl_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->index_name); } else { $xfer += $input->skip($ftype); } @@ -17590,15 +20143,20 @@ class ThriftHiveMetastore_get_config_value_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_config_value_args'); - if ($this->name !== null) { - $xfer += $output->writeFieldBegin('name', TType::STRING, 1); - $xfer += $output->writeString($this->name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_index_by_name_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); $xfer += $output->writeFieldEnd(); } - if ($this->defaultValue !== null) { - $xfer += $output->writeFieldBegin('defaultValue', TType::STRING, 2); - $xfer += $output->writeString($this->defaultValue); + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->index_name !== null) { + $xfer += $output->writeFieldBegin('index_name', TType::STRING, 3); + $xfer += $output->writeString($this->index_name); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -17608,23 +20166,30 @@ class ThriftHiveMetastore_get_config_value_args { } -class ThriftHiveMetastore_get_config_value_result { +class ThriftHiveMetastore_get_index_by_name_result { static $_TSPEC; public $success = null; public $o1 = null; + public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::STRING, + 'type' => TType::STRUCT, + 'class' => '\metastore\Index', ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\ConfigValSecurityException', + 'class' => '\metastore\MetaException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\NoSuchObjectException', ), ); } @@ -17635,11 +20200,14 @@ class ThriftHiveMetastore_get_config_value_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } } } public function getName() { - return 'ThriftHiveMetastore_get_config_value_result'; + return 'ThriftHiveMetastore_get_index_by_name_result'; } public function read($input) @@ -17658,20 +20226,29 @@ class ThriftHiveMetastore_get_config_value_result { switch ($fid) { case 0: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->success); + if ($ftype == TType::STRUCT) { + $this->success = new \metastore\Index(); + $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\ConfigValSecurityException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\NoSuchObjectException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -17684,10 +20261,13 @@ class ThriftHiveMetastore_get_config_value_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_config_value_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_index_by_name_result'); if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::STRING, 0); - $xfer += $output->writeString($this->success); + if (!is_object($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -17695,6 +20275,11 @@ class ThriftHiveMetastore_get_config_value_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -17702,29 +20287,45 @@ class ThriftHiveMetastore_get_config_value_result { } -class ThriftHiveMetastore_partition_name_to_vals_args { +class ThriftHiveMetastore_get_indexes_args { static $_TSPEC; - public $part_name = null; + public $db_name = null; + public $tbl_name = null; + public $max_indexes = -1; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'part_name', + 'var' => 'db_name', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'tbl_name', 'type' => TType::STRING, ), + 3 => array( + 'var' => 'max_indexes', + 'type' => TType::I16, + ), ); } if (is_array($vals)) { - if (isset($vals['part_name'])) { - $this->part_name = $vals['part_name']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; + } + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; + } + if (isset($vals['max_indexes'])) { + $this->max_indexes = $vals['max_indexes']; } } } public function getName() { - return 'ThriftHiveMetastore_partition_name_to_vals_args'; + return 'ThriftHiveMetastore_get_indexes_args'; } public function read($input) @@ -17744,7 +20345,21 @@ class ThriftHiveMetastore_partition_name_to_vals_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->part_name); + $xfer += $input->readString($this->db_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->tbl_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::I16) { + $xfer += $input->readI16($this->max_indexes); } else { $xfer += $input->skip($ftype); } @@ -17761,10 +20376,20 @@ class ThriftHiveMetastore_partition_name_to_vals_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_partition_name_to_vals_args'); - if ($this->part_name !== null) { - $xfer += $output->writeFieldBegin('part_name', TType::STRING, 1); - $xfer += $output->writeString($this->part_name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_indexes_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->max_indexes !== null) { + $xfer += $output->writeFieldBegin('max_indexes', TType::I16, 3); + $xfer += $output->writeI16($this->max_indexes); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -17774,11 +20399,12 @@ class ThriftHiveMetastore_partition_name_to_vals_args { } -class ThriftHiveMetastore_partition_name_to_vals_result { +class ThriftHiveMetastore_get_indexes_result { static $_TSPEC; public $success = null; public $o1 = null; + public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -17786,14 +20412,20 @@ class ThriftHiveMetastore_partition_name_to_vals_result { 0 => array( 'var' => 'success', 'type' => TType::LST, - 'etype' => TType::STRING, + 'etype' => TType::STRUCT, 'elem' => array( - 'type' => TType::STRING, + 'type' => TType::STRUCT, + 'class' => '\metastore\Index', ), ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, + 'class' => '\metastore\NoSuchObjectException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), ); @@ -17805,11 +20437,14 @@ class ThriftHiveMetastore_partition_name_to_vals_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } } } public function getName() { - return 'ThriftHiveMetastore_partition_name_to_vals_result'; + return 'ThriftHiveMetastore_get_indexes_result'; } public function read($input) @@ -17830,14 +20465,15 @@ class ThriftHiveMetastore_partition_name_to_vals_result { case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size483 = 0; - $_etype486 = 0; - $xfer += $input->readListBegin($_etype486, $_size483); - for ($_i487 = 0; $_i487 < $_size483; ++$_i487) + $_size517 = 0; + $_etype520 = 0; + $xfer += $input->readListBegin($_etype520, $_size517); + for ($_i521 = 0; $_i521 < $_size517; ++$_i521) { - $elem488 = null; - $xfer += $input->readString($elem488); - $this->success []= $elem488; + $elem522 = null; + $elem522 = new \metastore\Index(); + $xfer += $elem522->read($input); + $this->success []= $elem522; } $xfer += $input->readListEnd(); } else { @@ -17846,12 +20482,20 @@ class ThriftHiveMetastore_partition_name_to_vals_result { break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\MetaException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -17864,18 +20508,18 @@ class ThriftHiveMetastore_partition_name_to_vals_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_partition_name_to_vals_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_indexes_result'); if ($this->success !== null) { if (!is_array($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } $xfer += $output->writeFieldBegin('success', TType::LST, 0); { - $output->writeListBegin(TType::STRING, count($this->success)); + $output->writeListBegin(TType::STRUCT, count($this->success)); { - foreach ($this->success as $iter489) + foreach ($this->success as $iter523) { - $xfer += $output->writeString($iter489); + $xfer += $iter523->write($output); } } $output->writeListEnd(); @@ -17887,6 +20531,11 @@ class ThriftHiveMetastore_partition_name_to_vals_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -17894,29 +20543,45 @@ class ThriftHiveMetastore_partition_name_to_vals_result { } -class ThriftHiveMetastore_partition_name_to_spec_args { +class ThriftHiveMetastore_get_index_names_args { static $_TSPEC; - public $part_name = null; + public $db_name = null; + public $tbl_name = null; + public $max_indexes = -1; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'part_name', + 'var' => 'db_name', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'tbl_name', 'type' => TType::STRING, ), + 3 => array( + 'var' => 'max_indexes', + 'type' => TType::I16, + ), ); } if (is_array($vals)) { - if (isset($vals['part_name'])) { - $this->part_name = $vals['part_name']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; + } + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; + } + if (isset($vals['max_indexes'])) { + $this->max_indexes = $vals['max_indexes']; } } } public function getName() { - return 'ThriftHiveMetastore_partition_name_to_spec_args'; + return 'ThriftHiveMetastore_get_index_names_args'; } public function read($input) @@ -17936,7 +20601,21 @@ class ThriftHiveMetastore_partition_name_to_spec_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->part_name); + $xfer += $input->readString($this->db_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->tbl_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::I16) { + $xfer += $input->readI16($this->max_indexes); } else { $xfer += $input->skip($ftype); } @@ -17953,10 +20632,20 @@ class ThriftHiveMetastore_partition_name_to_spec_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_partition_name_to_spec_args'); - if ($this->part_name !== null) { - $xfer += $output->writeFieldBegin('part_name', TType::STRING, 1); - $xfer += $output->writeString($this->part_name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_index_names_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->max_indexes !== null) { + $xfer += $output->writeFieldBegin('max_indexes', TType::I16, 3); + $xfer += $output->writeI16($this->max_indexes); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -17966,29 +20655,25 @@ class ThriftHiveMetastore_partition_name_to_spec_args { } -class ThriftHiveMetastore_partition_name_to_spec_result { +class ThriftHiveMetastore_get_index_names_result { static $_TSPEC; public $success = null; - public $o1 = null; + public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::MAP, - 'ktype' => TType::STRING, - 'vtype' => TType::STRING, - 'key' => array( - 'type' => TType::STRING, - ), - 'val' => array( + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( 'type' => TType::STRING, ), ), 1 => array( - 'var' => 'o1', + 'var' => 'o2', 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), @@ -17998,14 +20683,14 @@ class ThriftHiveMetastore_partition_name_to_spec_result { if (isset($vals['success'])) { $this->success = $vals['success']; } - if (isset($vals['o1'])) { - $this->o1 = $vals['o1']; + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; } } } public function getName() { - return 'ThriftHiveMetastore_partition_name_to_spec_result'; + return 'ThriftHiveMetastore_get_index_names_result'; } public function read($input) @@ -18024,29 +20709,26 @@ class ThriftHiveMetastore_partition_name_to_spec_result { switch ($fid) { case 0: - if ($ftype == TType::MAP) { + if ($ftype == TType::LST) { $this->success = array(); - $_size490 = 0; - $_ktype491 = 0; - $_vtype492 = 0; - $xfer += $input->readMapBegin($_ktype491, $_vtype492, $_size490); - for ($_i494 = 0; $_i494 < $_size490; ++$_i494) + $_size524 = 0; + $_etype527 = 0; + $xfer += $input->readListBegin($_etype527, $_size524); + for ($_i528 = 0; $_i528 < $_size524; ++$_i528) { - $key495 = ''; - $val496 = ''; - $xfer += $input->readString($key495); - $xfer += $input->readString($val496); - $this->success[$key495] = $val496; + $elem529 = null; + $xfer += $input->readString($elem529); + $this->success []= $elem529; } - $xfer += $input->readMapEnd(); + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); - $xfer += $this->o1->read($input); + $this->o2 = new \metastore\MetaException(); + $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } @@ -18063,28 +20745,27 @@ class ThriftHiveMetastore_partition_name_to_spec_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_partition_name_to_spec_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_index_names_result'); if ($this->success !== null) { if (!is_array($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('success', TType::MAP, 0); + $xfer += $output->writeFieldBegin('success', TType::LST, 0); { - $output->writeMapBegin(TType::STRING, TType::STRING, count($this->success)); + $output->writeListBegin(TType::STRING, count($this->success)); { - foreach ($this->success as $kiter497 => $viter498) + foreach ($this->success as $iter530) { - $xfer += $output->writeString($kiter497); - $xfer += $output->writeString($viter498); + $xfer += $output->writeString($iter530); } } - $output->writeMapEnd(); + $output->writeListEnd(); } $xfer += $output->writeFieldEnd(); } - if ($this->o1 !== null) { - $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); - $xfer += $this->o1->write($output); + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 1); + $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -18094,61 +20775,30 @@ class ThriftHiveMetastore_partition_name_to_spec_result { } -class ThriftHiveMetastore_markPartitionForEvent_args { +class ThriftHiveMetastore_update_table_column_statistics_args { static $_TSPEC; - public $db_name = null; - public $tbl_name = null; - public $part_vals = null; - public $eventType = null; + public $stats_obj = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', - 'type' => TType::STRING, - ), - 2 => array( - 'var' => 'tbl_name', - 'type' => TType::STRING, - ), - 3 => array( - 'var' => 'part_vals', - 'type' => TType::MAP, - 'ktype' => TType::STRING, - 'vtype' => TType::STRING, - 'key' => array( - 'type' => TType::STRING, - ), - 'val' => array( - 'type' => TType::STRING, - ), - ), - 4 => array( - 'var' => 'eventType', - 'type' => TType::I32, + 'var' => 'stats_obj', + 'type' => TType::STRUCT, + 'class' => '\metastore\ColumnStatistics', ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; - } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; - } - if (isset($vals['part_vals'])) { - $this->part_vals = $vals['part_vals']; - } - if (isset($vals['eventType'])) { - $this->eventType = $vals['eventType']; + if (isset($vals['stats_obj'])) { + $this->stats_obj = $vals['stats_obj']; } } } public function getName() { - return 'ThriftHiveMetastore_markPartitionForEvent_args'; + return 'ThriftHiveMetastore_update_table_column_statistics_args'; } public function read($input) @@ -18167,90 +20817,32 @@ class ThriftHiveMetastore_markPartitionForEvent_args { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::MAP) { - $this->part_vals = array(); - $_size499 = 0; - $_ktype500 = 0; - $_vtype501 = 0; - $xfer += $input->readMapBegin($_ktype500, $_vtype501, $_size499); - for ($_i503 = 0; $_i503 < $_size499; ++$_i503) - { - $key504 = ''; - $val505 = ''; - $xfer += $input->readString($key504); - $xfer += $input->readString($val505); - $this->part_vals[$key504] = $val505; - } - $xfer += $input->readMapEnd(); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::I32) { - $xfer += $input->readI32($this->eventType); - } else { - $xfer += $input->skip($ftype); - } - break; - default: - $xfer += $input->skip($ftype); - break; - } - $xfer += $input->readFieldEnd(); - } - $xfer += $input->readStructEnd(); - return $xfer; - } - - public function write($output) { - $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_markPartitionForEvent_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->part_vals !== null) { - if (!is_array($this->part_vals)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('part_vals', TType::MAP, 3); - { - $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals)); - { - foreach ($this->part_vals as $kiter506 => $viter507) - { - $xfer += $output->writeString($kiter506); - $xfer += $output->writeString($viter507); + if ($ftype == TType::STRUCT) { + $this->stats_obj = new \metastore\ColumnStatistics(); + $xfer += $this->stats_obj->read($input); + } else { + $xfer += $input->skip($ftype); } - } - $output->writeMapEnd(); + break; + default: + $xfer += $input->skip($ftype); + break; } - $xfer += $output->writeFieldEnd(); + $xfer += $input->readFieldEnd(); } - if ($this->eventType !== null) { - $xfer += $output->writeFieldBegin('eventType', TType::I32, 4); - $xfer += $output->writeI32($this->eventType); + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_update_table_column_statistics_args'); + if ($this->stats_obj !== null) { + if (!is_object($this->stats_obj)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('stats_obj', TType::STRUCT, 1); + $xfer += $this->stats_obj->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -18260,52 +20852,48 @@ class ThriftHiveMetastore_markPartitionForEvent_args { } -class ThriftHiveMetastore_markPartitionForEvent_result { +class ThriftHiveMetastore_update_table_column_statistics_result { static $_TSPEC; + public $success = null; public $o1 = null; public $o2 = null; public $o3 = null; public $o4 = null; - public $o5 = null; - public $o6 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::BOOL, + ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchObjectException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\InvalidObjectException', ), 3 => array( 'var' => 'o3', 'type' => TType::STRUCT, - 'class' => '\metastore\UnknownDBException', + 'class' => '\metastore\MetaException', ), 4 => array( 'var' => 'o4', 'type' => TType::STRUCT, - 'class' => '\metastore\UnknownTableException', - ), - 5 => array( - 'var' => 'o5', - 'type' => TType::STRUCT, - 'class' => '\metastore\UnknownPartitionException', - ), - 6 => array( - 'var' => 'o6', - 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidPartitionException', + 'class' => '\metastore\InvalidInputException', ), ); } if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } @@ -18318,17 +20906,11 @@ class ThriftHiveMetastore_markPartitionForEvent_result { if (isset($vals['o4'])) { $this->o4 = $vals['o4']; } - if (isset($vals['o5'])) { - $this->o5 = $vals['o5']; - } - if (isset($vals['o6'])) { - $this->o6 = $vals['o6']; - } } } public function getName() { - return 'ThriftHiveMetastore_markPartitionForEvent_result'; + return 'ThriftHiveMetastore_update_table_column_statistics_result'; } public function read($input) @@ -18346,9 +20928,16 @@ class ThriftHiveMetastore_markPartitionForEvent_result { } switch ($fid) { + case 0: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->success); + } else { + $xfer += $input->skip($ftype); + } + break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -18356,7 +20945,7 @@ class ThriftHiveMetastore_markPartitionForEvent_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\NoSuchObjectException(); + $this->o2 = new \metastore\InvalidObjectException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -18364,7 +20953,7 @@ class ThriftHiveMetastore_markPartitionForEvent_result { break; case 3: if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\UnknownDBException(); + $this->o3 = new \metastore\MetaException(); $xfer += $this->o3->read($input); } else { $xfer += $input->skip($ftype); @@ -18372,28 +20961,12 @@ class ThriftHiveMetastore_markPartitionForEvent_result { break; case 4: if ($ftype == TType::STRUCT) { - $this->o4 = new \metastore\UnknownTableException(); + $this->o4 = new \metastore\InvalidInputException(); $xfer += $this->o4->read($input); } else { $xfer += $input->skip($ftype); } break; - case 5: - if ($ftype == TType::STRUCT) { - $this->o5 = new \metastore\UnknownPartitionException(); - $xfer += $this->o5->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 6: - if ($ftype == TType::STRUCT) { - $this->o6 = new \metastore\InvalidPartitionException(); - $xfer += $this->o6->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -18406,7 +20979,12 @@ class ThriftHiveMetastore_markPartitionForEvent_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_markPartitionForEvent_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_update_table_column_statistics_result'); + if ($this->success !== null) { + $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); + $xfer += $output->writeBool($this->success); + $xfer += $output->writeFieldEnd(); + } if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -18427,16 +21005,6 @@ class ThriftHiveMetastore_markPartitionForEvent_result { $xfer += $this->o4->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o5 !== null) { - $xfer += $output->writeFieldBegin('o5', TType::STRUCT, 5); - $xfer += $this->o5->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->o6 !== null) { - $xfer += $output->writeFieldBegin('o6', TType::STRUCT, 6); - $xfer += $this->o6->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -18444,61 +21012,30 @@ class ThriftHiveMetastore_markPartitionForEvent_result { } -class ThriftHiveMetastore_isPartitionMarkedForEvent_args { +class ThriftHiveMetastore_update_partition_column_statistics_args { static $_TSPEC; - public $db_name = null; - public $tbl_name = null; - public $part_vals = null; - public $eventType = null; + public $stats_obj = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', - 'type' => TType::STRING, - ), - 2 => array( - 'var' => 'tbl_name', - 'type' => TType::STRING, - ), - 3 => array( - 'var' => 'part_vals', - 'type' => TType::MAP, - 'ktype' => TType::STRING, - 'vtype' => TType::STRING, - 'key' => array( - 'type' => TType::STRING, - ), - 'val' => array( - 'type' => TType::STRING, - ), - ), - 4 => array( - 'var' => 'eventType', - 'type' => TType::I32, + 'var' => 'stats_obj', + 'type' => TType::STRUCT, + 'class' => '\metastore\ColumnStatistics', ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; - } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; - } - if (isset($vals['part_vals'])) { - $this->part_vals = $vals['part_vals']; - } - if (isset($vals['eventType'])) { - $this->eventType = $vals['eventType']; + if (isset($vals['stats_obj'])) { + $this->stats_obj = $vals['stats_obj']; } } } public function getName() { - return 'ThriftHiveMetastore_isPartitionMarkedForEvent_args'; + return 'ThriftHiveMetastore_update_partition_column_statistics_args'; } public function read($input) @@ -18517,42 +21054,9 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::MAP) { - $this->part_vals = array(); - $_size508 = 0; - $_ktype509 = 0; - $_vtype510 = 0; - $xfer += $input->readMapBegin($_ktype509, $_vtype510, $_size508); - for ($_i512 = 0; $_i512 < $_size508; ++$_i512) - { - $key513 = ''; - $val514 = ''; - $xfer += $input->readString($key513); - $xfer += $input->readString($val514); - $this->part_vals[$key513] = $val514; - } - $xfer += $input->readMapEnd(); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::I32) { - $xfer += $input->readI32($this->eventType); + if ($ftype == TType::STRUCT) { + $this->stats_obj = new \metastore\ColumnStatistics(); + $xfer += $this->stats_obj->read($input); } else { $xfer += $input->skip($ftype); } @@ -18569,38 +21073,13 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_isPartitionMarkedForEvent_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->part_vals !== null) { - if (!is_array($this->part_vals)) { + $xfer += $output->writeStructBegin('ThriftHiveMetastore_update_partition_column_statistics_args'); + if ($this->stats_obj !== null) { + if (!is_object($this->stats_obj)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('part_vals', TType::MAP, 3); - { - $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals)); - { - foreach ($this->part_vals as $kiter515 => $viter516) - { - $xfer += $output->writeString($kiter515); - $xfer += $output->writeString($viter516); - } - } - $output->writeMapEnd(); - } - $xfer += $output->writeFieldEnd(); - } - if ($this->eventType !== null) { - $xfer += $output->writeFieldBegin('eventType', TType::I32, 4); - $xfer += $output->writeI32($this->eventType); + $xfer += $output->writeFieldBegin('stats_obj', TType::STRUCT, 1); + $xfer += $this->stats_obj->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -18610,7 +21089,7 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args { } -class ThriftHiveMetastore_isPartitionMarkedForEvent_result { +class ThriftHiveMetastore_update_partition_column_statistics_result { static $_TSPEC; public $success = null; @@ -18618,8 +21097,6 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_result { public $o2 = null; public $o3 = null; public $o4 = null; - public $o5 = null; - public $o6 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -18631,32 +21108,22 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_result { 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchObjectException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\InvalidObjectException', ), 3 => array( 'var' => 'o3', 'type' => TType::STRUCT, - 'class' => '\metastore\UnknownDBException', + 'class' => '\metastore\MetaException', ), 4 => array( - 'var' => 'o4', - 'type' => TType::STRUCT, - 'class' => '\metastore\UnknownTableException', - ), - 5 => array( - 'var' => 'o5', - 'type' => TType::STRUCT, - 'class' => '\metastore\UnknownPartitionException', - ), - 6 => array( - 'var' => 'o6', + 'var' => 'o4', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidPartitionException', + 'class' => '\metastore\InvalidInputException', ), ); } @@ -18676,17 +21143,11 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_result { if (isset($vals['o4'])) { $this->o4 = $vals['o4']; } - if (isset($vals['o5'])) { - $this->o5 = $vals['o5']; - } - if (isset($vals['o6'])) { - $this->o6 = $vals['o6']; - } } } public function getName() { - return 'ThriftHiveMetastore_isPartitionMarkedForEvent_result'; + return 'ThriftHiveMetastore_update_partition_column_statistics_result'; } public function read($input) @@ -18713,7 +21174,7 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_result { break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -18721,7 +21182,7 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\NoSuchObjectException(); + $this->o2 = new \metastore\InvalidObjectException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -18729,7 +21190,7 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_result { break; case 3: if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\UnknownDBException(); + $this->o3 = new \metastore\MetaException(); $xfer += $this->o3->read($input); } else { $xfer += $input->skip($ftype); @@ -18737,28 +21198,12 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_result { break; case 4: if ($ftype == TType::STRUCT) { - $this->o4 = new \metastore\UnknownTableException(); + $this->o4 = new \metastore\InvalidInputException(); $xfer += $this->o4->read($input); } else { $xfer += $input->skip($ftype); } break; - case 5: - if ($ftype == TType::STRUCT) { - $this->o5 = new \metastore\UnknownPartitionException(); - $xfer += $this->o5->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 6: - if ($ftype == TType::STRUCT) { - $this->o6 = new \metastore\InvalidPartitionException(); - $xfer += $this->o6->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -18771,7 +21216,7 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_isPartitionMarkedForEvent_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_update_partition_column_statistics_result'); if ($this->success !== null) { $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); $xfer += $output->writeBool($this->success); @@ -18797,16 +21242,6 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_result { $xfer += $this->o4->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o5 !== null) { - $xfer += $output->writeFieldBegin('o5', TType::STRUCT, 5); - $xfer += $this->o5->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->o6 !== null) { - $xfer += $output->writeFieldBegin('o6', TType::STRUCT, 6); - $xfer += $this->o6->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -18814,39 +21249,45 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_result { } -class ThriftHiveMetastore_add_index_args { +class ThriftHiveMetastore_get_table_column_statistics_args { static $_TSPEC; - public $new_index = null; - public $index_table = null; + public $db_name = null; + public $tbl_name = null; + public $col_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'new_index', - 'type' => TType::STRUCT, - 'class' => '\metastore\Index', + 'var' => 'db_name', + 'type' => TType::STRING, ), 2 => array( - 'var' => 'index_table', - 'type' => TType::STRUCT, - 'class' => '\metastore\Table', + 'var' => 'tbl_name', + 'type' => TType::STRING, + ), + 3 => array( + 'var' => 'col_name', + 'type' => TType::STRING, ), ); } if (is_array($vals)) { - if (isset($vals['new_index'])) { - $this->new_index = $vals['new_index']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; } - if (isset($vals['index_table'])) { - $this->index_table = $vals['index_table']; + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; + } + if (isset($vals['col_name'])) { + $this->col_name = $vals['col_name']; } } } public function getName() { - return 'ThriftHiveMetastore_add_index_args'; + return 'ThriftHiveMetastore_get_table_column_statistics_args'; } public function read($input) @@ -18865,17 +21306,22 @@ class ThriftHiveMetastore_add_index_args { switch ($fid) { case 1: - if ($ftype == TType::STRUCT) { - $this->new_index = new \metastore\Index(); - $xfer += $this->new_index->read($input); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->db_name); } else { $xfer += $input->skip($ftype); } break; case 2: - if ($ftype == TType::STRUCT) { - $this->index_table = new \metastore\Table(); - $xfer += $this->index_table->read($input); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->tbl_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->col_name); } else { $xfer += $input->skip($ftype); } @@ -18892,21 +21338,20 @@ class ThriftHiveMetastore_add_index_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_index_args'); - if ($this->new_index !== null) { - if (!is_object($this->new_index)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('new_index', TType::STRUCT, 1); - $xfer += $this->new_index->write($output); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_column_statistics_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); $xfer += $output->writeFieldEnd(); } - if ($this->index_table !== null) { - if (!is_object($this->index_table)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('index_table', TType::STRUCT, 2); - $xfer += $this->index_table->write($output); + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->col_name !== null) { + $xfer += $output->writeFieldBegin('col_name', TType::STRING, 3); + $xfer += $output->writeString($this->col_name); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -18916,13 +21361,14 @@ class ThriftHiveMetastore_add_index_args { } -class ThriftHiveMetastore_add_index_result { +class ThriftHiveMetastore_get_table_column_statistics_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; public $o3 = null; + public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -18930,22 +21376,27 @@ class ThriftHiveMetastore_add_index_result { 0 => array( 'var' => 'success', 'type' => TType::STRUCT, - 'class' => '\metastore\Index', + 'class' => '\metastore\ColumnStatistics', ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', + 'class' => '\metastore\NoSuchObjectException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\AlreadyExistsException', + 'class' => '\metastore\MetaException', ), 3 => array( 'var' => 'o3', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\InvalidInputException', + ), + 4 => array( + 'var' => 'o4', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidObjectException', ), ); } @@ -18962,11 +21413,14 @@ class ThriftHiveMetastore_add_index_result { if (isset($vals['o3'])) { $this->o3 = $vals['o3']; } + if (isset($vals['o4'])) { + $this->o4 = $vals['o4']; + } } } public function getName() { - return 'ThriftHiveMetastore_add_index_result'; + return 'ThriftHiveMetastore_get_table_column_statistics_result'; } public function read($input) @@ -18986,7 +21440,7 @@ class ThriftHiveMetastore_add_index_result { { case 0: if ($ftype == TType::STRUCT) { - $this->success = new \metastore\Index(); + $this->success = new \metastore\ColumnStatistics(); $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); @@ -18994,7 +21448,7 @@ class ThriftHiveMetastore_add_index_result { break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidObjectException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -19002,7 +21456,7 @@ class ThriftHiveMetastore_add_index_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\AlreadyExistsException(); + $this->o2 = new \metastore\MetaException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -19010,12 +21464,20 @@ class ThriftHiveMetastore_add_index_result { break; case 3: if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); + $this->o3 = new \metastore\InvalidInputException(); $xfer += $this->o3->read($input); } else { $xfer += $input->skip($ftype); } break; + case 4: + if ($ftype == TType::STRUCT) { + $this->o4 = new \metastore\InvalidObjectException(); + $xfer += $this->o4->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -19028,7 +21490,7 @@ class ThriftHiveMetastore_add_index_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_index_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_column_statistics_result'); if ($this->success !== null) { if (!is_object($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); @@ -19052,6 +21514,11 @@ class ThriftHiveMetastore_add_index_result { $xfer += $this->o3->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o4 !== null) { + $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); + $xfer += $this->o4->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -19059,54 +21526,53 @@ class ThriftHiveMetastore_add_index_result { } -class ThriftHiveMetastore_alter_index_args { +class ThriftHiveMetastore_get_partition_column_statistics_args { static $_TSPEC; - public $dbname = null; - public $base_tbl_name = null; - public $idx_name = null; - public $new_idx = null; + public $db_name = null; + public $tbl_name = null; + public $part_name = null; + public $col_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'dbname', + 'var' => 'db_name', 'type' => TType::STRING, ), 2 => array( - 'var' => 'base_tbl_name', + 'var' => 'tbl_name', 'type' => TType::STRING, ), 3 => array( - 'var' => 'idx_name', + 'var' => 'part_name', 'type' => TType::STRING, ), 4 => array( - 'var' => 'new_idx', - 'type' => TType::STRUCT, - 'class' => '\metastore\Index', + 'var' => 'col_name', + 'type' => TType::STRING, ), ); } if (is_array($vals)) { - if (isset($vals['dbname'])) { - $this->dbname = $vals['dbname']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; } - if (isset($vals['base_tbl_name'])) { - $this->base_tbl_name = $vals['base_tbl_name']; + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['idx_name'])) { - $this->idx_name = $vals['idx_name']; + if (isset($vals['part_name'])) { + $this->part_name = $vals['part_name']; } - if (isset($vals['new_idx'])) { - $this->new_idx = $vals['new_idx']; + if (isset($vals['col_name'])) { + $this->col_name = $vals['col_name']; } } } public function getName() { - return 'ThriftHiveMetastore_alter_index_args'; + return 'ThriftHiveMetastore_get_partition_column_statistics_args'; } public function read($input) @@ -19126,29 +21592,28 @@ class ThriftHiveMetastore_alter_index_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->dbname); + $xfer += $input->readString($this->db_name); } else { $xfer += $input->skip($ftype); } break; case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->base_tbl_name); + $xfer += $input->readString($this->tbl_name); } else { $xfer += $input->skip($ftype); } break; case 3: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->idx_name); + $xfer += $input->readString($this->part_name); } else { $xfer += $input->skip($ftype); } break; case 4: - if ($ftype == TType::STRUCT) { - $this->new_idx = new \metastore\Index(); - $xfer += $this->new_idx->read($input); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->col_name); } else { $xfer += $input->skip($ftype); } @@ -19165,28 +21630,25 @@ class ThriftHiveMetastore_alter_index_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_index_args'); - if ($this->dbname !== null) { - $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); - $xfer += $output->writeString($this->dbname); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_column_statistics_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); $xfer += $output->writeFieldEnd(); } - if ($this->base_tbl_name !== null) { - $xfer += $output->writeFieldBegin('base_tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->base_tbl_name); + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->idx_name !== null) { - $xfer += $output->writeFieldBegin('idx_name', TType::STRING, 3); - $xfer += $output->writeString($this->idx_name); + if ($this->part_name !== null) { + $xfer += $output->writeFieldBegin('part_name', TType::STRING, 3); + $xfer += $output->writeString($this->part_name); $xfer += $output->writeFieldEnd(); } - if ($this->new_idx !== null) { - if (!is_object($this->new_idx)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('new_idx', TType::STRUCT, 4); - $xfer += $this->new_idx->write($output); + if ($this->col_name !== null) { + $xfer += $output->writeFieldBegin('col_name', TType::STRING, 4); + $xfer += $output->writeString($this->col_name); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -19196,39 +21658,66 @@ class ThriftHiveMetastore_alter_index_args { } -class ThriftHiveMetastore_alter_index_result { +class ThriftHiveMetastore_get_partition_column_statistics_result { static $_TSPEC; + public $success = null; public $o1 = null; public $o2 = null; + public $o3 = null; + public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::STRUCT, + 'class' => '\metastore\ColumnStatistics', + ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidOperationException', + 'class' => '\metastore\NoSuchObjectException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidInputException', + ), + 4 => array( + 'var' => 'o4', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidObjectException', + ), ); } if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } + if (isset($vals['o4'])) { + $this->o4 = $vals['o4']; + } } } public function getName() { - return 'ThriftHiveMetastore_alter_index_result'; + return 'ThriftHiveMetastore_get_partition_column_statistics_result'; } public function read($input) @@ -19246,9 +21735,17 @@ class ThriftHiveMetastore_alter_index_result { } switch ($fid) { + case 0: + if ($ftype == TType::STRUCT) { + $this->success = new \metastore\ColumnStatistics(); + $xfer += $this->success->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidOperationException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -19262,6 +21759,22 @@ class ThriftHiveMetastore_alter_index_result { $xfer += $input->skip($ftype); } break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\InvalidInputException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRUCT) { + $this->o4 = new \metastore\InvalidObjectException(); + $xfer += $this->o4->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -19274,7 +21787,15 @@ class ThriftHiveMetastore_alter_index_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_index_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_column_statistics_result'); + if ($this->success !== null) { + if (!is_object($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); + $xfer += $output->writeFieldEnd(); + } if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -19285,6 +21806,16 @@ class ThriftHiveMetastore_alter_index_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o4 !== null) { + $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); + $xfer += $this->o4->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -19292,13 +21823,13 @@ class ThriftHiveMetastore_alter_index_result { } -class ThriftHiveMetastore_drop_index_by_name_args { +class ThriftHiveMetastore_delete_partition_column_statistics_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; - public $index_name = null; - public $deleteData = null; + public $part_name = null; + public $col_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -19312,12 +21843,12 @@ class ThriftHiveMetastore_drop_index_by_name_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'index_name', + 'var' => 'part_name', 'type' => TType::STRING, ), 4 => array( - 'var' => 'deleteData', - 'type' => TType::BOOL, + 'var' => 'col_name', + 'type' => TType::STRING, ), ); } @@ -19328,17 +21859,17 @@ class ThriftHiveMetastore_drop_index_by_name_args { if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['index_name'])) { - $this->index_name = $vals['index_name']; + if (isset($vals['part_name'])) { + $this->part_name = $vals['part_name']; } - if (isset($vals['deleteData'])) { - $this->deleteData = $vals['deleteData']; + if (isset($vals['col_name'])) { + $this->col_name = $vals['col_name']; } } } public function getName() { - return 'ThriftHiveMetastore_drop_index_by_name_args'; + return 'ThriftHiveMetastore_delete_partition_column_statistics_args'; } public function read($input) @@ -19372,14 +21903,14 @@ class ThriftHiveMetastore_drop_index_by_name_args { break; case 3: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->index_name); + $xfer += $input->readString($this->part_name); } else { $xfer += $input->skip($ftype); } break; case 4: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->deleteData); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->col_name); } else { $xfer += $input->skip($ftype); } @@ -19396,7 +21927,7 @@ class ThriftHiveMetastore_drop_index_by_name_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_index_by_name_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_delete_partition_column_statistics_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -19407,14 +21938,14 @@ class ThriftHiveMetastore_drop_index_by_name_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->index_name !== null) { - $xfer += $output->writeFieldBegin('index_name', TType::STRING, 3); - $xfer += $output->writeString($this->index_name); + if ($this->part_name !== null) { + $xfer += $output->writeFieldBegin('part_name', TType::STRING, 3); + $xfer += $output->writeString($this->part_name); $xfer += $output->writeFieldEnd(); } - if ($this->deleteData !== null) { - $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 4); - $xfer += $output->writeBool($this->deleteData); + if ($this->col_name !== null) { + $xfer += $output->writeFieldBegin('col_name', TType::STRING, 4); + $xfer += $output->writeString($this->col_name); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -19424,12 +21955,14 @@ class ThriftHiveMetastore_drop_index_by_name_args { } -class ThriftHiveMetastore_drop_index_by_name_result { +class ThriftHiveMetastore_delete_partition_column_statistics_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; + public $o3 = null; + public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -19448,6 +21981,16 @@ class ThriftHiveMetastore_drop_index_by_name_result { 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidObjectException', + ), + 4 => array( + 'var' => 'o4', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidInputException', + ), ); } if (is_array($vals)) { @@ -19460,11 +22003,17 @@ class ThriftHiveMetastore_drop_index_by_name_result { if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } + if (isset($vals['o4'])) { + $this->o4 = $vals['o4']; + } } } public function getName() { - return 'ThriftHiveMetastore_drop_index_by_name_result'; + return 'ThriftHiveMetastore_delete_partition_column_statistics_result'; } public function read($input) @@ -19505,6 +22054,22 @@ class ThriftHiveMetastore_drop_index_by_name_result { $xfer += $input->skip($ftype); } break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\InvalidObjectException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRUCT) { + $this->o4 = new \metastore\InvalidInputException(); + $xfer += $this->o4->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -19517,7 +22082,7 @@ class ThriftHiveMetastore_drop_index_by_name_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_index_by_name_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_delete_partition_column_statistics_result'); if ($this->success !== null) { $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); $xfer += $output->writeBool($this->success); @@ -19533,6 +22098,16 @@ class ThriftHiveMetastore_drop_index_by_name_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o4 !== null) { + $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); + $xfer += $this->o4->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -19540,12 +22115,12 @@ class ThriftHiveMetastore_drop_index_by_name_result { } -class ThriftHiveMetastore_get_index_by_name_args { +class ThriftHiveMetastore_delete_table_column_statistics_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; - public $index_name = null; + public $col_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -19559,7 +22134,7 @@ class ThriftHiveMetastore_get_index_by_name_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'index_name', + 'var' => 'col_name', 'type' => TType::STRING, ), ); @@ -19571,14 +22146,14 @@ class ThriftHiveMetastore_get_index_by_name_args { if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['index_name'])) { - $this->index_name = $vals['index_name']; + if (isset($vals['col_name'])) { + $this->col_name = $vals['col_name']; } } } public function getName() { - return 'ThriftHiveMetastore_get_index_by_name_args'; + return 'ThriftHiveMetastore_delete_table_column_statistics_args'; } public function read($input) @@ -19612,7 +22187,7 @@ class ThriftHiveMetastore_get_index_by_name_args { break; case 3: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->index_name); + $xfer += $input->readString($this->col_name); } else { $xfer += $input->skip($ftype); } @@ -19629,7 +22204,7 @@ class ThriftHiveMetastore_get_index_by_name_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_index_by_name_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_delete_table_column_statistics_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -19640,9 +22215,9 @@ class ThriftHiveMetastore_get_index_by_name_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->index_name !== null) { - $xfer += $output->writeFieldBegin('index_name', TType::STRING, 3); - $xfer += $output->writeString($this->index_name); + if ($this->col_name !== null) { + $xfer += $output->writeFieldBegin('col_name', TType::STRING, 3); + $xfer += $output->writeString($this->col_name); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -19652,30 +22227,41 @@ class ThriftHiveMetastore_get_index_by_name_args { } -class ThriftHiveMetastore_get_index_by_name_result { +class ThriftHiveMetastore_delete_table_column_statistics_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; + public $o3 = null; + public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::STRUCT, - 'class' => '\metastore\Index', + 'type' => TType::BOOL, ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchObjectException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\MetaException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidObjectException', + ), + 4 => array( + 'var' => 'o4', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidInputException', ), ); } @@ -19689,11 +22275,17 @@ class ThriftHiveMetastore_get_index_by_name_result { if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } + if (isset($vals['o4'])) { + $this->o4 = $vals['o4']; + } } } public function getName() { - return 'ThriftHiveMetastore_get_index_by_name_result'; + return 'ThriftHiveMetastore_delete_table_column_statistics_result'; } public function read($input) @@ -19712,16 +22304,15 @@ class ThriftHiveMetastore_get_index_by_name_result { switch ($fid) { case 0: - if ($ftype == TType::STRUCT) { - $this->success = new \metastore\Index(); - $xfer += $this->success->read($input); + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->success); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -19729,12 +22320,28 @@ class ThriftHiveMetastore_get_index_by_name_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\NoSuchObjectException(); + $this->o2 = new \metastore\MetaException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\InvalidObjectException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRUCT) { + $this->o4 = new \metastore\InvalidInputException(); + $xfer += $this->o4->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -19747,13 +22354,10 @@ class ThriftHiveMetastore_get_index_by_name_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_index_by_name_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_delete_table_column_statistics_result'); if ($this->success !== null) { - if (!is_object($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); - $xfer += $this->success->write($output); + $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); + $xfer += $output->writeBool($this->success); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -19766,6 +22370,16 @@ class ThriftHiveMetastore_get_index_by_name_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o4 !== null) { + $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); + $xfer += $this->o4->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -19773,12 +22387,11 @@ class ThriftHiveMetastore_get_index_by_name_result { } -class ThriftHiveMetastore_get_indexes_args { +class ThriftHiveMetastore_streamingStatus_args { static $_TSPEC; public $db_name = null; - public $tbl_name = null; - public $max_indexes = -1; + public $table_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -19788,30 +22401,23 @@ class ThriftHiveMetastore_get_indexes_args { 'type' => TType::STRING, ), 2 => array( - 'var' => 'tbl_name', + 'var' => 'table_name', 'type' => TType::STRING, ), - 3 => array( - 'var' => 'max_indexes', - 'type' => TType::I16, - ), ); } if (is_array($vals)) { if (isset($vals['db_name'])) { $this->db_name = $vals['db_name']; } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; - } - if (isset($vals['max_indexes'])) { - $this->max_indexes = $vals['max_indexes']; + if (isset($vals['table_name'])) { + $this->table_name = $vals['table_name']; } } } public function getName() { - return 'ThriftHiveMetastore_get_indexes_args'; + return 'ThriftHiveMetastore_streamingStatus_args'; } public function read($input) @@ -19838,14 +22444,7 @@ class ThriftHiveMetastore_get_indexes_args { break; case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::I16) { - $xfer += $input->readI16($this->max_indexes); + $xfer += $input->readString($this->table_name); } else { $xfer += $input->skip($ftype); } @@ -19862,20 +22461,15 @@ class ThriftHiveMetastore_get_indexes_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_indexes_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_streamingStatus_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); $xfer += $output->writeFieldEnd(); } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->max_indexes !== null) { - $xfer += $output->writeFieldBegin('max_indexes', TType::I16, 3); - $xfer += $output->writeI16($this->max_indexes); + if ($this->table_name !== null) { + $xfer += $output->writeFieldBegin('table_name', TType::STRING, 2); + $xfer += $output->writeString($this->table_name); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -19885,33 +22479,42 @@ class ThriftHiveMetastore_get_indexes_args { } -class ThriftHiveMetastore_get_indexes_result { +class ThriftHiveMetastore_streamingStatus_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; + public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRUCT, - 'elem' => array( - 'type' => TType::STRUCT, - 'class' => '\metastore\Index', + 'type' => TType::MAP, + 'ktype' => TType::STRING, + 'vtype' => TType::STRING, + 'key' => array( + 'type' => TType::STRING, + ), + 'val' => array( + 'type' => TType::STRING, ), ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\InvalidInputException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidObjectException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), ); @@ -19926,11 +22529,14 @@ class ThriftHiveMetastore_get_indexes_result { if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } } } public function getName() { - return 'ThriftHiveMetastore_get_indexes_result'; + return 'ThriftHiveMetastore_streamingStatus_result'; } public function read($input) @@ -19949,26 +22555,28 @@ class ThriftHiveMetastore_get_indexes_result { switch ($fid) { case 0: - if ($ftype == TType::LST) { + if ($ftype == TType::MAP) { $this->success = array(); - $_size517 = 0; - $_etype520 = 0; - $xfer += $input->readListBegin($_etype520, $_size517); - for ($_i521 = 0; $_i521 < $_size517; ++$_i521) + $_size531 = 0; + $_ktype532 = 0; + $_vtype533 = 0; + $xfer += $input->readMapBegin($_ktype532, $_vtype533, $_size531); + for ($_i535 = 0; $_i535 < $_size531; ++$_i535) { - $elem522 = null; - $elem522 = new \metastore\Index(); - $xfer += $elem522->read($input); - $this->success []= $elem522; + $key536 = ''; + $val537 = ''; + $xfer += $input->readString($key536); + $xfer += $input->readString($val537); + $this->success[$key536] = $val537; } - $xfer += $input->readListEnd(); + $xfer += $input->readMapEnd(); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\InvalidInputException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -19976,12 +22584,20 @@ class ThriftHiveMetastore_get_indexes_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); + $this->o2 = new \metastore\InvalidObjectException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\MetaException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -19994,21 +22610,22 @@ class ThriftHiveMetastore_get_indexes_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_indexes_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_streamingStatus_result'); if ($this->success !== null) { if (!is_array($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); + $xfer += $output->writeFieldBegin('success', TType::MAP, 0); { - $output->writeListBegin(TType::STRUCT, count($this->success)); + $output->writeMapBegin(TType::STRING, TType::STRING, count($this->success)); { - foreach ($this->success as $iter523) + foreach ($this->success as $kiter538 => $viter539) { - $xfer += $iter523->write($output); + $xfer += $output->writeString($kiter538); + $xfer += $output->writeString($viter539); } } - $output->writeListEnd(); + $output->writeMapEnd(); } $xfer += $output->writeFieldEnd(); } @@ -20022,6 +22639,11 @@ class ThriftHiveMetastore_get_indexes_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -20029,12 +22651,14 @@ class ThriftHiveMetastore_get_indexes_result { } -class ThriftHiveMetastore_get_index_names_args { +class ThriftHiveMetastore_enableStreaming_args { static $_TSPEC; public $db_name = null; - public $tbl_name = null; - public $max_indexes = -1; + public $table_name = null; + public $streaming_tmp_dir = null; + public $partition_path = null; + public $partition_val = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -20044,12 +22668,20 @@ class ThriftHiveMetastore_get_index_names_args { 'type' => TType::STRING, ), 2 => array( - 'var' => 'tbl_name', + 'var' => 'table_name', 'type' => TType::STRING, ), 3 => array( - 'var' => 'max_indexes', - 'type' => TType::I16, + 'var' => 'streaming_tmp_dir', + 'type' => TType::STRING, + ), + 4 => array( + 'var' => 'partition_path', + 'type' => TType::STRING, + ), + 5 => array( + 'var' => 'partition_val', + 'type' => TType::STRING, ), ); } @@ -20057,17 +22689,23 @@ class ThriftHiveMetastore_get_index_names_args { if (isset($vals['db_name'])) { $this->db_name = $vals['db_name']; } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; + if (isset($vals['table_name'])) { + $this->table_name = $vals['table_name']; } - if (isset($vals['max_indexes'])) { - $this->max_indexes = $vals['max_indexes']; + if (isset($vals['streaming_tmp_dir'])) { + $this->streaming_tmp_dir = $vals['streaming_tmp_dir']; + } + if (isset($vals['partition_path'])) { + $this->partition_path = $vals['partition_path']; + } + if (isset($vals['partition_val'])) { + $this->partition_val = $vals['partition_val']; } } } public function getName() { - return 'ThriftHiveMetastore_get_index_names_args'; + return 'ThriftHiveMetastore_enableStreaming_args'; } public function read($input) @@ -20094,14 +22732,28 @@ class ThriftHiveMetastore_get_index_names_args { break; case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); + $xfer += $input->readString($this->table_name); } else { $xfer += $input->skip($ftype); } break; case 3: - if ($ftype == TType::I16) { - $xfer += $input->readI16($this->max_indexes); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->streaming_tmp_dir); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->partition_path); + } else { + $xfer += $input->skip($ftype); + } + break; + case 5: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->partition_val); } else { $xfer += $input->skip($ftype); } @@ -20118,20 +22770,30 @@ class ThriftHiveMetastore_get_index_names_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_index_names_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_enableStreaming_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); $xfer += $output->writeFieldEnd(); } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); + if ($this->table_name !== null) { + $xfer += $output->writeFieldBegin('table_name', TType::STRING, 2); + $xfer += $output->writeString($this->table_name); $xfer += $output->writeFieldEnd(); } - if ($this->max_indexes !== null) { - $xfer += $output->writeFieldBegin('max_indexes', TType::I16, 3); - $xfer += $output->writeI16($this->max_indexes); + if ($this->streaming_tmp_dir !== null) { + $xfer += $output->writeFieldBegin('streaming_tmp_dir', TType::STRING, 3); + $xfer += $output->writeString($this->streaming_tmp_dir); + $xfer += $output->writeFieldEnd(); + } + if ($this->partition_path !== null) { + $xfer += $output->writeFieldBegin('partition_path', TType::STRING, 4); + $xfer += $output->writeString($this->partition_path); + $xfer += $output->writeFieldEnd(); + } + if ($this->partition_val !== null) { + $xfer += $output->writeFieldBegin('partition_val', TType::STRING, 5); + $xfer += $output->writeString($this->partition_val); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -20141,42 +22803,57 @@ class ThriftHiveMetastore_get_index_names_args { } -class ThriftHiveMetastore_get_index_names_result { +class ThriftHiveMetastore_enableStreaming_result { static $_TSPEC; - public $success = null; + public $o1 = null; public $o2 = null; + public $o3 = null; + public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), - ), 1 => array( + 'var' => 'o1', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidInputException', + ), + 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidObjectException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidOperationException', + ), + 4 => array( + 'var' => 'o4', + 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), ); } if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; + if (isset($vals['o1'])) { + $this->o1 = $vals['o1']; } if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } + if (isset($vals['o4'])) { + $this->o4 = $vals['o4']; + } } } public function getName() { - return 'ThriftHiveMetastore_get_index_names_result'; + return 'ThriftHiveMetastore_enableStreaming_result'; } public function read($input) @@ -20194,66 +22871,71 @@ class ThriftHiveMetastore_get_index_names_result { } switch ($fid) { - case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size524 = 0; - $_etype527 = 0; - $xfer += $input->readListBegin($_etype527, $_size524); - for ($_i528 = 0; $_i528 < $_size524; ++$_i528) - { - $elem529 = null; - $xfer += $input->readString($elem529); - $this->success []= $elem529; - } - $xfer += $input->readListEnd(); + case 1: + if ($ftype == TType::STRUCT) { + $this->o1 = new \metastore\InvalidInputException(); + $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; - case 1: + case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); + $this->o2 = new \metastore\InvalidObjectException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; - default: - $xfer += $input->skip($ftype); + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\InvalidOperationException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } break; - } - $xfer += $input->readFieldEnd(); - } - $xfer += $input->readStructEnd(); - return $xfer; - } - - public function write($output) { - $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_index_names_result'); - if ($this->success !== null) { - if (!is_array($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRING, count($this->success)); - { - foreach ($this->success as $iter530) - { - $xfer += $output->writeString($iter530); + case 4: + if ($ftype == TType::STRUCT) { + $this->o4 = new \metastore\MetaException(); + $xfer += $this->o4->read($input); + } else { + $xfer += $input->skip($ftype); } - } - $output->writeListEnd(); + break; + default: + $xfer += $input->skip($ftype); + break; } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_enableStreaming_result'); + if ($this->o1 !== null) { + $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); + $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 1); + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o4 !== null) { + $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); + $xfer += $this->o4->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -20261,30 +22943,37 @@ class ThriftHiveMetastore_get_index_names_result { } -class ThriftHiveMetastore_update_table_column_statistics_args { +class ThriftHiveMetastore_disableStreaming_args { static $_TSPEC; - public $stats_obj = null; + public $db_name = null; + public $table_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'stats_obj', - 'type' => TType::STRUCT, - 'class' => '\metastore\ColumnStatistics', + 'var' => 'db_name', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'table_name', + 'type' => TType::STRING, ), ); } if (is_array($vals)) { - if (isset($vals['stats_obj'])) { - $this->stats_obj = $vals['stats_obj']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; + } + if (isset($vals['table_name'])) { + $this->table_name = $vals['table_name']; } } } public function getName() { - return 'ThriftHiveMetastore_update_table_column_statistics_args'; + return 'ThriftHiveMetastore_disableStreaming_args'; } public function read($input) @@ -20303,9 +22992,15 @@ class ThriftHiveMetastore_update_table_column_statistics_args { switch ($fid) { case 1: - if ($ftype == TType::STRUCT) { - $this->stats_obj = new \metastore\ColumnStatistics(); - $xfer += $this->stats_obj->read($input); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->db_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->table_name); } else { $xfer += $input->skip($ftype); } @@ -20322,13 +23017,15 @@ class ThriftHiveMetastore_update_table_column_statistics_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_update_table_column_statistics_args'); - if ($this->stats_obj !== null) { - if (!is_object($this->stats_obj)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('stats_obj', TType::STRUCT, 1); - $xfer += $this->stats_obj->write($output); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_disableStreaming_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->table_name !== null) { + $xfer += $output->writeFieldBegin('table_name', TType::STRING, 2); + $xfer += $output->writeString($this->table_name); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -20338,10 +23035,9 @@ class ThriftHiveMetastore_update_table_column_statistics_args { } -class ThriftHiveMetastore_update_table_column_statistics_result { +class ThriftHiveMetastore_disableStreaming_result { static $_TSPEC; - public $success = null; public $o1 = null; public $o2 = null; public $o3 = null; @@ -20350,14 +23046,10 @@ class ThriftHiveMetastore_update_table_column_statistics_result { public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::BOOL, - ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\InvalidInputException', ), 2 => array( 'var' => 'o2', @@ -20367,19 +23059,16 @@ class ThriftHiveMetastore_update_table_column_statistics_result { 3 => array( 'var' => 'o3', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\InvalidOperationException', ), 4 => array( 'var' => 'o4', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidInputException', + 'class' => '\metastore\MetaException', ), ); } if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; - } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } @@ -20396,7 +23085,7 @@ class ThriftHiveMetastore_update_table_column_statistics_result { } public function getName() { - return 'ThriftHiveMetastore_update_table_column_statistics_result'; + return 'ThriftHiveMetastore_disableStreaming_result'; } public function read($input) @@ -20414,16 +23103,9 @@ class ThriftHiveMetastore_update_table_column_statistics_result { } switch ($fid) { - case 0: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->success); - } else { - $xfer += $input->skip($ftype); - } - break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\InvalidInputException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -20439,7 +23121,7 @@ class ThriftHiveMetastore_update_table_column_statistics_result { break; case 3: if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); + $this->o3 = new \metastore\InvalidOperationException(); $xfer += $this->o3->read($input); } else { $xfer += $input->skip($ftype); @@ -20447,7 +23129,7 @@ class ThriftHiveMetastore_update_table_column_statistics_result { break; case 4: if ($ftype == TType::STRUCT) { - $this->o4 = new \metastore\InvalidInputException(); + $this->o4 = new \metastore\MetaException(); $xfer += $this->o4->read($input); } else { $xfer += $input->skip($ftype); @@ -20465,12 +23147,7 @@ class ThriftHiveMetastore_update_table_column_statistics_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_update_table_column_statistics_result'); - if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); - $xfer += $output->writeBool($this->success); - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_disableStreaming_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -20498,30 +23175,37 @@ class ThriftHiveMetastore_update_table_column_statistics_result { } -class ThriftHiveMetastore_update_partition_column_statistics_args { +class ThriftHiveMetastore_getCurrentStreamingPartitionPath_args { static $_TSPEC; - public $stats_obj = null; + public $dbName = null; + public $tableName = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'stats_obj', - 'type' => TType::STRUCT, - 'class' => '\metastore\ColumnStatistics', + 'var' => 'dbName', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'tableName', + 'type' => TType::STRING, ), ); } if (is_array($vals)) { - if (isset($vals['stats_obj'])) { - $this->stats_obj = $vals['stats_obj']; + if (isset($vals['dbName'])) { + $this->dbName = $vals['dbName']; + } + if (isset($vals['tableName'])) { + $this->tableName = $vals['tableName']; } } } public function getName() { - return 'ThriftHiveMetastore_update_partition_column_statistics_args'; + return 'ThriftHiveMetastore_getCurrentStreamingPartitionPath_args'; } public function read($input) @@ -20540,9 +23224,15 @@ class ThriftHiveMetastore_update_partition_column_statistics_args { switch ($fid) { case 1: - if ($ftype == TType::STRUCT) { - $this->stats_obj = new \metastore\ColumnStatistics(); - $xfer += $this->stats_obj->read($input); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->dbName); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->tableName); } else { $xfer += $input->skip($ftype); } @@ -20559,13 +23249,15 @@ class ThriftHiveMetastore_update_partition_column_statistics_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_update_partition_column_statistics_args'); - if ($this->stats_obj !== null) { - if (!is_object($this->stats_obj)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('stats_obj', TType::STRUCT, 1); - $xfer += $this->stats_obj->write($output); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_getCurrentStreamingPartitionPath_args'); + if ($this->dbName !== null) { + $xfer += $output->writeFieldBegin('dbName', TType::STRING, 1); + $xfer += $output->writeString($this->dbName); + $xfer += $output->writeFieldEnd(); + } + if ($this->tableName !== null) { + $xfer += $output->writeFieldBegin('tableName', TType::STRING, 2); + $xfer += $output->writeString($this->tableName); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -20575,7 +23267,7 @@ class ThriftHiveMetastore_update_partition_column_statistics_args { } -class ThriftHiveMetastore_update_partition_column_statistics_result { +class ThriftHiveMetastore_getCurrentStreamingPartitionPath_result { static $_TSPEC; public $success = null; @@ -20589,12 +23281,12 @@ class ThriftHiveMetastore_update_partition_column_statistics_result { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::BOOL, + 'type' => TType::STRING, ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\InvalidInputException', ), 2 => array( 'var' => 'o2', @@ -20604,12 +23296,12 @@ class ThriftHiveMetastore_update_partition_column_statistics_result { 3 => array( 'var' => 'o3', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\InvalidOperationException', ), 4 => array( 'var' => 'o4', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidInputException', + 'class' => '\metastore\MetaException', ), ); } @@ -20633,7 +23325,7 @@ class ThriftHiveMetastore_update_partition_column_statistics_result { } public function getName() { - return 'ThriftHiveMetastore_update_partition_column_statistics_result'; + return 'ThriftHiveMetastore_getCurrentStreamingPartitionPath_result'; } public function read($input) @@ -20652,15 +23344,15 @@ class ThriftHiveMetastore_update_partition_column_statistics_result { switch ($fid) { case 0: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->success); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->success); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\InvalidInputException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -20676,7 +23368,7 @@ class ThriftHiveMetastore_update_partition_column_statistics_result { break; case 3: if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); + $this->o3 = new \metastore\InvalidOperationException(); $xfer += $this->o3->read($input); } else { $xfer += $input->skip($ftype); @@ -20684,7 +23376,7 @@ class ThriftHiveMetastore_update_partition_column_statistics_result { break; case 4: if ($ftype == TType::STRUCT) { - $this->o4 = new \metastore\InvalidInputException(); + $this->o4 = new \metastore\MetaException(); $xfer += $this->o4->read($input); } else { $xfer += $input->skip($ftype); @@ -20702,10 +23394,10 @@ class ThriftHiveMetastore_update_partition_column_statistics_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_update_partition_column_statistics_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_getCurrentStreamingPartitionPath_result'); if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); - $xfer += $output->writeBool($this->success); + $xfer += $output->writeFieldBegin('success', TType::STRING, 0); + $xfer += $output->writeString($this->success); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -20735,45 +23427,53 @@ class ThriftHiveMetastore_update_partition_column_statistics_result { } -class ThriftHiveMetastore_get_table_column_statistics_args { +class ThriftHiveMetastore_rollStreamingPartition_args { static $_TSPEC; - public $db_name = null; - public $tbl_name = null; - public $col_name = null; + public $dbName = null; + public $tableName = null; + public $newPartitionPath = null; + public $newPartitionVal = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', + 'var' => 'dbName', 'type' => TType::STRING, ), 2 => array( - 'var' => 'tbl_name', + 'var' => 'tableName', 'type' => TType::STRING, ), 3 => array( - 'var' => 'col_name', + 'var' => 'newPartitionPath', + 'type' => TType::STRING, + ), + 4 => array( + 'var' => 'newPartitionVal', 'type' => TType::STRING, ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; + if (isset($vals['dbName'])) { + $this->dbName = $vals['dbName']; } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; + if (isset($vals['tableName'])) { + $this->tableName = $vals['tableName']; } - if (isset($vals['col_name'])) { - $this->col_name = $vals['col_name']; + if (isset($vals['newPartitionPath'])) { + $this->newPartitionPath = $vals['newPartitionPath']; + } + if (isset($vals['newPartitionVal'])) { + $this->newPartitionVal = $vals['newPartitionVal']; } } } public function getName() { - return 'ThriftHiveMetastore_get_table_column_statistics_args'; + return 'ThriftHiveMetastore_rollStreamingPartition_args'; } public function read($input) @@ -20793,21 +23493,28 @@ class ThriftHiveMetastore_get_table_column_statistics_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); + $xfer += $input->readString($this->dbName); } else { $xfer += $input->skip($ftype); } break; case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); + $xfer += $input->readString($this->tableName); } else { $xfer += $input->skip($ftype); } break; case 3: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->col_name); + $xfer += $input->readString($this->newPartitionPath); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->newPartitionVal); } else { $xfer += $input->skip($ftype); } @@ -20824,20 +23531,25 @@ class ThriftHiveMetastore_get_table_column_statistics_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_column_statistics_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_rollStreamingPartition_args'); + if ($this->dbName !== null) { + $xfer += $output->writeFieldBegin('dbName', TType::STRING, 1); + $xfer += $output->writeString($this->dbName); $xfer += $output->writeFieldEnd(); } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); + if ($this->tableName !== null) { + $xfer += $output->writeFieldBegin('tableName', TType::STRING, 2); + $xfer += $output->writeString($this->tableName); $xfer += $output->writeFieldEnd(); } - if ($this->col_name !== null) { - $xfer += $output->writeFieldBegin('col_name', TType::STRING, 3); - $xfer += $output->writeString($this->col_name); + if ($this->newPartitionPath !== null) { + $xfer += $output->writeFieldBegin('newPartitionPath', TType::STRING, 3); + $xfer += $output->writeString($this->newPartitionPath); + $xfer += $output->writeFieldEnd(); + } + if ($this->newPartitionVal !== null) { + $xfer += $output->writeFieldBegin('newPartitionVal', TType::STRING, 4); + $xfer += $output->writeString($this->newPartitionVal); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -20847,7 +23559,7 @@ class ThriftHiveMetastore_get_table_column_statistics_args { } -class ThriftHiveMetastore_get_table_column_statistics_result { +class ThriftHiveMetastore_rollStreamingPartition_result { static $_TSPEC; public $success = null; @@ -20855,34 +23567,39 @@ class ThriftHiveMetastore_get_table_column_statistics_result { public $o2 = null; public $o3 = null; public $o4 = null; + public $o5 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::STRUCT, - 'class' => '\metastore\ColumnStatistics', + 'type' => TType::STRING, ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\InvalidInputException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\InvalidObjectException', ), 3 => array( 'var' => 'o3', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidInputException', + 'class' => '\metastore\InvalidOperationException', ), 4 => array( 'var' => 'o4', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', + 'class' => '\metastore\AlreadyExistsException', + ), + 5 => array( + 'var' => 'o5', + 'type' => TType::STRUCT, + 'class' => '\metastore\MetaException', ), ); } @@ -20902,11 +23619,14 @@ class ThriftHiveMetastore_get_table_column_statistics_result { if (isset($vals['o4'])) { $this->o4 = $vals['o4']; } + if (isset($vals['o5'])) { + $this->o5 = $vals['o5']; + } } } public function getName() { - return 'ThriftHiveMetastore_get_table_column_statistics_result'; + return 'ThriftHiveMetastore_rollStreamingPartition_result'; } public function read($input) @@ -20925,16 +23645,15 @@ class ThriftHiveMetastore_get_table_column_statistics_result { switch ($fid) { case 0: - if ($ftype == TType::STRUCT) { - $this->success = new \metastore\ColumnStatistics(); - $xfer += $this->success->read($input); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->success); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\InvalidInputException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -20942,7 +23661,7 @@ class ThriftHiveMetastore_get_table_column_statistics_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); + $this->o2 = new \metastore\InvalidObjectException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -20950,7 +23669,7 @@ class ThriftHiveMetastore_get_table_column_statistics_result { break; case 3: if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\InvalidInputException(); + $this->o3 = new \metastore\InvalidOperationException(); $xfer += $this->o3->read($input); } else { $xfer += $input->skip($ftype); @@ -20958,12 +23677,20 @@ class ThriftHiveMetastore_get_table_column_statistics_result { break; case 4: if ($ftype == TType::STRUCT) { - $this->o4 = new \metastore\InvalidObjectException(); + $this->o4 = new \metastore\AlreadyExistsException(); $xfer += $this->o4->read($input); } else { $xfer += $input->skip($ftype); } break; + case 5: + if ($ftype == TType::STRUCT) { + $this->o5 = new \metastore\MetaException(); + $xfer += $this->o5->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -20976,13 +23703,10 @@ class ThriftHiveMetastore_get_table_column_statistics_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_column_statistics_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_rollStreamingPartition_result'); if ($this->success !== null) { - if (!is_object($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); - $xfer += $this->success->write($output); + $xfer += $output->writeFieldBegin('success', TType::STRING, 0); + $xfer += $output->writeString($this->success); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -21005,6 +23729,11 @@ class ThriftHiveMetastore_get_table_column_statistics_result { $xfer += $this->o4->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o5 !== null) { + $xfer += $output->writeFieldBegin('o5', TType::STRUCT, 5); + $xfer += $this->o5->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -21012,53 +23741,45 @@ class ThriftHiveMetastore_get_table_column_statistics_result { } -class ThriftHiveMetastore_get_partition_column_statistics_args { +class ThriftHiveMetastore_updateStreamingTempLocation_args { static $_TSPEC; - public $db_name = null; - public $tbl_name = null; - public $part_name = null; - public $col_name = null; - + public $dbName = null; + public $tableName = null; + public $streamingTempDir = null; + public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', + 'var' => 'dbName', 'type' => TType::STRING, ), 2 => array( - 'var' => 'tbl_name', + 'var' => 'tableName', 'type' => TType::STRING, ), 3 => array( - 'var' => 'part_name', - 'type' => TType::STRING, - ), - 4 => array( - 'var' => 'col_name', + 'var' => 'streamingTempDir', 'type' => TType::STRING, ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; - } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; + if (isset($vals['dbName'])) { + $this->dbName = $vals['dbName']; } - if (isset($vals['part_name'])) { - $this->part_name = $vals['part_name']; + if (isset($vals['tableName'])) { + $this->tableName = $vals['tableName']; } - if (isset($vals['col_name'])) { - $this->col_name = $vals['col_name']; + if (isset($vals['streamingTempDir'])) { + $this->streamingTempDir = $vals['streamingTempDir']; } } } public function getName() { - return 'ThriftHiveMetastore_get_partition_column_statistics_args'; + return 'ThriftHiveMetastore_updateStreamingTempLocation_args'; } public function read($input) @@ -21078,28 +23799,21 @@ class ThriftHiveMetastore_get_partition_column_statistics_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); + $xfer += $input->readString($this->dbName); } else { $xfer += $input->skip($ftype); } break; case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); + $xfer += $input->readString($this->tableName); } else { $xfer += $input->skip($ftype); } break; case 3: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->part_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->col_name); + $xfer += $input->readString($this->streamingTempDir); } else { $xfer += $input->skip($ftype); } @@ -21116,25 +23830,20 @@ class ThriftHiveMetastore_get_partition_column_statistics_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_column_statistics_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_updateStreamingTempLocation_args'); + if ($this->dbName !== null) { + $xfer += $output->writeFieldBegin('dbName', TType::STRING, 1); + $xfer += $output->writeString($this->dbName); $xfer += $output->writeFieldEnd(); } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->part_name !== null) { - $xfer += $output->writeFieldBegin('part_name', TType::STRING, 3); - $xfer += $output->writeString($this->part_name); + if ($this->tableName !== null) { + $xfer += $output->writeFieldBegin('tableName', TType::STRING, 2); + $xfer += $output->writeString($this->tableName); $xfer += $output->writeFieldEnd(); } - if ($this->col_name !== null) { - $xfer += $output->writeFieldBegin('col_name', TType::STRING, 4); - $xfer += $output->writeString($this->col_name); + if ($this->streamingTempDir !== null) { + $xfer += $output->writeFieldBegin('streamingTempDir', TType::STRING, 3); + $xfer += $output->writeString($this->streamingTempDir); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -21144,49 +23853,34 @@ class ThriftHiveMetastore_get_partition_column_statistics_args { } -class ThriftHiveMetastore_get_partition_column_statistics_result { +class ThriftHiveMetastore_updateStreamingTempLocation_result { static $_TSPEC; - public $success = null; public $o1 = null; public $o2 = null; public $o3 = null; - public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::STRUCT, - 'class' => '\metastore\ColumnStatistics', - ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\InvalidInputException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\InvalidObjectException', ), 3 => array( 'var' => 'o3', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidInputException', - ), - 4 => array( - 'var' => 'o4', - 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', + 'class' => '\metastore\MetaException', ), ); } if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; - } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } @@ -21196,14 +23890,11 @@ class ThriftHiveMetastore_get_partition_column_statistics_result { if (isset($vals['o3'])) { $this->o3 = $vals['o3']; } - if (isset($vals['o4'])) { - $this->o4 = $vals['o4']; - } } } public function getName() { - return 'ThriftHiveMetastore_get_partition_column_statistics_result'; + return 'ThriftHiveMetastore_updateStreamingTempLocation_result'; } public function read($input) @@ -21221,17 +23912,9 @@ class ThriftHiveMetastore_get_partition_column_statistics_result { } switch ($fid) { - case 0: - if ($ftype == TType::STRUCT) { - $this->success = new \metastore\ColumnStatistics(); - $xfer += $this->success->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\InvalidInputException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -21239,7 +23922,7 @@ class ThriftHiveMetastore_get_partition_column_statistics_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); + $this->o2 = new \metastore\InvalidObjectException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -21247,20 +23930,12 @@ class ThriftHiveMetastore_get_partition_column_statistics_result { break; case 3: if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\InvalidInputException(); + $this->o3 = new \metastore\MetaException(); $xfer += $this->o3->read($input); } else { $xfer += $input->skip($ftype); } break; - case 4: - if ($ftype == TType::STRUCT) { - $this->o4 = new \metastore\InvalidObjectException(); - $xfer += $this->o4->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -21273,15 +23948,7 @@ class ThriftHiveMetastore_get_partition_column_statistics_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_column_statistics_result'); - if ($this->success !== null) { - if (!is_object($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); - $xfer += $this->success->write($output); - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_updateStreamingTempLocation_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -21297,11 +23964,6 @@ class ThriftHiveMetastore_get_partition_column_statistics_result { $xfer += $this->o3->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o4 !== null) { - $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); - $xfer += $this->o4->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -21309,13 +23971,11 @@ class ThriftHiveMetastore_get_partition_column_statistics_result { } -class ThriftHiveMetastore_delete_partition_column_statistics_args { +class ThriftHiveMetastore_getNextChunkID_args { static $_TSPEC; public $db_name = null; - public $tbl_name = null; - public $part_name = null; - public $col_name = null; + public $table_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -21325,15 +23985,7 @@ class ThriftHiveMetastore_delete_partition_column_statistics_args { 'type' => TType::STRING, ), 2 => array( - 'var' => 'tbl_name', - 'type' => TType::STRING, - ), - 3 => array( - 'var' => 'part_name', - 'type' => TType::STRING, - ), - 4 => array( - 'var' => 'col_name', + 'var' => 'table_name', 'type' => TType::STRING, ), ); @@ -21342,20 +23994,14 @@ class ThriftHiveMetastore_delete_partition_column_statistics_args { if (isset($vals['db_name'])) { $this->db_name = $vals['db_name']; } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; - } - if (isset($vals['part_name'])) { - $this->part_name = $vals['part_name']; - } - if (isset($vals['col_name'])) { - $this->col_name = $vals['col_name']; + if (isset($vals['table_name'])) { + $this->table_name = $vals['table_name']; } } } public function getName() { - return 'ThriftHiveMetastore_delete_partition_column_statistics_args'; + return 'ThriftHiveMetastore_getNextChunkID_args'; } public function read($input) @@ -21382,21 +24028,7 @@ class ThriftHiveMetastore_delete_partition_column_statistics_args { break; case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->part_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->col_name); + $xfer += $input->readString($this->table_name); } else { $xfer += $input->skip($ftype); } @@ -21413,25 +24045,15 @@ class ThriftHiveMetastore_delete_partition_column_statistics_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_delete_partition_column_statistics_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_getNextChunkID_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); $xfer += $output->writeFieldEnd(); } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->part_name !== null) { - $xfer += $output->writeFieldBegin('part_name', TType::STRING, 3); - $xfer += $output->writeString($this->part_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->col_name !== null) { - $xfer += $output->writeFieldBegin('col_name', TType::STRING, 4); - $xfer += $output->writeString($this->col_name); + if ($this->table_name !== null) { + $xfer += $output->writeFieldBegin('table_name', TType::STRING, 2); + $xfer += $output->writeString($this->table_name); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -21441,7 +24063,7 @@ class ThriftHiveMetastore_delete_partition_column_statistics_args { } -class ThriftHiveMetastore_delete_partition_column_statistics_result { +class ThriftHiveMetastore_getNextChunkID_result { static $_TSPEC; public $success = null; @@ -21455,27 +24077,27 @@ class ThriftHiveMetastore_delete_partition_column_statistics_result { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::BOOL, + 'type' => TType::I64, ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\InvalidInputException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\InvalidObjectException', ), 3 => array( 'var' => 'o3', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', + 'class' => '\metastore\InvalidOperationException', ), 4 => array( 'var' => 'o4', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidInputException', + 'class' => '\metastore\MetaException', ), ); } @@ -21499,7 +24121,7 @@ class ThriftHiveMetastore_delete_partition_column_statistics_result { } public function getName() { - return 'ThriftHiveMetastore_delete_partition_column_statistics_result'; + return 'ThriftHiveMetastore_getNextChunkID_result'; } public function read($input) @@ -21518,15 +24140,15 @@ class ThriftHiveMetastore_delete_partition_column_statistics_result { switch ($fid) { case 0: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->success); + if ($ftype == TType::I64) { + $xfer += $input->readI64($this->success); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\InvalidInputException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -21534,7 +24156,7 @@ class ThriftHiveMetastore_delete_partition_column_statistics_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); + $this->o2 = new \metastore\InvalidObjectException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -21542,7 +24164,7 @@ class ThriftHiveMetastore_delete_partition_column_statistics_result { break; case 3: if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\InvalidObjectException(); + $this->o3 = new \metastore\InvalidOperationException(); $xfer += $this->o3->read($input); } else { $xfer += $input->skip($ftype); @@ -21550,7 +24172,7 @@ class ThriftHiveMetastore_delete_partition_column_statistics_result { break; case 4: if ($ftype == TType::STRUCT) { - $this->o4 = new \metastore\InvalidInputException(); + $this->o4 = new \metastore\MetaException(); $xfer += $this->o4->read($input); } else { $xfer += $input->skip($ftype); @@ -21568,10 +24190,10 @@ class ThriftHiveMetastore_delete_partition_column_statistics_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_delete_partition_column_statistics_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_getNextChunkID_result'); if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); - $xfer += $output->writeBool($this->success); + $xfer += $output->writeFieldBegin('success', TType::I64, 0); + $xfer += $output->writeI64($this->success); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -21601,45 +24223,37 @@ class ThriftHiveMetastore_delete_partition_column_statistics_result { } -class ThriftHiveMetastore_delete_table_column_statistics_args { +class ThriftHiveMetastore_getStreamingTmpDir_args { static $_TSPEC; - public $db_name = null; - public $tbl_name = null; - public $col_name = null; + public $dbName = null; + public $tableName = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', + 'var' => 'dbName', 'type' => TType::STRING, ), 2 => array( - 'var' => 'tbl_name', - 'type' => TType::STRING, - ), - 3 => array( - 'var' => 'col_name', + 'var' => 'tableName', 'type' => TType::STRING, ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; - } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; + if (isset($vals['dbName'])) { + $this->dbName = $vals['dbName']; } - if (isset($vals['col_name'])) { - $this->col_name = $vals['col_name']; + if (isset($vals['tableName'])) { + $this->tableName = $vals['tableName']; } } } public function getName() { - return 'ThriftHiveMetastore_delete_table_column_statistics_args'; + return 'ThriftHiveMetastore_getStreamingTmpDir_args'; } public function read($input) @@ -21659,21 +24273,14 @@ class ThriftHiveMetastore_delete_table_column_statistics_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); + $xfer += $input->readString($this->dbName); } else { $xfer += $input->skip($ftype); } break; case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->col_name); + $xfer += $input->readString($this->tableName); } else { $xfer += $input->skip($ftype); } @@ -21690,20 +24297,15 @@ class ThriftHiveMetastore_delete_table_column_statistics_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_delete_table_column_statistics_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_getStreamingTmpDir_args'); + if ($this->dbName !== null) { + $xfer += $output->writeFieldBegin('dbName', TType::STRING, 1); + $xfer += $output->writeString($this->dbName); $xfer += $output->writeFieldEnd(); } - if ($this->col_name !== null) { - $xfer += $output->writeFieldBegin('col_name', TType::STRING, 3); - $xfer += $output->writeString($this->col_name); + if ($this->tableName !== null) { + $xfer += $output->writeFieldBegin('tableName', TType::STRING, 2); + $xfer += $output->writeString($this->tableName); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -21713,7 +24315,7 @@ class ThriftHiveMetastore_delete_table_column_statistics_args { } -class ThriftHiveMetastore_delete_table_column_statistics_result { +class ThriftHiveMetastore_getStreamingTmpDir_result { static $_TSPEC; public $success = null; @@ -21727,27 +24329,27 @@ class ThriftHiveMetastore_delete_table_column_statistics_result { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::BOOL, + 'type' => TType::STRING, ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\InvalidInputException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\InvalidObjectException', ), 3 => array( 'var' => 'o3', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', + 'class' => '\metastore\InvalidOperationException', ), 4 => array( 'var' => 'o4', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidInputException', + 'class' => '\metastore\MetaException', ), ); } @@ -21771,7 +24373,7 @@ class ThriftHiveMetastore_delete_table_column_statistics_result { } public function getName() { - return 'ThriftHiveMetastore_delete_table_column_statistics_result'; + return 'ThriftHiveMetastore_getStreamingTmpDir_result'; } public function read($input) @@ -21790,15 +24392,15 @@ class ThriftHiveMetastore_delete_table_column_statistics_result { switch ($fid) { case 0: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->success); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->success); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\InvalidInputException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -21806,7 +24408,7 @@ class ThriftHiveMetastore_delete_table_column_statistics_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); + $this->o2 = new \metastore\InvalidObjectException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -21814,7 +24416,7 @@ class ThriftHiveMetastore_delete_table_column_statistics_result { break; case 3: if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\InvalidObjectException(); + $this->o3 = new \metastore\InvalidOperationException(); $xfer += $this->o3->read($input); } else { $xfer += $input->skip($ftype); @@ -21822,7 +24424,7 @@ class ThriftHiveMetastore_delete_table_column_statistics_result { break; case 4: if ($ftype == TType::STRUCT) { - $this->o4 = new \metastore\InvalidInputException(); + $this->o4 = new \metastore\MetaException(); $xfer += $this->o4->read($input); } else { $xfer += $input->skip($ftype); @@ -21840,10 +24442,10 @@ class ThriftHiveMetastore_delete_table_column_statistics_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_delete_table_column_statistics_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_getStreamingTmpDir_result'); if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); - $xfer += $output->writeBool($this->success); + $xfer += $output->writeFieldBegin('success', TType::STRING, 0); + $xfer += $output->writeString($this->success); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -22316,14 +24918,14 @@ class ThriftHiveMetastore_get_role_names_result { case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size531 = 0; - $_etype534 = 0; - $xfer += $input->readListBegin($_etype534, $_size531); - for ($_i535 = 0; $_i535 < $_size531; ++$_i535) + $_size540 = 0; + $_etype543 = 0; + $xfer += $input->readListBegin($_etype543, $_size540); + for ($_i544 = 0; $_i544 < $_size540; ++$_i544) { - $elem536 = null; - $xfer += $input->readString($elem536); - $this->success []= $elem536; + $elem545 = null; + $xfer += $input->readString($elem545); + $this->success []= $elem545; } $xfer += $input->readListEnd(); } else { @@ -22359,9 +24961,9 @@ class ThriftHiveMetastore_get_role_names_result { { $output->writeListBegin(TType::STRING, count($this->success)); { - foreach ($this->success as $iter537) + foreach ($this->success as $iter546) { - $xfer += $output->writeString($iter537); + $xfer += $output->writeString($iter546); } } $output->writeListEnd(); @@ -23001,15 +25603,15 @@ class ThriftHiveMetastore_list_roles_result { case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size538 = 0; - $_etype541 = 0; - $xfer += $input->readListBegin($_etype541, $_size538); - for ($_i542 = 0; $_i542 < $_size538; ++$_i542) + $_size547 = 0; + $_etype550 = 0; + $xfer += $input->readListBegin($_etype550, $_size547); + for ($_i551 = 0; $_i551 < $_size547; ++$_i551) { - $elem543 = null; - $elem543 = new \metastore\Role(); - $xfer += $elem543->read($input); - $this->success []= $elem543; + $elem552 = null; + $elem552 = new \metastore\Role(); + $xfer += $elem552->read($input); + $this->success []= $elem552; } $xfer += $input->readListEnd(); } else { @@ -23045,9 +25647,9 @@ class ThriftHiveMetastore_list_roles_result { { $output->writeListBegin(TType::STRUCT, count($this->success)); { - foreach ($this->success as $iter544) + foreach ($this->success as $iter553) { - $xfer += $iter544->write($output); + $xfer += $iter553->write($output); } } $output->writeListEnd(); @@ -23145,14 +25747,14 @@ class ThriftHiveMetastore_get_privilege_set_args { case 3: if ($ftype == TType::LST) { $this->group_names = array(); - $_size545 = 0; - $_etype548 = 0; - $xfer += $input->readListBegin($_etype548, $_size545); - for ($_i549 = 0; $_i549 < $_size545; ++$_i549) + $_size554 = 0; + $_etype557 = 0; + $xfer += $input->readListBegin($_etype557, $_size554); + for ($_i558 = 0; $_i558 < $_size554; ++$_i558) { - $elem550 = null; - $xfer += $input->readString($elem550); - $this->group_names []= $elem550; + $elem559 = null; + $xfer += $input->readString($elem559); + $this->group_names []= $elem559; } $xfer += $input->readListEnd(); } else { @@ -23193,9 +25795,9 @@ class ThriftHiveMetastore_get_privilege_set_args { { $output->writeListBegin(TType::STRING, count($this->group_names)); { - foreach ($this->group_names as $iter551) + foreach ($this->group_names as $iter560) { - $xfer += $output->writeString($iter551); + $xfer += $output->writeString($iter560); } } $output->writeListEnd(); @@ -23482,15 +26084,15 @@ class ThriftHiveMetastore_list_privileges_result { case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size552 = 0; - $_etype555 = 0; - $xfer += $input->readListBegin($_etype555, $_size552); - for ($_i556 = 0; $_i556 < $_size552; ++$_i556) + $_size561 = 0; + $_etype564 = 0; + $xfer += $input->readListBegin($_etype564, $_size561); + for ($_i565 = 0; $_i565 < $_size561; ++$_i565) { - $elem557 = null; - $elem557 = new \metastore\HiveObjectPrivilege(); - $xfer += $elem557->read($input); - $this->success []= $elem557; + $elem566 = null; + $elem566 = new \metastore\HiveObjectPrivilege(); + $xfer += $elem566->read($input); + $this->success []= $elem566; } $xfer += $input->readListEnd(); } else { @@ -23526,9 +26128,9 @@ class ThriftHiveMetastore_list_privileges_result { { $output->writeListBegin(TType::STRUCT, count($this->success)); { - foreach ($this->success as $iter558) + foreach ($this->success as $iter567) { - $xfer += $iter558->write($output); + $xfer += $iter567->write($output); } } $output->writeListEnd(); @@ -23951,14 +26553,14 @@ class ThriftHiveMetastore_set_ugi_args { case 2: if ($ftype == TType::LST) { $this->group_names = array(); - $_size559 = 0; - $_etype562 = 0; - $xfer += $input->readListBegin($_etype562, $_size559); - for ($_i563 = 0; $_i563 < $_size559; ++$_i563) + $_size568 = 0; + $_etype571 = 0; + $xfer += $input->readListBegin($_etype571, $_size568); + for ($_i572 = 0; $_i572 < $_size568; ++$_i572) { - $elem564 = null; - $xfer += $input->readString($elem564); - $this->group_names []= $elem564; + $elem573 = null; + $xfer += $input->readString($elem573); + $this->group_names []= $elem573; } $xfer += $input->readListEnd(); } else { @@ -23991,9 +26593,9 @@ class ThriftHiveMetastore_set_ugi_args { { $output->writeListBegin(TType::STRING, count($this->group_names)); { - foreach ($this->group_names as $iter565) + foreach ($this->group_names as $iter574) { - $xfer += $output->writeString($iter565); + $xfer += $output->writeString($iter574); } } $output->writeListEnd(); @@ -24063,14 +26665,14 @@ class ThriftHiveMetastore_set_ugi_result { case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size566 = 0; - $_etype569 = 0; - $xfer += $input->readListBegin($_etype569, $_size566); - for ($_i570 = 0; $_i570 < $_size566; ++$_i570) + $_size575 = 0; + $_etype578 = 0; + $xfer += $input->readListBegin($_etype578, $_size575); + for ($_i579 = 0; $_i579 < $_size575; ++$_i579) { - $elem571 = null; - $xfer += $input->readString($elem571); - $this->success []= $elem571; + $elem580 = null; + $xfer += $input->readString($elem580); + $this->success []= $elem580; } $xfer += $input->readListEnd(); } else { @@ -24106,9 +26708,9 @@ class ThriftHiveMetastore_set_ugi_result { { $output->writeListBegin(TType::STRING, count($this->success)); { - foreach ($this->success as $iter572) + foreach ($this->success as $iter581) { - $xfer += $output->writeString($iter572); + $xfer += $output->writeString($iter581); } } $output->writeListEnd(); diff --git metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote old mode 100644 new mode 100755 index 6fd2cce..7df9cc5 --- metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote +++ metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote @@ -91,6 +91,14 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help': print ' ColumnStatistics get_partition_column_statistics(string db_name, string tbl_name, string part_name, string col_name)' print ' bool delete_partition_column_statistics(string db_name, string tbl_name, string part_name, string col_name)' print ' bool delete_table_column_statistics(string db_name, string tbl_name, string col_name)' + print ' streamingStatus(string db_name, string table_name)' + print ' void enableStreaming(string db_name, string table_name, string streaming_tmp_dir, string partition_path, string partition_val)' + print ' void disableStreaming(string db_name, string table_name)' + print ' string getCurrentStreamingPartitionPath(string dbName, string tableName)' + print ' string rollStreamingPartition(string dbName, string tableName, string newPartitionPath, string newPartitionVal)' + print ' void updateStreamingTempLocation(string dbName, string tableName, string streamingTempDir)' + print ' i64 getNextChunkID(string db_name, string table_name)' + print ' string getStreamingTmpDir(string dbName, string tableName)' print ' bool create_role(Role role)' print ' bool drop_role(string role_name)' print ' get_role_names()' @@ -564,6 +572,54 @@ elif cmd == 'delete_table_column_statistics': sys.exit(1) pp.pprint(client.delete_table_column_statistics(args[0],args[1],args[2],)) +elif cmd == 'streamingStatus': + if len(args) != 2: + print 'streamingStatus requires 2 args' + sys.exit(1) + pp.pprint(client.streamingStatus(args[0],args[1],)) + +elif cmd == 'enableStreaming': + if len(args) != 5: + print 'enableStreaming requires 5 args' + sys.exit(1) + pp.pprint(client.enableStreaming(args[0],args[1],args[2],args[3],args[4],)) + +elif cmd == 'disableStreaming': + if len(args) != 2: + print 'disableStreaming requires 2 args' + sys.exit(1) + pp.pprint(client.disableStreaming(args[0],args[1],)) + +elif cmd == 'getCurrentStreamingPartitionPath': + if len(args) != 2: + print 'getCurrentStreamingPartitionPath requires 2 args' + sys.exit(1) + pp.pprint(client.getCurrentStreamingPartitionPath(args[0],args[1],)) + +elif cmd == 'rollStreamingPartition': + if len(args) != 4: + print 'rollStreamingPartition requires 4 args' + sys.exit(1) + pp.pprint(client.rollStreamingPartition(args[0],args[1],args[2],args[3],)) + +elif cmd == 'updateStreamingTempLocation': + if len(args) != 3: + print 'updateStreamingTempLocation requires 3 args' + sys.exit(1) + pp.pprint(client.updateStreamingTempLocation(args[0],args[1],args[2],)) + +elif cmd == 'getNextChunkID': + if len(args) != 2: + print 'getNextChunkID requires 2 args' + sys.exit(1) + pp.pprint(client.getNextChunkID(args[0],args[1],)) + +elif cmd == 'getStreamingTmpDir': + if len(args) != 2: + print 'getStreamingTmpDir requires 2 args' + sys.exit(1) + pp.pprint(client.getStreamingTmpDir(args[0],args[1],)) + elif cmd == 'create_role': if len(args) != 1: print 'create_role requires 1 args' diff --git metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py index 9b856e5..19586be 100644 --- metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py +++ metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py @@ -617,6 +617,76 @@ def delete_table_column_statistics(self, db_name, tbl_name, col_name): """ pass + def streamingStatus(self, db_name, table_name): + """ + Parameters: + - db_name + - table_name + """ + pass + + def enableStreaming(self, db_name, table_name, streaming_tmp_dir, partition_path, partition_val): + """ + Parameters: + - db_name + - table_name + - streaming_tmp_dir + - partition_path + - partition_val + """ + pass + + def disableStreaming(self, db_name, table_name): + """ + Parameters: + - db_name + - table_name + """ + pass + + def getCurrentStreamingPartitionPath(self, dbName, tableName): + """ + Parameters: + - dbName + - tableName + """ + pass + + def rollStreamingPartition(self, dbName, tableName, newPartitionPath, newPartitionVal): + """ + Parameters: + - dbName + - tableName + - newPartitionPath + - newPartitionVal + """ + pass + + def updateStreamingTempLocation(self, dbName, tableName, streamingTempDir): + """ + Parameters: + - dbName + - tableName + - streamingTempDir + """ + pass + + def getNextChunkID(self, db_name, table_name): + """ + Parameters: + - db_name + - table_name + """ + pass + + def getStreamingTmpDir(self, dbName, tableName): + """ + Parameters: + - dbName + - tableName + """ + pass + def create_role(self, role): """ Parameters: @@ -3316,474 +3386,798 @@ def recv_delete_table_column_statistics(self, ): raise result.o4 raise TApplicationException(TApplicationException.MISSING_RESULT, "delete_table_column_statistics failed: unknown result"); - def create_role(self, role): + def streamingStatus(self, db_name, table_name): """ Parameters: - - role + - db_name + - table_name """ - self.send_create_role(role) - return self.recv_create_role() + self.send_streamingStatus(db_name, table_name) + return self.recv_streamingStatus() - def send_create_role(self, role): - self._oprot.writeMessageBegin('create_role', TMessageType.CALL, self._seqid) - args = create_role_args() - args.role = role + def send_streamingStatus(self, db_name, table_name): + self._oprot.writeMessageBegin('streamingStatus', TMessageType.CALL, self._seqid) + args = streamingStatus_args() + args.db_name = db_name + args.table_name = table_name args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() - def recv_create_role(self, ): + def recv_streamingStatus(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x - result = create_role_result() + result = streamingStatus_result() result.read(self._iprot) self._iprot.readMessageEnd() if result.success is not None: return result.success if result.o1 is not None: raise result.o1 - raise TApplicationException(TApplicationException.MISSING_RESULT, "create_role failed: unknown result"); + if result.o2 is not None: + raise result.o2 + if result.o3 is not None: + raise result.o3 + raise TApplicationException(TApplicationException.MISSING_RESULT, "streamingStatus failed: unknown result"); - def drop_role(self, role_name): + def enableStreaming(self, db_name, table_name, streaming_tmp_dir, partition_path, partition_val): """ Parameters: - - role_name + - db_name + - table_name + - streaming_tmp_dir + - partition_path + - partition_val """ - self.send_drop_role(role_name) - return self.recv_drop_role() + self.send_enableStreaming(db_name, table_name, streaming_tmp_dir, partition_path, partition_val) + self.recv_enableStreaming() - def send_drop_role(self, role_name): - self._oprot.writeMessageBegin('drop_role', TMessageType.CALL, self._seqid) - args = drop_role_args() - args.role_name = role_name + def send_enableStreaming(self, db_name, table_name, streaming_tmp_dir, partition_path, partition_val): + self._oprot.writeMessageBegin('enableStreaming', TMessageType.CALL, self._seqid) + args = enableStreaming_args() + args.db_name = db_name + args.table_name = table_name + args.streaming_tmp_dir = streaming_tmp_dir + args.partition_path = partition_path + args.partition_val = partition_val args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() - def recv_drop_role(self, ): + def recv_enableStreaming(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x - result = drop_role_result() + result = enableStreaming_result() result.read(self._iprot) self._iprot.readMessageEnd() - if result.success is not None: - return result.success if result.o1 is not None: raise result.o1 - raise TApplicationException(TApplicationException.MISSING_RESULT, "drop_role failed: unknown result"); + if result.o2 is not None: + raise result.o2 + if result.o3 is not None: + raise result.o3 + if result.o4 is not None: + raise result.o4 + return - def get_role_names(self, ): - self.send_get_role_names() - return self.recv_get_role_names() + def disableStreaming(self, db_name, table_name): + """ + Parameters: + - db_name + - table_name + """ + self.send_disableStreaming(db_name, table_name) + self.recv_disableStreaming() - def send_get_role_names(self, ): - self._oprot.writeMessageBegin('get_role_names', TMessageType.CALL, self._seqid) - args = get_role_names_args() + def send_disableStreaming(self, db_name, table_name): + self._oprot.writeMessageBegin('disableStreaming', TMessageType.CALL, self._seqid) + args = disableStreaming_args() + args.db_name = db_name + args.table_name = table_name args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() - def recv_get_role_names(self, ): + def recv_disableStreaming(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x - result = get_role_names_result() + result = disableStreaming_result() result.read(self._iprot) self._iprot.readMessageEnd() - if result.success is not None: - return result.success if result.o1 is not None: raise result.o1 - raise TApplicationException(TApplicationException.MISSING_RESULT, "get_role_names failed: unknown result"); + if result.o2 is not None: + raise result.o2 + if result.o3 is not None: + raise result.o3 + if result.o4 is not None: + raise result.o4 + return - def grant_role(self, role_name, principal_name, principal_type, grantor, grantorType, grant_option): + def getCurrentStreamingPartitionPath(self, dbName, tableName): """ Parameters: - - role_name - - principal_name - - principal_type - - grantor - - grantorType - - grant_option + - dbName + - tableName """ - self.send_grant_role(role_name, principal_name, principal_type, grantor, grantorType, grant_option) - return self.recv_grant_role() + self.send_getCurrentStreamingPartitionPath(dbName, tableName) + return self.recv_getCurrentStreamingPartitionPath() - def send_grant_role(self, role_name, principal_name, principal_type, grantor, grantorType, grant_option): - self._oprot.writeMessageBegin('grant_role', TMessageType.CALL, self._seqid) - args = grant_role_args() - args.role_name = role_name - args.principal_name = principal_name - args.principal_type = principal_type - args.grantor = grantor - args.grantorType = grantorType - args.grant_option = grant_option + def send_getCurrentStreamingPartitionPath(self, dbName, tableName): + self._oprot.writeMessageBegin('getCurrentStreamingPartitionPath', TMessageType.CALL, self._seqid) + args = getCurrentStreamingPartitionPath_args() + args.dbName = dbName + args.tableName = tableName args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() - def recv_grant_role(self, ): + def recv_getCurrentStreamingPartitionPath(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x - result = grant_role_result() + result = getCurrentStreamingPartitionPath_result() result.read(self._iprot) self._iprot.readMessageEnd() if result.success is not None: return result.success if result.o1 is not None: raise result.o1 - raise TApplicationException(TApplicationException.MISSING_RESULT, "grant_role failed: unknown result"); + if result.o2 is not None: + raise result.o2 + if result.o3 is not None: + raise result.o3 + if result.o4 is not None: + raise result.o4 + raise TApplicationException(TApplicationException.MISSING_RESULT, "getCurrentStreamingPartitionPath failed: unknown result"); - def revoke_role(self, role_name, principal_name, principal_type): + def rollStreamingPartition(self, dbName, tableName, newPartitionPath, newPartitionVal): """ Parameters: - - role_name - - principal_name - - principal_type - """ - self.send_revoke_role(role_name, principal_name, principal_type) - return self.recv_revoke_role() - - def send_revoke_role(self, role_name, principal_name, principal_type): - self._oprot.writeMessageBegin('revoke_role', TMessageType.CALL, self._seqid) - args = revoke_role_args() - args.role_name = role_name - args.principal_name = principal_name - args.principal_type = principal_type + - dbName + - tableName + - newPartitionPath + - newPartitionVal + """ + self.send_rollStreamingPartition(dbName, tableName, newPartitionPath, newPartitionVal) + return self.recv_rollStreamingPartition() + + def send_rollStreamingPartition(self, dbName, tableName, newPartitionPath, newPartitionVal): + self._oprot.writeMessageBegin('rollStreamingPartition', TMessageType.CALL, self._seqid) + args = rollStreamingPartition_args() + args.dbName = dbName + args.tableName = tableName + args.newPartitionPath = newPartitionPath + args.newPartitionVal = newPartitionVal args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() - def recv_revoke_role(self, ): + def recv_rollStreamingPartition(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x - result = revoke_role_result() + result = rollStreamingPartition_result() result.read(self._iprot) self._iprot.readMessageEnd() if result.success is not None: return result.success if result.o1 is not None: raise result.o1 - raise TApplicationException(TApplicationException.MISSING_RESULT, "revoke_role failed: unknown result"); + if result.o2 is not None: + raise result.o2 + if result.o3 is not None: + raise result.o3 + if result.o4 is not None: + raise result.o4 + if result.o5 is not None: + raise result.o5 + raise TApplicationException(TApplicationException.MISSING_RESULT, "rollStreamingPartition failed: unknown result"); - def list_roles(self, principal_name, principal_type): + def updateStreamingTempLocation(self, dbName, tableName, streamingTempDir): """ Parameters: - - principal_name - - principal_type - """ - self.send_list_roles(principal_name, principal_type) - return self.recv_list_roles() - - def send_list_roles(self, principal_name, principal_type): - self._oprot.writeMessageBegin('list_roles', TMessageType.CALL, self._seqid) - args = list_roles_args() - args.principal_name = principal_name - args.principal_type = principal_type + - dbName + - tableName + - streamingTempDir + """ + self.send_updateStreamingTempLocation(dbName, tableName, streamingTempDir) + self.recv_updateStreamingTempLocation() + + def send_updateStreamingTempLocation(self, dbName, tableName, streamingTempDir): + self._oprot.writeMessageBegin('updateStreamingTempLocation', TMessageType.CALL, self._seqid) + args = updateStreamingTempLocation_args() + args.dbName = dbName + args.tableName = tableName + args.streamingTempDir = streamingTempDir args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() - def recv_list_roles(self, ): + def recv_updateStreamingTempLocation(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x - result = list_roles_result() + result = updateStreamingTempLocation_result() result.read(self._iprot) self._iprot.readMessageEnd() - if result.success is not None: - return result.success if result.o1 is not None: raise result.o1 - raise TApplicationException(TApplicationException.MISSING_RESULT, "list_roles failed: unknown result"); + if result.o2 is not None: + raise result.o2 + if result.o3 is not None: + raise result.o3 + return - def get_privilege_set(self, hiveObject, user_name, group_names): + def getNextChunkID(self, db_name, table_name): """ Parameters: - - hiveObject - - user_name - - group_names + - db_name + - table_name """ - self.send_get_privilege_set(hiveObject, user_name, group_names) - return self.recv_get_privilege_set() + self.send_getNextChunkID(db_name, table_name) + return self.recv_getNextChunkID() - def send_get_privilege_set(self, hiveObject, user_name, group_names): - self._oprot.writeMessageBegin('get_privilege_set', TMessageType.CALL, self._seqid) - args = get_privilege_set_args() - args.hiveObject = hiveObject - args.user_name = user_name - args.group_names = group_names + def send_getNextChunkID(self, db_name, table_name): + self._oprot.writeMessageBegin('getNextChunkID', TMessageType.CALL, self._seqid) + args = getNextChunkID_args() + args.db_name = db_name + args.table_name = table_name args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() - def recv_get_privilege_set(self, ): + def recv_getNextChunkID(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x - result = get_privilege_set_result() + result = getNextChunkID_result() result.read(self._iprot) self._iprot.readMessageEnd() if result.success is not None: return result.success if result.o1 is not None: raise result.o1 - raise TApplicationException(TApplicationException.MISSING_RESULT, "get_privilege_set failed: unknown result"); + if result.o2 is not None: + raise result.o2 + if result.o3 is not None: + raise result.o3 + if result.o4 is not None: + raise result.o4 + raise TApplicationException(TApplicationException.MISSING_RESULT, "getNextChunkID failed: unknown result"); - def list_privileges(self, principal_name, principal_type, hiveObject): + def getStreamingTmpDir(self, dbName, tableName): """ Parameters: - - principal_name - - principal_type - - hiveObject + - dbName + - tableName """ - self.send_list_privileges(principal_name, principal_type, hiveObject) - return self.recv_list_privileges() + self.send_getStreamingTmpDir(dbName, tableName) + return self.recv_getStreamingTmpDir() - def send_list_privileges(self, principal_name, principal_type, hiveObject): - self._oprot.writeMessageBegin('list_privileges', TMessageType.CALL, self._seqid) - args = list_privileges_args() - args.principal_name = principal_name - args.principal_type = principal_type - args.hiveObject = hiveObject + def send_getStreamingTmpDir(self, dbName, tableName): + self._oprot.writeMessageBegin('getStreamingTmpDir', TMessageType.CALL, self._seqid) + args = getStreamingTmpDir_args() + args.dbName = dbName + args.tableName = tableName args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() - def recv_list_privileges(self, ): + def recv_getStreamingTmpDir(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x - result = list_privileges_result() + result = getStreamingTmpDir_result() result.read(self._iprot) self._iprot.readMessageEnd() if result.success is not None: return result.success if result.o1 is not None: raise result.o1 - raise TApplicationException(TApplicationException.MISSING_RESULT, "list_privileges failed: unknown result"); + if result.o2 is not None: + raise result.o2 + if result.o3 is not None: + raise result.o3 + if result.o4 is not None: + raise result.o4 + raise TApplicationException(TApplicationException.MISSING_RESULT, "getStreamingTmpDir failed: unknown result"); - def grant_privileges(self, privileges): + def create_role(self, role): """ Parameters: - - privileges + - role """ - self.send_grant_privileges(privileges) - return self.recv_grant_privileges() + self.send_create_role(role) + return self.recv_create_role() - def send_grant_privileges(self, privileges): - self._oprot.writeMessageBegin('grant_privileges', TMessageType.CALL, self._seqid) - args = grant_privileges_args() - args.privileges = privileges + def send_create_role(self, role): + self._oprot.writeMessageBegin('create_role', TMessageType.CALL, self._seqid) + args = create_role_args() + args.role = role args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() - def recv_grant_privileges(self, ): + def recv_create_role(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x - result = grant_privileges_result() + result = create_role_result() result.read(self._iprot) self._iprot.readMessageEnd() if result.success is not None: return result.success if result.o1 is not None: raise result.o1 - raise TApplicationException(TApplicationException.MISSING_RESULT, "grant_privileges failed: unknown result"); + raise TApplicationException(TApplicationException.MISSING_RESULT, "create_role failed: unknown result"); - def revoke_privileges(self, privileges): + def drop_role(self, role_name): """ Parameters: - - privileges + - role_name """ - self.send_revoke_privileges(privileges) - return self.recv_revoke_privileges() + self.send_drop_role(role_name) + return self.recv_drop_role() - def send_revoke_privileges(self, privileges): - self._oprot.writeMessageBegin('revoke_privileges', TMessageType.CALL, self._seqid) - args = revoke_privileges_args() - args.privileges = privileges + def send_drop_role(self, role_name): + self._oprot.writeMessageBegin('drop_role', TMessageType.CALL, self._seqid) + args = drop_role_args() + args.role_name = role_name args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() - def recv_revoke_privileges(self, ): + def recv_drop_role(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x - result = revoke_privileges_result() + result = drop_role_result() result.read(self._iprot) self._iprot.readMessageEnd() if result.success is not None: return result.success if result.o1 is not None: raise result.o1 - raise TApplicationException(TApplicationException.MISSING_RESULT, "revoke_privileges failed: unknown result"); + raise TApplicationException(TApplicationException.MISSING_RESULT, "drop_role failed: unknown result"); - def set_ugi(self, user_name, group_names): + def get_role_names(self, ): + self.send_get_role_names() + return self.recv_get_role_names() + + def send_get_role_names(self, ): + self._oprot.writeMessageBegin('get_role_names', TMessageType.CALL, self._seqid) + args = get_role_names_args() + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_get_role_names(self, ): + (fname, mtype, rseqid) = self._iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(self._iprot) + self._iprot.readMessageEnd() + raise x + result = get_role_names_result() + result.read(self._iprot) + self._iprot.readMessageEnd() + if result.success is not None: + return result.success + if result.o1 is not None: + raise result.o1 + raise TApplicationException(TApplicationException.MISSING_RESULT, "get_role_names failed: unknown result"); + + def grant_role(self, role_name, principal_name, principal_type, grantor, grantorType, grant_option): + """ + Parameters: + - role_name + - principal_name + - principal_type + - grantor + - grantorType + - grant_option + """ + self.send_grant_role(role_name, principal_name, principal_type, grantor, grantorType, grant_option) + return self.recv_grant_role() + + def send_grant_role(self, role_name, principal_name, principal_type, grantor, grantorType, grant_option): + self._oprot.writeMessageBegin('grant_role', TMessageType.CALL, self._seqid) + args = grant_role_args() + args.role_name = role_name + args.principal_name = principal_name + args.principal_type = principal_type + args.grantor = grantor + args.grantorType = grantorType + args.grant_option = grant_option + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_grant_role(self, ): + (fname, mtype, rseqid) = self._iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(self._iprot) + self._iprot.readMessageEnd() + raise x + result = grant_role_result() + result.read(self._iprot) + self._iprot.readMessageEnd() + if result.success is not None: + return result.success + if result.o1 is not None: + raise result.o1 + raise TApplicationException(TApplicationException.MISSING_RESULT, "grant_role failed: unknown result"); + + def revoke_role(self, role_name, principal_name, principal_type): + """ + Parameters: + - role_name + - principal_name + - principal_type + """ + self.send_revoke_role(role_name, principal_name, principal_type) + return self.recv_revoke_role() + + def send_revoke_role(self, role_name, principal_name, principal_type): + self._oprot.writeMessageBegin('revoke_role', TMessageType.CALL, self._seqid) + args = revoke_role_args() + args.role_name = role_name + args.principal_name = principal_name + args.principal_type = principal_type + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_revoke_role(self, ): + (fname, mtype, rseqid) = self._iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(self._iprot) + self._iprot.readMessageEnd() + raise x + result = revoke_role_result() + result.read(self._iprot) + self._iprot.readMessageEnd() + if result.success is not None: + return result.success + if result.o1 is not None: + raise result.o1 + raise TApplicationException(TApplicationException.MISSING_RESULT, "revoke_role failed: unknown result"); + + def list_roles(self, principal_name, principal_type): + """ + Parameters: + - principal_name + - principal_type + """ + self.send_list_roles(principal_name, principal_type) + return self.recv_list_roles() + + def send_list_roles(self, principal_name, principal_type): + self._oprot.writeMessageBegin('list_roles', TMessageType.CALL, self._seqid) + args = list_roles_args() + args.principal_name = principal_name + args.principal_type = principal_type + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_list_roles(self, ): + (fname, mtype, rseqid) = self._iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(self._iprot) + self._iprot.readMessageEnd() + raise x + result = list_roles_result() + result.read(self._iprot) + self._iprot.readMessageEnd() + if result.success is not None: + return result.success + if result.o1 is not None: + raise result.o1 + raise TApplicationException(TApplicationException.MISSING_RESULT, "list_roles failed: unknown result"); + + def get_privilege_set(self, hiveObject, user_name, group_names): """ Parameters: + - hiveObject - user_name - group_names """ - self.send_set_ugi(user_name, group_names) - return self.recv_set_ugi() + self.send_get_privilege_set(hiveObject, user_name, group_names) + return self.recv_get_privilege_set() - def send_set_ugi(self, user_name, group_names): - self._oprot.writeMessageBegin('set_ugi', TMessageType.CALL, self._seqid) - args = set_ugi_args() + def send_get_privilege_set(self, hiveObject, user_name, group_names): + self._oprot.writeMessageBegin('get_privilege_set', TMessageType.CALL, self._seqid) + args = get_privilege_set_args() + args.hiveObject = hiveObject args.user_name = user_name args.group_names = group_names args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() - def recv_set_ugi(self, ): + def recv_get_privilege_set(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x - result = set_ugi_result() + result = get_privilege_set_result() result.read(self._iprot) self._iprot.readMessageEnd() if result.success is not None: return result.success if result.o1 is not None: raise result.o1 - raise TApplicationException(TApplicationException.MISSING_RESULT, "set_ugi failed: unknown result"); + raise TApplicationException(TApplicationException.MISSING_RESULT, "get_privilege_set failed: unknown result"); - def get_delegation_token(self, token_owner, renewer_kerberos_principal_name): + def list_privileges(self, principal_name, principal_type, hiveObject): """ Parameters: - - token_owner - - renewer_kerberos_principal_name + - principal_name + - principal_type + - hiveObject """ - self.send_get_delegation_token(token_owner, renewer_kerberos_principal_name) - return self.recv_get_delegation_token() + self.send_list_privileges(principal_name, principal_type, hiveObject) + return self.recv_list_privileges() - def send_get_delegation_token(self, token_owner, renewer_kerberos_principal_name): - self._oprot.writeMessageBegin('get_delegation_token', TMessageType.CALL, self._seqid) - args = get_delegation_token_args() - args.token_owner = token_owner - args.renewer_kerberos_principal_name = renewer_kerberos_principal_name + def send_list_privileges(self, principal_name, principal_type, hiveObject): + self._oprot.writeMessageBegin('list_privileges', TMessageType.CALL, self._seqid) + args = list_privileges_args() + args.principal_name = principal_name + args.principal_type = principal_type + args.hiveObject = hiveObject args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() - def recv_get_delegation_token(self, ): + def recv_list_privileges(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x - result = get_delegation_token_result() + result = list_privileges_result() result.read(self._iprot) self._iprot.readMessageEnd() if result.success is not None: return result.success if result.o1 is not None: raise result.o1 - raise TApplicationException(TApplicationException.MISSING_RESULT, "get_delegation_token failed: unknown result"); + raise TApplicationException(TApplicationException.MISSING_RESULT, "list_privileges failed: unknown result"); - def renew_delegation_token(self, token_str_form): + def grant_privileges(self, privileges): """ Parameters: - - token_str_form + - privileges """ - self.send_renew_delegation_token(token_str_form) - return self.recv_renew_delegation_token() + self.send_grant_privileges(privileges) + return self.recv_grant_privileges() - def send_renew_delegation_token(self, token_str_form): - self._oprot.writeMessageBegin('renew_delegation_token', TMessageType.CALL, self._seqid) - args = renew_delegation_token_args() - args.token_str_form = token_str_form + def send_grant_privileges(self, privileges): + self._oprot.writeMessageBegin('grant_privileges', TMessageType.CALL, self._seqid) + args = grant_privileges_args() + args.privileges = privileges args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() - def recv_renew_delegation_token(self, ): + def recv_grant_privileges(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x - result = renew_delegation_token_result() + result = grant_privileges_result() result.read(self._iprot) self._iprot.readMessageEnd() if result.success is not None: return result.success if result.o1 is not None: raise result.o1 - raise TApplicationException(TApplicationException.MISSING_RESULT, "renew_delegation_token failed: unknown result"); + raise TApplicationException(TApplicationException.MISSING_RESULT, "grant_privileges failed: unknown result"); - def cancel_delegation_token(self, token_str_form): + def revoke_privileges(self, privileges): """ Parameters: - - token_str_form + - privileges """ - self.send_cancel_delegation_token(token_str_form) - self.recv_cancel_delegation_token() + self.send_revoke_privileges(privileges) + return self.recv_revoke_privileges() - def send_cancel_delegation_token(self, token_str_form): - self._oprot.writeMessageBegin('cancel_delegation_token', TMessageType.CALL, self._seqid) - args = cancel_delegation_token_args() - args.token_str_form = token_str_form + def send_revoke_privileges(self, privileges): + self._oprot.writeMessageBegin('revoke_privileges', TMessageType.CALL, self._seqid) + args = revoke_privileges_args() + args.privileges = privileges args.write(self._oprot) self._oprot.writeMessageEnd() self._oprot.trans.flush() - def recv_cancel_delegation_token(self, ): + def recv_revoke_privileges(self, ): (fname, mtype, rseqid) = self._iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() x.read(self._iprot) self._iprot.readMessageEnd() raise x - result = cancel_delegation_token_result() + result = revoke_privileges_result() result.read(self._iprot) self._iprot.readMessageEnd() + if result.success is not None: + return result.success if result.o1 is not None: raise result.o1 - return + raise TApplicationException(TApplicationException.MISSING_RESULT, "revoke_privileges failed: unknown result"); + + def set_ugi(self, user_name, group_names): + """ + Parameters: + - user_name + - group_names + """ + self.send_set_ugi(user_name, group_names) + return self.recv_set_ugi() + + def send_set_ugi(self, user_name, group_names): + self._oprot.writeMessageBegin('set_ugi', TMessageType.CALL, self._seqid) + args = set_ugi_args() + args.user_name = user_name + args.group_names = group_names + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_set_ugi(self, ): + (fname, mtype, rseqid) = self._iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(self._iprot) + self._iprot.readMessageEnd() + raise x + result = set_ugi_result() + result.read(self._iprot) + self._iprot.readMessageEnd() + if result.success is not None: + return result.success + if result.o1 is not None: + raise result.o1 + raise TApplicationException(TApplicationException.MISSING_RESULT, "set_ugi failed: unknown result"); + + def get_delegation_token(self, token_owner, renewer_kerberos_principal_name): + """ + Parameters: + - token_owner + - renewer_kerberos_principal_name + """ + self.send_get_delegation_token(token_owner, renewer_kerberos_principal_name) + return self.recv_get_delegation_token() + + def send_get_delegation_token(self, token_owner, renewer_kerberos_principal_name): + self._oprot.writeMessageBegin('get_delegation_token', TMessageType.CALL, self._seqid) + args = get_delegation_token_args() + args.token_owner = token_owner + args.renewer_kerberos_principal_name = renewer_kerberos_principal_name + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_get_delegation_token(self, ): + (fname, mtype, rseqid) = self._iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(self._iprot) + self._iprot.readMessageEnd() + raise x + result = get_delegation_token_result() + result.read(self._iprot) + self._iprot.readMessageEnd() + if result.success is not None: + return result.success + if result.o1 is not None: + raise result.o1 + raise TApplicationException(TApplicationException.MISSING_RESULT, "get_delegation_token failed: unknown result"); + + def renew_delegation_token(self, token_str_form): + """ + Parameters: + - token_str_form + """ + self.send_renew_delegation_token(token_str_form) + return self.recv_renew_delegation_token() + + def send_renew_delegation_token(self, token_str_form): + self._oprot.writeMessageBegin('renew_delegation_token', TMessageType.CALL, self._seqid) + args = renew_delegation_token_args() + args.token_str_form = token_str_form + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_renew_delegation_token(self, ): + (fname, mtype, rseqid) = self._iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(self._iprot) + self._iprot.readMessageEnd() + raise x + result = renew_delegation_token_result() + result.read(self._iprot) + self._iprot.readMessageEnd() + if result.success is not None: + return result.success + if result.o1 is not None: + raise result.o1 + raise TApplicationException(TApplicationException.MISSING_RESULT, "renew_delegation_token failed: unknown result"); + + def cancel_delegation_token(self, token_str_form): + """ + Parameters: + - token_str_form + """ + self.send_cancel_delegation_token(token_str_form) + self.recv_cancel_delegation_token() + + def send_cancel_delegation_token(self, token_str_form): + self._oprot.writeMessageBegin('cancel_delegation_token', TMessageType.CALL, self._seqid) + args = cancel_delegation_token_args() + args.token_str_form = token_str_form + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_cancel_delegation_token(self, ): + (fname, mtype, rseqid) = self._iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(self._iprot) + self._iprot.readMessageEnd() + raise x + result = cancel_delegation_token_result() + result.read(self._iprot) + self._iprot.readMessageEnd() + if result.o1 is not None: + raise result.o1 + return class Processor(fb303.FacebookService.Processor, Iface, TProcessor): @@ -3857,6 +4251,14 @@ def __init__(self, handler): self._processMap["get_partition_column_statistics"] = Processor.process_get_partition_column_statistics self._processMap["delete_partition_column_statistics"] = Processor.process_delete_partition_column_statistics self._processMap["delete_table_column_statistics"] = Processor.process_delete_table_column_statistics + self._processMap["streamingStatus"] = Processor.process_streamingStatus + self._processMap["enableStreaming"] = Processor.process_enableStreaming + self._processMap["disableStreaming"] = Processor.process_disableStreaming + self._processMap["getCurrentStreamingPartitionPath"] = Processor.process_getCurrentStreamingPartitionPath + self._processMap["rollStreamingPartition"] = Processor.process_rollStreamingPartition + self._processMap["updateStreamingTempLocation"] = Processor.process_updateStreamingTempLocation + self._processMap["getNextChunkID"] = Processor.process_getNextChunkID + self._processMap["getStreamingTmpDir"] = Processor.process_getStreamingTmpDir self._processMap["create_role"] = Processor.process_create_role self._processMap["drop_role"] = Processor.process_drop_role self._processMap["get_role_names"] = Processor.process_get_role_names @@ -5035,6 +5437,164 @@ def process_delete_table_column_statistics(self, seqid, iprot, oprot): oprot.writeMessageEnd() oprot.trans.flush() + def process_streamingStatus(self, seqid, iprot, oprot): + args = streamingStatus_args() + args.read(iprot) + iprot.readMessageEnd() + result = streamingStatus_result() + try: + result.success = self._handler.streamingStatus(args.db_name, args.table_name) + except InvalidInputException as o1: + result.o1 = o1 + except InvalidObjectException as o2: + result.o2 = o2 + except MetaException as o3: + result.o3 = o3 + oprot.writeMessageBegin("streamingStatus", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_enableStreaming(self, seqid, iprot, oprot): + args = enableStreaming_args() + args.read(iprot) + iprot.readMessageEnd() + result = enableStreaming_result() + try: + self._handler.enableStreaming(args.db_name, args.table_name, args.streaming_tmp_dir, args.partition_path, args.partition_val) + except InvalidInputException as o1: + result.o1 = o1 + except InvalidObjectException as o2: + result.o2 = o2 + except InvalidOperationException as o3: + result.o3 = o3 + except MetaException as o4: + result.o4 = o4 + oprot.writeMessageBegin("enableStreaming", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_disableStreaming(self, seqid, iprot, oprot): + args = disableStreaming_args() + args.read(iprot) + iprot.readMessageEnd() + result = disableStreaming_result() + try: + self._handler.disableStreaming(args.db_name, args.table_name) + except InvalidInputException as o1: + result.o1 = o1 + except InvalidObjectException as o2: + result.o2 = o2 + except InvalidOperationException as o3: + result.o3 = o3 + except MetaException as o4: + result.o4 = o4 + oprot.writeMessageBegin("disableStreaming", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_getCurrentStreamingPartitionPath(self, seqid, iprot, oprot): + args = getCurrentStreamingPartitionPath_args() + args.read(iprot) + iprot.readMessageEnd() + result = getCurrentStreamingPartitionPath_result() + try: + result.success = self._handler.getCurrentStreamingPartitionPath(args.dbName, args.tableName) + except InvalidInputException as o1: + result.o1 = o1 + except InvalidObjectException as o2: + result.o2 = o2 + except InvalidOperationException as o3: + result.o3 = o3 + except MetaException as o4: + result.o4 = o4 + oprot.writeMessageBegin("getCurrentStreamingPartitionPath", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_rollStreamingPartition(self, seqid, iprot, oprot): + args = rollStreamingPartition_args() + args.read(iprot) + iprot.readMessageEnd() + result = rollStreamingPartition_result() + try: + result.success = self._handler.rollStreamingPartition(args.dbName, args.tableName, args.newPartitionPath, args.newPartitionVal) + except InvalidInputException as o1: + result.o1 = o1 + except InvalidObjectException as o2: + result.o2 = o2 + except InvalidOperationException as o3: + result.o3 = o3 + except AlreadyExistsException as o4: + result.o4 = o4 + except MetaException as o5: + result.o5 = o5 + oprot.writeMessageBegin("rollStreamingPartition", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_updateStreamingTempLocation(self, seqid, iprot, oprot): + args = updateStreamingTempLocation_args() + args.read(iprot) + iprot.readMessageEnd() + result = updateStreamingTempLocation_result() + try: + self._handler.updateStreamingTempLocation(args.dbName, args.tableName, args.streamingTempDir) + except InvalidInputException as o1: + result.o1 = o1 + except InvalidObjectException as o2: + result.o2 = o2 + except MetaException as o3: + result.o3 = o3 + oprot.writeMessageBegin("updateStreamingTempLocation", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_getNextChunkID(self, seqid, iprot, oprot): + args = getNextChunkID_args() + args.read(iprot) + iprot.readMessageEnd() + result = getNextChunkID_result() + try: + result.success = self._handler.getNextChunkID(args.db_name, args.table_name) + except InvalidInputException as o1: + result.o1 = o1 + except InvalidObjectException as o2: + result.o2 = o2 + except InvalidOperationException as o3: + result.o3 = o3 + except MetaException as o4: + result.o4 = o4 + oprot.writeMessageBegin("getNextChunkID", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_getStreamingTmpDir(self, seqid, iprot, oprot): + args = getStreamingTmpDir_args() + args.read(iprot) + iprot.readMessageEnd() + result = getStreamingTmpDir_result() + try: + result.success = self._handler.getStreamingTmpDir(args.dbName, args.tableName) + except InvalidInputException as o1: + result.o1 = o1 + except InvalidObjectException as o2: + result.o2 = o2 + except InvalidOperationException as o3: + result.o3 = o3 + except MetaException as o4: + result.o4 = o4 + oprot.writeMessageBegin("getStreamingTmpDir", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + def process_create_role(self, seqid, iprot, oprot): args = create_role_args() args.read(iprot) @@ -5326,22 +5886,1198 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRUCT: - self.o1 = AlreadyExistsException() + self.o1 = AlreadyExistsException() + self.o1.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = InvalidObjectException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = MetaException() + self.o3.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('create_database_result') + if self.o1 is not None: + oprot.writeFieldBegin('o1', TType.STRUCT, 1) + self.o1.write(oprot) + oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class get_database_args: + """ + Attributes: + - name + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'name', None, None, ), # 1 + ) + + def __init__(self, name=None,): + self.name = name + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.name = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('get_database_args') + if self.name is not None: + oprot.writeFieldBegin('name', TType.STRING, 1) + oprot.writeString(self.name) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class get_database_result: + """ + Attributes: + - success + - o1 + - o2 + """ + + thrift_spec = ( + (0, TType.STRUCT, 'success', (Database, Database.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + ) + + def __init__(self, success=None, o1=None, o2=None,): + self.success = success + self.o1 = o1 + self.o2 = o2 + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.STRUCT: + self.success = Database() + self.success.read(iprot) + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.o1 = NoSuchObjectException() + self.o1.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = MetaException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('get_database_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) + oprot.writeFieldEnd() + if self.o1 is not None: + oprot.writeFieldBegin('o1', TType.STRUCT, 1) + self.o1.write(oprot) + oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class drop_database_args: + """ + Attributes: + - name + - deleteData + - cascade + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'name', None, None, ), # 1 + (2, TType.BOOL, 'deleteData', None, None, ), # 2 + (3, TType.BOOL, 'cascade', None, None, ), # 3 + ) + + def __init__(self, name=None, deleteData=None, cascade=None,): + self.name = name + self.deleteData = deleteData + self.cascade = cascade + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.BOOL: + self.deleteData = iprot.readBool(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.BOOL: + self.cascade = iprot.readBool(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('drop_database_args') + if self.name is not None: + oprot.writeFieldBegin('name', TType.STRING, 1) + oprot.writeString(self.name) + oprot.writeFieldEnd() + if self.deleteData is not None: + oprot.writeFieldBegin('deleteData', TType.BOOL, 2) + oprot.writeBool(self.deleteData) + oprot.writeFieldEnd() + if self.cascade is not None: + oprot.writeFieldBegin('cascade', TType.BOOL, 3) + oprot.writeBool(self.cascade) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class drop_database_result: + """ + Attributes: + - o1 + - o2 + - o3 + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + ) + + def __init__(self, o1=None, o2=None, o3=None,): + self.o1 = o1 + self.o2 = o2 + self.o3 = o3 + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.o1 = NoSuchObjectException() + self.o1.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = InvalidOperationException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = MetaException() + self.o3.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('drop_database_result') + if self.o1 is not None: + oprot.writeFieldBegin('o1', TType.STRUCT, 1) + self.o1.write(oprot) + oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class get_databases_args: + """ + Attributes: + - pattern + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'pattern', None, None, ), # 1 + ) + + def __init__(self, pattern=None,): + self.pattern = pattern + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.pattern = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('get_databases_args') + if self.pattern is not None: + oprot.writeFieldBegin('pattern', TType.STRING, 1) + oprot.writeString(self.pattern) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class get_databases_result: + """ + Attributes: + - success + - o1 + """ + + thrift_spec = ( + (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + ) + + def __init__(self, success=None, o1=None,): + self.success = success + self.o1 = o1 + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.LIST: + self.success = [] + (_etype230, _size227) = iprot.readListBegin() + for _i231 in xrange(_size227): + _elem232 = iprot.readString(); + self.success.append(_elem232) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.o1 = MetaException() + self.o1.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('get_databases_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRING, len(self.success)) + for iter233 in self.success: + oprot.writeString(iter233) + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.o1 is not None: + oprot.writeFieldBegin('o1', TType.STRUCT, 1) + self.o1.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class get_all_databases_args: + + thrift_spec = ( + ) + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('get_all_databases_args') + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class get_all_databases_result: + """ + Attributes: + - success + - o1 + """ + + thrift_spec = ( + (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + ) + + def __init__(self, success=None, o1=None,): + self.success = success + self.o1 = o1 + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.LIST: + self.success = [] + (_etype237, _size234) = iprot.readListBegin() + for _i238 in xrange(_size234): + _elem239 = iprot.readString(); + self.success.append(_elem239) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.o1 = MetaException() + self.o1.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('get_all_databases_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRING, len(self.success)) + for iter240 in self.success: + oprot.writeString(iter240) + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.o1 is not None: + oprot.writeFieldBegin('o1', TType.STRUCT, 1) + self.o1.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class alter_database_args: + """ + Attributes: + - dbname + - db + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'dbname', None, None, ), # 1 + (2, TType.STRUCT, 'db', (Database, Database.thrift_spec), None, ), # 2 + ) + + def __init__(self, dbname=None, db=None,): + self.dbname = dbname + self.db = db + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.dbname = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.db = Database() + self.db.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('alter_database_args') + if self.dbname is not None: + oprot.writeFieldBegin('dbname', TType.STRING, 1) + oprot.writeString(self.dbname) + oprot.writeFieldEnd() + if self.db is not None: + oprot.writeFieldBegin('db', TType.STRUCT, 2) + self.db.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class alter_database_result: + """ + Attributes: + - o1 + - o2 + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 + ) + + def __init__(self, o1=None, o2=None,): + self.o1 = o1 + self.o2 = o2 + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.o1 = MetaException() + self.o1.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = NoSuchObjectException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('alter_database_result') + if self.o1 is not None: + oprot.writeFieldBegin('o1', TType.STRUCT, 1) + self.o1.write(oprot) + oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class get_type_args: + """ + Attributes: + - name + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'name', None, None, ), # 1 + ) + + def __init__(self, name=None,): + self.name = name + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.name = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('get_type_args') + if self.name is not None: + oprot.writeFieldBegin('name', TType.STRING, 1) + oprot.writeString(self.name) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class get_type_result: + """ + Attributes: + - success + - o1 + - o2 + """ + + thrift_spec = ( + (0, TType.STRUCT, 'success', (Type, Type.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 + ) + + def __init__(self, success=None, o1=None, o2=None,): + self.success = success + self.o1 = o1 + self.o2 = o2 + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.STRUCT: + self.success = Type() + self.success.read(iprot) + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.o1 = MetaException() + self.o1.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = NoSuchObjectException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('get_type_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) + oprot.writeFieldEnd() + if self.o1 is not None: + oprot.writeFieldBegin('o1', TType.STRUCT, 1) + self.o1.write(oprot) + oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class create_type_args: + """ + Attributes: + - type + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'type', (Type, Type.thrift_spec), None, ), # 1 + ) + + def __init__(self, type=None,): + self.type = type + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.type = Type() + self.type.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('create_type_args') + if self.type is not None: + oprot.writeFieldBegin('type', TType.STRUCT, 1) + self.type.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class create_type_result: + """ + Attributes: + - success + - o1 + - o2 + - o3 + """ + + thrift_spec = ( + (0, TType.BOOL, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + ) + + def __init__(self, success=None, o1=None, o2=None, o3=None,): + self.success = success + self.o1 = o1 + self.o2 = o2 + self.o3 = o3 + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.BOOL: + self.success = iprot.readBool(); + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.o1 = AlreadyExistsException() + self.o1.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = InvalidObjectException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = MetaException() + self.o3.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('create_type_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.BOOL, 0) + oprot.writeBool(self.success) + oprot.writeFieldEnd() + if self.o1 is not None: + oprot.writeFieldBegin('o1', TType.STRUCT, 1) + self.o1.write(oprot) + oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class drop_type_args: + """ + Attributes: + - type + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'type', None, None, ), # 1 + ) + + def __init__(self, type=None,): + self.type = type + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.type = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('drop_type_args') + if self.type is not None: + oprot.writeFieldBegin('type', TType.STRING, 1) + oprot.writeString(self.type) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class drop_type_result: + """ + Attributes: + - success + - o1 + - o2 + """ + + thrift_spec = ( + (0, TType.BOOL, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 + ) + + def __init__(self, success=None, o1=None, o2=None,): + self.success = success + self.o1 = o1 + self.o2 = o2 + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.BOOL: + self.success = iprot.readBool(); + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = InvalidObjectException() + self.o2 = NoSuchObjectException() self.o2.read(iprot) else: iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = MetaException() - self.o3.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -5351,7 +7087,11 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('create_database_result') + oprot.writeStructBegin('drop_type_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.BOOL, 0) + oprot.writeBool(self.success) + oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -5360,10 +7100,6 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -5382,7 +7118,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_database_args: +class get_type_all_args: """ Attributes: - name @@ -5419,7 +7155,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_database_args') + oprot.writeStructBegin('get_type_all_args') if self.name is not None: oprot.writeFieldBegin('name', TType.STRING, 1) oprot.writeString(self.name) @@ -5442,23 +7178,20 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_database_result: +class get_type_all_result: """ Attributes: - success - - o1 - o2 """ thrift_spec = ( - (0, TType.STRUCT, 'success', (Database, Database.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + (0, TType.MAP, 'success', (TType.STRING,None,TType.STRUCT,(Type, Type.thrift_spec)), None, ), # 0 + (1, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 1 ) - def __init__(self, success=None, o1=None, o2=None,): + def __init__(self, success=None, o2=None,): self.success = success - self.o1 = o1 self.o2 = o2 def read(self, iprot): @@ -5471,19 +7204,19 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.STRUCT: - self.success = Database() - self.success.read(iprot) + if ftype == TType.MAP: + self.success = {} + (_ktype242, _vtype243, _size241 ) = iprot.readMapBegin() + for _i245 in xrange(_size241): + _key246 = iprot.readString(); + _val247 = Type() + _val247.read(iprot) + self.success[_key246] = _val247 + iprot.readMapEnd() else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() - self.o1.read(iprot) - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: self.o2 = MetaException() self.o2.read(iprot) else: @@ -5497,17 +7230,17 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_database_result') + oprot.writeStructBegin('get_type_all_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) - oprot.writeFieldEnd() - if self.o1 is not None: - oprot.writeFieldBegin('o1', TType.STRUCT, 1) - self.o1.write(oprot) + oprot.writeFieldBegin('success', TType.MAP, 0) + oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.success)) + for kiter248,viter249 in self.success.items(): + oprot.writeString(kiter248) + viter249.write(oprot) + oprot.writeMapEnd() oprot.writeFieldEnd() if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 2) + oprot.writeFieldBegin('o2', TType.STRUCT, 1) self.o2.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() @@ -5528,25 +7261,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_database_args: +class get_fields_args: """ Attributes: - - name - - deleteData - - cascade + - db_name + - table_name """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'name', None, None, ), # 1 - (2, TType.BOOL, 'deleteData', None, None, ), # 2 - (3, TType.BOOL, 'cascade', None, None, ), # 3 + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'table_name', None, None, ), # 2 ) - def __init__(self, name=None, deleteData=None, cascade=None,): - self.name = name - self.deleteData = deleteData - self.cascade = cascade + def __init__(self, db_name=None, table_name=None,): + self.db_name = db_name + self.table_name = table_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -5559,17 +7289,12 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.name = iprot.readString(); + self.db_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: - if ftype == TType.BOOL: - self.deleteData = iprot.readBool(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.BOOL: - self.cascade = iprot.readBool(); + if ftype == TType.STRING: + self.table_name = iprot.readString(); else: iprot.skip(ftype) else: @@ -5581,18 +7306,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_database_args') - if self.name is not None: - oprot.writeFieldBegin('name', TType.STRING, 1) - oprot.writeString(self.name) - oprot.writeFieldEnd() - if self.deleteData is not None: - oprot.writeFieldBegin('deleteData', TType.BOOL, 2) - oprot.writeBool(self.deleteData) + oprot.writeStructBegin('get_fields_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) oprot.writeFieldEnd() - if self.cascade is not None: - oprot.writeFieldBegin('cascade', TType.BOOL, 3) - oprot.writeBool(self.cascade) + if self.table_name is not None: + oprot.writeFieldBegin('table_name', TType.STRING, 2) + oprot.writeString(self.table_name) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -5612,22 +7333,24 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_database_result: +class get_fields_result: """ Attributes: + - success - o1 - o2 - o3 """ thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + (0, TType.LIST, 'success', (TType.STRUCT,(FieldSchema, FieldSchema.thrift_spec)), None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (UnknownTableException, UnknownTableException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (UnknownDBException, UnknownDBException.thrift_spec), None, ), # 3 ) - def __init__(self, o1=None, o2=None, o3=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None,): + self.success = success self.o1 = o1 self.o2 = o2 self.o3 = o3 @@ -5641,21 +7364,32 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 1: + if fid == 0: + if ftype == TType.LIST: + self.success = [] + (_etype253, _size250) = iprot.readListBegin() + for _i254 in xrange(_size250): + _elem255 = FieldSchema() + _elem255.read(iprot) + self.success.append(_elem255) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = InvalidOperationException() + self.o2 = UnknownTableException() self.o2.read(iprot) else: iprot.skip(ftype) elif fid == 3: if ftype == TType.STRUCT: - self.o3 = MetaException() + self.o3 = UnknownDBException() self.o3.read(iprot) else: iprot.skip(ftype) @@ -5668,7 +7402,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_database_result') + oprot.writeStructBegin('get_fields_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRUCT, len(self.success)) + for iter256 in self.success: + iter256.write(oprot) + oprot.writeListEnd() + oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -5699,19 +7440,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_databases_args: +class get_schema_args: """ Attributes: - - pattern + - db_name + - table_name """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'pattern', None, None, ), # 1 + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'table_name', None, None, ), # 2 ) - def __init__(self, pattern=None,): - self.pattern = pattern + def __init__(self, db_name=None, table_name=None,): + self.db_name = db_name + self.table_name = table_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -5724,7 +7468,12 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.pattern = iprot.readString(); + self.db_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.table_name = iprot.readString(); else: iprot.skip(ftype) else: @@ -5736,10 +7485,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_databases_args') - if self.pattern is not None: - oprot.writeFieldBegin('pattern', TType.STRING, 1) - oprot.writeString(self.pattern) + oprot.writeStructBegin('get_schema_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) + oprot.writeFieldEnd() + if self.table_name is not None: + oprot.writeFieldBegin('table_name', TType.STRING, 2) + oprot.writeString(self.table_name) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -5759,21 +7512,27 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_databases_result: +class get_schema_result: """ Attributes: - success - o1 + - o2 + - o3 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 + (0, TType.LIST, 'success', (TType.STRUCT,(FieldSchema, FieldSchema.thrift_spec)), None, ), # 0 (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (UnknownTableException, UnknownTableException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (UnknownDBException, UnknownDBException.thrift_spec), None, ), # 3 ) - def __init__(self, success=None, o1=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None,): self.success = success self.o1 = o1 + self.o2 = o2 + self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -5787,10 +7546,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype230, _size227) = iprot.readListBegin() - for _i231 in xrange(_size227): - _elem232 = iprot.readString(); - self.success.append(_elem232) + (_etype260, _size257) = iprot.readListBegin() + for _i261 in xrange(_size257): + _elem262 = FieldSchema() + _elem262.read(iprot) + self.success.append(_elem262) iprot.readListEnd() else: iprot.skip(ftype) @@ -5800,6 +7560,18 @@ def read(self, iprot): self.o1.read(iprot) else: iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = UnknownTableException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = UnknownDBException() + self.o3.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -5809,18 +7581,26 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_databases_result') + oprot.writeStructBegin('get_schema_result') if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRING, len(self.success)) - for iter233 in self.success: - oprot.writeString(iter233) + oprot.writeListBegin(TType.STRUCT, len(self.success)) + for iter263 in self.success: + iter263.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -5839,11 +7619,20 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_all_databases_args: +class create_table_args: + """ + Attributes: + - tbl + """ thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'tbl', (Table, Table.thrift_spec), None, ), # 1 ) + def __init__(self, tbl=None,): + self.tbl = tbl + def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) @@ -5853,6 +7642,12 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break + if fid == 1: + if ftype == TType.STRUCT: + self.tbl = Table() + self.tbl.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -5862,7 +7657,11 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_all_databases_args') + oprot.writeStructBegin('create_table_args') + if self.tbl is not None: + oprot.writeFieldBegin('tbl', TType.STRUCT, 1) + self.tbl.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -5881,21 +7680,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_all_databases_result: +class create_table_result: """ Attributes: - - success - o1 + - o2 + - o3 + - o4 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + None, # 0 + (1, TType.STRUCT, 'o1', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 4 ) - def __init__(self, success=None, o1=None,): - self.success = success + def __init__(self, o1=None, o2=None, o3=None, o4=None,): self.o1 = o1 + self.o2 = o2 + self.o3 = o3 + self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -5906,20 +7712,28 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype237, _size234) = iprot.readListBegin() - for _i238 in xrange(_size234): - _elem239 = iprot.readString(); - self.success.append(_elem239) - iprot.readListEnd() + if fid == 1: + if ftype == TType.STRUCT: + self.o1 = AlreadyExistsException() + self.o1.read(iprot) else: iprot.skip(ftype) - elif fid == 1: + elif fid == 2: if ftype == TType.STRUCT: - self.o1 = MetaException() - self.o1.read(iprot) + self.o2 = InvalidObjectException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = MetaException() + self.o3.read(iprot) + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.o4 = NoSuchObjectException() + self.o4.read(iprot) else: iprot.skip(ftype) else: @@ -5931,18 +7745,23 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_all_databases_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRING, len(self.success)) - for iter240 in self.success: - oprot.writeString(iter240) - oprot.writeListEnd() - oprot.writeFieldEnd() + oprot.writeStructBegin('create_table_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() + if self.o4 is not None: + oprot.writeFieldBegin('o4', TType.STRUCT, 4) + self.o4.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -5961,22 +7780,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_database_args: +class create_table_with_environment_context_args: """ Attributes: - - dbname - - db + - tbl + - environment_context """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'dbname', None, None, ), # 1 - (2, TType.STRUCT, 'db', (Database, Database.thrift_spec), None, ), # 2 + (1, TType.STRUCT, 'tbl', (Table, Table.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 2 ) - def __init__(self, dbname=None, db=None,): - self.dbname = dbname - self.db = db + def __init__(self, tbl=None, environment_context=None,): + self.tbl = tbl + self.environment_context = environment_context def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -5988,14 +7807,15 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.dbname = iprot.readString(); + if ftype == TType.STRUCT: + self.tbl = Table() + self.tbl.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.db = Database() - self.db.read(iprot) + self.environment_context = EnvironmentContext() + self.environment_context.read(iprot) else: iprot.skip(ftype) else: @@ -6007,14 +7827,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_database_args') - if self.dbname is not None: - oprot.writeFieldBegin('dbname', TType.STRING, 1) - oprot.writeString(self.dbname) + oprot.writeStructBegin('create_table_with_environment_context_args') + if self.tbl is not None: + oprot.writeFieldBegin('tbl', TType.STRUCT, 1) + self.tbl.write(oprot) oprot.writeFieldEnd() - if self.db is not None: - oprot.writeFieldBegin('db', TType.STRUCT, 2) - self.db.write(oprot) + if self.environment_context is not None: + oprot.writeFieldBegin('environment_context', TType.STRUCT, 2) + self.environment_context.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -6034,22 +7854,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_database_result: +class create_table_with_environment_context_result: """ Attributes: - o1 - o2 + - o3 + - o4 """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 + (1, TType.STRUCT, 'o1', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 4 ) - def __init__(self, o1=None, o2=None,): + def __init__(self, o1=None, o2=None, o3=None, o4=None,): self.o1 = o1 self.o2 = o2 + self.o3 = o3 + self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -6062,16 +7888,28 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = AlreadyExistsException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = NoSuchObjectException() + self.o2 = InvalidObjectException() self.o2.read(iprot) else: iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = MetaException() + self.o3.read(iprot) + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.o4 = NoSuchObjectException() + self.o4.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -6081,7 +7919,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_database_result') + oprot.writeStructBegin('create_table_with_environment_context_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -6090,6 +7928,14 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() + if self.o4 is not None: + oprot.writeFieldBegin('o4', TType.STRUCT, 4) + self.o4.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -6108,19 +7954,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_type_args: +class drop_table_args: """ Attributes: + - dbname - name + - deleteData """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'name', None, None, ), # 1 + (1, TType.STRING, 'dbname', None, None, ), # 1 + (2, TType.STRING, 'name', None, None, ), # 2 + (3, TType.BOOL, 'deleteData', None, None, ), # 3 ) - def __init__(self, name=None,): + def __init__(self, dbname=None, name=None, deleteData=None,): + self.dbname = dbname self.name = name + self.deleteData = deleteData def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -6133,9 +7985,19 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: + self.dbname = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: self.name = iprot.readString(); else: iprot.skip(ftype) + elif fid == 3: + if ftype == TType.BOOL: + self.deleteData = iprot.readBool(); + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -6145,11 +8007,19 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_type_args') + oprot.writeStructBegin('drop_table_args') + if self.dbname is not None: + oprot.writeFieldBegin('dbname', TType.STRING, 1) + oprot.writeString(self.dbname) + oprot.writeFieldEnd() if self.name is not None: - oprot.writeFieldBegin('name', TType.STRING, 1) + oprot.writeFieldBegin('name', TType.STRING, 2) oprot.writeString(self.name) oprot.writeFieldEnd() + if self.deleteData is not None: + oprot.writeFieldBegin('deleteData', TType.BOOL, 3) + oprot.writeBool(self.deleteData) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -6168,24 +8038,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_type_result: +class drop_table_result: """ Attributes: - - success - o1 - - o2 + - o3 """ thrift_spec = ( - (0, TType.STRUCT, 'success', (Type, Type.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 + None, # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None,): - self.success = success + def __init__(self, o1=None, o3=None,): self.o1 = o1 - self.o2 = o2 + self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -6196,22 +8064,16 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.STRUCT: - self.success = Type() - self.success.read(iprot) - else: - iprot.skip(ftype) - elif fid == 1: + if fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = NoSuchObjectException() - self.o2.read(iprot) + self.o3 = MetaException() + self.o3.read(iprot) else: iprot.skip(ftype) else: @@ -6223,18 +8085,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_type_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) - oprot.writeFieldEnd() + oprot.writeStructBegin('drop_table_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() - if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 2) - self.o2.write(oprot) + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 2) + self.o3.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -6254,19 +8112,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class create_type_args: +class drop_table_with_environment_context_args: """ Attributes: - - type + - dbname + - name + - deleteData + - environment_context """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'type', (Type, Type.thrift_spec), None, ), # 1 + (1, TType.STRING, 'dbname', None, None, ), # 1 + (2, TType.STRING, 'name', None, None, ), # 2 + (3, TType.BOOL, 'deleteData', None, None, ), # 3 + (4, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 4 ) - def __init__(self, type=None,): - self.type = type + def __init__(self, dbname=None, name=None, deleteData=None, environment_context=None,): + self.dbname = dbname + self.name = name + self.deleteData = deleteData + self.environment_context = environment_context def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -6278,9 +8145,24 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: + if ftype == TType.STRING: + self.dbname = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.BOOL: + self.deleteData = iprot.readBool(); + else: + iprot.skip(ftype) + elif fid == 4: if ftype == TType.STRUCT: - self.type = Type() - self.type.read(iprot) + self.environment_context = EnvironmentContext() + self.environment_context.read(iprot) else: iprot.skip(ftype) else: @@ -6292,10 +8174,22 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('create_type_args') - if self.type is not None: - oprot.writeFieldBegin('type', TType.STRUCT, 1) - self.type.write(oprot) + oprot.writeStructBegin('drop_table_with_environment_context_args') + if self.dbname is not None: + oprot.writeFieldBegin('dbname', TType.STRING, 1) + oprot.writeString(self.dbname) + oprot.writeFieldEnd() + if self.name is not None: + oprot.writeFieldBegin('name', TType.STRING, 2) + oprot.writeString(self.name) + oprot.writeFieldEnd() + if self.deleteData is not None: + oprot.writeFieldBegin('deleteData', TType.BOOL, 3) + oprot.writeBool(self.deleteData) + oprot.writeFieldEnd() + if self.environment_context is not None: + oprot.writeFieldBegin('environment_context', TType.STRUCT, 4) + self.environment_context.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -6315,26 +8209,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class create_type_result: +class drop_table_with_environment_context_result: """ Attributes: - - success - o1 - - o2 - o3 """ thrift_spec = ( - (0, TType.BOOL, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + None, # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None, o3=None,): - self.success = success + def __init__(self, o1=None, o3=None,): self.o1 = o1 - self.o2 = o2 self.o3 = o3 def read(self, iprot): @@ -6346,25 +8235,14 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.BOOL: - self.success = iprot.readBool(); - else: - iprot.skip(ftype) - elif fid == 1: + if fid == 1: if ftype == TType.STRUCT: - self.o1 = AlreadyExistsException() + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = InvalidObjectException() - self.o2.read(iprot) - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: self.o3 = MetaException() self.o3.read(iprot) else: @@ -6378,21 +8256,13 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('create_type_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.BOOL, 0) - oprot.writeBool(self.success) - oprot.writeFieldEnd() + oprot.writeStructBegin('drop_table_with_environment_context_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() - if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 2) - self.o2.write(oprot) - oprot.writeFieldEnd() if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) + oprot.writeFieldBegin('o3', TType.STRUCT, 2) self.o3.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() @@ -6413,19 +8283,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_type_args: +class get_tables_args: """ Attributes: - - type + - db_name + - pattern """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'type', None, None, ), # 1 + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'pattern', None, None, ), # 2 ) - def __init__(self, type=None,): - self.type = type + def __init__(self, db_name=None, pattern=None,): + self.db_name = db_name + self.pattern = pattern def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -6438,7 +8311,12 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.type = iprot.readString(); + self.db_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.pattern = iprot.readString(); else: iprot.skip(ftype) else: @@ -6450,10 +8328,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_type_args') - if self.type is not None: - oprot.writeFieldBegin('type', TType.STRING, 1) - oprot.writeString(self.type) + oprot.writeStructBegin('get_tables_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) + oprot.writeFieldEnd() + if self.pattern is not None: + oprot.writeFieldBegin('pattern', TType.STRING, 2) + oprot.writeString(self.pattern) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -6473,24 +8355,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_type_result: +class get_tables_result: """ Attributes: - success - o1 - - o2 """ thrift_spec = ( - (0, TType.BOOL, 'success', None, None, ), # 0 + (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None,): + def __init__(self, success=None, o1=None,): self.success = success self.o1 = o1 - self.o2 = o2 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -6502,8 +8381,13 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.BOOL: - self.success = iprot.readBool(); + if ftype == TType.LIST: + self.success = [] + (_etype267, _size264) = iprot.readListBegin() + for _i268 in xrange(_size264): + _elem269 = iprot.readString(); + self.success.append(_elem269) + iprot.readListEnd() else: iprot.skip(ftype) elif fid == 1: @@ -6512,12 +8396,6 @@ def read(self, iprot): self.o1.read(iprot) else: iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.o2 = NoSuchObjectException() - self.o2.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -6527,19 +8405,18 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_type_result') + oprot.writeStructBegin('get_tables_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.BOOL, 0) - oprot.writeBool(self.success) + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRING, len(self.success)) + for iter270 in self.success: + oprot.writeString(iter270) + oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() - if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 2) - self.o2.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -6558,19 +8435,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_type_all_args: +class get_all_tables_args: """ Attributes: - - name + - db_name """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'name', None, None, ), # 1 + (1, TType.STRING, 'db_name', None, None, ), # 1 ) - def __init__(self, name=None,): - self.name = name + def __init__(self, db_name=None,): + self.db_name = db_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -6583,7 +8460,7 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.name = iprot.readString(); + self.db_name = iprot.readString(); else: iprot.skip(ftype) else: @@ -6595,10 +8472,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_type_all_args') - if self.name is not None: - oprot.writeFieldBegin('name', TType.STRING, 1) - oprot.writeString(self.name) + oprot.writeStructBegin('get_all_tables_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -6618,21 +8495,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_type_all_result: +class get_all_tables_result: """ Attributes: - success - - o2 + - o1 """ thrift_spec = ( - (0, TType.MAP, 'success', (TType.STRING,None,TType.STRUCT,(Type, Type.thrift_spec)), None, ), # 0 - (1, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 1 + (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 ) - def __init__(self, success=None, o2=None,): + def __init__(self, success=None, o1=None,): self.success = success - self.o2 = o2 + self.o1 = o1 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -6644,21 +8521,19 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.MAP: - self.success = {} - (_ktype242, _vtype243, _size241 ) = iprot.readMapBegin() - for _i245 in xrange(_size241): - _key246 = iprot.readString(); - _val247 = Type() - _val247.read(iprot) - self.success[_key246] = _val247 - iprot.readMapEnd() + if ftype == TType.LIST: + self.success = [] + (_etype274, _size271) = iprot.readListBegin() + for _i275 in xrange(_size271): + _elem276 = iprot.readString(); + self.success.append(_elem276) + iprot.readListEnd() else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o2 = MetaException() - self.o2.read(iprot) + self.o1 = MetaException() + self.o1.read(iprot) else: iprot.skip(ftype) else: @@ -6670,18 +8545,17 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_type_all_result') + oprot.writeStructBegin('get_all_tables_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.MAP, 0) - oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.success)) - for kiter248,viter249 in self.success.items(): - oprot.writeString(kiter248) - viter249.write(oprot) - oprot.writeMapEnd() + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRING, len(self.success)) + for iter277 in self.success: + oprot.writeString(iter277) + oprot.writeListEnd() oprot.writeFieldEnd() - if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 1) - self.o2.write(oprot) + if self.o1 is not None: + oprot.writeFieldBegin('o1', TType.STRUCT, 1) + self.o1.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -6701,22 +8575,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_fields_args: +class get_table_args: """ Attributes: - - db_name - - table_name + - dbname + - tbl_name """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'table_name', None, None, ), # 2 + (1, TType.STRING, 'dbname', None, None, ), # 1 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 ) - def __init__(self, db_name=None, table_name=None,): - self.db_name = db_name - self.table_name = table_name + def __init__(self, dbname=None, tbl_name=None,): + self.dbname = dbname + self.tbl_name = tbl_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -6729,12 +8603,12 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.db_name = iprot.readString(); + self.dbname = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: - self.table_name = iprot.readString(); + self.tbl_name = iprot.readString(); else: iprot.skip(ftype) else: @@ -6746,14 +8620,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_fields_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) + oprot.writeStructBegin('get_table_args') + if self.dbname is not None: + oprot.writeFieldBegin('dbname', TType.STRING, 1) + oprot.writeString(self.dbname) oprot.writeFieldEnd() - if self.table_name is not None: - oprot.writeFieldBegin('table_name', TType.STRING, 2) - oprot.writeString(self.table_name) + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -6773,27 +8647,24 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_fields_result: +class get_table_result: """ Attributes: - success - o1 - o2 - - o3 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRUCT,(FieldSchema, FieldSchema.thrift_spec)), None, ), # 0 + (0, TType.STRUCT, 'success', (Table, Table.thrift_spec), None, ), # 0 (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (UnknownTableException, UnknownTableException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (UnknownDBException, UnknownDBException.thrift_spec), None, ), # 3 + (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None, o3=None,): + def __init__(self, success=None, o1=None, o2=None,): self.success = success self.o1 = o1 self.o2 = o2 - self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -6805,14 +8676,9 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype253, _size250) = iprot.readListBegin() - for _i254 in xrange(_size250): - _elem255 = FieldSchema() - _elem255.read(iprot) - self.success.append(_elem255) - iprot.readListEnd() + if ftype == TType.STRUCT: + self.success = Table() + self.success.read(iprot) else: iprot.skip(ftype) elif fid == 1: @@ -6823,16 +8689,10 @@ def read(self, iprot): iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = UnknownTableException() + self.o2 = NoSuchObjectException() self.o2.read(iprot) else: iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = UnknownDBException() - self.o3.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -6842,13 +8702,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_fields_result') + oprot.writeStructBegin('get_table_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter256 in self.success: - iter256.write(oprot) - oprot.writeListEnd() + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -6858,10 +8715,6 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -6880,22 +8733,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_schema_args: +class get_table_objects_by_name_args: """ Attributes: - - db_name - - table_name + - dbname + - tbl_names """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'table_name', None, None, ), # 2 + (1, TType.STRING, 'dbname', None, None, ), # 1 + (2, TType.LIST, 'tbl_names', (TType.STRING,None), None, ), # 2 ) - def __init__(self, db_name=None, table_name=None,): - self.db_name = db_name - self.table_name = table_name + def __init__(self, dbname=None, tbl_names=None,): + self.dbname = dbname + self.tbl_names = tbl_names def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -6908,12 +8761,17 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.db_name = iprot.readString(); + self.dbname = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: - if ftype == TType.STRING: - self.table_name = iprot.readString(); + if ftype == TType.LIST: + self.tbl_names = [] + (_etype281, _size278) = iprot.readListBegin() + for _i282 in xrange(_size278): + _elem283 = iprot.readString(); + self.tbl_names.append(_elem283) + iprot.readListEnd() else: iprot.skip(ftype) else: @@ -6925,14 +8783,17 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_schema_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) + oprot.writeStructBegin('get_table_objects_by_name_args') + if self.dbname is not None: + oprot.writeFieldBegin('dbname', TType.STRING, 1) + oprot.writeString(self.dbname) oprot.writeFieldEnd() - if self.table_name is not None: - oprot.writeFieldBegin('table_name', TType.STRING, 2) - oprot.writeString(self.table_name) + if self.tbl_names is not None: + oprot.writeFieldBegin('tbl_names', TType.LIST, 2) + oprot.writeListBegin(TType.STRING, len(self.tbl_names)) + for iter284 in self.tbl_names: + oprot.writeString(iter284) + oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -6952,7 +8813,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_schema_result: +class get_table_objects_by_name_result: """ Attributes: - success @@ -6962,9 +8823,9 @@ class get_schema_result: """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRUCT,(FieldSchema, FieldSchema.thrift_spec)), None, ), # 0 + (0, TType.LIST, 'success', (TType.STRUCT,(Table, Table.thrift_spec)), None, ), # 0 (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (UnknownTableException, UnknownTableException.thrift_spec), None, ), # 2 + (2, TType.STRUCT, 'o2', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 2 (3, TType.STRUCT, 'o3', (UnknownDBException, UnknownDBException.thrift_spec), None, ), # 3 ) @@ -6986,11 +8847,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype260, _size257) = iprot.readListBegin() - for _i261 in xrange(_size257): - _elem262 = FieldSchema() - _elem262.read(iprot) - self.success.append(_elem262) + (_etype288, _size285) = iprot.readListBegin() + for _i289 in xrange(_size285): + _elem290 = Table() + _elem290.read(iprot) + self.success.append(_elem290) iprot.readListEnd() else: iprot.skip(ftype) @@ -7002,7 +8863,7 @@ def read(self, iprot): iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = UnknownTableException() + self.o2 = InvalidOperationException() self.o2.read(iprot) else: iprot.skip(ftype) @@ -7021,12 +8882,12 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_schema_result') + oprot.writeStructBegin('get_table_objects_by_name_result') if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter263 in self.success: - iter263.write(oprot) + for iter291 in self.success: + iter291.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -7059,19 +8920,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class create_table_args: +class get_table_names_by_filter_args: """ Attributes: - - tbl + - dbname + - filter + - max_tables """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'tbl', (Table, Table.thrift_spec), None, ), # 1 + (1, TType.STRING, 'dbname', None, None, ), # 1 + (2, TType.STRING, 'filter', None, None, ), # 2 + (3, TType.I16, 'max_tables', None, -1, ), # 3 ) - def __init__(self, tbl=None,): - self.tbl = tbl + def __init__(self, dbname=None, filter=None, max_tables=thrift_spec[3][4],): + self.dbname = dbname + self.filter = filter + self.max_tables = max_tables def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -7083,9 +8950,18 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRUCT: - self.tbl = Table() - self.tbl.read(iprot) + if ftype == TType.STRING: + self.dbname = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.filter = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.I16: + self.max_tables = iprot.readI16(); else: iprot.skip(ftype) else: @@ -7097,10 +8973,18 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('create_table_args') - if self.tbl is not None: - oprot.writeFieldBegin('tbl', TType.STRUCT, 1) - self.tbl.write(oprot) + oprot.writeStructBegin('get_table_names_by_filter_args') + if self.dbname is not None: + oprot.writeFieldBegin('dbname', TType.STRING, 1) + oprot.writeString(self.dbname) + oprot.writeFieldEnd() + if self.filter is not None: + oprot.writeFieldBegin('filter', TType.STRING, 2) + oprot.writeString(self.filter) + oprot.writeFieldEnd() + if self.max_tables is not None: + oprot.writeFieldBegin('max_tables', TType.I16, 3) + oprot.writeI16(self.max_tables) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -7120,28 +9004,27 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class create_table_result: +class get_table_names_by_filter_result: """ Attributes: + - success - o1 - o2 - o3 - - o4 """ thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'o1', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 - (4, TType.STRUCT, 'o4', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 4 + (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (UnknownDBException, UnknownDBException.thrift_spec), None, ), # 3 ) - def __init__(self, o1=None, o2=None, o3=None, o4=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None,): + self.success = success self.o1 = o1 self.o2 = o2 self.o3 = o3 - self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -7152,30 +9035,34 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 1: + if fid == 0: + if ftype == TType.LIST: + self.success = [] + (_etype295, _size292) = iprot.readListBegin() + for _i296 in xrange(_size292): + _elem297 = iprot.readString(); + self.success.append(_elem297) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 1: if ftype == TType.STRUCT: - self.o1 = AlreadyExistsException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = InvalidObjectException() + self.o2 = InvalidOperationException() self.o2.read(iprot) else: iprot.skip(ftype) elif fid == 3: if ftype == TType.STRUCT: - self.o3 = MetaException() + self.o3 = UnknownDBException() self.o3.read(iprot) else: iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRUCT: - self.o4 = NoSuchObjectException() - self.o4.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -7185,7 +9072,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('create_table_result') + oprot.writeStructBegin('get_table_names_by_filter_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRING, len(self.success)) + for iter298 in self.success: + oprot.writeString(iter298) + oprot.writeListEnd() + oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -7198,10 +9092,6 @@ def write(self, oprot): oprot.writeFieldBegin('o3', TType.STRUCT, 3) self.o3.write(oprot) oprot.writeFieldEnd() - if self.o4 is not None: - oprot.writeFieldBegin('o4', TType.STRUCT, 4) - self.o4.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -7220,22 +9110,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class create_table_with_environment_context_args: +class alter_table_args: """ Attributes: - - tbl - - environment_context + - dbname + - tbl_name + - new_tbl """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'tbl', (Table, Table.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 2 + (1, TType.STRING, 'dbname', None, None, ), # 1 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.STRUCT, 'new_tbl', (Table, Table.thrift_spec), None, ), # 3 ) - def __init__(self, tbl=None, environment_context=None,): - self.tbl = tbl - self.environment_context = environment_context + def __init__(self, dbname=None, tbl_name=None, new_tbl=None,): + self.dbname = dbname + self.tbl_name = tbl_name + self.new_tbl = new_tbl def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -7247,15 +9140,19 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRUCT: - self.tbl = Table() - self.tbl.read(iprot) + if ftype == TType.STRING: + self.dbname = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: + if ftype == TType.STRING: + self.tbl_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: if ftype == TType.STRUCT: - self.environment_context = EnvironmentContext() - self.environment_context.read(iprot) + self.new_tbl = Table() + self.new_tbl.read(iprot) else: iprot.skip(ftype) else: @@ -7267,14 +9164,18 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('create_table_with_environment_context_args') - if self.tbl is not None: - oprot.writeFieldBegin('tbl', TType.STRUCT, 1) - self.tbl.write(oprot) + oprot.writeStructBegin('alter_table_args') + if self.dbname is not None: + oprot.writeFieldBegin('dbname', TType.STRING, 1) + oprot.writeString(self.dbname) + oprot.writeFieldEnd() + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.environment_context is not None: - oprot.writeFieldBegin('environment_context', TType.STRUCT, 2) - self.environment_context.write(oprot) + if self.new_tbl is not None: + oprot.writeFieldBegin('new_tbl', TType.STRUCT, 3) + self.new_tbl.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -7294,28 +9195,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class create_table_with_environment_context_result: +class alter_table_result: """ Attributes: - o1 - o2 - - o3 - - o4 """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'o1', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 - (4, TType.STRUCT, 'o4', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 4 + (1, TType.STRUCT, 'o1', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, o1=None, o2=None, o3=None, o4=None,): + def __init__(self, o1=None, o2=None,): self.o1 = o1 self.o2 = o2 - self.o3 = o3 - self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -7328,28 +9223,16 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRUCT: - self.o1 = AlreadyExistsException() + self.o1 = InvalidOperationException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = InvalidObjectException() + self.o2 = MetaException() self.o2.read(iprot) else: iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = MetaException() - self.o3.read(iprot) - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRUCT: - self.o4 = NoSuchObjectException() - self.o4.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -7359,7 +9242,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('create_table_with_environment_context_result') + oprot.writeStructBegin('alter_table_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -7368,14 +9251,6 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() - if self.o4 is not None: - oprot.writeFieldBegin('o4', TType.STRUCT, 4) - self.o4.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -7394,25 +9269,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_table_args: +class alter_table_with_environment_context_args: """ Attributes: - dbname - - name - - deleteData + - tbl_name + - new_tbl + - environment_context """ thrift_spec = ( None, # 0 (1, TType.STRING, 'dbname', None, None, ), # 1 - (2, TType.STRING, 'name', None, None, ), # 2 - (3, TType.BOOL, 'deleteData', None, None, ), # 3 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.STRUCT, 'new_tbl', (Table, Table.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 4 ) - def __init__(self, dbname=None, name=None, deleteData=None,): + def __init__(self, dbname=None, tbl_name=None, new_tbl=None, environment_context=None,): self.dbname = dbname - self.name = name - self.deleteData = deleteData + self.tbl_name = tbl_name + self.new_tbl = new_tbl + self.environment_context = environment_context def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -7430,12 +9308,19 @@ def read(self, iprot): iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: - self.name = iprot.readString(); + self.tbl_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.BOOL: - self.deleteData = iprot.readBool(); + if ftype == TType.STRUCT: + self.new_tbl = Table() + self.new_tbl.read(iprot) + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.environment_context = EnvironmentContext() + self.environment_context.read(iprot) else: iprot.skip(ftype) else: @@ -7447,18 +9332,22 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_table_args') + oprot.writeStructBegin('alter_table_with_environment_context_args') if self.dbname is not None: oprot.writeFieldBegin('dbname', TType.STRING, 1) oprot.writeString(self.dbname) oprot.writeFieldEnd() - if self.name is not None: - oprot.writeFieldBegin('name', TType.STRING, 2) - oprot.writeString(self.name) + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.deleteData is not None: - oprot.writeFieldBegin('deleteData', TType.BOOL, 3) - oprot.writeBool(self.deleteData) + if self.new_tbl is not None: + oprot.writeFieldBegin('new_tbl', TType.STRUCT, 3) + self.new_tbl.write(oprot) + oprot.writeFieldEnd() + if self.environment_context is not None: + oprot.writeFieldBegin('environment_context', TType.STRUCT, 4) + self.environment_context.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -7478,22 +9367,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_table_result: +class alter_table_with_environment_context_result: """ Attributes: - o1 - - o3 + - o2 """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 2 + (1, TType.STRUCT, 'o1', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, o1=None, o3=None,): + def __init__(self, o1=None, o2=None,): self.o1 = o1 - self.o3 = o3 + self.o2 = o2 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -7506,14 +9395,14 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = InvalidOperationException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o3 = MetaException() - self.o3.read(iprot) + self.o2 = MetaException() + self.o2.read(iprot) else: iprot.skip(ftype) else: @@ -7525,14 +9414,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_table_result') + oprot.writeStructBegin('alter_table_with_environment_context_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 2) - self.o3.write(oprot) + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -7552,28 +9441,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_table_with_environment_context_args: +class add_partition_args: """ Attributes: - - dbname - - name - - deleteData - - environment_context + - new_part """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'dbname', None, None, ), # 1 - (2, TType.STRING, 'name', None, None, ), # 2 - (3, TType.BOOL, 'deleteData', None, None, ), # 3 - (4, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 4 + (1, TType.STRUCT, 'new_part', (Partition, Partition.thrift_spec), None, ), # 1 ) - def __init__(self, dbname=None, name=None, deleteData=None, environment_context=None,): - self.dbname = dbname - self.name = name - self.deleteData = deleteData - self.environment_context = environment_context + def __init__(self, new_part=None,): + self.new_part = new_part def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -7585,24 +9465,9 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.dbname = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.BOOL: - self.deleteData = iprot.readBool(); - else: - iprot.skip(ftype) - elif fid == 4: if ftype == TType.STRUCT: - self.environment_context = EnvironmentContext() - self.environment_context.read(iprot) + self.new_part = Partition() + self.new_part.read(iprot) else: iprot.skip(ftype) else: @@ -7614,22 +9479,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_table_with_environment_context_args') - if self.dbname is not None: - oprot.writeFieldBegin('dbname', TType.STRING, 1) - oprot.writeString(self.dbname) - oprot.writeFieldEnd() - if self.name is not None: - oprot.writeFieldBegin('name', TType.STRING, 2) - oprot.writeString(self.name) - oprot.writeFieldEnd() - if self.deleteData is not None: - oprot.writeFieldBegin('deleteData', TType.BOOL, 3) - oprot.writeBool(self.deleteData) - oprot.writeFieldEnd() - if self.environment_context is not None: - oprot.writeFieldBegin('environment_context', TType.STRUCT, 4) - self.environment_context.write(oprot) + oprot.writeStructBegin('add_partition_args') + if self.new_part is not None: + oprot.writeFieldBegin('new_part', TType.STRUCT, 1) + self.new_part.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -7649,21 +9502,26 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_table_with_environment_context_result: +class add_partition_result: """ Attributes: + - success - o1 + - o2 - o3 """ thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 2 + (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 ) - def __init__(self, o1=None, o3=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None,): + self.success = success self.o1 = o1 + self.o2 = o2 self.o3 = o3 def read(self, iprot): @@ -7675,14 +9533,26 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 1: + if fid == 0: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.success = Partition() + self.success.read(iprot) + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.o1 = InvalidObjectException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: + self.o2 = AlreadyExistsException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: self.o3 = MetaException() self.o3.read(iprot) else: @@ -7696,13 +9566,21 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_table_with_environment_context_result') + oprot.writeStructBegin('add_partition_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) + oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 2) + oprot.writeFieldBegin('o3', TType.STRUCT, 3) self.o3.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() @@ -7723,22 +9601,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_tables_args: +class add_partition_with_environment_context_args: """ Attributes: - - db_name - - pattern + - new_part + - environment_context """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'pattern', None, None, ), # 2 + (1, TType.STRUCT, 'new_part', (Partition, Partition.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 2 ) - def __init__(self, db_name=None, pattern=None,): - self.db_name = db_name - self.pattern = pattern + def __init__(self, new_part=None, environment_context=None,): + self.new_part = new_part + self.environment_context = environment_context def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -7750,13 +9628,15 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.db_name = iprot.readString(); + if ftype == TType.STRUCT: + self.new_part = Partition() + self.new_part.read(iprot) else: iprot.skip(ftype) elif fid == 2: - if ftype == TType.STRING: - self.pattern = iprot.readString(); + if ftype == TType.STRUCT: + self.environment_context = EnvironmentContext() + self.environment_context.read(iprot) else: iprot.skip(ftype) else: @@ -7768,14 +9648,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_tables_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) + oprot.writeStructBegin('add_partition_with_environment_context_args') + if self.new_part is not None: + oprot.writeFieldBegin('new_part', TType.STRUCT, 1) + self.new_part.write(oprot) oprot.writeFieldEnd() - if self.pattern is not None: - oprot.writeFieldBegin('pattern', TType.STRING, 2) - oprot.writeString(self.pattern) + if self.environment_context is not None: + oprot.writeFieldBegin('environment_context', TType.STRUCT, 2) + self.environment_context.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -7795,21 +9675,27 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_tables_result: +class add_partition_with_environment_context_result: """ Attributes: - success - o1 + - o2 + - o3 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 ) - def __init__(self, success=None, o1=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None,): self.success = success self.o1 = o1 + self.o2 = o2 + self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -7821,21 +9707,29 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype267, _size264) = iprot.readListBegin() - for _i268 in xrange(_size264): - _elem269 = iprot.readString(); - self.success.append(_elem269) - iprot.readListEnd() + if ftype == TType.STRUCT: + self.success = Partition() + self.success.read(iprot) else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = InvalidObjectException() self.o1.read(iprot) else: iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = AlreadyExistsException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = MetaException() + self.o3.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -7845,18 +9739,23 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_tables_result') + oprot.writeStructBegin('add_partition_with_environment_context_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRING, len(self.success)) - for iter270 in self.success: - oprot.writeString(iter270) - oprot.writeListEnd() + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -7875,19 +9774,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_all_tables_args: +class add_partitions_args: """ Attributes: - - db_name + - new_parts """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 + (1, TType.LIST, 'new_parts', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 1 ) - def __init__(self, db_name=None,): - self.db_name = db_name + def __init__(self, new_parts=None,): + self.new_parts = new_parts def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -7899,8 +9798,14 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.db_name = iprot.readString(); + if ftype == TType.LIST: + self.new_parts = [] + (_etype302, _size299) = iprot.readListBegin() + for _i303 in xrange(_size299): + _elem304 = Partition() + _elem304.read(iprot) + self.new_parts.append(_elem304) + iprot.readListEnd() else: iprot.skip(ftype) else: @@ -7912,10 +9817,13 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_all_tables_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) + oprot.writeStructBegin('add_partitions_args') + if self.new_parts is not None: + oprot.writeFieldBegin('new_parts', TType.LIST, 1) + oprot.writeListBegin(TType.STRUCT, len(self.new_parts)) + for iter305 in self.new_parts: + iter305.write(oprot) + oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -7935,21 +9843,27 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_all_tables_result: +class add_partitions_result: """ Attributes: - success - o1 + - o2 + - o3 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (0, TType.I32, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 ) - def __init__(self, success=None, o1=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None,): self.success = success self.o1 = o1 + self.o2 = o2 + self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -7961,21 +9875,28 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype274, _size271) = iprot.readListBegin() - for _i275 in xrange(_size271): - _elem276 = iprot.readString(); - self.success.append(_elem276) - iprot.readListEnd() + if ftype == TType.I32: + self.success = iprot.readI32(); else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = InvalidObjectException() self.o1.read(iprot) else: iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = AlreadyExistsException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = MetaException() + self.o3.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -7985,18 +9906,23 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_all_tables_result') + oprot.writeStructBegin('add_partitions_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRING, len(self.success)) - for iter277 in self.success: - oprot.writeString(iter277) - oprot.writeListEnd() + oprot.writeFieldBegin('success', TType.I32, 0) + oprot.writeI32(self.success) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -8015,22 +9941,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_table_args: +class append_partition_args: """ Attributes: - - dbname + - db_name - tbl_name + - part_vals """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'dbname', None, None, ), # 1 + (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 ) - def __init__(self, dbname=None, tbl_name=None,): - self.dbname = dbname + def __init__(self, db_name=None, tbl_name=None, part_vals=None,): + self.db_name = db_name self.tbl_name = tbl_name + self.part_vals = part_vals def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -8043,7 +9972,7 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.dbname = iprot.readString(); + self.db_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: @@ -8051,6 +9980,16 @@ def read(self, iprot): self.tbl_name = iprot.readString(); else: iprot.skip(ftype) + elif fid == 3: + if ftype == TType.LIST: + self.part_vals = [] + (_etype309, _size306) = iprot.readListBegin() + for _i310 in xrange(_size306): + _elem311 = iprot.readString(); + self.part_vals.append(_elem311) + iprot.readListEnd() + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -8060,15 +9999,22 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_table_args') - if self.dbname is not None: - oprot.writeFieldBegin('dbname', TType.STRING, 1) - oprot.writeString(self.dbname) + oprot.writeStructBegin('append_partition_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) oprot.writeFieldEnd() if self.tbl_name is not None: oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() + if self.part_vals is not None: + oprot.writeFieldBegin('part_vals', TType.LIST, 3) + oprot.writeListBegin(TType.STRING, len(self.part_vals)) + for iter312 in self.part_vals: + oprot.writeString(iter312) + oprot.writeListEnd() + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -8087,24 +10033,27 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_table_result: +class append_partition_result: """ Attributes: - success - o1 - o2 + - o3 """ thrift_spec = ( - (0, TType.STRUCT, 'success', (Table, Table.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 + (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 ) - def __init__(self, success=None, o1=None, o2=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None,): self.success = success self.o1 = o1 self.o2 = o2 + self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -8117,22 +10066,28 @@ def read(self, iprot): break if fid == 0: if ftype == TType.STRUCT: - self.success = Table() + self.success = Partition() self.success.read(iprot) else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = InvalidObjectException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = NoSuchObjectException() + self.o2 = AlreadyExistsException() self.o2.read(iprot) else: iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = MetaException() + self.o3.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -8142,7 +10097,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_table_result') + oprot.writeStructBegin('append_partition_result') if self.success is not None: oprot.writeFieldBegin('success', TType.STRUCT, 0) self.success.write(oprot) @@ -8155,6 +10110,10 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -8173,22 +10132,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_table_objects_by_name_args: +class append_partition_with_environment_context_args: """ Attributes: - - dbname - - tbl_names + - db_name + - tbl_name + - part_vals + - environment_context """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'dbname', None, None, ), # 1 - (2, TType.LIST, 'tbl_names', (TType.STRING,None), None, ), # 2 + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 + (4, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 4 ) - def __init__(self, dbname=None, tbl_names=None,): - self.dbname = dbname - self.tbl_names = tbl_names + def __init__(self, db_name=None, tbl_name=None, part_vals=None, environment_context=None,): + self.db_name = db_name + self.tbl_name = tbl_name + self.part_vals = part_vals + self.environment_context = environment_context def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -8201,17 +10166,28 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.dbname = iprot.readString(); + self.db_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.tbl_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.LIST: + self.part_vals = [] + (_etype316, _size313) = iprot.readListBegin() + for _i317 in xrange(_size313): + _elem318 = iprot.readString(); + self.part_vals.append(_elem318) + iprot.readListEnd() else: iprot.skip(ftype) - elif fid == 2: - if ftype == TType.LIST: - self.tbl_names = [] - (_etype281, _size278) = iprot.readListBegin() - for _i282 in xrange(_size278): - _elem283 = iprot.readString(); - self.tbl_names.append(_elem283) - iprot.readListEnd() + elif fid == 4: + if ftype == TType.STRUCT: + self.environment_context = EnvironmentContext() + self.environment_context.read(iprot) else: iprot.skip(ftype) else: @@ -8223,18 +10199,26 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_table_objects_by_name_args') - if self.dbname is not None: - oprot.writeFieldBegin('dbname', TType.STRING, 1) - oprot.writeString(self.dbname) + oprot.writeStructBegin('append_partition_with_environment_context_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) oprot.writeFieldEnd() - if self.tbl_names is not None: - oprot.writeFieldBegin('tbl_names', TType.LIST, 2) - oprot.writeListBegin(TType.STRING, len(self.tbl_names)) - for iter284 in self.tbl_names: - oprot.writeString(iter284) + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) + oprot.writeFieldEnd() + if self.part_vals is not None: + oprot.writeFieldBegin('part_vals', TType.LIST, 3) + oprot.writeListBegin(TType.STRING, len(self.part_vals)) + for iter319 in self.part_vals: + oprot.writeString(iter319) oprot.writeListEnd() oprot.writeFieldEnd() + if self.environment_context is not None: + oprot.writeFieldBegin('environment_context', TType.STRUCT, 4) + self.environment_context.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -8253,7 +10237,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_table_objects_by_name_result: +class append_partition_with_environment_context_result: """ Attributes: - success @@ -8263,10 +10247,10 @@ class get_table_objects_by_name_result: """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRUCT,(Table, Table.thrift_spec)), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (UnknownDBException, UnknownDBException.thrift_spec), None, ), # 3 + (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 ) def __init__(self, success=None, o1=None, o2=None, o3=None,): @@ -8285,31 +10269,26 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype288, _size285) = iprot.readListBegin() - for _i289 in xrange(_size285): - _elem290 = Table() - _elem290.read(iprot) - self.success.append(_elem290) - iprot.readListEnd() + if ftype == TType.STRUCT: + self.success = Partition() + self.success.read(iprot) else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = InvalidObjectException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = InvalidOperationException() + self.o2 = AlreadyExistsException() self.o2.read(iprot) else: iprot.skip(ftype) elif fid == 3: if ftype == TType.STRUCT: - self.o3 = UnknownDBException() + self.o3 = MetaException() self.o3.read(iprot) else: iprot.skip(ftype) @@ -8322,13 +10301,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_table_objects_by_name_result') + oprot.writeStructBegin('append_partition_with_environment_context_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter291 in self.success: - iter291.write(oprot) - oprot.writeListEnd() + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -8360,25 +10336,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_table_names_by_filter_args: +class append_partition_by_name_args: """ Attributes: - - dbname - - filter - - max_tables + - db_name + - tbl_name + - part_name """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'dbname', None, None, ), # 1 - (2, TType.STRING, 'filter', None, None, ), # 2 - (3, TType.I16, 'max_tables', None, -1, ), # 3 + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.STRING, 'part_name', None, None, ), # 3 ) - def __init__(self, dbname=None, filter=None, max_tables=thrift_spec[3][4],): - self.dbname = dbname - self.filter = filter - self.max_tables = max_tables + def __init__(self, db_name=None, tbl_name=None, part_name=None,): + self.db_name = db_name + self.tbl_name = tbl_name + self.part_name = part_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -8391,17 +10367,17 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.dbname = iprot.readString(); + self.db_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: - self.filter = iprot.readString(); + self.tbl_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.I16: - self.max_tables = iprot.readI16(); + if ftype == TType.STRING: + self.part_name = iprot.readString(); else: iprot.skip(ftype) else: @@ -8413,18 +10389,18 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_table_names_by_filter_args') - if self.dbname is not None: - oprot.writeFieldBegin('dbname', TType.STRING, 1) - oprot.writeString(self.dbname) + oprot.writeStructBegin('append_partition_by_name_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) oprot.writeFieldEnd() - if self.filter is not None: - oprot.writeFieldBegin('filter', TType.STRING, 2) - oprot.writeString(self.filter) + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.max_tables is not None: - oprot.writeFieldBegin('max_tables', TType.I16, 3) - oprot.writeI16(self.max_tables) + if self.part_name is not None: + oprot.writeFieldBegin('part_name', TType.STRING, 3) + oprot.writeString(self.part_name) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -8444,7 +10420,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_table_names_by_filter_result: +class append_partition_by_name_result: """ Attributes: - success @@ -8454,10 +10430,10 @@ class get_table_names_by_filter_result: """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (UnknownDBException, UnknownDBException.thrift_spec), None, ), # 3 + (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 ) def __init__(self, success=None, o1=None, o2=None, o3=None,): @@ -8476,30 +10452,26 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype295, _size292) = iprot.readListBegin() - for _i296 in xrange(_size292): - _elem297 = iprot.readString(); - self.success.append(_elem297) - iprot.readListEnd() + if ftype == TType.STRUCT: + self.success = Partition() + self.success.read(iprot) else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = InvalidObjectException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = InvalidOperationException() + self.o2 = AlreadyExistsException() self.o2.read(iprot) else: iprot.skip(ftype) elif fid == 3: if ftype == TType.STRUCT: - self.o3 = UnknownDBException() + self.o3 = MetaException() self.o3.read(iprot) else: iprot.skip(ftype) @@ -8512,13 +10484,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_table_names_by_filter_result') + oprot.writeStructBegin('append_partition_by_name_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRING, len(self.success)) - for iter298 in self.success: - oprot.writeString(iter298) - oprot.writeListEnd() + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -8550,25 +10519,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_table_args: +class append_partition_by_name_with_environment_context_args: """ Attributes: - - dbname + - db_name - tbl_name - - new_tbl + - part_name + - environment_context """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'dbname', None, None, ), # 1 + (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRUCT, 'new_tbl', (Table, Table.thrift_spec), None, ), # 3 + (3, TType.STRING, 'part_name', None, None, ), # 3 + (4, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 4 ) - def __init__(self, dbname=None, tbl_name=None, new_tbl=None,): - self.dbname = dbname + def __init__(self, db_name=None, tbl_name=None, part_name=None, environment_context=None,): + self.db_name = db_name self.tbl_name = tbl_name - self.new_tbl = new_tbl + self.part_name = part_name + self.environment_context = environment_context def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -8581,7 +10553,7 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.dbname = iprot.readString(); + self.db_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: @@ -8590,9 +10562,14 @@ def read(self, iprot): else: iprot.skip(ftype) elif fid == 3: + if ftype == TType.STRING: + self.part_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 4: if ftype == TType.STRUCT: - self.new_tbl = Table() - self.new_tbl.read(iprot) + self.environment_context = EnvironmentContext() + self.environment_context.read(iprot) else: iprot.skip(ftype) else: @@ -8604,18 +10581,22 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_table_args') - if self.dbname is not None: - oprot.writeFieldBegin('dbname', TType.STRING, 1) - oprot.writeString(self.dbname) + oprot.writeStructBegin('append_partition_by_name_with_environment_context_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) oprot.writeFieldEnd() if self.tbl_name is not None: oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.new_tbl is not None: - oprot.writeFieldBegin('new_tbl', TType.STRUCT, 3) - self.new_tbl.write(oprot) + if self.part_name is not None: + oprot.writeFieldBegin('part_name', TType.STRING, 3) + oprot.writeString(self.part_name) + oprot.writeFieldEnd() + if self.environment_context is not None: + oprot.writeFieldBegin('environment_context', TType.STRUCT, 4) + self.environment_context.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -8635,22 +10616,27 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_table_result: +class append_partition_by_name_with_environment_context_result: """ Attributes: + - success - o1 - o2 + - o3 """ thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'o1', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 ) - def __init__(self, o1=None, o2=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None,): + self.success = success self.o1 = o1 self.o2 = o2 + self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -8661,18 +10647,30 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 1: + if fid == 0: if ftype == TType.STRUCT: - self.o1 = InvalidOperationException() + self.success = Partition() + self.success.read(iprot) + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.o1 = InvalidObjectException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = MetaException() + self.o2 = AlreadyExistsException() self.o2.read(iprot) else: iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = MetaException() + self.o3.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -8682,7 +10680,11 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_table_result') + oprot.writeStructBegin('append_partition_by_name_with_environment_context_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) + oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -8691,6 +10693,10 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -8709,28 +10715,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_table_with_environment_context_args: +class drop_partition_args: """ Attributes: - - dbname + - db_name - tbl_name - - new_tbl - - environment_context + - part_vals + - deleteData """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'dbname', None, None, ), # 1 + (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRUCT, 'new_tbl', (Table, Table.thrift_spec), None, ), # 3 - (4, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 4 + (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 + (4, TType.BOOL, 'deleteData', None, None, ), # 4 ) - def __init__(self, dbname=None, tbl_name=None, new_tbl=None, environment_context=None,): - self.dbname = dbname + def __init__(self, db_name=None, tbl_name=None, part_vals=None, deleteData=None,): + self.db_name = db_name self.tbl_name = tbl_name - self.new_tbl = new_tbl - self.environment_context = environment_context + self.part_vals = part_vals + self.deleteData = deleteData def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -8743,7 +10749,7 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.dbname = iprot.readString(); + self.db_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: @@ -8752,15 +10758,18 @@ def read(self, iprot): else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.STRUCT: - self.new_tbl = Table() - self.new_tbl.read(iprot) + if ftype == TType.LIST: + self.part_vals = [] + (_etype323, _size320) = iprot.readListBegin() + for _i324 in xrange(_size320): + _elem325 = iprot.readString(); + self.part_vals.append(_elem325) + iprot.readListEnd() else: iprot.skip(ftype) elif fid == 4: - if ftype == TType.STRUCT: - self.environment_context = EnvironmentContext() - self.environment_context.read(iprot) + if ftype == TType.BOOL: + self.deleteData = iprot.readBool(); else: iprot.skip(ftype) else: @@ -8772,22 +10781,25 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_table_with_environment_context_args') - if self.dbname is not None: - oprot.writeFieldBegin('dbname', TType.STRING, 1) - oprot.writeString(self.dbname) + oprot.writeStructBegin('drop_partition_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) oprot.writeFieldEnd() if self.tbl_name is not None: oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.new_tbl is not None: - oprot.writeFieldBegin('new_tbl', TType.STRUCT, 3) - self.new_tbl.write(oprot) + if self.part_vals is not None: + oprot.writeFieldBegin('part_vals', TType.LIST, 3) + oprot.writeListBegin(TType.STRING, len(self.part_vals)) + for iter326 in self.part_vals: + oprot.writeString(iter326) + oprot.writeListEnd() oprot.writeFieldEnd() - if self.environment_context is not None: - oprot.writeFieldBegin('environment_context', TType.STRUCT, 4) - self.environment_context.write(oprot) + if self.deleteData is not None: + oprot.writeFieldBegin('deleteData', TType.BOOL, 4) + oprot.writeBool(self.deleteData) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -8807,20 +10819,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_table_with_environment_context_result: +class drop_partition_result: """ Attributes: + - success - o1 - o2 """ thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'o1', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 1 + (0, TType.BOOL, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, o1=None, o2=None,): + def __init__(self, success=None, o1=None, o2=None,): + self.success = success self.o1 = o1 self.o2 = o2 @@ -8833,9 +10847,14 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 1: + if fid == 0: + if ftype == TType.BOOL: + self.success = iprot.readBool(); + else: + iprot.skip(ftype) + elif fid == 1: if ftype == TType.STRUCT: - self.o1 = InvalidOperationException() + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) @@ -8854,7 +10873,11 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_table_with_environment_context_result') + oprot.writeStructBegin('drop_partition_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.BOOL, 0) + oprot.writeBool(self.success) + oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -8881,19 +10904,31 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class add_partition_args: +class drop_partition_with_environment_context_args: """ Attributes: - - new_part + - db_name + - tbl_name + - part_vals + - deleteData + - environment_context """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'new_part', (Partition, Partition.thrift_spec), None, ), # 1 + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 + (4, TType.BOOL, 'deleteData', None, None, ), # 4 + (5, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 5 ) - def __init__(self, new_part=None,): - self.new_part = new_part + def __init__(self, db_name=None, tbl_name=None, part_vals=None, deleteData=None, environment_context=None,): + self.db_name = db_name + self.tbl_name = tbl_name + self.part_vals = part_vals + self.deleteData = deleteData + self.environment_context = environment_context def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -8905,9 +10940,34 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: + if ftype == TType.STRING: + self.db_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.tbl_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.LIST: + self.part_vals = [] + (_etype330, _size327) = iprot.readListBegin() + for _i331 in xrange(_size327): + _elem332 = iprot.readString(); + self.part_vals.append(_elem332) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.BOOL: + self.deleteData = iprot.readBool(); + else: + iprot.skip(ftype) + elif fid == 5: if ftype == TType.STRUCT: - self.new_part = Partition() - self.new_part.read(iprot) + self.environment_context = EnvironmentContext() + self.environment_context.read(iprot) else: iprot.skip(ftype) else: @@ -8919,10 +10979,29 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('add_partition_args') - if self.new_part is not None: - oprot.writeFieldBegin('new_part', TType.STRUCT, 1) - self.new_part.write(oprot) + oprot.writeStructBegin('drop_partition_with_environment_context_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) + oprot.writeFieldEnd() + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) + oprot.writeFieldEnd() + if self.part_vals is not None: + oprot.writeFieldBegin('part_vals', TType.LIST, 3) + oprot.writeListBegin(TType.STRING, len(self.part_vals)) + for iter333 in self.part_vals: + oprot.writeString(iter333) + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.deleteData is not None: + oprot.writeFieldBegin('deleteData', TType.BOOL, 4) + oprot.writeBool(self.deleteData) + oprot.writeFieldEnd() + if self.environment_context is not None: + oprot.writeFieldBegin('environment_context', TType.STRUCT, 5) + self.environment_context.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -8942,27 +11021,24 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class add_partition_result: +class drop_partition_with_environment_context_result: """ Attributes: - success - o1 - o2 - - o3 """ thrift_spec = ( - (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + (0, TType.BOOL, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None, o3=None,): + def __init__(self, success=None, o1=None, o2=None,): self.success = success self.o1 = o1 self.o2 = o2 - self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -8974,29 +11050,22 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.STRUCT: - self.success = Partition() - self.success.read(iprot) + if ftype == TType.BOOL: + self.success = iprot.readBool(); else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = InvalidObjectException() + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = AlreadyExistsException() + self.o2 = MetaException() self.o2.read(iprot) else: iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = MetaException() - self.o3.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -9006,10 +11075,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('add_partition_result') + oprot.writeStructBegin('drop_partition_with_environment_context_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) + oprot.writeFieldBegin('success', TType.BOOL, 0) + oprot.writeBool(self.success) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -9019,10 +11088,6 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -9041,22 +11106,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class add_partition_with_environment_context_args: +class drop_partition_by_name_args: """ Attributes: - - new_part - - environment_context + - db_name + - tbl_name + - part_name + - deleteData """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'new_part', (Partition, Partition.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 2 + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.STRING, 'part_name', None, None, ), # 3 + (4, TType.BOOL, 'deleteData', None, None, ), # 4 ) - def __init__(self, new_part=None, environment_context=None,): - self.new_part = new_part - self.environment_context = environment_context + def __init__(self, db_name=None, tbl_name=None, part_name=None, deleteData=None,): + self.db_name = db_name + self.tbl_name = tbl_name + self.part_name = part_name + self.deleteData = deleteData def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -9068,15 +11139,23 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRUCT: - self.new_part = Partition() - self.new_part.read(iprot) + if ftype == TType.STRING: + self.db_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: - if ftype == TType.STRUCT: - self.environment_context = EnvironmentContext() - self.environment_context.read(iprot) + if ftype == TType.STRING: + self.tbl_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRING: + self.part_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.BOOL: + self.deleteData = iprot.readBool(); else: iprot.skip(ftype) else: @@ -9088,14 +11167,22 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('add_partition_with_environment_context_args') - if self.new_part is not None: - oprot.writeFieldBegin('new_part', TType.STRUCT, 1) - self.new_part.write(oprot) + oprot.writeStructBegin('drop_partition_by_name_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) + oprot.writeFieldEnd() + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) + oprot.writeFieldEnd() + if self.part_name is not None: + oprot.writeFieldBegin('part_name', TType.STRING, 3) + oprot.writeString(self.part_name) oprot.writeFieldEnd() - if self.environment_context is not None: - oprot.writeFieldBegin('environment_context', TType.STRUCT, 2) - self.environment_context.write(oprot) + if self.deleteData is not None: + oprot.writeFieldBegin('deleteData', TType.BOOL, 4) + oprot.writeBool(self.deleteData) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -9115,27 +11202,24 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class add_partition_with_environment_context_result: +class drop_partition_by_name_result: """ Attributes: - success - o1 - o2 - - o3 """ thrift_spec = ( - (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + (0, TType.BOOL, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None, o3=None,): + def __init__(self, success=None, o1=None, o2=None,): self.success = success self.o1 = o1 self.o2 = o2 - self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -9147,29 +11231,22 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.STRUCT: - self.success = Partition() - self.success.read(iprot) + if ftype == TType.BOOL: + self.success = iprot.readBool(); else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = InvalidObjectException() + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = AlreadyExistsException() + self.o2 = MetaException() self.o2.read(iprot) else: iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = MetaException() - self.o3.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -9179,10 +11256,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('add_partition_with_environment_context_result') + oprot.writeStructBegin('drop_partition_by_name_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) + oprot.writeFieldBegin('success', TType.BOOL, 0) + oprot.writeBool(self.success) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -9192,10 +11269,6 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -9214,19 +11287,31 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class add_partitions_args: +class drop_partition_by_name_with_environment_context_args: """ Attributes: - - new_parts + - db_name + - tbl_name + - part_name + - deleteData + - environment_context """ thrift_spec = ( None, # 0 - (1, TType.LIST, 'new_parts', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 1 + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.STRING, 'part_name', None, None, ), # 3 + (4, TType.BOOL, 'deleteData', None, None, ), # 4 + (5, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 5 ) - def __init__(self, new_parts=None,): - self.new_parts = new_parts + def __init__(self, db_name=None, tbl_name=None, part_name=None, deleteData=None, environment_context=None,): + self.db_name = db_name + self.tbl_name = tbl_name + self.part_name = part_name + self.deleteData = deleteData + self.environment_context = environment_context def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -9238,14 +11323,29 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.LIST: - self.new_parts = [] - (_etype302, _size299) = iprot.readListBegin() - for _i303 in xrange(_size299): - _elem304 = Partition() - _elem304.read(iprot) - self.new_parts.append(_elem304) - iprot.readListEnd() + if ftype == TType.STRING: + self.db_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.tbl_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRING: + self.part_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.BOOL: + self.deleteData = iprot.readBool(); + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.STRUCT: + self.environment_context = EnvironmentContext() + self.environment_context.read(iprot) else: iprot.skip(ftype) else: @@ -9257,13 +11357,26 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('add_partitions_args') - if self.new_parts is not None: - oprot.writeFieldBegin('new_parts', TType.LIST, 1) - oprot.writeListBegin(TType.STRUCT, len(self.new_parts)) - for iter305 in self.new_parts: - iter305.write(oprot) - oprot.writeListEnd() + oprot.writeStructBegin('drop_partition_by_name_with_environment_context_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) + oprot.writeFieldEnd() + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) + oprot.writeFieldEnd() + if self.part_name is not None: + oprot.writeFieldBegin('part_name', TType.STRING, 3) + oprot.writeString(self.part_name) + oprot.writeFieldEnd() + if self.deleteData is not None: + oprot.writeFieldBegin('deleteData', TType.BOOL, 4) + oprot.writeBool(self.deleteData) + oprot.writeFieldEnd() + if self.environment_context is not None: + oprot.writeFieldBegin('environment_context', TType.STRUCT, 5) + self.environment_context.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -9283,27 +11396,24 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class add_partitions_result: +class drop_partition_by_name_with_environment_context_result: """ Attributes: - success - o1 - o2 - - o3 """ thrift_spec = ( - (0, TType.I32, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + (0, TType.BOOL, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None, o3=None,): + def __init__(self, success=None, o1=None, o2=None,): self.success = success self.o1 = o1 self.o2 = o2 - self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -9315,28 +11425,22 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.I32: - self.success = iprot.readI32(); + if ftype == TType.BOOL: + self.success = iprot.readBool(); else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = InvalidObjectException() + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = AlreadyExistsException() + self.o2 = MetaException() self.o2.read(iprot) else: iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = MetaException() - self.o3.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -9346,10 +11450,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('add_partitions_result') + oprot.writeStructBegin('drop_partition_by_name_with_environment_context_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.I32, 0) - oprot.writeI32(self.success) + oprot.writeFieldBegin('success', TType.BOOL, 0) + oprot.writeBool(self.success) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -9359,10 +11463,6 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -9381,7 +11481,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class append_partition_args: +class get_partition_args: """ Attributes: - db_name @@ -9423,10 +11523,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.part_vals = [] - (_etype309, _size306) = iprot.readListBegin() - for _i310 in xrange(_size306): - _elem311 = iprot.readString(); - self.part_vals.append(_elem311) + (_etype337, _size334) = iprot.readListBegin() + for _i338 in xrange(_size334): + _elem339 = iprot.readString(); + self.part_vals.append(_elem339) iprot.readListEnd() else: iprot.skip(ftype) @@ -9439,7 +11539,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('append_partition_args') + oprot.writeStructBegin('get_partition_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -9451,8 +11551,8 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter312 in self.part_vals: - oprot.writeString(iter312) + for iter340 in self.part_vals: + oprot.writeString(iter340) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -9473,27 +11573,24 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class append_partition_result: +class get_partition_result: """ Attributes: - success - o1 - o2 - - o3 """ thrift_spec = ( (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None, o3=None,): + def __init__(self, success=None, o1=None, o2=None,): self.success = success self.o1 = o1 self.o2 = o2 - self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -9512,22 +11609,16 @@ def read(self, iprot): iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = InvalidObjectException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = AlreadyExistsException() + self.o2 = NoSuchObjectException() self.o2.read(iprot) else: iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = MetaException() - self.o3.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -9537,7 +11628,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('append_partition_result') + oprot.writeStructBegin('get_partition_result') if self.success is not None: oprot.writeFieldBegin('success', TType.STRUCT, 0) self.success.write(oprot) @@ -9550,10 +11641,6 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -9572,28 +11659,31 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class append_partition_with_environment_context_args: +class exchange_partition_args: """ Attributes: - - db_name - - tbl_name - - part_vals - - environment_context + - partitionSpecs + - source_db + - source_table_name + - dest_db + - dest_table_name """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 - (4, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 4 + (1, TType.MAP, 'partitionSpecs', (TType.STRING,None,TType.STRING,None), None, ), # 1 + (2, TType.STRING, 'source_db', None, None, ), # 2 + (3, TType.STRING, 'source_table_name', None, None, ), # 3 + (4, TType.STRING, 'dest_db', None, None, ), # 4 + (5, TType.STRING, 'dest_table_name', None, None, ), # 5 ) - def __init__(self, db_name=None, tbl_name=None, part_vals=None, environment_context=None,): - self.db_name = db_name - self.tbl_name = tbl_name - self.part_vals = part_vals - self.environment_context = environment_context + def __init__(self, partitionSpecs=None, source_db=None, source_table_name=None, dest_db=None, dest_table_name=None,): + self.partitionSpecs = partitionSpecs + self.source_db = source_db + self.source_table_name = source_table_name + self.dest_db = dest_db + self.dest_table_name = dest_table_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -9605,29 +11695,34 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.db_name = iprot.readString(); + if ftype == TType.MAP: + self.partitionSpecs = {} + (_ktype342, _vtype343, _size341 ) = iprot.readMapBegin() + for _i345 in xrange(_size341): + _key346 = iprot.readString(); + _val347 = iprot.readString(); + self.partitionSpecs[_key346] = _val347 + iprot.readMapEnd() else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: - self.tbl_name = iprot.readString(); + self.source_db = iprot.readString(); else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.LIST: - self.part_vals = [] - (_etype316, _size313) = iprot.readListBegin() - for _i317 in xrange(_size313): - _elem318 = iprot.readString(); - self.part_vals.append(_elem318) - iprot.readListEnd() + if ftype == TType.STRING: + self.source_table_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 4: - if ftype == TType.STRUCT: - self.environment_context = EnvironmentContext() - self.environment_context.read(iprot) + if ftype == TType.STRING: + self.dest_db = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.STRING: + self.dest_table_name = iprot.readString(); else: iprot.skip(ftype) else: @@ -9639,25 +11734,30 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('append_partition_with_environment_context_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) + oprot.writeStructBegin('exchange_partition_args') + if self.partitionSpecs is not None: + oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1) + oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs)) + for kiter348,viter349 in self.partitionSpecs.items(): + oprot.writeString(kiter348) + oprot.writeString(viter349) + oprot.writeMapEnd() oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) + if self.source_db is not None: + oprot.writeFieldBegin('source_db', TType.STRING, 2) + oprot.writeString(self.source_db) oprot.writeFieldEnd() - if self.part_vals is not None: - oprot.writeFieldBegin('part_vals', TType.LIST, 3) - oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter319 in self.part_vals: - oprot.writeString(iter319) - oprot.writeListEnd() + if self.source_table_name is not None: + oprot.writeFieldBegin('source_table_name', TType.STRING, 3) + oprot.writeString(self.source_table_name) oprot.writeFieldEnd() - if self.environment_context is not None: - oprot.writeFieldBegin('environment_context', TType.STRUCT, 4) - self.environment_context.write(oprot) + if self.dest_db is not None: + oprot.writeFieldBegin('dest_db', TType.STRING, 4) + oprot.writeString(self.dest_db) + oprot.writeFieldEnd() + if self.dest_table_name is not None: + oprot.writeFieldBegin('dest_table_name', TType.STRING, 5) + oprot.writeString(self.dest_table_name) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -9677,27 +11777,30 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class append_partition_with_environment_context_result: +class exchange_partition_result: """ Attributes: - success - o1 - o2 - o3 + - o4 """ thrift_spec = ( (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 4 ) - def __init__(self, success=None, o1=None, o2=None, o3=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None,): self.success = success self.o1 = o1 self.o2 = o2 self.o3 = o3 + self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -9716,22 +11819,28 @@ def read(self, iprot): iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = InvalidObjectException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = AlreadyExistsException() + self.o2 = NoSuchObjectException() self.o2.read(iprot) else: iprot.skip(ftype) elif fid == 3: if ftype == TType.STRUCT: - self.o3 = MetaException() + self.o3 = InvalidObjectException() self.o3.read(iprot) else: iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.o4 = InvalidInputException() + self.o4.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -9741,7 +11850,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('append_partition_with_environment_context_result') + oprot.writeStructBegin('exchange_partition_result') if self.success is not None: oprot.writeFieldBegin('success', TType.STRUCT, 0) self.success.write(oprot) @@ -9758,6 +11867,10 @@ def write(self, oprot): oprot.writeFieldBegin('o3', TType.STRUCT, 3) self.o3.write(oprot) oprot.writeFieldEnd() + if self.o4 is not None: + oprot.writeFieldBegin('o4', TType.STRUCT, 4) + self.o4.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -9776,25 +11889,31 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class append_partition_by_name_args: +class get_partition_with_auth_args: """ Attributes: - db_name - tbl_name - - part_name + - part_vals + - user_name + - group_names """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRING, 'part_name', None, None, ), # 3 + (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 + (4, TType.STRING, 'user_name', None, None, ), # 4 + (5, TType.LIST, 'group_names', (TType.STRING,None), None, ), # 5 ) - def __init__(self, db_name=None, tbl_name=None, part_name=None,): + def __init__(self, db_name=None, tbl_name=None, part_vals=None, user_name=None, group_names=None,): self.db_name = db_name self.tbl_name = tbl_name - self.part_name = part_name + self.part_vals = part_vals + self.user_name = user_name + self.group_names = group_names def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -9816,8 +11935,28 @@ def read(self, iprot): else: iprot.skip(ftype) elif fid == 3: + if ftype == TType.LIST: + self.part_vals = [] + (_etype353, _size350) = iprot.readListBegin() + for _i354 in xrange(_size350): + _elem355 = iprot.readString(); + self.part_vals.append(_elem355) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 4: if ftype == TType.STRING: - self.part_name = iprot.readString(); + self.user_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.LIST: + self.group_names = [] + (_etype359, _size356) = iprot.readListBegin() + for _i360 in xrange(_size356): + _elem361 = iprot.readString(); + self.group_names.append(_elem361) + iprot.readListEnd() else: iprot.skip(ftype) else: @@ -9829,7 +11968,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('append_partition_by_name_args') + oprot.writeStructBegin('get_partition_with_auth_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -9838,9 +11977,23 @@ def write(self, oprot): oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.part_name is not None: - oprot.writeFieldBegin('part_name', TType.STRING, 3) - oprot.writeString(self.part_name) + if self.part_vals is not None: + oprot.writeFieldBegin('part_vals', TType.LIST, 3) + oprot.writeListBegin(TType.STRING, len(self.part_vals)) + for iter362 in self.part_vals: + oprot.writeString(iter362) + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.user_name is not None: + oprot.writeFieldBegin('user_name', TType.STRING, 4) + oprot.writeString(self.user_name) + oprot.writeFieldEnd() + if self.group_names is not None: + oprot.writeFieldBegin('group_names', TType.LIST, 5) + oprot.writeListBegin(TType.STRING, len(self.group_names)) + for iter363 in self.group_names: + oprot.writeString(iter363) + oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -9860,27 +12013,24 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class append_partition_by_name_result: +class get_partition_with_auth_result: """ Attributes: - success - o1 - o2 - - o3 """ thrift_spec = ( (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None, o3=None,): + def __init__(self, success=None, o1=None, o2=None,): self.success = success self.o1 = o1 self.o2 = o2 - self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -9899,22 +12049,16 @@ def read(self, iprot): iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = InvalidObjectException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = AlreadyExistsException() + self.o2 = NoSuchObjectException() self.o2.read(iprot) else: iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = MetaException() - self.o3.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -9924,7 +12068,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('append_partition_by_name_result') + oprot.writeStructBegin('get_partition_with_auth_result') if self.success is not None: oprot.writeFieldBegin('success', TType.STRUCT, 0) self.success.write(oprot) @@ -9937,10 +12081,6 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -9959,13 +12099,12 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class append_partition_by_name_with_environment_context_args: +class get_partition_by_name_args: """ Attributes: - db_name - tbl_name - part_name - - environment_context """ thrift_spec = ( @@ -9973,14 +12112,12 @@ class append_partition_by_name_with_environment_context_args: (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 (3, TType.STRING, 'part_name', None, None, ), # 3 - (4, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 4 ) - def __init__(self, db_name=None, tbl_name=None, part_name=None, environment_context=None,): + def __init__(self, db_name=None, tbl_name=None, part_name=None,): self.db_name = db_name self.tbl_name = tbl_name self.part_name = part_name - self.environment_context = environment_context def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -10006,12 +12143,6 @@ def read(self, iprot): self.part_name = iprot.readString(); else: iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRUCT: - self.environment_context = EnvironmentContext() - self.environment_context.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -10021,7 +12152,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('append_partition_by_name_with_environment_context_args') + oprot.writeStructBegin('get_partition_by_name_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -10034,10 +12165,6 @@ def write(self, oprot): oprot.writeFieldBegin('part_name', TType.STRING, 3) oprot.writeString(self.part_name) oprot.writeFieldEnd() - if self.environment_context is not None: - oprot.writeFieldBegin('environment_context', TType.STRUCT, 4) - self.environment_context.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -10056,27 +12183,24 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class append_partition_by_name_with_environment_context_result: +class get_partition_by_name_result: """ Attributes: - success - o1 - o2 - - o3 """ thrift_spec = ( (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None, o3=None,): + def __init__(self, success=None, o1=None, o2=None,): self.success = success self.o1 = o1 self.o2 = o2 - self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -10095,22 +12219,16 @@ def read(self, iprot): iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = InvalidObjectException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = AlreadyExistsException() + self.o2 = NoSuchObjectException() self.o2.read(iprot) else: iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = MetaException() - self.o3.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -10120,7 +12238,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('append_partition_by_name_with_environment_context_result') + oprot.writeStructBegin('get_partition_by_name_result') if self.success is not None: oprot.writeFieldBegin('success', TType.STRUCT, 0) self.success.write(oprot) @@ -10133,10 +12251,6 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -10155,28 +12269,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_partition_args: +class get_partitions_args: """ Attributes: - db_name - tbl_name - - part_vals - - deleteData + - max_parts """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 - (4, TType.BOOL, 'deleteData', None, None, ), # 4 + (3, TType.I16, 'max_parts', None, -1, ), # 3 ) - def __init__(self, db_name=None, tbl_name=None, part_vals=None, deleteData=None,): + def __init__(self, db_name=None, tbl_name=None, max_parts=thrift_spec[3][4],): self.db_name = db_name self.tbl_name = tbl_name - self.part_vals = part_vals - self.deleteData = deleteData + self.max_parts = max_parts def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -10198,18 +12309,8 @@ def read(self, iprot): else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.LIST: - self.part_vals = [] - (_etype323, _size320) = iprot.readListBegin() - for _i324 in xrange(_size320): - _elem325 = iprot.readString(); - self.part_vals.append(_elem325) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.BOOL: - self.deleteData = iprot.readBool(); + if ftype == TType.I16: + self.max_parts = iprot.readI16(); else: iprot.skip(ftype) else: @@ -10221,7 +12322,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_partition_args') + oprot.writeStructBegin('get_partitions_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -10230,16 +12331,9 @@ def write(self, oprot): oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.part_vals is not None: - oprot.writeFieldBegin('part_vals', TType.LIST, 3) - oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter326 in self.part_vals: - oprot.writeString(iter326) - oprot.writeListEnd() - oprot.writeFieldEnd() - if self.deleteData is not None: - oprot.writeFieldBegin('deleteData', TType.BOOL, 4) - oprot.writeBool(self.deleteData) + if self.max_parts is not None: + oprot.writeFieldBegin('max_parts', TType.I16, 3) + oprot.writeI16(self.max_parts) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -10259,7 +12353,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_partition_result: +class get_partitions_result: """ Attributes: - success @@ -10268,7 +12362,7 @@ class drop_partition_result: """ thrift_spec = ( - (0, TType.BOOL, 'success', None, None, ), # 0 + (0, TType.LIST, 'success', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 0 (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) @@ -10288,8 +12382,14 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.BOOL: - self.success = iprot.readBool(); + if ftype == TType.LIST: + self.success = [] + (_etype367, _size364) = iprot.readListBegin() + for _i368 in xrange(_size364): + _elem369 = Partition() + _elem369.read(iprot) + self.success.append(_elem369) + iprot.readListEnd() else: iprot.skip(ftype) elif fid == 1: @@ -10313,10 +12413,13 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_partition_result') + oprot.writeStructBegin('get_partitions_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.BOOL, 0) - oprot.writeBool(self.success) + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRUCT, len(self.success)) + for iter370 in self.success: + iter370.write(oprot) + oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -10344,31 +12447,31 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_partition_with_environment_context_args: +class get_partitions_with_auth_args: """ Attributes: - db_name - tbl_name - - part_vals - - deleteData - - environment_context + - max_parts + - user_name + - group_names """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 - (4, TType.BOOL, 'deleteData', None, None, ), # 4 - (5, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 5 + (3, TType.I16, 'max_parts', None, -1, ), # 3 + (4, TType.STRING, 'user_name', None, None, ), # 4 + (5, TType.LIST, 'group_names', (TType.STRING,None), None, ), # 5 ) - def __init__(self, db_name=None, tbl_name=None, part_vals=None, deleteData=None, environment_context=None,): + def __init__(self, db_name=None, tbl_name=None, max_parts=thrift_spec[3][4], user_name=None, group_names=None,): self.db_name = db_name self.tbl_name = tbl_name - self.part_vals = part_vals - self.deleteData = deleteData - self.environment_context = environment_context + self.max_parts = max_parts + self.user_name = user_name + self.group_names = group_names def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -10390,24 +12493,23 @@ def read(self, iprot): else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.LIST: - self.part_vals = [] - (_etype330, _size327) = iprot.readListBegin() - for _i331 in xrange(_size327): - _elem332 = iprot.readString(); - self.part_vals.append(_elem332) - iprot.readListEnd() + if ftype == TType.I16: + self.max_parts = iprot.readI16(); else: iprot.skip(ftype) elif fid == 4: - if ftype == TType.BOOL: - self.deleteData = iprot.readBool(); + if ftype == TType.STRING: + self.user_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 5: - if ftype == TType.STRUCT: - self.environment_context = EnvironmentContext() - self.environment_context.read(iprot) + if ftype == TType.LIST: + self.group_names = [] + (_etype374, _size371) = iprot.readListBegin() + for _i375 in xrange(_size371): + _elem376 = iprot.readString(); + self.group_names.append(_elem376) + iprot.readListEnd() else: iprot.skip(ftype) else: @@ -10419,7 +12521,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_partition_with_environment_context_args') + oprot.writeStructBegin('get_partitions_with_auth_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -10428,20 +12530,20 @@ def write(self, oprot): oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.part_vals is not None: - oprot.writeFieldBegin('part_vals', TType.LIST, 3) - oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter333 in self.part_vals: - oprot.writeString(iter333) - oprot.writeListEnd() + if self.max_parts is not None: + oprot.writeFieldBegin('max_parts', TType.I16, 3) + oprot.writeI16(self.max_parts) oprot.writeFieldEnd() - if self.deleteData is not None: - oprot.writeFieldBegin('deleteData', TType.BOOL, 4) - oprot.writeBool(self.deleteData) + if self.user_name is not None: + oprot.writeFieldBegin('user_name', TType.STRING, 4) + oprot.writeString(self.user_name) oprot.writeFieldEnd() - if self.environment_context is not None: - oprot.writeFieldBegin('environment_context', TType.STRUCT, 5) - self.environment_context.write(oprot) + if self.group_names is not None: + oprot.writeFieldBegin('group_names', TType.LIST, 5) + oprot.writeListBegin(TType.STRING, len(self.group_names)) + for iter377 in self.group_names: + oprot.writeString(iter377) + oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -10461,7 +12563,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_partition_with_environment_context_result: +class get_partitions_with_auth_result: """ Attributes: - success @@ -10470,7 +12572,7 @@ class drop_partition_with_environment_context_result: """ thrift_spec = ( - (0, TType.BOOL, 'success', None, None, ), # 0 + (0, TType.LIST, 'success', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 0 (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) @@ -10490,8 +12592,14 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.BOOL: - self.success = iprot.readBool(); + if ftype == TType.LIST: + self.success = [] + (_etype381, _size378) = iprot.readListBegin() + for _i382 in xrange(_size378): + _elem383 = Partition() + _elem383.read(iprot) + self.success.append(_elem383) + iprot.readListEnd() else: iprot.skip(ftype) elif fid == 1: @@ -10515,10 +12623,13 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_partition_with_environment_context_result') + oprot.writeStructBegin('get_partitions_with_auth_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.BOOL, 0) - oprot.writeBool(self.success) + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRUCT, len(self.success)) + for iter384 in self.success: + iter384.write(oprot) + oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -10546,28 +12657,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_partition_by_name_args: +class get_partition_names_args: """ Attributes: - db_name - tbl_name - - part_name - - deleteData + - max_parts """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRING, 'part_name', None, None, ), # 3 - (4, TType.BOOL, 'deleteData', None, None, ), # 4 + (3, TType.I16, 'max_parts', None, -1, ), # 3 ) - def __init__(self, db_name=None, tbl_name=None, part_name=None, deleteData=None,): + def __init__(self, db_name=None, tbl_name=None, max_parts=thrift_spec[3][4],): self.db_name = db_name self.tbl_name = tbl_name - self.part_name = part_name - self.deleteData = deleteData + self.max_parts = max_parts def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -10589,13 +12697,8 @@ def read(self, iprot): else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.STRING: - self.part_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.BOOL: - self.deleteData = iprot.readBool(); + if ftype == TType.I16: + self.max_parts = iprot.readI16(); else: iprot.skip(ftype) else: @@ -10607,7 +12710,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_partition_by_name_args') + oprot.writeStructBegin('get_partition_names_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -10616,13 +12719,9 @@ def write(self, oprot): oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.part_name is not None: - oprot.writeFieldBegin('part_name', TType.STRING, 3) - oprot.writeString(self.part_name) - oprot.writeFieldEnd() - if self.deleteData is not None: - oprot.writeFieldBegin('deleteData', TType.BOOL, 4) - oprot.writeBool(self.deleteData) + if self.max_parts is not None: + oprot.writeFieldBegin('max_parts', TType.I16, 3) + oprot.writeI16(self.max_parts) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -10642,23 +12741,20 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_partition_by_name_result: +class get_partition_names_result: """ Attributes: - success - - o1 - o2 """ thrift_spec = ( - (0, TType.BOOL, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 + (1, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 1 ) - def __init__(self, success=None, o1=None, o2=None,): + def __init__(self, success=None, o2=None,): self.success = success - self.o1 = o1 self.o2 = o2 def read(self, iprot): @@ -10671,18 +12767,17 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.BOOL: - self.success = iprot.readBool(); + if ftype == TType.LIST: + self.success = [] + (_etype388, _size385) = iprot.readListBegin() + for _i389 in xrange(_size385): + _elem390 = iprot.readString(); + self.success.append(_elem390) + iprot.readListEnd() else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() - self.o1.read(iprot) - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: self.o2 = MetaException() self.o2.read(iprot) else: @@ -10696,17 +12791,16 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_partition_by_name_result') + oprot.writeStructBegin('get_partition_names_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.BOOL, 0) - oprot.writeBool(self.success) - oprot.writeFieldEnd() - if self.o1 is not None: - oprot.writeFieldBegin('o1', TType.STRUCT, 1) - self.o1.write(oprot) + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRING, len(self.success)) + for iter391 in self.success: + oprot.writeString(iter391) + oprot.writeListEnd() oprot.writeFieldEnd() if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 2) + oprot.writeFieldBegin('o2', TType.STRUCT, 1) self.o2.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() @@ -10727,31 +12821,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_partition_by_name_with_environment_context_args: +class get_partitions_ps_args: """ Attributes: - db_name - tbl_name - - part_name - - deleteData - - environment_context + - part_vals + - max_parts """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRING, 'part_name', None, None, ), # 3 - (4, TType.BOOL, 'deleteData', None, None, ), # 4 - (5, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 5 + (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 + (4, TType.I16, 'max_parts', None, -1, ), # 4 ) - def __init__(self, db_name=None, tbl_name=None, part_name=None, deleteData=None, environment_context=None,): + def __init__(self, db_name=None, tbl_name=None, part_vals=None, max_parts=thrift_spec[4][4],): self.db_name = db_name self.tbl_name = tbl_name - self.part_name = part_name - self.deleteData = deleteData - self.environment_context = environment_context + self.part_vals = part_vals + self.max_parts = max_parts def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -10773,19 +12864,18 @@ def read(self, iprot): else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.STRING: - self.part_name = iprot.readString(); + if ftype == TType.LIST: + self.part_vals = [] + (_etype395, _size392) = iprot.readListBegin() + for _i396 in xrange(_size392): + _elem397 = iprot.readString(); + self.part_vals.append(_elem397) + iprot.readListEnd() else: iprot.skip(ftype) elif fid == 4: - if ftype == TType.BOOL: - self.deleteData = iprot.readBool(); - else: - iprot.skip(ftype) - elif fid == 5: - if ftype == TType.STRUCT: - self.environment_context = EnvironmentContext() - self.environment_context.read(iprot) + if ftype == TType.I16: + self.max_parts = iprot.readI16(); else: iprot.skip(ftype) else: @@ -10797,7 +12887,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_partition_by_name_with_environment_context_args') + oprot.writeStructBegin('get_partitions_ps_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -10806,17 +12896,16 @@ def write(self, oprot): oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.part_name is not None: - oprot.writeFieldBegin('part_name', TType.STRING, 3) - oprot.writeString(self.part_name) - oprot.writeFieldEnd() - if self.deleteData is not None: - oprot.writeFieldBegin('deleteData', TType.BOOL, 4) - oprot.writeBool(self.deleteData) + if self.part_vals is not None: + oprot.writeFieldBegin('part_vals', TType.LIST, 3) + oprot.writeListBegin(TType.STRING, len(self.part_vals)) + for iter398 in self.part_vals: + oprot.writeString(iter398) + oprot.writeListEnd() oprot.writeFieldEnd() - if self.environment_context is not None: - oprot.writeFieldBegin('environment_context', TType.STRUCT, 5) - self.environment_context.write(oprot) + if self.max_parts is not None: + oprot.writeFieldBegin('max_parts', TType.I16, 4) + oprot.writeI16(self.max_parts) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -10836,18 +12925,18 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_partition_by_name_with_environment_context_result: +class get_partitions_ps_result: """ Attributes: - success - o1 - o2 """ - - thrift_spec = ( - (0, TType.BOOL, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + + thrift_spec = ( + (0, TType.LIST, 'success', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 ) def __init__(self, success=None, o1=None, o2=None,): @@ -10865,19 +12954,25 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.BOOL: - self.success = iprot.readBool(); + if ftype == TType.LIST: + self.success = [] + (_etype402, _size399) = iprot.readListBegin() + for _i403 in xrange(_size399): + _elem404 = Partition() + _elem404.read(iprot) + self.success.append(_elem404) + iprot.readListEnd() else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = MetaException() + self.o2 = NoSuchObjectException() self.o2.read(iprot) else: iprot.skip(ftype) @@ -10890,10 +12985,13 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_partition_by_name_with_environment_context_result') + oprot.writeStructBegin('get_partitions_ps_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.BOOL, 0) - oprot.writeBool(self.success) + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRUCT, len(self.success)) + for iter405 in self.success: + iter405.write(oprot) + oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -10921,12 +13019,15 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partition_args: +class get_partitions_ps_with_auth_args: """ Attributes: - db_name - tbl_name - part_vals + - max_parts + - user_name + - group_names """ thrift_spec = ( @@ -10934,12 +13035,18 @@ class get_partition_args: (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 + (4, TType.I16, 'max_parts', None, -1, ), # 4 + (5, TType.STRING, 'user_name', None, None, ), # 5 + (6, TType.LIST, 'group_names', (TType.STRING,None), None, ), # 6 ) - def __init__(self, db_name=None, tbl_name=None, part_vals=None,): + def __init__(self, db_name=None, tbl_name=None, part_vals=None, max_parts=thrift_spec[4][4], user_name=None, group_names=None,): self.db_name = db_name self.tbl_name = tbl_name self.part_vals = part_vals + self.max_parts = max_parts + self.user_name = user_name + self.group_names = group_names def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -10963,10 +13070,30 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.part_vals = [] - (_etype337, _size334) = iprot.readListBegin() - for _i338 in xrange(_size334): - _elem339 = iprot.readString(); - self.part_vals.append(_elem339) + (_etype409, _size406) = iprot.readListBegin() + for _i410 in xrange(_size406): + _elem411 = iprot.readString(); + self.part_vals.append(_elem411) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.I16: + self.max_parts = iprot.readI16(); + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.STRING: + self.user_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 6: + if ftype == TType.LIST: + self.group_names = [] + (_etype415, _size412) = iprot.readListBegin() + for _i416 in xrange(_size412): + _elem417 = iprot.readString(); + self.group_names.append(_elem417) iprot.readListEnd() else: iprot.skip(ftype) @@ -10979,7 +13106,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partition_args') + oprot.writeStructBegin('get_partitions_ps_with_auth_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -10991,8 +13118,23 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter340 in self.part_vals: - oprot.writeString(iter340) + for iter418 in self.part_vals: + oprot.writeString(iter418) + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.max_parts is not None: + oprot.writeFieldBegin('max_parts', TType.I16, 4) + oprot.writeI16(self.max_parts) + oprot.writeFieldEnd() + if self.user_name is not None: + oprot.writeFieldBegin('user_name', TType.STRING, 5) + oprot.writeString(self.user_name) + oprot.writeFieldEnd() + if self.group_names is not None: + oprot.writeFieldBegin('group_names', TType.LIST, 6) + oprot.writeListBegin(TType.STRING, len(self.group_names)) + for iter419 in self.group_names: + oprot.writeString(iter419) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -11013,7 +13155,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partition_result: +class get_partitions_ps_with_auth_result: """ Attributes: - success @@ -11022,9 +13164,9 @@ class get_partition_result: """ thrift_spec = ( - (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 + (0, TType.LIST, 'success', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) def __init__(self, success=None, o1=None, o2=None,): @@ -11042,20 +13184,25 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.STRUCT: - self.success = Partition() - self.success.read(iprot) + if ftype == TType.LIST: + self.success = [] + (_etype423, _size420) = iprot.readListBegin() + for _i424 in xrange(_size420): + _elem425 = Partition() + _elem425.read(iprot) + self.success.append(_elem425) + iprot.readListEnd() else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = NoSuchObjectException() + self.o2 = MetaException() self.o2.read(iprot) else: iprot.skip(ftype) @@ -11068,10 +13215,13 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partition_result') + oprot.writeStructBegin('get_partitions_ps_with_auth_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRUCT, len(self.success)) + for iter426 in self.success: + iter426.write(oprot) + oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -11099,31 +13249,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class exchange_partition_args: +class get_partition_names_ps_args: """ Attributes: - - partitionSpecs - - source_db - - source_table_name - - dest_db - - dest_table_name + - db_name + - tbl_name + - part_vals + - max_parts """ thrift_spec = ( None, # 0 - (1, TType.MAP, 'partitionSpecs', (TType.STRING,None,TType.STRING,None), None, ), # 1 - (2, TType.STRING, 'source_db', None, None, ), # 2 - (3, TType.STRING, 'source_table_name', None, None, ), # 3 - (4, TType.STRING, 'dest_db', None, None, ), # 4 - (5, TType.STRING, 'dest_table_name', None, None, ), # 5 + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 + (4, TType.I16, 'max_parts', None, -1, ), # 4 ) - def __init__(self, partitionSpecs=None, source_db=None, source_table_name=None, dest_db=None, dest_table_name=None,): - self.partitionSpecs = partitionSpecs - self.source_db = source_db - self.source_table_name = source_table_name - self.dest_db = dest_db - self.dest_table_name = dest_table_name + def __init__(self, db_name=None, tbl_name=None, part_vals=None, max_parts=thrift_spec[4][4],): + self.db_name = db_name + self.tbl_name = tbl_name + self.part_vals = part_vals + self.max_parts = max_parts def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -11135,34 +13282,28 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.MAP: - self.partitionSpecs = {} - (_ktype342, _vtype343, _size341 ) = iprot.readMapBegin() - for _i345 in xrange(_size341): - _key346 = iprot.readString(); - _val347 = iprot.readString(); - self.partitionSpecs[_key346] = _val347 - iprot.readMapEnd() + if ftype == TType.STRING: + self.db_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: - self.source_db = iprot.readString(); + self.tbl_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.STRING: - self.source_table_name = iprot.readString(); + if ftype == TType.LIST: + self.part_vals = [] + (_etype430, _size427) = iprot.readListBegin() + for _i431 in xrange(_size427): + _elem432 = iprot.readString(); + self.part_vals.append(_elem432) + iprot.readListEnd() else: iprot.skip(ftype) elif fid == 4: - if ftype == TType.STRING: - self.dest_db = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 5: - if ftype == TType.STRING: - self.dest_table_name = iprot.readString(); + if ftype == TType.I16: + self.max_parts = iprot.readI16(); else: iprot.skip(ftype) else: @@ -11174,30 +13315,25 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('exchange_partition_args') - if self.partitionSpecs is not None: - oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1) - oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs)) - for kiter348,viter349 in self.partitionSpecs.items(): - oprot.writeString(kiter348) - oprot.writeString(viter349) - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.source_db is not None: - oprot.writeFieldBegin('source_db', TType.STRING, 2) - oprot.writeString(self.source_db) + oprot.writeStructBegin('get_partition_names_ps_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) oprot.writeFieldEnd() - if self.source_table_name is not None: - oprot.writeFieldBegin('source_table_name', TType.STRING, 3) - oprot.writeString(self.source_table_name) + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.dest_db is not None: - oprot.writeFieldBegin('dest_db', TType.STRING, 4) - oprot.writeString(self.dest_db) + if self.part_vals is not None: + oprot.writeFieldBegin('part_vals', TType.LIST, 3) + oprot.writeListBegin(TType.STRING, len(self.part_vals)) + for iter433 in self.part_vals: + oprot.writeString(iter433) + oprot.writeListEnd() oprot.writeFieldEnd() - if self.dest_table_name is not None: - oprot.writeFieldBegin('dest_table_name', TType.STRING, 5) - oprot.writeString(self.dest_table_name) + if self.max_parts is not None: + oprot.writeFieldBegin('max_parts', TType.I16, 4) + oprot.writeI16(self.max_parts) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -11217,30 +13353,24 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class exchange_partition_result: +class get_partition_names_ps_result: """ Attributes: - success - o1 - o2 - - o3 - - o4 """ thrift_spec = ( - (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 + (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 3 - (4, TType.STRUCT, 'o4', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 4 ) - def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None,): + def __init__(self, success=None, o1=None, o2=None,): self.success = success self.o1 = o1 self.o2 = o2 - self.o3 = o3 - self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -11252,9 +13382,13 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.STRUCT: - self.success = Partition() - self.success.read(iprot) + if ftype == TType.LIST: + self.success = [] + (_etype437, _size434) = iprot.readListBegin() + for _i438 in xrange(_size434): + _elem439 = iprot.readString(); + self.success.append(_elem439) + iprot.readListEnd() else: iprot.skip(ftype) elif fid == 1: @@ -11265,20 +13399,8 @@ def read(self, iprot): iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = NoSuchObjectException() - self.o2.read(iprot) - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = InvalidObjectException() - self.o3.read(iprot) - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRUCT: - self.o4 = InvalidInputException() - self.o4.read(iprot) + self.o2 = NoSuchObjectException() + self.o2.read(iprot) else: iprot.skip(ftype) else: @@ -11290,10 +13412,13 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('exchange_partition_result') + oprot.writeStructBegin('get_partition_names_ps_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRING, len(self.success)) + for iter440 in self.success: + oprot.writeString(iter440) + oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -11303,14 +13428,6 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() - if self.o4 is not None: - oprot.writeFieldBegin('o4', TType.STRUCT, 4) - self.o4.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -11329,31 +13446,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partition_with_auth_args: +class get_partitions_by_filter_args: """ Attributes: - db_name - tbl_name - - part_vals - - user_name - - group_names + - filter + - max_parts """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 - (4, TType.STRING, 'user_name', None, None, ), # 4 - (5, TType.LIST, 'group_names', (TType.STRING,None), None, ), # 5 + (3, TType.STRING, 'filter', None, None, ), # 3 + (4, TType.I16, 'max_parts', None, -1, ), # 4 ) - def __init__(self, db_name=None, tbl_name=None, part_vals=None, user_name=None, group_names=None,): + def __init__(self, db_name=None, tbl_name=None, filter=None, max_parts=thrift_spec[4][4],): self.db_name = db_name self.tbl_name = tbl_name - self.part_vals = part_vals - self.user_name = user_name - self.group_names = group_names + self.filter = filter + self.max_parts = max_parts def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -11375,28 +13489,13 @@ def read(self, iprot): else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.LIST: - self.part_vals = [] - (_etype353, _size350) = iprot.readListBegin() - for _i354 in xrange(_size350): - _elem355 = iprot.readString(); - self.part_vals.append(_elem355) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 4: if ftype == TType.STRING: - self.user_name = iprot.readString(); + self.filter = iprot.readString(); else: iprot.skip(ftype) - elif fid == 5: - if ftype == TType.LIST: - self.group_names = [] - (_etype359, _size356) = iprot.readListBegin() - for _i360 in xrange(_size356): - _elem361 = iprot.readString(); - self.group_names.append(_elem361) - iprot.readListEnd() + elif fid == 4: + if ftype == TType.I16: + self.max_parts = iprot.readI16(); else: iprot.skip(ftype) else: @@ -11408,7 +13507,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partition_with_auth_args') + oprot.writeStructBegin('get_partitions_by_filter_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -11417,23 +13516,13 @@ def write(self, oprot): oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.part_vals is not None: - oprot.writeFieldBegin('part_vals', TType.LIST, 3) - oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter362 in self.part_vals: - oprot.writeString(iter362) - oprot.writeListEnd() - oprot.writeFieldEnd() - if self.user_name is not None: - oprot.writeFieldBegin('user_name', TType.STRING, 4) - oprot.writeString(self.user_name) + if self.filter is not None: + oprot.writeFieldBegin('filter', TType.STRING, 3) + oprot.writeString(self.filter) oprot.writeFieldEnd() - if self.group_names is not None: - oprot.writeFieldBegin('group_names', TType.LIST, 5) - oprot.writeListBegin(TType.STRING, len(self.group_names)) - for iter363 in self.group_names: - oprot.writeString(iter363) - oprot.writeListEnd() + if self.max_parts is not None: + oprot.writeFieldBegin('max_parts', TType.I16, 4) + oprot.writeI16(self.max_parts) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -11453,7 +13542,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partition_with_auth_result: +class get_partitions_by_filter_result: """ Attributes: - success @@ -11462,7 +13551,7 @@ class get_partition_with_auth_result: """ thrift_spec = ( - (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 + (0, TType.LIST, 'success', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 0 (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 ) @@ -11482,9 +13571,14 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.STRUCT: - self.success = Partition() - self.success.read(iprot) + if ftype == TType.LIST: + self.success = [] + (_etype444, _size441) = iprot.readListBegin() + for _i445 in xrange(_size441): + _elem446 = Partition() + _elem446.read(iprot) + self.success.append(_elem446) + iprot.readListEnd() else: iprot.skip(ftype) elif fid == 1: @@ -11508,10 +13602,13 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partition_with_auth_result') + oprot.writeStructBegin('get_partitions_by_filter_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRUCT, len(self.success)) + for iter447 in self.success: + iter447.write(oprot) + oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -11539,25 +13636,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partition_by_name_args: +class get_partitions_by_names_args: """ Attributes: - db_name - tbl_name - - part_name + - names """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRING, 'part_name', None, None, ), # 3 + (3, TType.LIST, 'names', (TType.STRING,None), None, ), # 3 ) - def __init__(self, db_name=None, tbl_name=None, part_name=None,): + def __init__(self, db_name=None, tbl_name=None, names=None,): self.db_name = db_name self.tbl_name = tbl_name - self.part_name = part_name + self.names = names def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -11579,8 +13676,13 @@ def read(self, iprot): else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.STRING: - self.part_name = iprot.readString(); + if ftype == TType.LIST: + self.names = [] + (_etype451, _size448) = iprot.readListBegin() + for _i452 in xrange(_size448): + _elem453 = iprot.readString(); + self.names.append(_elem453) + iprot.readListEnd() else: iprot.skip(ftype) else: @@ -11592,7 +13694,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partition_by_name_args') + oprot.writeStructBegin('get_partitions_by_names_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -11601,9 +13703,12 @@ def write(self, oprot): oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.part_name is not None: - oprot.writeFieldBegin('part_name', TType.STRING, 3) - oprot.writeString(self.part_name) + if self.names is not None: + oprot.writeFieldBegin('names', TType.LIST, 3) + oprot.writeListBegin(TType.STRING, len(self.names)) + for iter454 in self.names: + oprot.writeString(iter454) + oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -11623,7 +13728,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partition_by_name_result: +class get_partitions_by_names_result: """ Attributes: - success @@ -11632,7 +13737,7 @@ class get_partition_by_name_result: """ thrift_spec = ( - (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 + (0, TType.LIST, 'success', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 0 (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 ) @@ -11652,9 +13757,14 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.STRUCT: - self.success = Partition() - self.success.read(iprot) + if ftype == TType.LIST: + self.success = [] + (_etype458, _size455) = iprot.readListBegin() + for _i459 in xrange(_size455): + _elem460 = Partition() + _elem460.read(iprot) + self.success.append(_elem460) + iprot.readListEnd() else: iprot.skip(ftype) elif fid == 1: @@ -11678,10 +13788,13 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partition_by_name_result') + oprot.writeStructBegin('get_partitions_by_names_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRUCT, len(self.success)) + for iter461 in self.success: + iter461.write(oprot) + oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -11709,25 +13822,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partitions_args: +class alter_partition_args: """ Attributes: - db_name - tbl_name - - max_parts + - new_part """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.I16, 'max_parts', None, -1, ), # 3 + (3, TType.STRUCT, 'new_part', (Partition, Partition.thrift_spec), None, ), # 3 ) - def __init__(self, db_name=None, tbl_name=None, max_parts=thrift_spec[3][4],): + def __init__(self, db_name=None, tbl_name=None, new_part=None,): self.db_name = db_name self.tbl_name = tbl_name - self.max_parts = max_parts + self.new_part = new_part def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -11749,8 +13862,9 @@ def read(self, iprot): else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.I16: - self.max_parts = iprot.readI16(); + if ftype == TType.STRUCT: + self.new_part = Partition() + self.new_part.read(iprot) else: iprot.skip(ftype) else: @@ -11762,7 +13876,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partitions_args') + oprot.writeStructBegin('alter_partition_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -11771,9 +13885,9 @@ def write(self, oprot): oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.max_parts is not None: - oprot.writeFieldBegin('max_parts', TType.I16, 3) - oprot.writeI16(self.max_parts) + if self.new_part is not None: + oprot.writeFieldBegin('new_part', TType.STRUCT, 3) + self.new_part.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -11793,22 +13907,20 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partitions_result: +class alter_partition_result: """ Attributes: - - success - o1 - o2 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + None, # 0 + (1, TType.STRUCT, 'o1', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None,): - self.success = success + def __init__(self, o1=None, o2=None,): self.o1 = o1 self.o2 = o2 @@ -11821,20 +13933,9 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype367, _size364) = iprot.readListBegin() - for _i368 in xrange(_size364): - _elem369 = Partition() - _elem369.read(iprot) - self.success.append(_elem369) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 1: + if fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = InvalidOperationException() self.o1.read(iprot) else: iprot.skip(ftype) @@ -11853,14 +13954,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partitions_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter370 in self.success: - iter370.write(oprot) - oprot.writeListEnd() - oprot.writeFieldEnd() + oprot.writeStructBegin('alter_partition_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -11887,31 +13981,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partitions_with_auth_args: +class alter_partitions_args: """ Attributes: - db_name - tbl_name - - max_parts - - user_name - - group_names + - new_parts """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.I16, 'max_parts', None, -1, ), # 3 - (4, TType.STRING, 'user_name', None, None, ), # 4 - (5, TType.LIST, 'group_names', (TType.STRING,None), None, ), # 5 + (3, TType.LIST, 'new_parts', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 3 ) - def __init__(self, db_name=None, tbl_name=None, max_parts=thrift_spec[3][4], user_name=None, group_names=None,): + def __init__(self, db_name=None, tbl_name=None, new_parts=None,): self.db_name = db_name self.tbl_name = tbl_name - self.max_parts = max_parts - self.user_name = user_name - self.group_names = group_names + self.new_parts = new_parts def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -11933,22 +14021,13 @@ def read(self, iprot): else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.I16: - self.max_parts = iprot.readI16(); - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRING: - self.user_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 5: if ftype == TType.LIST: - self.group_names = [] - (_etype374, _size371) = iprot.readListBegin() - for _i375 in xrange(_size371): - _elem376 = iprot.readString(); - self.group_names.append(_elem376) + self.new_parts = [] + (_etype465, _size462) = iprot.readListBegin() + for _i466 in xrange(_size462): + _elem467 = Partition() + _elem467.read(iprot) + self.new_parts.append(_elem467) iprot.readListEnd() else: iprot.skip(ftype) @@ -11961,7 +14040,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partitions_with_auth_args') + oprot.writeStructBegin('alter_partitions_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -11970,19 +14049,11 @@ def write(self, oprot): oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.max_parts is not None: - oprot.writeFieldBegin('max_parts', TType.I16, 3) - oprot.writeI16(self.max_parts) - oprot.writeFieldEnd() - if self.user_name is not None: - oprot.writeFieldBegin('user_name', TType.STRING, 4) - oprot.writeString(self.user_name) - oprot.writeFieldEnd() - if self.group_names is not None: - oprot.writeFieldBegin('group_names', TType.LIST, 5) - oprot.writeListBegin(TType.STRING, len(self.group_names)) - for iter377 in self.group_names: - oprot.writeString(iter377) + if self.new_parts is not None: + oprot.writeFieldBegin('new_parts', TType.LIST, 3) + oprot.writeListBegin(TType.STRUCT, len(self.new_parts)) + for iter468 in self.new_parts: + iter468.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -12003,22 +14074,20 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partitions_with_auth_result: +class alter_partitions_result: """ Attributes: - - success - o1 - o2 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + None, # 0 + (1, TType.STRUCT, 'o1', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None,): - self.success = success + def __init__(self, o1=None, o2=None,): self.o1 = o1 self.o2 = o2 @@ -12031,20 +14100,9 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype381, _size378) = iprot.readListBegin() - for _i382 in xrange(_size378): - _elem383 = Partition() - _elem383.read(iprot) - self.success.append(_elem383) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 1: + if fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = InvalidOperationException() self.o1.read(iprot) else: iprot.skip(ftype) @@ -12063,14 +14121,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partitions_with_auth_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter384 in self.success: - iter384.write(oprot) - oprot.writeListEnd() - oprot.writeFieldEnd() + oprot.writeStructBegin('alter_partitions_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -12097,25 +14148,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partition_names_args: +class alter_partition_with_environment_context_args: """ Attributes: - db_name - tbl_name - - max_parts + - new_part + - environment_context """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.I16, 'max_parts', None, -1, ), # 3 + (3, TType.STRUCT, 'new_part', (Partition, Partition.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 4 ) - def __init__(self, db_name=None, tbl_name=None, max_parts=thrift_spec[3][4],): + def __init__(self, db_name=None, tbl_name=None, new_part=None, environment_context=None,): self.db_name = db_name self.tbl_name = tbl_name - self.max_parts = max_parts + self.new_part = new_part + self.environment_context = environment_context def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -12137,8 +14191,15 @@ def read(self, iprot): else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.I16: - self.max_parts = iprot.readI16(); + if ftype == TType.STRUCT: + self.new_part = Partition() + self.new_part.read(iprot) + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.environment_context = EnvironmentContext() + self.environment_context.read(iprot) else: iprot.skip(ftype) else: @@ -12150,7 +14211,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partition_names_args') + oprot.writeStructBegin('alter_partition_with_environment_context_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -12159,9 +14220,13 @@ def write(self, oprot): oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.max_parts is not None: - oprot.writeFieldBegin('max_parts', TType.I16, 3) - oprot.writeI16(self.max_parts) + if self.new_part is not None: + oprot.writeFieldBegin('new_part', TType.STRUCT, 3) + self.new_part.write(oprot) + oprot.writeFieldEnd() + if self.environment_context is not None: + oprot.writeFieldBegin('environment_context', TType.STRUCT, 4) + self.environment_context.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -12181,20 +14246,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partition_names_result: +class alter_partition_with_environment_context_result: """ Attributes: - - success + - o1 - o2 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 - (1, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 1 + None, # 0 + (1, TType.STRUCT, 'o1', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o2=None,): - self.success = success + def __init__(self, o1=None, o2=None,): + self.o1 = o1 self.o2 = o2 def read(self, iprot): @@ -12206,17 +14272,13 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype388, _size385) = iprot.readListBegin() - for _i389 in xrange(_size385): - _elem390 = iprot.readString(); - self.success.append(_elem390) - iprot.readListEnd() + if fid == 1: + if ftype == TType.STRUCT: + self.o1 = InvalidOperationException() + self.o1.read(iprot) else: iprot.skip(ftype) - elif fid == 1: + elif fid == 2: if ftype == TType.STRUCT: self.o2 = MetaException() self.o2.read(iprot) @@ -12231,16 +14293,13 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partition_names_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRING, len(self.success)) - for iter391 in self.success: - oprot.writeString(iter391) - oprot.writeListEnd() + oprot.writeStructBegin('alter_partition_with_environment_context_result') + if self.o1 is not None: + oprot.writeFieldBegin('o1', TType.STRUCT, 1) + self.o1.write(oprot) oprot.writeFieldEnd() if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 1) + oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() @@ -12261,13 +14320,13 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partitions_ps_args: +class rename_partition_args: """ Attributes: - db_name - tbl_name - part_vals - - max_parts + - new_part """ thrift_spec = ( @@ -12275,14 +14334,14 @@ class get_partitions_ps_args: (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 - (4, TType.I16, 'max_parts', None, -1, ), # 4 + (4, TType.STRUCT, 'new_part', (Partition, Partition.thrift_spec), None, ), # 4 ) - def __init__(self, db_name=None, tbl_name=None, part_vals=None, max_parts=thrift_spec[4][4],): + def __init__(self, db_name=None, tbl_name=None, part_vals=None, new_part=None,): self.db_name = db_name self.tbl_name = tbl_name self.part_vals = part_vals - self.max_parts = max_parts + self.new_part = new_part def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -12306,16 +14365,17 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.part_vals = [] - (_etype395, _size392) = iprot.readListBegin() - for _i396 in xrange(_size392): - _elem397 = iprot.readString(); - self.part_vals.append(_elem397) + (_etype472, _size469) = iprot.readListBegin() + for _i473 in xrange(_size469): + _elem474 = iprot.readString(); + self.part_vals.append(_elem474) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 4: - if ftype == TType.I16: - self.max_parts = iprot.readI16(); + if ftype == TType.STRUCT: + self.new_part = Partition() + self.new_part.read(iprot) else: iprot.skip(ftype) else: @@ -12327,7 +14387,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partitions_ps_args') + oprot.writeStructBegin('rename_partition_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -12339,13 +14399,13 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter398 in self.part_vals: - oprot.writeString(iter398) + for iter475 in self.part_vals: + oprot.writeString(iter475) oprot.writeListEnd() oprot.writeFieldEnd() - if self.max_parts is not None: - oprot.writeFieldBegin('max_parts', TType.I16, 4) - oprot.writeI16(self.max_parts) + if self.new_part is not None: + oprot.writeFieldBegin('new_part', TType.STRUCT, 4) + self.new_part.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -12365,22 +14425,20 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partitions_ps_result: +class rename_partition_result: """ Attributes: - - success - o1 - o2 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 + None, # 0 + (1, TType.STRUCT, 'o1', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None,): - self.success = success + def __init__(self, o1=None, o2=None,): self.o1 = o1 self.o2 = o2 @@ -12393,26 +14451,15 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype402, _size399) = iprot.readListBegin() - for _i403 in xrange(_size399): - _elem404 = Partition() - _elem404.read(iprot) - self.success.append(_elem404) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 1: + if fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = InvalidOperationException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = NoSuchObjectException() + self.o2 = MetaException() self.o2.read(iprot) else: iprot.skip(ftype) @@ -12425,14 +14472,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partitions_ps_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter405 in self.success: - iter405.write(oprot) - oprot.writeListEnd() - oprot.writeFieldEnd() + oprot.writeStructBegin('rename_partition_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -12459,34 +14499,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partitions_ps_with_auth_args: +class partition_name_has_valid_characters_args: """ Attributes: - - db_name - - tbl_name - part_vals - - max_parts - - user_name - - group_names + - throw_exception """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 - (4, TType.I16, 'max_parts', None, -1, ), # 4 - (5, TType.STRING, 'user_name', None, None, ), # 5 - (6, TType.LIST, 'group_names', (TType.STRING,None), None, ), # 6 + (1, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 1 + (2, TType.BOOL, 'throw_exception', None, None, ), # 2 ) - def __init__(self, db_name=None, tbl_name=None, part_vals=None, max_parts=thrift_spec[4][4], user_name=None, group_names=None,): - self.db_name = db_name - self.tbl_name = tbl_name + def __init__(self, part_vals=None, throw_exception=None,): self.part_vals = part_vals - self.max_parts = max_parts - self.user_name = user_name - self.group_names = group_names + self.throw_exception = throw_exception def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -12498,43 +14526,18 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.db_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.tbl_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: if ftype == TType.LIST: self.part_vals = [] - (_etype409, _size406) = iprot.readListBegin() - for _i410 in xrange(_size406): - _elem411 = iprot.readString(); - self.part_vals.append(_elem411) + (_etype479, _size476) = iprot.readListBegin() + for _i480 in xrange(_size476): + _elem481 = iprot.readString(); + self.part_vals.append(_elem481) iprot.readListEnd() else: iprot.skip(ftype) - elif fid == 4: - if ftype == TType.I16: - self.max_parts = iprot.readI16(); - else: - iprot.skip(ftype) - elif fid == 5: - if ftype == TType.STRING: - self.user_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 6: - if ftype == TType.LIST: - self.group_names = [] - (_etype415, _size412) = iprot.readListBegin() - for _i416 in xrange(_size412): - _elem417 = iprot.readString(); - self.group_names.append(_elem417) - iprot.readListEnd() + elif fid == 2: + if ftype == TType.BOOL: + self.throw_exception = iprot.readBool(); else: iprot.skip(ftype) else: @@ -12546,36 +14549,17 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partitions_ps_with_auth_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) - oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) - oprot.writeFieldEnd() + oprot.writeStructBegin('partition_name_has_valid_characters_args') if self.part_vals is not None: - oprot.writeFieldBegin('part_vals', TType.LIST, 3) + oprot.writeFieldBegin('part_vals', TType.LIST, 1) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter418 in self.part_vals: - oprot.writeString(iter418) + for iter482 in self.part_vals: + oprot.writeString(iter482) oprot.writeListEnd() oprot.writeFieldEnd() - if self.max_parts is not None: - oprot.writeFieldBegin('max_parts', TType.I16, 4) - oprot.writeI16(self.max_parts) - oprot.writeFieldEnd() - if self.user_name is not None: - oprot.writeFieldBegin('user_name', TType.STRING, 5) - oprot.writeString(self.user_name) - oprot.writeFieldEnd() - if self.group_names is not None: - oprot.writeFieldBegin('group_names', TType.LIST, 6) - oprot.writeListBegin(TType.STRING, len(self.group_names)) - for iter419 in self.group_names: - oprot.writeString(iter419) - oprot.writeListEnd() + if self.throw_exception is not None: + oprot.writeFieldBegin('throw_exception', TType.BOOL, 2) + oprot.writeBool(self.throw_exception) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -12595,24 +14579,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partitions_ps_with_auth_result: +class partition_name_has_valid_characters_result: """ Attributes: - success - o1 - - o2 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + (0, TType.BOOL, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 ) - def __init__(self, success=None, o1=None, o2=None,): + def __init__(self, success=None, o1=None,): self.success = success self.o1 = o1 - self.o2 = o2 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -12624,28 +14605,16 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype423, _size420) = iprot.readListBegin() - for _i424 in xrange(_size420): - _elem425 = Partition() - _elem425.read(iprot) - self.success.append(_elem425) - iprot.readListEnd() + if ftype == TType.BOOL: + self.success = iprot.readBool(); else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.o2 = MetaException() - self.o2.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -12655,22 +14624,15 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partitions_ps_with_auth_result') + oprot.writeStructBegin('partition_name_has_valid_characters_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter426 in self.success: - iter426.write(oprot) - oprot.writeListEnd() + oprot.writeFieldBegin('success', TType.BOOL, 0) + oprot.writeBool(self.success) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() - if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 2) - self.o2.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -12689,28 +14651,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partition_names_ps_args: +class get_config_value_args: """ Attributes: - - db_name - - tbl_name - - part_vals - - max_parts + - name + - defaultValue """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 - (4, TType.I16, 'max_parts', None, -1, ), # 4 + (1, TType.STRING, 'name', None, None, ), # 1 + (2, TType.STRING, 'defaultValue', None, None, ), # 2 ) - def __init__(self, db_name=None, tbl_name=None, part_vals=None, max_parts=thrift_spec[4][4],): - self.db_name = db_name - self.tbl_name = tbl_name - self.part_vals = part_vals - self.max_parts = max_parts + def __init__(self, name=None, defaultValue=None,): + self.name = name + self.defaultValue = defaultValue def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -12723,27 +14679,12 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.db_name = iprot.readString(); + self.name = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: - self.tbl_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.LIST: - self.part_vals = [] - (_etype430, _size427) = iprot.readListBegin() - for _i431 in xrange(_size427): - _elem432 = iprot.readString(); - self.part_vals.append(_elem432) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.I16: - self.max_parts = iprot.readI16(); + self.defaultValue = iprot.readString(); else: iprot.skip(ftype) else: @@ -12755,25 +14696,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partition_names_ps_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) - oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) - oprot.writeFieldEnd() - if self.part_vals is not None: - oprot.writeFieldBegin('part_vals', TType.LIST, 3) - oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter433 in self.part_vals: - oprot.writeString(iter433) - oprot.writeListEnd() + oprot.writeStructBegin('get_config_value_args') + if self.name is not None: + oprot.writeFieldBegin('name', TType.STRING, 1) + oprot.writeString(self.name) oprot.writeFieldEnd() - if self.max_parts is not None: - oprot.writeFieldBegin('max_parts', TType.I16, 4) - oprot.writeI16(self.max_parts) + if self.defaultValue is not None: + oprot.writeFieldBegin('defaultValue', TType.STRING, 2) + oprot.writeString(self.defaultValue) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -12793,24 +14723,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partition_names_ps_result: +class get_config_value_result: """ Attributes: - success - o1 - - o2 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 + (0, TType.STRING, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (ConfigValSecurityException, ConfigValSecurityException.thrift_spec), None, ), # 1 ) - def __init__(self, success=None, o1=None, o2=None,): + def __init__(self, success=None, o1=None,): self.success = success self.o1 = o1 - self.o2 = o2 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -12822,27 +14749,16 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype437, _size434) = iprot.readListBegin() - for _i438 in xrange(_size434): - _elem439 = iprot.readString(); - self.success.append(_elem439) - iprot.readListEnd() + if ftype == TType.STRING: + self.success = iprot.readString(); else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = ConfigValSecurityException() self.o1.read(iprot) else: iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.o2 = NoSuchObjectException() - self.o2.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -12852,22 +14768,15 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partition_names_ps_result') + oprot.writeStructBegin('get_config_value_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRING, len(self.success)) - for iter440 in self.success: - oprot.writeString(iter440) - oprot.writeListEnd() + oprot.writeFieldBegin('success', TType.STRING, 0) + oprot.writeString(self.success) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() - if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 2) - self.o2.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -12886,28 +14795,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partitions_by_filter_args: +class partition_name_to_vals_args: """ Attributes: - - db_name - - tbl_name - - filter - - max_parts + - part_name """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRING, 'filter', None, None, ), # 3 - (4, TType.I16, 'max_parts', None, -1, ), # 4 + (1, TType.STRING, 'part_name', None, None, ), # 1 ) - def __init__(self, db_name=None, tbl_name=None, filter=None, max_parts=thrift_spec[4][4],): - self.db_name = db_name - self.tbl_name = tbl_name - self.filter = filter - self.max_parts = max_parts + def __init__(self, part_name=None,): + self.part_name = part_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -12920,22 +14820,7 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.db_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.tbl_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRING: - self.filter = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.I16: - self.max_parts = iprot.readI16(); + self.part_name = iprot.readString(); else: iprot.skip(ftype) else: @@ -12947,22 +14832,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partitions_by_filter_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) - oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) - oprot.writeFieldEnd() - if self.filter is not None: - oprot.writeFieldBegin('filter', TType.STRING, 3) - oprot.writeString(self.filter) - oprot.writeFieldEnd() - if self.max_parts is not None: - oprot.writeFieldBegin('max_parts', TType.I16, 4) - oprot.writeI16(self.max_parts) + oprot.writeStructBegin('partition_name_to_vals_args') + if self.part_name is not None: + oprot.writeFieldBegin('part_name', TType.STRING, 1) + oprot.writeString(self.part_name) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -12982,24 +14855,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partitions_by_filter_result: +class partition_name_to_vals_result: """ Attributes: - success - o1 - - o2 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 0 + (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None,): + def __init__(self, success=None, o1=None,): self.success = success self.o1 = o1 - self.o2 = o2 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -13013,11 +14883,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype444, _size441) = iprot.readListBegin() - for _i445 in xrange(_size441): - _elem446 = Partition() - _elem446.read(iprot) - self.success.append(_elem446) + (_etype486, _size483) = iprot.readListBegin() + for _i487 in xrange(_size483): + _elem488 = iprot.readString(); + self.success.append(_elem488) iprot.readListEnd() else: iprot.skip(ftype) @@ -13027,12 +14896,6 @@ def read(self, iprot): self.o1.read(iprot) else: iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.o2 = NoSuchObjectException() - self.o2.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -13042,22 +14905,18 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partitions_by_filter_result') + oprot.writeStructBegin('partition_name_to_vals_result') if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter447 in self.success: - iter447.write(oprot) + oprot.writeListBegin(TType.STRING, len(self.success)) + for iter489 in self.success: + oprot.writeString(iter489) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() - if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 2) - self.o2.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -13076,25 +14935,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partitions_by_names_args: +class partition_name_to_spec_args: """ Attributes: - - db_name - - tbl_name - - names + - part_name """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.LIST, 'names', (TType.STRING,None), None, ), # 3 + (1, TType.STRING, 'part_name', None, None, ), # 1 ) - def __init__(self, db_name=None, tbl_name=None, names=None,): - self.db_name = db_name - self.tbl_name = tbl_name - self.names = names + def __init__(self, part_name=None,): + self.part_name = part_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -13107,22 +14960,7 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.db_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.tbl_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.LIST: - self.names = [] - (_etype451, _size448) = iprot.readListBegin() - for _i452 in xrange(_size448): - _elem453 = iprot.readString(); - self.names.append(_elem453) - iprot.readListEnd() + self.part_name = iprot.readString(); else: iprot.skip(ftype) else: @@ -13134,21 +14972,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partitions_by_names_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) - oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) - oprot.writeFieldEnd() - if self.names is not None: - oprot.writeFieldBegin('names', TType.LIST, 3) - oprot.writeListBegin(TType.STRING, len(self.names)) - for iter454 in self.names: - oprot.writeString(iter454) - oprot.writeListEnd() + oprot.writeStructBegin('partition_name_to_spec_args') + if self.part_name is not None: + oprot.writeFieldBegin('part_name', TType.STRING, 1) + oprot.writeString(self.part_name) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -13168,24 +14995,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partitions_by_names_result: +class partition_name_to_spec_result: """ Attributes: - success - o1 - - o2 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 0 + (0, TType.MAP, 'success', (TType.STRING,None,TType.STRING,None), None, ), # 0 (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None,): + def __init__(self, success=None, o1=None,): self.success = success self.o1 = o1 - self.o2 = o2 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -13197,14 +15021,14 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype458, _size455) = iprot.readListBegin() - for _i459 in xrange(_size455): - _elem460 = Partition() - _elem460.read(iprot) - self.success.append(_elem460) - iprot.readListEnd() + if ftype == TType.MAP: + self.success = {} + (_ktype491, _vtype492, _size490 ) = iprot.readMapBegin() + for _i494 in xrange(_size490): + _key495 = iprot.readString(); + _val496 = iprot.readString(); + self.success[_key495] = _val496 + iprot.readMapEnd() else: iprot.skip(ftype) elif fid == 1: @@ -13213,12 +15037,6 @@ def read(self, iprot): self.o1.read(iprot) else: iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.o2 = NoSuchObjectException() - self.o2.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -13228,22 +15046,19 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partitions_by_names_result') + oprot.writeStructBegin('partition_name_to_spec_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter461 in self.success: - iter461.write(oprot) - oprot.writeListEnd() + oprot.writeFieldBegin('success', TType.MAP, 0) + oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.success)) + for kiter497,viter498 in self.success.items(): + oprot.writeString(kiter497) + oprot.writeString(viter498) + oprot.writeMapEnd() oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() - if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 2) - self.o2.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -13262,25 +15077,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_partition_args: +class markPartitionForEvent_args: """ Attributes: - db_name - tbl_name - - new_part + - part_vals + - eventType """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRUCT, 'new_part', (Partition, Partition.thrift_spec), None, ), # 3 + (3, TType.MAP, 'part_vals', (TType.STRING,None,TType.STRING,None), None, ), # 3 + (4, TType.I32, 'eventType', None, None, ), # 4 ) - def __init__(self, db_name=None, tbl_name=None, new_part=None,): + def __init__(self, db_name=None, tbl_name=None, part_vals=None, eventType=None,): self.db_name = db_name self.tbl_name = tbl_name - self.new_part = new_part + self.part_vals = part_vals + self.eventType = eventType def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -13302,9 +15120,19 @@ def read(self, iprot): else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.STRUCT: - self.new_part = Partition() - self.new_part.read(iprot) + if ftype == TType.MAP: + self.part_vals = {} + (_ktype500, _vtype501, _size499 ) = iprot.readMapBegin() + for _i503 in xrange(_size499): + _key504 = iprot.readString(); + _val505 = iprot.readString(); + self.part_vals[_key504] = _val505 + iprot.readMapEnd() + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.I32: + self.eventType = iprot.readI32(); else: iprot.skip(ftype) else: @@ -13316,7 +15144,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_partition_args') + oprot.writeStructBegin('markPartitionForEvent_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -13325,9 +15153,17 @@ def write(self, oprot): oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.new_part is not None: - oprot.writeFieldBegin('new_part', TType.STRUCT, 3) - self.new_part.write(oprot) + if self.part_vals is not None: + oprot.writeFieldBegin('part_vals', TType.MAP, 3) + oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals)) + for kiter506,viter507 in self.part_vals.items(): + oprot.writeString(kiter506) + oprot.writeString(viter507) + oprot.writeMapEnd() + oprot.writeFieldEnd() + if self.eventType is not None: + oprot.writeFieldBegin('eventType', TType.I32, 4) + oprot.writeI32(self.eventType) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -13347,22 +15183,34 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_partition_result: +class markPartitionForEvent_result: """ Attributes: - o1 - o2 + - o3 + - o4 + - o5 + - o6 """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'o1', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (UnknownDBException, UnknownDBException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (UnknownTableException, UnknownTableException.thrift_spec), None, ), # 4 + (5, TType.STRUCT, 'o5', (UnknownPartitionException, UnknownPartitionException.thrift_spec), None, ), # 5 + (6, TType.STRUCT, 'o6', (InvalidPartitionException, InvalidPartitionException.thrift_spec), None, ), # 6 ) - def __init__(self, o1=None, o2=None,): + def __init__(self, o1=None, o2=None, o3=None, o4=None, o5=None, o6=None,): self.o1 = o1 self.o2 = o2 + self.o3 = o3 + self.o4 = o4 + self.o5 = o5 + self.o6 = o6 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -13375,16 +15223,40 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRUCT: - self.o1 = InvalidOperationException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = MetaException() + self.o2 = NoSuchObjectException() self.o2.read(iprot) else: iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = UnknownDBException() + self.o3.read(iprot) + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.o4 = UnknownTableException() + self.o4.read(iprot) + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.STRUCT: + self.o5 = UnknownPartitionException() + self.o5.read(iprot) + else: + iprot.skip(ftype) + elif fid == 6: + if ftype == TType.STRUCT: + self.o6 = InvalidPartitionException() + self.o6.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -13394,7 +15266,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_partition_result') + oprot.writeStructBegin('markPartitionForEvent_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -13403,6 +15275,22 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() + if self.o4 is not None: + oprot.writeFieldBegin('o4', TType.STRUCT, 4) + self.o4.write(oprot) + oprot.writeFieldEnd() + if self.o5 is not None: + oprot.writeFieldBegin('o5', TType.STRUCT, 5) + self.o5.write(oprot) + oprot.writeFieldEnd() + if self.o6 is not None: + oprot.writeFieldBegin('o6', TType.STRUCT, 6) + self.o6.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -13421,25 +15309,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_partitions_args: +class isPartitionMarkedForEvent_args: """ Attributes: - db_name - tbl_name - - new_parts + - part_vals + - eventType """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.LIST, 'new_parts', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 3 + (3, TType.MAP, 'part_vals', (TType.STRING,None,TType.STRING,None), None, ), # 3 + (4, TType.I32, 'eventType', None, None, ), # 4 ) - def __init__(self, db_name=None, tbl_name=None, new_parts=None,): + def __init__(self, db_name=None, tbl_name=None, part_vals=None, eventType=None,): self.db_name = db_name self.tbl_name = tbl_name - self.new_parts = new_parts + self.part_vals = part_vals + self.eventType = eventType def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -13460,15 +15351,20 @@ def read(self, iprot): self.tbl_name = iprot.readString(); else: iprot.skip(ftype) - elif fid == 3: - if ftype == TType.LIST: - self.new_parts = [] - (_etype465, _size462) = iprot.readListBegin() - for _i466 in xrange(_size462): - _elem467 = Partition() - _elem467.read(iprot) - self.new_parts.append(_elem467) - iprot.readListEnd() + elif fid == 3: + if ftype == TType.MAP: + self.part_vals = {} + (_ktype509, _vtype510, _size508 ) = iprot.readMapBegin() + for _i512 in xrange(_size508): + _key513 = iprot.readString(); + _val514 = iprot.readString(); + self.part_vals[_key513] = _val514 + iprot.readMapEnd() + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.I32: + self.eventType = iprot.readI32(); else: iprot.skip(ftype) else: @@ -13480,7 +15376,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_partitions_args') + oprot.writeStructBegin('isPartitionMarkedForEvent_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -13489,12 +15385,17 @@ def write(self, oprot): oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.new_parts is not None: - oprot.writeFieldBegin('new_parts', TType.LIST, 3) - oprot.writeListBegin(TType.STRUCT, len(self.new_parts)) - for iter468 in self.new_parts: - iter468.write(oprot) - oprot.writeListEnd() + if self.part_vals is not None: + oprot.writeFieldBegin('part_vals', TType.MAP, 3) + oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals)) + for kiter515,viter516 in self.part_vals.items(): + oprot.writeString(kiter515) + oprot.writeString(viter516) + oprot.writeMapEnd() + oprot.writeFieldEnd() + if self.eventType is not None: + oprot.writeFieldBegin('eventType', TType.I32, 4) + oprot.writeI32(self.eventType) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -13514,22 +15415,36 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_partitions_result: +class isPartitionMarkedForEvent_result: """ Attributes: + - success - o1 - o2 + - o3 + - o4 + - o5 + - o6 """ thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'o1', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + (0, TType.BOOL, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (UnknownDBException, UnknownDBException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (UnknownTableException, UnknownTableException.thrift_spec), None, ), # 4 + (5, TType.STRUCT, 'o5', (UnknownPartitionException, UnknownPartitionException.thrift_spec), None, ), # 5 + (6, TType.STRUCT, 'o6', (InvalidPartitionException, InvalidPartitionException.thrift_spec), None, ), # 6 ) - def __init__(self, o1=None, o2=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None, o5=None, o6=None,): + self.success = success self.o1 = o1 self.o2 = o2 + self.o3 = o3 + self.o4 = o4 + self.o5 = o5 + self.o6 = o6 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -13540,18 +15455,47 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 1: + if fid == 0: + if ftype == TType.BOOL: + self.success = iprot.readBool(); + else: + iprot.skip(ftype) + elif fid == 1: if ftype == TType.STRUCT: - self.o1 = InvalidOperationException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = MetaException() + self.o2 = NoSuchObjectException() self.o2.read(iprot) else: iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = UnknownDBException() + self.o3.read(iprot) + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.o4 = UnknownTableException() + self.o4.read(iprot) + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.STRUCT: + self.o5 = UnknownPartitionException() + self.o5.read(iprot) + else: + iprot.skip(ftype) + elif fid == 6: + if ftype == TType.STRUCT: + self.o6 = InvalidPartitionException() + self.o6.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -13561,7 +15505,11 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_partitions_result') + oprot.writeStructBegin('isPartitionMarkedForEvent_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.BOOL, 0) + oprot.writeBool(self.success) + oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -13570,6 +15518,22 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() + if self.o4 is not None: + oprot.writeFieldBegin('o4', TType.STRUCT, 4) + self.o4.write(oprot) + oprot.writeFieldEnd() + if self.o5 is not None: + oprot.writeFieldBegin('o5', TType.STRUCT, 5) + self.o5.write(oprot) + oprot.writeFieldEnd() + if self.o6 is not None: + oprot.writeFieldBegin('o6', TType.STRUCT, 6) + self.o6.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -13588,28 +15552,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_partition_with_environment_context_args: +class add_index_args: """ Attributes: - - db_name - - tbl_name - - new_part - - environment_context + - new_index + - index_table """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRUCT, 'new_part', (Partition, Partition.thrift_spec), None, ), # 3 - (4, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 4 + (1, TType.STRUCT, 'new_index', (Index, Index.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'index_table', (Table, Table.thrift_spec), None, ), # 2 ) - def __init__(self, db_name=None, tbl_name=None, new_part=None, environment_context=None,): - self.db_name = db_name - self.tbl_name = tbl_name - self.new_part = new_part - self.environment_context = environment_context + def __init__(self, new_index=None, index_table=None,): + self.new_index = new_index + self.index_table = index_table def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -13621,25 +15579,15 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.db_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.tbl_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: if ftype == TType.STRUCT: - self.new_part = Partition() - self.new_part.read(iprot) + self.new_index = Index() + self.new_index.read(iprot) else: iprot.skip(ftype) - elif fid == 4: + elif fid == 2: if ftype == TType.STRUCT: - self.environment_context = EnvironmentContext() - self.environment_context.read(iprot) + self.index_table = Table() + self.index_table.read(iprot) else: iprot.skip(ftype) else: @@ -13651,22 +15599,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_partition_with_environment_context_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) - oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) - oprot.writeFieldEnd() - if self.new_part is not None: - oprot.writeFieldBegin('new_part', TType.STRUCT, 3) - self.new_part.write(oprot) + oprot.writeStructBegin('add_index_args') + if self.new_index is not None: + oprot.writeFieldBegin('new_index', TType.STRUCT, 1) + self.new_index.write(oprot) oprot.writeFieldEnd() - if self.environment_context is not None: - oprot.writeFieldBegin('environment_context', TType.STRUCT, 4) - self.environment_context.write(oprot) + if self.index_table is not None: + oprot.writeFieldBegin('index_table', TType.STRUCT, 2) + self.index_table.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -13686,22 +15626,27 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_partition_with_environment_context_result: +class add_index_result: """ Attributes: + - success - o1 - o2 + - o3 """ thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'o1', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + (0, TType.STRUCT, 'success', (Index, Index.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 ) - def __init__(self, o1=None, o2=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None,): + self.success = success self.o1 = o1 self.o2 = o2 + self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -13712,18 +15657,30 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 1: + if fid == 0: if ftype == TType.STRUCT: - self.o1 = InvalidOperationException() + self.success = Index() + self.success.read(iprot) + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.o1 = InvalidObjectException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = MetaException() + self.o2 = AlreadyExistsException() self.o2.read(iprot) else: iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = MetaException() + self.o3.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -13733,7 +15690,11 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_partition_with_environment_context_result') + oprot.writeStructBegin('add_index_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) + oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -13742,6 +15703,10 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -13760,28 +15725,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class rename_partition_args: +class alter_index_args: """ Attributes: - - db_name - - tbl_name - - part_vals - - new_part + - dbname + - base_tbl_name + - idx_name + - new_idx """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 - (4, TType.STRUCT, 'new_part', (Partition, Partition.thrift_spec), None, ), # 4 + (1, TType.STRING, 'dbname', None, None, ), # 1 + (2, TType.STRING, 'base_tbl_name', None, None, ), # 2 + (3, TType.STRING, 'idx_name', None, None, ), # 3 + (4, TType.STRUCT, 'new_idx', (Index, Index.thrift_spec), None, ), # 4 ) - def __init__(self, db_name=None, tbl_name=None, part_vals=None, new_part=None,): - self.db_name = db_name - self.tbl_name = tbl_name - self.part_vals = part_vals - self.new_part = new_part + def __init__(self, dbname=None, base_tbl_name=None, idx_name=None, new_idx=None,): + self.dbname = dbname + self.base_tbl_name = base_tbl_name + self.idx_name = idx_name + self.new_idx = new_idx def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -13794,28 +15759,23 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.db_name = iprot.readString(); + self.dbname = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: - self.tbl_name = iprot.readString(); + self.base_tbl_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.LIST: - self.part_vals = [] - (_etype472, _size469) = iprot.readListBegin() - for _i473 in xrange(_size469): - _elem474 = iprot.readString(); - self.part_vals.append(_elem474) - iprot.readListEnd() + if ftype == TType.STRING: + self.idx_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 4: if ftype == TType.STRUCT: - self.new_part = Partition() - self.new_part.read(iprot) + self.new_idx = Index() + self.new_idx.read(iprot) else: iprot.skip(ftype) else: @@ -13827,25 +15787,22 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('rename_partition_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) + oprot.writeStructBegin('alter_index_args') + if self.dbname is not None: + oprot.writeFieldBegin('dbname', TType.STRING, 1) + oprot.writeString(self.dbname) oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) + if self.base_tbl_name is not None: + oprot.writeFieldBegin('base_tbl_name', TType.STRING, 2) + oprot.writeString(self.base_tbl_name) oprot.writeFieldEnd() - if self.part_vals is not None: - oprot.writeFieldBegin('part_vals', TType.LIST, 3) - oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter475 in self.part_vals: - oprot.writeString(iter475) - oprot.writeListEnd() + if self.idx_name is not None: + oprot.writeFieldBegin('idx_name', TType.STRING, 3) + oprot.writeString(self.idx_name) oprot.writeFieldEnd() - if self.new_part is not None: - oprot.writeFieldBegin('new_part', TType.STRUCT, 4) - self.new_part.write(oprot) + if self.new_idx is not None: + oprot.writeFieldBegin('new_idx', TType.STRUCT, 4) + self.new_idx.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -13865,7 +15822,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class rename_partition_result: +class alter_index_result: """ Attributes: - o1 @@ -13912,7 +15869,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('rename_partition_result') + oprot.writeStructBegin('alter_index_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -13939,22 +15896,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class partition_name_has_valid_characters_args: +class drop_index_by_name_args: """ Attributes: - - part_vals - - throw_exception + - db_name + - tbl_name + - index_name + - deleteData """ thrift_spec = ( None, # 0 - (1, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 1 - (2, TType.BOOL, 'throw_exception', None, None, ), # 2 + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.STRING, 'index_name', None, None, ), # 3 + (4, TType.BOOL, 'deleteData', None, None, ), # 4 ) - def __init__(self, part_vals=None, throw_exception=None,): - self.part_vals = part_vals - self.throw_exception = throw_exception + def __init__(self, db_name=None, tbl_name=None, index_name=None, deleteData=None,): + self.db_name = db_name + self.tbl_name = tbl_name + self.index_name = index_name + self.deleteData = deleteData def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -13966,18 +15929,23 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.LIST: - self.part_vals = [] - (_etype479, _size476) = iprot.readListBegin() - for _i480 in xrange(_size476): - _elem481 = iprot.readString(); - self.part_vals.append(_elem481) - iprot.readListEnd() + if ftype == TType.STRING: + self.db_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: + if ftype == TType.STRING: + self.tbl_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRING: + self.index_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 4: if ftype == TType.BOOL: - self.throw_exception = iprot.readBool(); + self.deleteData = iprot.readBool(); else: iprot.skip(ftype) else: @@ -13989,17 +15957,22 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('partition_name_has_valid_characters_args') - if self.part_vals is not None: - oprot.writeFieldBegin('part_vals', TType.LIST, 1) - oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter482 in self.part_vals: - oprot.writeString(iter482) - oprot.writeListEnd() + oprot.writeStructBegin('drop_index_by_name_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) oprot.writeFieldEnd() - if self.throw_exception is not None: - oprot.writeFieldBegin('throw_exception', TType.BOOL, 2) - oprot.writeBool(self.throw_exception) + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) + oprot.writeFieldEnd() + if self.index_name is not None: + oprot.writeFieldBegin('index_name', TType.STRING, 3) + oprot.writeString(self.index_name) + oprot.writeFieldEnd() + if self.deleteData is not None: + oprot.writeFieldBegin('deleteData', TType.BOOL, 4) + oprot.writeBool(self.deleteData) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -14019,21 +15992,24 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class partition_name_has_valid_characters_result: +class drop_index_by_name_result: """ Attributes: - success - o1 + - o2 """ thrift_spec = ( (0, TType.BOOL, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None,): + def __init__(self, success=None, o1=None, o2=None,): self.success = success self.o1 = o1 + self.o2 = o2 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -14051,10 +16027,16 @@ def read(self, iprot): iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = MetaException() + self.o2.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -14064,7 +16046,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('partition_name_has_valid_characters_result') + oprot.writeStructBegin('drop_index_by_name_result') if self.success is not None: oprot.writeFieldBegin('success', TType.BOOL, 0) oprot.writeBool(self.success) @@ -14073,6 +16055,10 @@ def write(self, oprot): oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -14091,22 +16077,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_config_value_args: +class get_index_by_name_args: """ Attributes: - - name - - defaultValue + - db_name + - tbl_name + - index_name """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'name', None, None, ), # 1 - (2, TType.STRING, 'defaultValue', None, None, ), # 2 + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.STRING, 'index_name', None, None, ), # 3 ) - def __init__(self, name=None, defaultValue=None,): - self.name = name - self.defaultValue = defaultValue + def __init__(self, db_name=None, tbl_name=None, index_name=None,): + self.db_name = db_name + self.tbl_name = tbl_name + self.index_name = index_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -14119,12 +16108,17 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.name = iprot.readString(); + self.db_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: - self.defaultValue = iprot.readString(); + self.tbl_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRING: + self.index_name = iprot.readString(); else: iprot.skip(ftype) else: @@ -14136,14 +16130,18 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_config_value_args') - if self.name is not None: - oprot.writeFieldBegin('name', TType.STRING, 1) - oprot.writeString(self.name) + oprot.writeStructBegin('get_index_by_name_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) oprot.writeFieldEnd() - if self.defaultValue is not None: - oprot.writeFieldBegin('defaultValue', TType.STRING, 2) - oprot.writeString(self.defaultValue) + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) + oprot.writeFieldEnd() + if self.index_name is not None: + oprot.writeFieldBegin('index_name', TType.STRING, 3) + oprot.writeString(self.index_name) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -14163,21 +16161,24 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_config_value_result: +class get_index_by_name_result: """ Attributes: - success - o1 + - o2 """ thrift_spec = ( - (0, TType.STRING, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (ConfigValSecurityException, ConfigValSecurityException.thrift_spec), None, ), # 1 + (0, TType.STRUCT, 'success', (Index, Index.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None,): + def __init__(self, success=None, o1=None, o2=None,): self.success = success self.o1 = o1 + self.o2 = o2 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -14189,16 +16190,23 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.STRING: - self.success = iprot.readString(); + if ftype == TType.STRUCT: + self.success = Index() + self.success.read(iprot) else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = ConfigValSecurityException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = NoSuchObjectException() + self.o2.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -14208,15 +16216,19 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_config_value_result') + oprot.writeStructBegin('get_index_by_name_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.STRING, 0) - oprot.writeString(self.success) + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -14235,19 +16247,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class partition_name_to_vals_args: +class get_indexes_args: """ Attributes: - - part_name + - db_name + - tbl_name + - max_indexes """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'part_name', None, None, ), # 1 + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.I16, 'max_indexes', None, -1, ), # 3 ) - def __init__(self, part_name=None,): - self.part_name = part_name + def __init__(self, db_name=None, tbl_name=None, max_indexes=thrift_spec[3][4],): + self.db_name = db_name + self.tbl_name = tbl_name + self.max_indexes = max_indexes def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -14260,7 +16278,17 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.part_name = iprot.readString(); + self.db_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.tbl_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.I16: + self.max_indexes = iprot.readI16(); else: iprot.skip(ftype) else: @@ -14272,10 +16300,18 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('partition_name_to_vals_args') - if self.part_name is not None: - oprot.writeFieldBegin('part_name', TType.STRING, 1) - oprot.writeString(self.part_name) + oprot.writeStructBegin('get_indexes_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) + oprot.writeFieldEnd() + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) + oprot.writeFieldEnd() + if self.max_indexes is not None: + oprot.writeFieldBegin('max_indexes', TType.I16, 3) + oprot.writeI16(self.max_indexes) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -14295,21 +16331,24 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class partition_name_to_vals_result: +class get_indexes_result: """ Attributes: - success - o1 + - o2 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (0, TType.LIST, 'success', (TType.STRUCT,(Index, Index.thrift_spec)), None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None,): + def __init__(self, success=None, o1=None, o2=None,): self.success = success self.o1 = o1 + self.o2 = o2 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -14323,19 +16362,26 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype486, _size483) = iprot.readListBegin() - for _i487 in xrange(_size483): - _elem488 = iprot.readString(); - self.success.append(_elem488) + (_etype520, _size517) = iprot.readListBegin() + for _i521 in xrange(_size517): + _elem522 = Index() + _elem522.read(iprot) + self.success.append(_elem522) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = MetaException() + self.o2.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -14345,18 +16391,22 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('partition_name_to_vals_result') + oprot.writeStructBegin('get_indexes_result') if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRING, len(self.success)) - for iter489 in self.success: - oprot.writeString(iter489) + oprot.writeListBegin(TType.STRUCT, len(self.success)) + for iter523 in self.success: + iter523.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -14375,19 +16425,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class partition_name_to_spec_args: +class get_index_names_args: """ Attributes: - - part_name + - db_name + - tbl_name + - max_indexes """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'part_name', None, None, ), # 1 + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.I16, 'max_indexes', None, -1, ), # 3 ) - def __init__(self, part_name=None,): - self.part_name = part_name + def __init__(self, db_name=None, tbl_name=None, max_indexes=thrift_spec[3][4],): + self.db_name = db_name + self.tbl_name = tbl_name + self.max_indexes = max_indexes def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -14400,7 +16456,17 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.part_name = iprot.readString(); + self.db_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.tbl_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.I16: + self.max_indexes = iprot.readI16(); else: iprot.skip(ftype) else: @@ -14412,10 +16478,18 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('partition_name_to_spec_args') - if self.part_name is not None: - oprot.writeFieldBegin('part_name', TType.STRING, 1) - oprot.writeString(self.part_name) + oprot.writeStructBegin('get_index_names_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) + oprot.writeFieldEnd() + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) + oprot.writeFieldEnd() + if self.max_indexes is not None: + oprot.writeFieldBegin('max_indexes', TType.I16, 3) + oprot.writeI16(self.max_indexes) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -14435,21 +16509,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class partition_name_to_spec_result: +class get_index_names_result: """ Attributes: - success - - o1 + - o2 """ thrift_spec = ( - (0, TType.MAP, 'success', (TType.STRING,None,TType.STRING,None), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 + (1, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 1 ) - def __init__(self, success=None, o1=None,): + def __init__(self, success=None, o2=None,): self.success = success - self.o1 = o1 + self.o2 = o2 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -14461,20 +16535,19 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.MAP: - self.success = {} - (_ktype491, _vtype492, _size490 ) = iprot.readMapBegin() - for _i494 in xrange(_size490): - _key495 = iprot.readString(); - _val496 = iprot.readString(); - self.success[_key495] = _val496 - iprot.readMapEnd() + if ftype == TType.LIST: + self.success = [] + (_etype527, _size524) = iprot.readListBegin() + for _i528 in xrange(_size524): + _elem529 = iprot.readString(); + self.success.append(_elem529) + iprot.readListEnd() else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() - self.o1.read(iprot) + self.o2 = MetaException() + self.o2.read(iprot) else: iprot.skip(ftype) else: @@ -14486,18 +16559,17 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('partition_name_to_spec_result') + oprot.writeStructBegin('get_index_names_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.MAP, 0) - oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.success)) - for kiter497,viter498 in self.success.items(): - oprot.writeString(kiter497) - oprot.writeString(viter498) - oprot.writeMapEnd() + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRING, len(self.success)) + for iter530 in self.success: + oprot.writeString(iter530) + oprot.writeListEnd() oprot.writeFieldEnd() - if self.o1 is not None: - oprot.writeFieldBegin('o1', TType.STRUCT, 1) - self.o1.write(oprot) + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 1) + self.o2.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -14517,28 +16589,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class markPartitionForEvent_args: +class update_table_column_statistics_args: """ Attributes: - - db_name - - tbl_name - - part_vals - - eventType + - stats_obj """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.MAP, 'part_vals', (TType.STRING,None,TType.STRING,None), None, ), # 3 - (4, TType.I32, 'eventType', None, None, ), # 4 + (1, TType.STRUCT, 'stats_obj', (ColumnStatistics, ColumnStatistics.thrift_spec), None, ), # 1 ) - def __init__(self, db_name=None, tbl_name=None, part_vals=None, eventType=None,): - self.db_name = db_name - self.tbl_name = tbl_name - self.part_vals = part_vals - self.eventType = eventType + def __init__(self, stats_obj=None,): + self.stats_obj = stats_obj def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -14550,29 +16613,9 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.db_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.tbl_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.MAP: - self.part_vals = {} - (_ktype500, _vtype501, _size499 ) = iprot.readMapBegin() - for _i503 in xrange(_size499): - _key504 = iprot.readString(); - _val505 = iprot.readString(); - self.part_vals[_key504] = _val505 - iprot.readMapEnd() - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.I32: - self.eventType = iprot.readI32(); + if ftype == TType.STRUCT: + self.stats_obj = ColumnStatistics() + self.stats_obj.read(iprot) else: iprot.skip(ftype) else: @@ -14584,26 +16627,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('markPartitionForEvent_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) - oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) - oprot.writeFieldEnd() - if self.part_vals is not None: - oprot.writeFieldBegin('part_vals', TType.MAP, 3) - oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals)) - for kiter506,viter507 in self.part_vals.items(): - oprot.writeString(kiter506) - oprot.writeString(viter507) - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.eventType is not None: - oprot.writeFieldBegin('eventType', TType.I32, 4) - oprot.writeI32(self.eventType) + oprot.writeStructBegin('update_table_column_statistics_args') + if self.stats_obj is not None: + oprot.writeFieldBegin('stats_obj', TType.STRUCT, 1) + self.stats_obj.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -14623,34 +16650,30 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class markPartitionForEvent_result: +class update_table_column_statistics_result: """ Attributes: + - success - o1 - o2 - o3 - o4 - - o5 - - o6 """ thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (UnknownDBException, UnknownDBException.thrift_spec), None, ), # 3 - (4, TType.STRUCT, 'o4', (UnknownTableException, UnknownTableException.thrift_spec), None, ), # 4 - (5, TType.STRUCT, 'o5', (UnknownPartitionException, UnknownPartitionException.thrift_spec), None, ), # 5 - (6, TType.STRUCT, 'o6', (InvalidPartitionException, InvalidPartitionException.thrift_spec), None, ), # 6 + (0, TType.BOOL, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 4 ) - def __init__(self, o1=None, o2=None, o3=None, o4=None, o5=None, o6=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None,): + self.success = success self.o1 = o1 self.o2 = o2 self.o3 = o3 self.o4 = o4 - self.o5 = o5 - self.o6 = o6 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -14661,40 +16684,33 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 1: - if ftype == TType.STRUCT: - self.o1 = MetaException() - self.o1.read(iprot) - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.o2 = NoSuchObjectException() - self.o2.read(iprot) + if fid == 0: + if ftype == TType.BOOL: + self.success = iprot.readBool(); else: iprot.skip(ftype) - elif fid == 3: + elif fid == 1: if ftype == TType.STRUCT: - self.o3 = UnknownDBException() - self.o3.read(iprot) + self.o1 = NoSuchObjectException() + self.o1.read(iprot) else: iprot.skip(ftype) - elif fid == 4: + elif fid == 2: if ftype == TType.STRUCT: - self.o4 = UnknownTableException() - self.o4.read(iprot) + self.o2 = InvalidObjectException() + self.o2.read(iprot) else: iprot.skip(ftype) - elif fid == 5: + elif fid == 3: if ftype == TType.STRUCT: - self.o5 = UnknownPartitionException() - self.o5.read(iprot) + self.o3 = MetaException() + self.o3.read(iprot) else: iprot.skip(ftype) - elif fid == 6: + elif fid == 4: if ftype == TType.STRUCT: - self.o6 = InvalidPartitionException() - self.o6.read(iprot) + self.o4 = InvalidInputException() + self.o4.read(iprot) else: iprot.skip(ftype) else: @@ -14706,7 +16722,11 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('markPartitionForEvent_result') + oprot.writeStructBegin('update_table_column_statistics_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.BOOL, 0) + oprot.writeBool(self.success) + oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -14723,14 +16743,6 @@ def write(self, oprot): oprot.writeFieldBegin('o4', TType.STRUCT, 4) self.o4.write(oprot) oprot.writeFieldEnd() - if self.o5 is not None: - oprot.writeFieldBegin('o5', TType.STRUCT, 5) - self.o5.write(oprot) - oprot.writeFieldEnd() - if self.o6 is not None: - oprot.writeFieldBegin('o6', TType.STRUCT, 6) - self.o6.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -14749,28 +16761,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class isPartitionMarkedForEvent_args: +class update_partition_column_statistics_args: """ Attributes: - - db_name - - tbl_name - - part_vals - - eventType + - stats_obj """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.MAP, 'part_vals', (TType.STRING,None,TType.STRING,None), None, ), # 3 - (4, TType.I32, 'eventType', None, None, ), # 4 + (1, TType.STRUCT, 'stats_obj', (ColumnStatistics, ColumnStatistics.thrift_spec), None, ), # 1 ) - def __init__(self, db_name=None, tbl_name=None, part_vals=None, eventType=None,): - self.db_name = db_name - self.tbl_name = tbl_name - self.part_vals = part_vals - self.eventType = eventType + def __init__(self, stats_obj=None,): + self.stats_obj = stats_obj def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -14782,29 +16785,9 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.db_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.tbl_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.MAP: - self.part_vals = {} - (_ktype509, _vtype510, _size508 ) = iprot.readMapBegin() - for _i512 in xrange(_size508): - _key513 = iprot.readString(); - _val514 = iprot.readString(); - self.part_vals[_key513] = _val514 - iprot.readMapEnd() - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.I32: - self.eventType = iprot.readI32(); + if ftype == TType.STRUCT: + self.stats_obj = ColumnStatistics() + self.stats_obj.read(iprot) else: iprot.skip(ftype) else: @@ -14816,26 +16799,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('isPartitionMarkedForEvent_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) - oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) - oprot.writeFieldEnd() - if self.part_vals is not None: - oprot.writeFieldBegin('part_vals', TType.MAP, 3) - oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals)) - for kiter515,viter516 in self.part_vals.items(): - oprot.writeString(kiter515) - oprot.writeString(viter516) - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.eventType is not None: - oprot.writeFieldBegin('eventType', TType.I32, 4) - oprot.writeI32(self.eventType) + oprot.writeStructBegin('update_partition_column_statistics_args') + if self.stats_obj is not None: + oprot.writeFieldBegin('stats_obj', TType.STRUCT, 1) + self.stats_obj.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -14855,7 +16822,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class isPartitionMarkedForEvent_result: +class update_partition_column_statistics_result: """ Attributes: - success @@ -14863,28 +16830,22 @@ class isPartitionMarkedForEvent_result: - o2 - o3 - o4 - - o5 - - o6 """ thrift_spec = ( (0, TType.BOOL, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (UnknownDBException, UnknownDBException.thrift_spec), None, ), # 3 - (4, TType.STRUCT, 'o4', (UnknownTableException, UnknownTableException.thrift_spec), None, ), # 4 - (5, TType.STRUCT, 'o5', (UnknownPartitionException, UnknownPartitionException.thrift_spec), None, ), # 5 - (6, TType.STRUCT, 'o6', (InvalidPartitionException, InvalidPartitionException.thrift_spec), None, ), # 6 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 4 ) - def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None, o5=None, o6=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None,): self.success = success self.o1 = o1 self.o2 = o2 self.o3 = o3 self.o4 = o4 - self.o5 = o5 - self.o6 = o6 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -14902,40 +16863,28 @@ def read(self, iprot): iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = NoSuchObjectException() + self.o2 = InvalidObjectException() self.o2.read(iprot) else: iprot.skip(ftype) elif fid == 3: if ftype == TType.STRUCT: - self.o3 = UnknownDBException() + self.o3 = MetaException() self.o3.read(iprot) else: iprot.skip(ftype) elif fid == 4: if ftype == TType.STRUCT: - self.o4 = UnknownTableException() + self.o4 = InvalidInputException() self.o4.read(iprot) else: iprot.skip(ftype) - elif fid == 5: - if ftype == TType.STRUCT: - self.o5 = UnknownPartitionException() - self.o5.read(iprot) - else: - iprot.skip(ftype) - elif fid == 6: - if ftype == TType.STRUCT: - self.o6 = InvalidPartitionException() - self.o6.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -14945,7 +16894,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('isPartitionMarkedForEvent_result') + oprot.writeStructBegin('update_partition_column_statistics_result') if self.success is not None: oprot.writeFieldBegin('success', TType.BOOL, 0) oprot.writeBool(self.success) @@ -14966,14 +16915,6 @@ def write(self, oprot): oprot.writeFieldBegin('o4', TType.STRUCT, 4) self.o4.write(oprot) oprot.writeFieldEnd() - if self.o5 is not None: - oprot.writeFieldBegin('o5', TType.STRUCT, 5) - self.o5.write(oprot) - oprot.writeFieldEnd() - if self.o6 is not None: - oprot.writeFieldBegin('o6', TType.STRUCT, 6) - self.o6.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -14992,22 +16933,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class add_index_args: +class get_table_column_statistics_args: """ Attributes: - - new_index - - index_table + - db_name + - tbl_name + - col_name """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'new_index', (Index, Index.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'index_table', (Table, Table.thrift_spec), None, ), # 2 + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.STRING, 'col_name', None, None, ), # 3 ) - def __init__(self, new_index=None, index_table=None,): - self.new_index = new_index - self.index_table = index_table + def __init__(self, db_name=None, tbl_name=None, col_name=None,): + self.db_name = db_name + self.tbl_name = tbl_name + self.col_name = col_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -15019,15 +16963,18 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRUCT: - self.new_index = Index() - self.new_index.read(iprot) + if ftype == TType.STRING: + self.db_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: - if ftype == TType.STRUCT: - self.index_table = Table() - self.index_table.read(iprot) + if ftype == TType.STRING: + self.tbl_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRING: + self.col_name = iprot.readString(); else: iprot.skip(ftype) else: @@ -15039,14 +16986,18 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('add_index_args') - if self.new_index is not None: - oprot.writeFieldBegin('new_index', TType.STRUCT, 1) - self.new_index.write(oprot) + oprot.writeStructBegin('get_table_column_statistics_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) oprot.writeFieldEnd() - if self.index_table is not None: - oprot.writeFieldBegin('index_table', TType.STRUCT, 2) - self.index_table.write(oprot) + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) + oprot.writeFieldEnd() + if self.col_name is not None: + oprot.writeFieldBegin('col_name', TType.STRING, 3) + oprot.writeString(self.col_name) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -15066,27 +17017,30 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class add_index_result: +class get_table_column_statistics_result: """ Attributes: - success - o1 - o2 - o3 + - o4 """ thrift_spec = ( - (0, TType.STRUCT, 'success', (Index, Index.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + (0, TType.STRUCT, 'success', (ColumnStatistics, ColumnStatistics.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 4 ) - def __init__(self, success=None, o1=None, o2=None, o3=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None,): self.success = success self.o1 = o1 self.o2 = o2 self.o3 = o3 + self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -15099,28 +17053,34 @@ def read(self, iprot): break if fid == 0: if ftype == TType.STRUCT: - self.success = Index() + self.success = ColumnStatistics() self.success.read(iprot) else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = InvalidObjectException() + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = AlreadyExistsException() + self.o2 = MetaException() self.o2.read(iprot) else: iprot.skip(ftype) elif fid == 3: if ftype == TType.STRUCT: - self.o3 = MetaException() + self.o3 = InvalidInputException() self.o3.read(iprot) else: iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.o4 = InvalidObjectException() + self.o4.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -15130,7 +17090,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('add_index_result') + oprot.writeStructBegin('get_table_column_statistics_result') if self.success is not None: oprot.writeFieldBegin('success', TType.STRUCT, 0) self.success.write(oprot) @@ -15147,6 +17107,10 @@ def write(self, oprot): oprot.writeFieldBegin('o3', TType.STRUCT, 3) self.o3.write(oprot) oprot.writeFieldEnd() + if self.o4 is not None: + oprot.writeFieldBegin('o4', TType.STRUCT, 4) + self.o4.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -15165,29 +17129,29 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_index_args: +class get_partition_column_statistics_args: """ Attributes: - - dbname - - base_tbl_name - - idx_name - - new_idx + - db_name + - tbl_name + - part_name + - col_name """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'dbname', None, None, ), # 1 - (2, TType.STRING, 'base_tbl_name', None, None, ), # 2 - (3, TType.STRING, 'idx_name', None, None, ), # 3 - (4, TType.STRUCT, 'new_idx', (Index, Index.thrift_spec), None, ), # 4 + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.STRING, 'part_name', None, None, ), # 3 + (4, TType.STRING, 'col_name', None, None, ), # 4 ) - def __init__(self, dbname=None, base_tbl_name=None, idx_name=None, new_idx=None,): - self.dbname = dbname - self.base_tbl_name = base_tbl_name - self.idx_name = idx_name - self.new_idx = new_idx - + def __init__(self, db_name=None, tbl_name=None, part_name=None, col_name=None,): + self.db_name = db_name + self.tbl_name = tbl_name + self.part_name = part_name + self.col_name = col_name + def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) @@ -15199,23 +17163,22 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.dbname = iprot.readString(); + self.db_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: - self.base_tbl_name = iprot.readString(); + self.tbl_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 3: if ftype == TType.STRING: - self.idx_name = iprot.readString(); + self.part_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 4: - if ftype == TType.STRUCT: - self.new_idx = Index() - self.new_idx.read(iprot) + if ftype == TType.STRING: + self.col_name = iprot.readString(); else: iprot.skip(ftype) else: @@ -15227,22 +17190,22 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_index_args') - if self.dbname is not None: - oprot.writeFieldBegin('dbname', TType.STRING, 1) - oprot.writeString(self.dbname) + oprot.writeStructBegin('get_partition_column_statistics_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) oprot.writeFieldEnd() - if self.base_tbl_name is not None: - oprot.writeFieldBegin('base_tbl_name', TType.STRING, 2) - oprot.writeString(self.base_tbl_name) + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.idx_name is not None: - oprot.writeFieldBegin('idx_name', TType.STRING, 3) - oprot.writeString(self.idx_name) + if self.part_name is not None: + oprot.writeFieldBegin('part_name', TType.STRING, 3) + oprot.writeString(self.part_name) oprot.writeFieldEnd() - if self.new_idx is not None: - oprot.writeFieldBegin('new_idx', TType.STRUCT, 4) - self.new_idx.write(oprot) + if self.col_name is not None: + oprot.writeFieldBegin('col_name', TType.STRING, 4) + oprot.writeString(self.col_name) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -15262,22 +17225,30 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_index_result: +class get_partition_column_statistics_result: """ Attributes: + - success - o1 - o2 + - o3 + - o4 """ thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'o1', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 1 + (0, TType.STRUCT, 'success', (ColumnStatistics, ColumnStatistics.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 4 ) - def __init__(self, o1=None, o2=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None,): + self.success = success self.o1 = o1 self.o2 = o2 + self.o3 = o3 + self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -15288,9 +17259,15 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 1: + if fid == 0: if ftype == TType.STRUCT: - self.o1 = InvalidOperationException() + self.success = ColumnStatistics() + self.success.read(iprot) + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) @@ -15300,6 +17277,18 @@ def read(self, iprot): self.o2.read(iprot) else: iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = InvalidInputException() + self.o3.read(iprot) + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.o4 = InvalidObjectException() + self.o4.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -15309,7 +17298,11 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_index_result') + oprot.writeStructBegin('get_partition_column_statistics_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) + oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -15318,6 +17311,14 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() + if self.o4 is not None: + oprot.writeFieldBegin('o4', TType.STRUCT, 4) + self.o4.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -15336,28 +17337,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_index_by_name_args: +class delete_partition_column_statistics_args: """ Attributes: - db_name - tbl_name - - index_name - - deleteData + - part_name + - col_name """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRING, 'index_name', None, None, ), # 3 - (4, TType.BOOL, 'deleteData', None, None, ), # 4 + (3, TType.STRING, 'part_name', None, None, ), # 3 + (4, TType.STRING, 'col_name', None, None, ), # 4 ) - def __init__(self, db_name=None, tbl_name=None, index_name=None, deleteData=None,): + def __init__(self, db_name=None, tbl_name=None, part_name=None, col_name=None,): self.db_name = db_name self.tbl_name = tbl_name - self.index_name = index_name - self.deleteData = deleteData + self.part_name = part_name + self.col_name = col_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -15380,12 +17381,12 @@ def read(self, iprot): iprot.skip(ftype) elif fid == 3: if ftype == TType.STRING: - self.index_name = iprot.readString(); + self.part_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 4: - if ftype == TType.BOOL: - self.deleteData = iprot.readBool(); + if ftype == TType.STRING: + self.col_name = iprot.readString(); else: iprot.skip(ftype) else: @@ -15397,7 +17398,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_index_by_name_args') + oprot.writeStructBegin('delete_partition_column_statistics_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -15406,13 +17407,13 @@ def write(self, oprot): oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.index_name is not None: - oprot.writeFieldBegin('index_name', TType.STRING, 3) - oprot.writeString(self.index_name) + if self.part_name is not None: + oprot.writeFieldBegin('part_name', TType.STRING, 3) + oprot.writeString(self.part_name) oprot.writeFieldEnd() - if self.deleteData is not None: - oprot.writeFieldBegin('deleteData', TType.BOOL, 4) - oprot.writeBool(self.deleteData) + if self.col_name is not None: + oprot.writeFieldBegin('col_name', TType.STRING, 4) + oprot.writeString(self.col_name) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -15432,24 +17433,30 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_index_by_name_result: +class delete_partition_column_statistics_result: """ Attributes: - success - o1 - o2 + - o3 + - o4 """ thrift_spec = ( (0, TType.BOOL, 'success', None, None, ), # 0 (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 4 ) - def __init__(self, success=None, o1=None, o2=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None,): self.success = success self.o1 = o1 self.o2 = o2 + self.o3 = o3 + self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -15477,6 +17484,18 @@ def read(self, iprot): self.o2.read(iprot) else: iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = InvalidObjectException() + self.o3.read(iprot) + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.o4 = InvalidInputException() + self.o4.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -15486,7 +17505,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_index_by_name_result') + oprot.writeStructBegin('delete_partition_column_statistics_result') if self.success is not None: oprot.writeFieldBegin('success', TType.BOOL, 0) oprot.writeBool(self.success) @@ -15499,6 +17518,14 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() + if self.o4 is not None: + oprot.writeFieldBegin('o4', TType.STRUCT, 4) + self.o4.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -15517,25 +17544,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_index_by_name_args: +class delete_table_column_statistics_args: """ Attributes: - db_name - tbl_name - - index_name + - col_name """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRING, 'index_name', None, None, ), # 3 + (3, TType.STRING, 'col_name', None, None, ), # 3 ) - def __init__(self, db_name=None, tbl_name=None, index_name=None,): + def __init__(self, db_name=None, tbl_name=None, col_name=None,): self.db_name = db_name self.tbl_name = tbl_name - self.index_name = index_name + self.col_name = col_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -15558,7 +17585,7 @@ def read(self, iprot): iprot.skip(ftype) elif fid == 3: if ftype == TType.STRING: - self.index_name = iprot.readString(); + self.col_name = iprot.readString(); else: iprot.skip(ftype) else: @@ -15570,7 +17597,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_index_by_name_args') + oprot.writeStructBegin('delete_table_column_statistics_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -15579,9 +17606,9 @@ def write(self, oprot): oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.index_name is not None: - oprot.writeFieldBegin('index_name', TType.STRING, 3) - oprot.writeString(self.index_name) + if self.col_name is not None: + oprot.writeFieldBegin('col_name', TType.STRING, 3) + oprot.writeString(self.col_name) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -15601,24 +17628,30 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_index_by_name_result: +class delete_table_column_statistics_result: """ Attributes: - success - o1 - o2 + - o3 + - o4 """ thrift_spec = ( - (0, TType.STRUCT, 'success', (Index, Index.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 + (0, TType.BOOL, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 4 ) - def __init__(self, success=None, o1=None, o2=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None,): self.success = success self.o1 = o1 self.o2 = o2 + self.o3 = o3 + self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -15630,23 +17663,34 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.STRUCT: - self.success = Index() - self.success.read(iprot) + if ftype == TType.BOOL: + self.success = iprot.readBool(); else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = NoSuchObjectException() + self.o2 = MetaException() self.o2.read(iprot) else: iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = InvalidObjectException() + self.o3.read(iprot) + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.o4 = InvalidInputException() + self.o4.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -15656,10 +17700,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_index_by_name_result') + oprot.writeStructBegin('delete_table_column_statistics_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) + oprot.writeFieldBegin('success', TType.BOOL, 0) + oprot.writeBool(self.success) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -15669,6 +17713,14 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() + if self.o4 is not None: + oprot.writeFieldBegin('o4', TType.STRUCT, 4) + self.o4.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -15687,25 +17739,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_indexes_args: +class streamingStatus_args: """ Attributes: - db_name - - tbl_name - - max_indexes + - table_name """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.I16, 'max_indexes', None, -1, ), # 3 + (2, TType.STRING, 'table_name', None, None, ), # 2 ) - def __init__(self, db_name=None, tbl_name=None, max_indexes=thrift_spec[3][4],): + def __init__(self, db_name=None, table_name=None,): self.db_name = db_name - self.tbl_name = tbl_name - self.max_indexes = max_indexes + self.table_name = table_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -15721,16 +17770,11 @@ def read(self, iprot): self.db_name = iprot.readString(); else: iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.tbl_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.I16: - self.max_indexes = iprot.readI16(); - else: - iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.table_name = iprot.readString(); + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -15740,18 +17784,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_indexes_args') + oprot.writeStructBegin('streamingStatus_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) - oprot.writeFieldEnd() - if self.max_indexes is not None: - oprot.writeFieldBegin('max_indexes', TType.I16, 3) - oprot.writeI16(self.max_indexes) + if self.table_name is not None: + oprot.writeFieldBegin('table_name', TType.STRING, 2) + oprot.writeString(self.table_name) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -15771,24 +17811,27 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_indexes_result: +class streamingStatus_result: """ Attributes: - success - o1 - o2 + - o3 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRUCT,(Index, Index.thrift_spec)), None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + (0, TType.MAP, 'success', (TType.STRING,None,TType.STRING,None), None, ), # 0 + (1, TType.STRUCT, 'o1', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 ) - def __init__(self, success=None, o1=None, o2=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None,): self.success = success self.o1 = o1 self.o2 = o2 + self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -15800,28 +17843,34 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype520, _size517) = iprot.readListBegin() - for _i521 in xrange(_size517): - _elem522 = Index() - _elem522.read(iprot) - self.success.append(_elem522) - iprot.readListEnd() + if ftype == TType.MAP: + self.success = {} + (_ktype532, _vtype533, _size531 ) = iprot.readMapBegin() + for _i535 in xrange(_size531): + _key536 = iprot.readString(); + _val537 = iprot.readString(); + self.success[_key536] = _val537 + iprot.readMapEnd() else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = InvalidInputException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = MetaException() + self.o2 = InvalidObjectException() self.o2.read(iprot) else: iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = MetaException() + self.o3.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -15831,13 +17880,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_indexes_result') + oprot.writeStructBegin('streamingStatus_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter523 in self.success: - iter523.write(oprot) - oprot.writeListEnd() + oprot.writeFieldBegin('success', TType.MAP, 0) + oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.success)) + for kiter538,viter539 in self.success.items(): + oprot.writeString(kiter538) + oprot.writeString(viter539) + oprot.writeMapEnd() oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -15847,6 +17897,10 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -15865,25 +17919,31 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_index_names_args: +class enableStreaming_args: """ Attributes: - db_name - - tbl_name - - max_indexes + - table_name + - streaming_tmp_dir + - partition_path + - partition_val """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.I16, 'max_indexes', None, -1, ), # 3 + (2, TType.STRING, 'table_name', None, None, ), # 2 + (3, TType.STRING, 'streaming_tmp_dir', None, None, ), # 3 + (4, TType.STRING, 'partition_path', None, None, ), # 4 + (5, TType.STRING, 'partition_val', None, None, ), # 5 ) - def __init__(self, db_name=None, tbl_name=None, max_indexes=thrift_spec[3][4],): + def __init__(self, db_name=None, table_name=None, streaming_tmp_dir=None, partition_path=None, partition_val=None,): self.db_name = db_name - self.tbl_name = tbl_name - self.max_indexes = max_indexes + self.table_name = table_name + self.streaming_tmp_dir = streaming_tmp_dir + self.partition_path = partition_path + self.partition_val = partition_val def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -15901,12 +17961,22 @@ def read(self, iprot): iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: - self.tbl_name = iprot.readString(); + self.table_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.I16: - self.max_indexes = iprot.readI16(); + if ftype == TType.STRING: + self.streaming_tmp_dir = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRING: + self.partition_path = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.STRING: + self.partition_val = iprot.readString(); else: iprot.skip(ftype) else: @@ -15918,18 +17988,26 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_index_names_args') + oprot.writeStructBegin('enableStreaming_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) + if self.table_name is not None: + oprot.writeFieldBegin('table_name', TType.STRING, 2) + oprot.writeString(self.table_name) oprot.writeFieldEnd() - if self.max_indexes is not None: - oprot.writeFieldBegin('max_indexes', TType.I16, 3) - oprot.writeI16(self.max_indexes) + if self.streaming_tmp_dir is not None: + oprot.writeFieldBegin('streaming_tmp_dir', TType.STRING, 3) + oprot.writeString(self.streaming_tmp_dir) + oprot.writeFieldEnd() + if self.partition_path is not None: + oprot.writeFieldBegin('partition_path', TType.STRING, 4) + oprot.writeString(self.partition_path) + oprot.writeFieldEnd() + if self.partition_val is not None: + oprot.writeFieldBegin('partition_val', TType.STRING, 5) + oprot.writeString(self.partition_val) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -15949,21 +18027,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_index_names_result: +class enableStreaming_result: """ Attributes: - - success + - o1 - o2 + - o3 + - o4 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 - (1, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 1 + None, # 0 + (1, TType.STRUCT, 'o1', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (MetaException, MetaException.thrift_spec), None, ), # 4 ) - def __init__(self, success=None, o2=None,): - self.success = success + def __init__(self, o1=None, o2=None, o3=None, o4=None,): + self.o1 = o1 self.o2 = o2 + self.o3 = o3 + self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -15974,22 +18059,30 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype527, _size524) = iprot.readListBegin() - for _i528 in xrange(_size524): - _elem529 = iprot.readString(); - self.success.append(_elem529) - iprot.readListEnd() + if fid == 1: + if ftype == TType.STRUCT: + self.o1 = InvalidInputException() + self.o1.read(iprot) else: iprot.skip(ftype) - elif fid == 1: + elif fid == 2: if ftype == TType.STRUCT: - self.o2 = MetaException() + self.o2 = InvalidObjectException() self.o2.read(iprot) else: iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = InvalidOperationException() + self.o3.read(iprot) + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.o4 = MetaException() + self.o4.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -15999,18 +18092,23 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_index_names_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRING, len(self.success)) - for iter530 in self.success: - oprot.writeString(iter530) - oprot.writeListEnd() + oprot.writeStructBegin('enableStreaming_result') + if self.o1 is not None: + oprot.writeFieldBegin('o1', TType.STRUCT, 1) + self.o1.write(oprot) oprot.writeFieldEnd() if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 1) + oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() + if self.o4 is not None: + oprot.writeFieldBegin('o4', TType.STRUCT, 4) + self.o4.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -16029,19 +18127,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class update_table_column_statistics_args: +class disableStreaming_args: """ Attributes: - - stats_obj + - db_name + - table_name """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'stats_obj', (ColumnStatistics, ColumnStatistics.thrift_spec), None, ), # 1 + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'table_name', None, None, ), # 2 ) - def __init__(self, stats_obj=None,): - self.stats_obj = stats_obj + def __init__(self, db_name=None, table_name=None,): + self.db_name = db_name + self.table_name = table_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -16053,9 +18154,13 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRUCT: - self.stats_obj = ColumnStatistics() - self.stats_obj.read(iprot) + if ftype == TType.STRING: + self.db_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.table_name = iprot.readString(); else: iprot.skip(ftype) else: @@ -16067,10 +18172,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('update_table_column_statistics_args') - if self.stats_obj is not None: - oprot.writeFieldBegin('stats_obj', TType.STRUCT, 1) - self.stats_obj.write(oprot) + oprot.writeStructBegin('disableStreaming_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) + oprot.writeFieldEnd() + if self.table_name is not None: + oprot.writeFieldBegin('table_name', TType.STRING, 2) + oprot.writeString(self.table_name) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -16090,10 +18199,9 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class update_table_column_statistics_result: +class disableStreaming_result: """ Attributes: - - success - o1 - o2 - o3 @@ -16101,15 +18209,14 @@ class update_table_column_statistics_result: """ thrift_spec = ( - (0, TType.BOOL, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + None, # 0 + (1, TType.STRUCT, 'o1', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 - (4, TType.STRUCT, 'o4', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 4 + (3, TType.STRUCT, 'o3', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (MetaException, MetaException.thrift_spec), None, ), # 4 ) - def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None,): - self.success = success + def __init__(self, o1=None, o2=None, o3=None, o4=None,): self.o1 = o1 self.o2 = o2 self.o3 = o3 @@ -16124,14 +18231,9 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.BOOL: - self.success = iprot.readBool(); - else: - iprot.skip(ftype) - elif fid == 1: + if fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = InvalidInputException() self.o1.read(iprot) else: iprot.skip(ftype) @@ -16143,13 +18245,13 @@ def read(self, iprot): iprot.skip(ftype) elif fid == 3: if ftype == TType.STRUCT: - self.o3 = MetaException() + self.o3 = InvalidOperationException() self.o3.read(iprot) else: iprot.skip(ftype) elif fid == 4: if ftype == TType.STRUCT: - self.o4 = InvalidInputException() + self.o4 = MetaException() self.o4.read(iprot) else: iprot.skip(ftype) @@ -16162,11 +18264,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('update_table_column_statistics_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.BOOL, 0) - oprot.writeBool(self.success) - oprot.writeFieldEnd() + oprot.writeStructBegin('disableStreaming_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -16201,19 +18299,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class update_partition_column_statistics_args: +class getCurrentStreamingPartitionPath_args: """ Attributes: - - stats_obj + - dbName + - tableName """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'stats_obj', (ColumnStatistics, ColumnStatistics.thrift_spec), None, ), # 1 + (1, TType.STRING, 'dbName', None, None, ), # 1 + (2, TType.STRING, 'tableName', None, None, ), # 2 ) - def __init__(self, stats_obj=None,): - self.stats_obj = stats_obj + def __init__(self, dbName=None, tableName=None,): + self.dbName = dbName + self.tableName = tableName def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -16225,9 +18326,13 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRUCT: - self.stats_obj = ColumnStatistics() - self.stats_obj.read(iprot) + if ftype == TType.STRING: + self.dbName = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.tableName = iprot.readString(); else: iprot.skip(ftype) else: @@ -16239,10 +18344,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('update_partition_column_statistics_args') - if self.stats_obj is not None: - oprot.writeFieldBegin('stats_obj', TType.STRUCT, 1) - self.stats_obj.write(oprot) + oprot.writeStructBegin('getCurrentStreamingPartitionPath_args') + if self.dbName is not None: + oprot.writeFieldBegin('dbName', TType.STRING, 1) + oprot.writeString(self.dbName) + oprot.writeFieldEnd() + if self.tableName is not None: + oprot.writeFieldBegin('tableName', TType.STRING, 2) + oprot.writeString(self.tableName) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -16262,7 +18371,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class update_partition_column_statistics_result: +class getCurrentStreamingPartitionPath_result: """ Attributes: - success @@ -16273,11 +18382,11 @@ class update_partition_column_statistics_result: """ thrift_spec = ( - (0, TType.BOOL, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (0, TType.STRING, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 - (4, TType.STRUCT, 'o4', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 4 + (3, TType.STRUCT, 'o3', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (MetaException, MetaException.thrift_spec), None, ), # 4 ) def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None,): @@ -16297,13 +18406,13 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.BOOL: - self.success = iprot.readBool(); + if ftype == TType.STRING: + self.success = iprot.readString(); else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = InvalidInputException() self.o1.read(iprot) else: iprot.skip(ftype) @@ -16315,13 +18424,13 @@ def read(self, iprot): iprot.skip(ftype) elif fid == 3: if ftype == TType.STRUCT: - self.o3 = MetaException() + self.o3 = InvalidOperationException() self.o3.read(iprot) else: iprot.skip(ftype) elif fid == 4: if ftype == TType.STRUCT: - self.o4 = InvalidInputException() + self.o4 = MetaException() self.o4.read(iprot) else: iprot.skip(ftype) @@ -16334,10 +18443,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('update_partition_column_statistics_result') + oprot.writeStructBegin('getCurrentStreamingPartitionPath_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.BOOL, 0) - oprot.writeBool(self.success) + oprot.writeFieldBegin('success', TType.STRING, 0) + oprot.writeString(self.success) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -16373,25 +18482,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_table_column_statistics_args: +class rollStreamingPartition_args: """ - Attributes: - - db_name - - tbl_name - - col_name + Attributes: + - dbName + - tableName + - newPartitionPath + - newPartitionVal """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRING, 'col_name', None, None, ), # 3 + (1, TType.STRING, 'dbName', None, None, ), # 1 + (2, TType.STRING, 'tableName', None, None, ), # 2 + (3, TType.STRING, 'newPartitionPath', None, None, ), # 3 + (4, TType.STRING, 'newPartitionVal', None, None, ), # 4 ) - def __init__(self, db_name=None, tbl_name=None, col_name=None,): - self.db_name = db_name - self.tbl_name = tbl_name - self.col_name = col_name + def __init__(self, dbName=None, tableName=None, newPartitionPath=None, newPartitionVal=None,): + self.dbName = dbName + self.tableName = tableName + self.newPartitionPath = newPartitionPath + self.newPartitionVal = newPartitionVal def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -16404,17 +18516,22 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.db_name = iprot.readString(); + self.dbName = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: - self.tbl_name = iprot.readString(); + self.tableName = iprot.readString(); else: iprot.skip(ftype) elif fid == 3: if ftype == TType.STRING: - self.col_name = iprot.readString(); + self.newPartitionPath = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRING: + self.newPartitionVal = iprot.readString(); else: iprot.skip(ftype) else: @@ -16426,18 +18543,22 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_table_column_statistics_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) + oprot.writeStructBegin('rollStreamingPartition_args') + if self.dbName is not None: + oprot.writeFieldBegin('dbName', TType.STRING, 1) + oprot.writeString(self.dbName) oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) + if self.tableName is not None: + oprot.writeFieldBegin('tableName', TType.STRING, 2) + oprot.writeString(self.tableName) oprot.writeFieldEnd() - if self.col_name is not None: - oprot.writeFieldBegin('col_name', TType.STRING, 3) - oprot.writeString(self.col_name) + if self.newPartitionPath is not None: + oprot.writeFieldBegin('newPartitionPath', TType.STRING, 3) + oprot.writeString(self.newPartitionPath) + oprot.writeFieldEnd() + if self.newPartitionVal is not None: + oprot.writeFieldBegin('newPartitionVal', TType.STRING, 4) + oprot.writeString(self.newPartitionVal) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -16457,7 +18578,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_table_column_statistics_result: +class rollStreamingPartition_result: """ Attributes: - success @@ -16465,22 +18586,25 @@ class get_table_column_statistics_result: - o2 - o3 - o4 + - o5 """ thrift_spec = ( - (0, TType.STRUCT, 'success', (ColumnStatistics, ColumnStatistics.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 3 - (4, TType.STRUCT, 'o4', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 4 + (0, TType.STRING, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 4 + (5, TType.STRUCT, 'o5', (MetaException, MetaException.thrift_spec), None, ), # 5 ) - def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None, o5=None,): self.success = success self.o1 = o1 self.o2 = o2 self.o3 = o3 self.o4 = o4 + self.o5 = o5 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -16492,35 +18616,40 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.STRUCT: - self.success = ColumnStatistics() - self.success.read(iprot) + if ftype == TType.STRING: + self.success = iprot.readString(); else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = InvalidInputException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = MetaException() + self.o2 = InvalidObjectException() self.o2.read(iprot) else: iprot.skip(ftype) elif fid == 3: if ftype == TType.STRUCT: - self.o3 = InvalidInputException() + self.o3 = InvalidOperationException() self.o3.read(iprot) else: iprot.skip(ftype) elif fid == 4: if ftype == TType.STRUCT: - self.o4 = InvalidObjectException() + self.o4 = AlreadyExistsException() self.o4.read(iprot) else: iprot.skip(ftype) + elif fid == 5: + if ftype == TType.STRUCT: + self.o5 = MetaException() + self.o5.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -16530,10 +18659,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_table_column_statistics_result') + oprot.writeStructBegin('rollStreamingPartition_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) + oprot.writeFieldBegin('success', TType.STRING, 0) + oprot.writeString(self.success) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -16551,6 +18680,10 @@ def write(self, oprot): oprot.writeFieldBegin('o4', TType.STRUCT, 4) self.o4.write(oprot) oprot.writeFieldEnd() + if self.o5 is not None: + oprot.writeFieldBegin('o5', TType.STRUCT, 5) + self.o5.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -16569,28 +18702,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partition_column_statistics_args: +class updateStreamingTempLocation_args: """ Attributes: - - db_name - - tbl_name - - part_name - - col_name + - dbName + - tableName + - streamingTempDir """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRING, 'part_name', None, None, ), # 3 - (4, TType.STRING, 'col_name', None, None, ), # 4 + (1, TType.STRING, 'dbName', None, None, ), # 1 + (2, TType.STRING, 'tableName', None, None, ), # 2 + (3, TType.STRING, 'streamingTempDir', None, None, ), # 3 ) - def __init__(self, db_name=None, tbl_name=None, part_name=None, col_name=None,): - self.db_name = db_name - self.tbl_name = tbl_name - self.part_name = part_name - self.col_name = col_name + def __init__(self, dbName=None, tableName=None, streamingTempDir=None,): + self.dbName = dbName + self.tableName = tableName + self.streamingTempDir = streamingTempDir def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -16603,22 +18733,17 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.db_name = iprot.readString(); + self.dbName = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: - self.tbl_name = iprot.readString(); + self.tableName = iprot.readString(); else: iprot.skip(ftype) elif fid == 3: if ftype == TType.STRING: - self.part_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRING: - self.col_name = iprot.readString(); + self.streamingTempDir = iprot.readString(); else: iprot.skip(ftype) else: @@ -16630,22 +18755,18 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partition_column_statistics_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) - oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) + oprot.writeStructBegin('updateStreamingTempLocation_args') + if self.dbName is not None: + oprot.writeFieldBegin('dbName', TType.STRING, 1) + oprot.writeString(self.dbName) oprot.writeFieldEnd() - if self.part_name is not None: - oprot.writeFieldBegin('part_name', TType.STRING, 3) - oprot.writeString(self.part_name) + if self.tableName is not None: + oprot.writeFieldBegin('tableName', TType.STRING, 2) + oprot.writeString(self.tableName) oprot.writeFieldEnd() - if self.col_name is not None: - oprot.writeFieldBegin('col_name', TType.STRING, 4) - oprot.writeString(self.col_name) + if self.streamingTempDir is not None: + oprot.writeFieldBegin('streamingTempDir', TType.STRING, 3) + oprot.writeString(self.streamingTempDir) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -16665,30 +18786,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partition_column_statistics_result: +class updateStreamingTempLocation_result: """ Attributes: - - success - o1 - o2 - o3 - - o4 """ thrift_spec = ( - (0, TType.STRUCT, 'success', (ColumnStatistics, ColumnStatistics.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 3 - (4, TType.STRUCT, 'o4', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 4 + None, # 0 + (1, TType.STRUCT, 'o1', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 ) - def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None,): - self.success = success + def __init__(self, o1=None, o2=None, o3=None,): self.o1 = o1 self.o2 = o2 self.o3 = o3 - self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -16699,36 +18815,24 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.STRUCT: - self.success = ColumnStatistics() - self.success.read(iprot) - else: - iprot.skip(ftype) - elif fid == 1: + if fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = InvalidInputException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = MetaException() + self.o2 = InvalidObjectException() self.o2.read(iprot) else: iprot.skip(ftype) elif fid == 3: if ftype == TType.STRUCT: - self.o3 = InvalidInputException() + self.o3 = MetaException() self.o3.read(iprot) else: iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRUCT: - self.o4 = InvalidObjectException() - self.o4.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -16738,11 +18842,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partition_column_statistics_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) - oprot.writeFieldEnd() + oprot.writeStructBegin('updateStreamingTempLocation_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -16755,10 +18855,6 @@ def write(self, oprot): oprot.writeFieldBegin('o3', TType.STRUCT, 3) self.o3.write(oprot) oprot.writeFieldEnd() - if self.o4 is not None: - oprot.writeFieldBegin('o4', TType.STRUCT, 4) - self.o4.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -16777,28 +18873,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class delete_partition_column_statistics_args: +class getNextChunkID_args: """ Attributes: - db_name - - tbl_name - - part_name - - col_name + - table_name """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRING, 'part_name', None, None, ), # 3 - (4, TType.STRING, 'col_name', None, None, ), # 4 + (2, TType.STRING, 'table_name', None, None, ), # 2 ) - def __init__(self, db_name=None, tbl_name=None, part_name=None, col_name=None,): + def __init__(self, db_name=None, table_name=None,): self.db_name = db_name - self.tbl_name = tbl_name - self.part_name = part_name - self.col_name = col_name + self.table_name = table_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -16816,17 +18906,7 @@ def read(self, iprot): iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: - self.tbl_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRING: - self.part_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRING: - self.col_name = iprot.readString(); + self.table_name = iprot.readString(); else: iprot.skip(ftype) else: @@ -16838,22 +18918,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('delete_partition_column_statistics_args') + oprot.writeStructBegin('getNextChunkID_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) - oprot.writeFieldEnd() - if self.part_name is not None: - oprot.writeFieldBegin('part_name', TType.STRING, 3) - oprot.writeString(self.part_name) - oprot.writeFieldEnd() - if self.col_name is not None: - oprot.writeFieldBegin('col_name', TType.STRING, 4) - oprot.writeString(self.col_name) + if self.table_name is not None: + oprot.writeFieldBegin('table_name', TType.STRING, 2) + oprot.writeString(self.table_name) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -16873,7 +18945,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class delete_partition_column_statistics_result: +class getNextChunkID_result: """ Attributes: - success @@ -16884,11 +18956,11 @@ class delete_partition_column_statistics_result: """ thrift_spec = ( - (0, TType.BOOL, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 3 - (4, TType.STRUCT, 'o4', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 4 + (0, TType.I64, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (MetaException, MetaException.thrift_spec), None, ), # 4 ) def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None,): @@ -16908,31 +18980,31 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.BOOL: - self.success = iprot.readBool(); + if ftype == TType.I64: + self.success = iprot.readI64(); else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = InvalidInputException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = MetaException() + self.o2 = InvalidObjectException() self.o2.read(iprot) else: iprot.skip(ftype) elif fid == 3: if ftype == TType.STRUCT: - self.o3 = InvalidObjectException() + self.o3 = InvalidOperationException() self.o3.read(iprot) else: iprot.skip(ftype) elif fid == 4: if ftype == TType.STRUCT: - self.o4 = InvalidInputException() + self.o4 = MetaException() self.o4.read(iprot) else: iprot.skip(ftype) @@ -16945,10 +19017,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('delete_partition_column_statistics_result') + oprot.writeStructBegin('getNextChunkID_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.BOOL, 0) - oprot.writeBool(self.success) + oprot.writeFieldBegin('success', TType.I64, 0) + oprot.writeI64(self.success) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -16984,25 +19056,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class delete_table_column_statistics_args: +class getStreamingTmpDir_args: """ Attributes: - - db_name - - tbl_name - - col_name + - dbName + - tableName """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRING, 'col_name', None, None, ), # 3 + (1, TType.STRING, 'dbName', None, None, ), # 1 + (2, TType.STRING, 'tableName', None, None, ), # 2 ) - def __init__(self, db_name=None, tbl_name=None, col_name=None,): - self.db_name = db_name - self.tbl_name = tbl_name - self.col_name = col_name + def __init__(self, dbName=None, tableName=None,): + self.dbName = dbName + self.tableName = tableName def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -17015,17 +19084,12 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.db_name = iprot.readString(); + self.dbName = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: - self.tbl_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRING: - self.col_name = iprot.readString(); + self.tableName = iprot.readString(); else: iprot.skip(ftype) else: @@ -17037,18 +19101,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('delete_table_column_statistics_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) - oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) + oprot.writeStructBegin('getStreamingTmpDir_args') + if self.dbName is not None: + oprot.writeFieldBegin('dbName', TType.STRING, 1) + oprot.writeString(self.dbName) oprot.writeFieldEnd() - if self.col_name is not None: - oprot.writeFieldBegin('col_name', TType.STRING, 3) - oprot.writeString(self.col_name) + if self.tableName is not None: + oprot.writeFieldBegin('tableName', TType.STRING, 2) + oprot.writeString(self.tableName) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -17068,7 +19128,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class delete_table_column_statistics_result: +class getStreamingTmpDir_result: """ Attributes: - success @@ -17079,11 +19139,11 @@ class delete_table_column_statistics_result: """ thrift_spec = ( - (0, TType.BOOL, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 3 - (4, TType.STRUCT, 'o4', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 4 + (0, TType.STRING, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (MetaException, MetaException.thrift_spec), None, ), # 4 ) def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None,): @@ -17103,31 +19163,31 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.BOOL: - self.success = iprot.readBool(); + if ftype == TType.STRING: + self.success = iprot.readString(); else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = InvalidInputException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = MetaException() + self.o2 = InvalidObjectException() self.o2.read(iprot) else: iprot.skip(ftype) elif fid == 3: if ftype == TType.STRUCT: - self.o3 = InvalidObjectException() + self.o3 = InvalidOperationException() self.o3.read(iprot) else: iprot.skip(ftype) elif fid == 4: if ftype == TType.STRUCT: - self.o4 = InvalidInputException() + self.o4 = MetaException() self.o4.read(iprot) else: iprot.skip(ftype) @@ -17140,10 +19200,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('delete_table_column_statistics_result') + oprot.writeStructBegin('getStreamingTmpDir_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.BOOL, 0) - oprot.writeBool(self.success) + oprot.writeFieldBegin('success', TType.STRING, 0) + oprot.writeString(self.success) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -17514,10 +19574,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype534, _size531) = iprot.readListBegin() - for _i535 in xrange(_size531): - _elem536 = iprot.readString(); - self.success.append(_elem536) + (_etype543, _size540) = iprot.readListBegin() + for _i544 in xrange(_size540): + _elem545 = iprot.readString(); + self.success.append(_elem545) iprot.readListEnd() else: iprot.skip(ftype) @@ -17540,8 +19600,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter537 in self.success: - oprot.writeString(iter537) + for iter546 in self.success: + oprot.writeString(iter546) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -18014,11 +20074,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype541, _size538) = iprot.readListBegin() - for _i542 in xrange(_size538): - _elem543 = Role() - _elem543.read(iprot) - self.success.append(_elem543) + (_etype550, _size547) = iprot.readListBegin() + for _i551 in xrange(_size547): + _elem552 = Role() + _elem552.read(iprot) + self.success.append(_elem552) iprot.readListEnd() else: iprot.skip(ftype) @@ -18041,8 +20101,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter544 in self.success: - iter544.write(oprot) + for iter553 in self.success: + iter553.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -18110,10 +20170,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.group_names = [] - (_etype548, _size545) = iprot.readListBegin() - for _i549 in xrange(_size545): - _elem550 = iprot.readString(); - self.group_names.append(_elem550) + (_etype557, _size554) = iprot.readListBegin() + for _i558 in xrange(_size554): + _elem559 = iprot.readString(); + self.group_names.append(_elem559) iprot.readListEnd() else: iprot.skip(ftype) @@ -18138,8 +20198,8 @@ def write(self, oprot): if self.group_names is not None: oprot.writeFieldBegin('group_names', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.group_names)) - for iter551 in self.group_names: - oprot.writeString(iter551) + for iter560 in self.group_names: + oprot.writeString(iter560) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -18346,11 +20406,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype555, _size552) = iprot.readListBegin() - for _i556 in xrange(_size552): - _elem557 = HiveObjectPrivilege() - _elem557.read(iprot) - self.success.append(_elem557) + (_etype564, _size561) = iprot.readListBegin() + for _i565 in xrange(_size561): + _elem566 = HiveObjectPrivilege() + _elem566.read(iprot) + self.success.append(_elem566) iprot.readListEnd() else: iprot.skip(ftype) @@ -18373,8 +20433,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter558 in self.success: - iter558.write(oprot) + for iter567 in self.success: + iter567.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -18699,10 +20759,10 @@ def read(self, iprot): elif fid == 2: if ftype == TType.LIST: self.group_names = [] - (_etype562, _size559) = iprot.readListBegin() - for _i563 in xrange(_size559): - _elem564 = iprot.readString(); - self.group_names.append(_elem564) + (_etype571, _size568) = iprot.readListBegin() + for _i572 in xrange(_size568): + _elem573 = iprot.readString(); + self.group_names.append(_elem573) iprot.readListEnd() else: iprot.skip(ftype) @@ -18723,8 +20783,8 @@ def write(self, oprot): if self.group_names is not None: oprot.writeFieldBegin('group_names', TType.LIST, 2) oprot.writeListBegin(TType.STRING, len(self.group_names)) - for iter565 in self.group_names: - oprot.writeString(iter565) + for iter574 in self.group_names: + oprot.writeString(iter574) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -18773,10 +20833,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype569, _size566) = iprot.readListBegin() - for _i570 in xrange(_size566): - _elem571 = iprot.readString(); - self.success.append(_elem571) + (_etype578, _size575) = iprot.readListBegin() + for _i579 in xrange(_size575): + _elem580 = iprot.readString(); + self.success.append(_elem580) iprot.readListEnd() else: iprot.skip(ftype) @@ -18799,8 +20859,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter572 in self.success: - oprot.writeString(iter572) + for iter581 in self.success: + oprot.writeString(iter581) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: diff --git metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb index 25aa30c..0180fe5 100644 --- metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb +++ metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb @@ -1183,6 +1183,154 @@ module ThriftHiveMetastore raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'delete_table_column_statistics failed: unknown result') end + def streamingStatus(db_name, table_name) + send_streamingStatus(db_name, table_name) + return recv_streamingStatus() + end + + def send_streamingStatus(db_name, table_name) + send_message('streamingStatus', StreamingStatus_args, :db_name => db_name, :table_name => table_name) + end + + def recv_streamingStatus() + result = receive_message(StreamingStatus_result) + return result.success unless result.success.nil? + raise result.o1 unless result.o1.nil? + raise result.o2 unless result.o2.nil? + raise result.o3 unless result.o3.nil? + raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'streamingStatus failed: unknown result') + end + + def enableStreaming(db_name, table_name, streaming_tmp_dir, partition_path, partition_val) + send_enableStreaming(db_name, table_name, streaming_tmp_dir, partition_path, partition_val) + recv_enableStreaming() + end + + def send_enableStreaming(db_name, table_name, streaming_tmp_dir, partition_path, partition_val) + send_message('enableStreaming', EnableStreaming_args, :db_name => db_name, :table_name => table_name, :streaming_tmp_dir => streaming_tmp_dir, :partition_path => partition_path, :partition_val => partition_val) + end + + def recv_enableStreaming() + result = receive_message(EnableStreaming_result) + raise result.o1 unless result.o1.nil? + raise result.o2 unless result.o2.nil? + raise result.o3 unless result.o3.nil? + raise result.o4 unless result.o4.nil? + return + end + + def disableStreaming(db_name, table_name) + send_disableStreaming(db_name, table_name) + recv_disableStreaming() + end + + def send_disableStreaming(db_name, table_name) + send_message('disableStreaming', DisableStreaming_args, :db_name => db_name, :table_name => table_name) + end + + def recv_disableStreaming() + result = receive_message(DisableStreaming_result) + raise result.o1 unless result.o1.nil? + raise result.o2 unless result.o2.nil? + raise result.o3 unless result.o3.nil? + raise result.o4 unless result.o4.nil? + return + end + + def getCurrentStreamingPartitionPath(dbName, tableName) + send_getCurrentStreamingPartitionPath(dbName, tableName) + return recv_getCurrentStreamingPartitionPath() + end + + def send_getCurrentStreamingPartitionPath(dbName, tableName) + send_message('getCurrentStreamingPartitionPath', GetCurrentStreamingPartitionPath_args, :dbName => dbName, :tableName => tableName) + end + + def recv_getCurrentStreamingPartitionPath() + result = receive_message(GetCurrentStreamingPartitionPath_result) + return result.success unless result.success.nil? + raise result.o1 unless result.o1.nil? + raise result.o2 unless result.o2.nil? + raise result.o3 unless result.o3.nil? + raise result.o4 unless result.o4.nil? + raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'getCurrentStreamingPartitionPath failed: unknown result') + end + + def rollStreamingPartition(dbName, tableName, newPartitionPath, newPartitionVal) + send_rollStreamingPartition(dbName, tableName, newPartitionPath, newPartitionVal) + return recv_rollStreamingPartition() + end + + def send_rollStreamingPartition(dbName, tableName, newPartitionPath, newPartitionVal) + send_message('rollStreamingPartition', RollStreamingPartition_args, :dbName => dbName, :tableName => tableName, :newPartitionPath => newPartitionPath, :newPartitionVal => newPartitionVal) + end + + def recv_rollStreamingPartition() + result = receive_message(RollStreamingPartition_result) + return result.success unless result.success.nil? + raise result.o1 unless result.o1.nil? + raise result.o2 unless result.o2.nil? + raise result.o3 unless result.o3.nil? + raise result.o4 unless result.o4.nil? + raise result.o5 unless result.o5.nil? + raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'rollStreamingPartition failed: unknown result') + end + + def updateStreamingTempLocation(dbName, tableName, streamingTempDir) + send_updateStreamingTempLocation(dbName, tableName, streamingTempDir) + recv_updateStreamingTempLocation() + end + + def send_updateStreamingTempLocation(dbName, tableName, streamingTempDir) + send_message('updateStreamingTempLocation', UpdateStreamingTempLocation_args, :dbName => dbName, :tableName => tableName, :streamingTempDir => streamingTempDir) + end + + def recv_updateStreamingTempLocation() + result = receive_message(UpdateStreamingTempLocation_result) + raise result.o1 unless result.o1.nil? + raise result.o2 unless result.o2.nil? + raise result.o3 unless result.o3.nil? + return + end + + def getNextChunkID(db_name, table_name) + send_getNextChunkID(db_name, table_name) + return recv_getNextChunkID() + end + + def send_getNextChunkID(db_name, table_name) + send_message('getNextChunkID', GetNextChunkID_args, :db_name => db_name, :table_name => table_name) + end + + def recv_getNextChunkID() + result = receive_message(GetNextChunkID_result) + return result.success unless result.success.nil? + raise result.o1 unless result.o1.nil? + raise result.o2 unless result.o2.nil? + raise result.o3 unless result.o3.nil? + raise result.o4 unless result.o4.nil? + raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'getNextChunkID failed: unknown result') + end + + def getStreamingTmpDir(dbName, tableName) + send_getStreamingTmpDir(dbName, tableName) + return recv_getStreamingTmpDir() + end + + def send_getStreamingTmpDir(dbName, tableName) + send_message('getStreamingTmpDir', GetStreamingTmpDir_args, :dbName => dbName, :tableName => tableName) + end + + def recv_getStreamingTmpDir() + result = receive_message(GetStreamingTmpDir_result) + return result.success unless result.success.nil? + raise result.o1 unless result.o1.nil? + raise result.o2 unless result.o2.nil? + raise result.o3 unless result.o3.nil? + raise result.o4 unless result.o4.nil? + raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'getStreamingTmpDir failed: unknown result') + end + def create_role(role) send_create_role(role) return recv_create_role() @@ -2355,6 +2503,140 @@ module ThriftHiveMetastore write_result(result, oprot, 'delete_table_column_statistics', seqid) end + def process_streamingStatus(seqid, iprot, oprot) + args = read_args(iprot, StreamingStatus_args) + result = StreamingStatus_result.new() + begin + result.success = @handler.streamingStatus(args.db_name, args.table_name) + rescue ::InvalidInputException => o1 + result.o1 = o1 + rescue ::InvalidObjectException => o2 + result.o2 = o2 + rescue ::MetaException => o3 + result.o3 = o3 + end + write_result(result, oprot, 'streamingStatus', seqid) + end + + def process_enableStreaming(seqid, iprot, oprot) + args = read_args(iprot, EnableStreaming_args) + result = EnableStreaming_result.new() + begin + @handler.enableStreaming(args.db_name, args.table_name, args.streaming_tmp_dir, args.partition_path, args.partition_val) + rescue ::InvalidInputException => o1 + result.o1 = o1 + rescue ::InvalidObjectException => o2 + result.o2 = o2 + rescue ::InvalidOperationException => o3 + result.o3 = o3 + rescue ::MetaException => o4 + result.o4 = o4 + end + write_result(result, oprot, 'enableStreaming', seqid) + end + + def process_disableStreaming(seqid, iprot, oprot) + args = read_args(iprot, DisableStreaming_args) + result = DisableStreaming_result.new() + begin + @handler.disableStreaming(args.db_name, args.table_name) + rescue ::InvalidInputException => o1 + result.o1 = o1 + rescue ::InvalidObjectException => o2 + result.o2 = o2 + rescue ::InvalidOperationException => o3 + result.o3 = o3 + rescue ::MetaException => o4 + result.o4 = o4 + end + write_result(result, oprot, 'disableStreaming', seqid) + end + + def process_getCurrentStreamingPartitionPath(seqid, iprot, oprot) + args = read_args(iprot, GetCurrentStreamingPartitionPath_args) + result = GetCurrentStreamingPartitionPath_result.new() + begin + result.success = @handler.getCurrentStreamingPartitionPath(args.dbName, args.tableName) + rescue ::InvalidInputException => o1 + result.o1 = o1 + rescue ::InvalidObjectException => o2 + result.o2 = o2 + rescue ::InvalidOperationException => o3 + result.o3 = o3 + rescue ::MetaException => o4 + result.o4 = o4 + end + write_result(result, oprot, 'getCurrentStreamingPartitionPath', seqid) + end + + def process_rollStreamingPartition(seqid, iprot, oprot) + args = read_args(iprot, RollStreamingPartition_args) + result = RollStreamingPartition_result.new() + begin + result.success = @handler.rollStreamingPartition(args.dbName, args.tableName, args.newPartitionPath, args.newPartitionVal) + rescue ::InvalidInputException => o1 + result.o1 = o1 + rescue ::InvalidObjectException => o2 + result.o2 = o2 + rescue ::InvalidOperationException => o3 + result.o3 = o3 + rescue ::AlreadyExistsException => o4 + result.o4 = o4 + rescue ::MetaException => o5 + result.o5 = o5 + end + write_result(result, oprot, 'rollStreamingPartition', seqid) + end + + def process_updateStreamingTempLocation(seqid, iprot, oprot) + args = read_args(iprot, UpdateStreamingTempLocation_args) + result = UpdateStreamingTempLocation_result.new() + begin + @handler.updateStreamingTempLocation(args.dbName, args.tableName, args.streamingTempDir) + rescue ::InvalidInputException => o1 + result.o1 = o1 + rescue ::InvalidObjectException => o2 + result.o2 = o2 + rescue ::MetaException => o3 + result.o3 = o3 + end + write_result(result, oprot, 'updateStreamingTempLocation', seqid) + end + + def process_getNextChunkID(seqid, iprot, oprot) + args = read_args(iprot, GetNextChunkID_args) + result = GetNextChunkID_result.new() + begin + result.success = @handler.getNextChunkID(args.db_name, args.table_name) + rescue ::InvalidInputException => o1 + result.o1 = o1 + rescue ::InvalidObjectException => o2 + result.o2 = o2 + rescue ::InvalidOperationException => o3 + result.o3 = o3 + rescue ::MetaException => o4 + result.o4 = o4 + end + write_result(result, oprot, 'getNextChunkID', seqid) + end + + def process_getStreamingTmpDir(seqid, iprot, oprot) + args = read_args(iprot, GetStreamingTmpDir_args) + result = GetStreamingTmpDir_result.new() + begin + result.success = @handler.getStreamingTmpDir(args.dbName, args.tableName) + rescue ::InvalidInputException => o1 + result.o1 = o1 + rescue ::InvalidObjectException => o2 + result.o2 = o2 + rescue ::InvalidOperationException => o3 + result.o3 = o3 + rescue ::MetaException => o4 + result.o4 = o4 + end + write_result(result, oprot, 'getStreamingTmpDir', seqid) + end + def process_create_role(seqid, iprot, oprot) args = read_args(iprot, Create_role_args) result = Create_role_result.new() @@ -5242,6 +5524,346 @@ module ThriftHiveMetastore ::Thrift::Struct.generate_accessors self end + class StreamingStatus_args + include ::Thrift::Struct, ::Thrift::Struct_Union + DB_NAME = 1 + TABLE_NAME = 2 + + FIELDS = { + DB_NAME => {:type => ::Thrift::Types::STRING, :name => 'db_name'}, + TABLE_NAME => {:type => ::Thrift::Types::STRING, :name => 'table_name'} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class StreamingStatus_result + include ::Thrift::Struct, ::Thrift::Struct_Union + SUCCESS = 0 + O1 = 1 + O2 = 2 + O3 = 3 + + FIELDS = { + SUCCESS => {:type => ::Thrift::Types::MAP, :name => 'success', :key => {:type => ::Thrift::Types::STRING}, :value => {:type => ::Thrift::Types::STRING}}, + O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::InvalidInputException}, + O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::InvalidObjectException}, + O3 => {:type => ::Thrift::Types::STRUCT, :name => 'o3', :class => ::MetaException} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class EnableStreaming_args + include ::Thrift::Struct, ::Thrift::Struct_Union + DB_NAME = 1 + TABLE_NAME = 2 + STREAMING_TMP_DIR = 3 + PARTITION_PATH = 4 + PARTITION_VAL = 5 + + FIELDS = { + DB_NAME => {:type => ::Thrift::Types::STRING, :name => 'db_name'}, + TABLE_NAME => {:type => ::Thrift::Types::STRING, :name => 'table_name'}, + STREAMING_TMP_DIR => {:type => ::Thrift::Types::STRING, :name => 'streaming_tmp_dir'}, + PARTITION_PATH => {:type => ::Thrift::Types::STRING, :name => 'partition_path'}, + PARTITION_VAL => {:type => ::Thrift::Types::STRING, :name => 'partition_val'} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class EnableStreaming_result + include ::Thrift::Struct, ::Thrift::Struct_Union + O1 = 1 + O2 = 2 + O3 = 3 + O4 = 4 + + FIELDS = { + O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::InvalidInputException}, + O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::InvalidObjectException}, + O3 => {:type => ::Thrift::Types::STRUCT, :name => 'o3', :class => ::InvalidOperationException}, + O4 => {:type => ::Thrift::Types::STRUCT, :name => 'o4', :class => ::MetaException} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class DisableStreaming_args + include ::Thrift::Struct, ::Thrift::Struct_Union + DB_NAME = 1 + TABLE_NAME = 2 + + FIELDS = { + DB_NAME => {:type => ::Thrift::Types::STRING, :name => 'db_name'}, + TABLE_NAME => {:type => ::Thrift::Types::STRING, :name => 'table_name'} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class DisableStreaming_result + include ::Thrift::Struct, ::Thrift::Struct_Union + O1 = 1 + O2 = 2 + O3 = 3 + O4 = 4 + + FIELDS = { + O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::InvalidInputException}, + O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::InvalidObjectException}, + O3 => {:type => ::Thrift::Types::STRUCT, :name => 'o3', :class => ::InvalidOperationException}, + O4 => {:type => ::Thrift::Types::STRUCT, :name => 'o4', :class => ::MetaException} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class GetCurrentStreamingPartitionPath_args + include ::Thrift::Struct, ::Thrift::Struct_Union + DBNAME = 1 + TABLENAME = 2 + + FIELDS = { + DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'}, + TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tableName'} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class GetCurrentStreamingPartitionPath_result + include ::Thrift::Struct, ::Thrift::Struct_Union + SUCCESS = 0 + O1 = 1 + O2 = 2 + O3 = 3 + O4 = 4 + + FIELDS = { + SUCCESS => {:type => ::Thrift::Types::STRING, :name => 'success'}, + O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::InvalidInputException}, + O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::InvalidObjectException}, + O3 => {:type => ::Thrift::Types::STRUCT, :name => 'o3', :class => ::InvalidOperationException}, + O4 => {:type => ::Thrift::Types::STRUCT, :name => 'o4', :class => ::MetaException} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class RollStreamingPartition_args + include ::Thrift::Struct, ::Thrift::Struct_Union + DBNAME = 1 + TABLENAME = 2 + NEWPARTITIONPATH = 3 + NEWPARTITIONVAL = 4 + + FIELDS = { + DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'}, + TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tableName'}, + NEWPARTITIONPATH => {:type => ::Thrift::Types::STRING, :name => 'newPartitionPath'}, + NEWPARTITIONVAL => {:type => ::Thrift::Types::STRING, :name => 'newPartitionVal'} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class RollStreamingPartition_result + include ::Thrift::Struct, ::Thrift::Struct_Union + SUCCESS = 0 + O1 = 1 + O2 = 2 + O3 = 3 + O4 = 4 + O5 = 5 + + FIELDS = { + SUCCESS => {:type => ::Thrift::Types::STRING, :name => 'success'}, + O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::InvalidInputException}, + O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::InvalidObjectException}, + O3 => {:type => ::Thrift::Types::STRUCT, :name => 'o3', :class => ::InvalidOperationException}, + O4 => {:type => ::Thrift::Types::STRUCT, :name => 'o4', :class => ::AlreadyExistsException}, + O5 => {:type => ::Thrift::Types::STRUCT, :name => 'o5', :class => ::MetaException} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class UpdateStreamingTempLocation_args + include ::Thrift::Struct, ::Thrift::Struct_Union + DBNAME = 1 + TABLENAME = 2 + STREAMINGTEMPDIR = 3 + + FIELDS = { + DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'}, + TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tableName'}, + STREAMINGTEMPDIR => {:type => ::Thrift::Types::STRING, :name => 'streamingTempDir'} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class UpdateStreamingTempLocation_result + include ::Thrift::Struct, ::Thrift::Struct_Union + O1 = 1 + O2 = 2 + O3 = 3 + + FIELDS = { + O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::InvalidInputException}, + O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::InvalidObjectException}, + O3 => {:type => ::Thrift::Types::STRUCT, :name => 'o3', :class => ::MetaException} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class GetNextChunkID_args + include ::Thrift::Struct, ::Thrift::Struct_Union + DB_NAME = 1 + TABLE_NAME = 2 + + FIELDS = { + DB_NAME => {:type => ::Thrift::Types::STRING, :name => 'db_name'}, + TABLE_NAME => {:type => ::Thrift::Types::STRING, :name => 'table_name'} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class GetNextChunkID_result + include ::Thrift::Struct, ::Thrift::Struct_Union + SUCCESS = 0 + O1 = 1 + O2 = 2 + O3 = 3 + O4 = 4 + + FIELDS = { + SUCCESS => {:type => ::Thrift::Types::I64, :name => 'success'}, + O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::InvalidInputException}, + O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::InvalidObjectException}, + O3 => {:type => ::Thrift::Types::STRUCT, :name => 'o3', :class => ::InvalidOperationException}, + O4 => {:type => ::Thrift::Types::STRUCT, :name => 'o4', :class => ::MetaException} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class GetStreamingTmpDir_args + include ::Thrift::Struct, ::Thrift::Struct_Union + DBNAME = 1 + TABLENAME = 2 + + FIELDS = { + DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'}, + TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tableName'} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class GetStreamingTmpDir_result + include ::Thrift::Struct, ::Thrift::Struct_Union + SUCCESS = 0 + O1 = 1 + O2 = 2 + O3 = 3 + O4 = 4 + + FIELDS = { + SUCCESS => {:type => ::Thrift::Types::STRING, :name => 'success'}, + O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::InvalidInputException}, + O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::InvalidObjectException}, + O3 => {:type => ::Thrift::Types::STRUCT, :name => 'o3', :class => ::InvalidOperationException}, + O4 => {:type => ::Thrift::Types::STRUCT, :name => 'o4', :class => ::MetaException} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + class Create_role_args include ::Thrift::Struct, ::Thrift::Struct_Union ROLE = 1 diff --git metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java index 39dda92..788c62e 100644 --- metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java +++ metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java @@ -19,9 +19,7 @@ package org.apache.hadoop.hive.metastore; import static org.apache.commons.lang.StringUtils.join; -import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_COMMENT; -import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME; -import static org.apache.hadoop.hive.metastore.MetaStoreUtils.validateName; +import static org.apache.hadoop.hive.metastore.MetaStoreUtils.*; import java.io.IOException; import java.util.AbstractMap; @@ -171,8 +169,7 @@ public TTransport getTransport(TTransport trans) { } } - public static class HMSHandler extends FacebookBase implements - IHMSHandler { + public static class HMSHandler extends FacebookBase implements IHMSHandler { public static final Log LOG = HiveMetaStore.LOG; private static boolean createDefaultDB = false; private String rawStoreClassName; @@ -4053,6 +4050,172 @@ public boolean partition_name_has_valid_characters(List part_vals, endFunction("partition_name_has_valid_characters", true, null); return ret; } + + @Override + public Map streamingStatus(String dbName, String tableName) + throws InvalidInputException, InvalidObjectException, MetaException { + startFunction("streamingStatus", ": DB=" + dbName + ", Table="+ tableName); + boolean success = true; + try { + RawStore metastore = getMS(); + Map result = metastore.streamingStatus(dbName, tableName); + success= true; + return result; + } finally { + endFunction("streamingStatus", success, null); + } + } + + @Override + public void enableStreaming(String dbName, String tableName, String streamingTmpDir + , String partitionPath, String partitionVal) + throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException { + startFunction("enableStreaming", ": DB=" + dbName + ", Table="+ tableName + + ", StreamingTmpDir=" + streamingTmpDir + ", partitionPath=" + partitionPath + + ", partitionVal=" + partitionVal); + boolean success = true; + try { + RawStore metastore = getMS(); + metastore.enableStreaming(dbName, tableName, streamingTmpDir, partitionPath, partitionVal); + success= true; + } finally { + endFunction("enableStreaming", success, null); + } + } + + @Override + public void disableStreaming(String dbName, String tableName) + throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException { + startFunction("disableStreaming", ": DB=" + dbName + ", Table="+ tableName); + boolean success = false; + try { + RawStore metastore = getMS(); + metastore.disableStreaming(dbName, tableName); + success = true; + } finally { + endFunction("disableStreaming", success, null); + } + } + + @Override + public String getCurrentStreamingPartitionPath(String dbName, String tableName) + throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException { + startFunction("getCurrentStreamingPartition", ": DB=" + dbName + ", Table="+ tableName); + boolean success = false; + try { + RawStore metastore = getMS(); + if(dbName == null || tableName==null) + throw new InvalidInputException("dbName and tableName parameters cannot be null"); + String result = metastore.getCurrentStreamingPartitionPath(dbName, tableName); + success = true; + return result; + } finally { + endFunction("getCurrentStreamingPartition", success, null); + } + } + + private static Partition makePartitionObject( Table tbl, String partLocation, String partVal) { + Partition newPart = new Partition(); + newPart.setDbName(tbl.getDbName()); + newPart.setTableName(tbl.getTableName()); + List ptnVals = new ArrayList(); + ptnVals.add(partVal); + newPart.setValues(ptnVals); + newPart.setParameters(new HashMap()); + newPart.setSd(tbl.getSd().deepCopy()); + newPart.getSd().setSerdeInfo(tbl.getSd().getSerdeInfo().deepCopy()); + newPart.getSd().setLocation(partLocation); + return newPart; + } + + @Override + public String rollStreamingPartition(String dbName, String tableName + , String newPartitionPath, String newPartitionVal) + throws InvalidInputException, InvalidObjectException, InvalidOperationException + , AlreadyExistsException, MetaException { + startFunction("rollStreamingPartition", ": DB=" + dbName + ", Table="+ tableName + + ", newPartitionPath=" + newPartitionPath + ", newPartitionVal=" + newPartitionVal); + boolean success = false; + Exception ex = null; + if(dbName == null || tableName==null) + throw new InvalidInputException("dbName and tableName parameters cannot be null"); + + RawStore metastore = getMS(); + try { + metastore.openTransaction(); + // 1) check if streaming enabled + Map res = metastore.streamingStatus(dbName, tableName); + if( res.get("streaming_enabled").equalsIgnoreCase("false") ) { + throw new InvalidOperationException("Cannot roll partition as streaming is disabled on table : " + dbName + "." + tableName); + } + String currPath = metastore.getCurrentStreamingPartitionPath(dbName, tableName); + String currVal = metastore.getCurrentStreamingPartitionVal(dbName, tableName); + if( newPartitionVal.compareToIgnoreCase(currVal)==0 ) // TODO: check if this should be case sensitive + throw new InvalidOperationException("New streaming partition's value is same as current streaming partition"); + + // 2) Point streaming partition to new location + metastore.updateCurrentStreamingPartition(dbName, tableName, newPartitionPath, newPartitionVal); + + // 3) Convert old streaming partition to standard partition by adding it to table + Table tbl = metastore.getTable(dbName,tableName); + Partition currPart = makePartitionObject(tbl, currPath, currVal); + add_partition_core_notxn(metastore, currPart, null); + + metastore.commitTransaction(); + success = true; + return currPath; + } finally { + if(!success) + metastore.rollbackTransaction(); + endFunction("rollStreamingPartition", success, null); + } + } + + @Override + public void updateStreamingTempLocation(String dbName, String tableName, String streamingTempDir) + throws InvalidInputException, InvalidObjectException, MetaException { + startFunction("updateStreamingTempLocation", ": DB=" + dbName + ", Table="+ tableName); + boolean success = false; + Exception ex = null; + try { + RawStore metastore = getMS(); + metastore.updateStreamingTempLocation(dbName, tableName, streamingTempDir); + success = true; + } finally { + endFunction("updateStreamingTempLocation", success, null); + } + } + + @Override + public long getNextChunkID(String dbName, String tableName) + throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException { + startFunction("getNextChunkID", ": DB=" + dbName + ", Table=" + tableName); + boolean success = false; + try { + RawStore metastore = getMS(); + Long result = metastore.getNextChunkID(dbName, tableName); + success = true; + return result; + } finally { + endFunction("getNextChunkID", success, null); + } + } + + @Override + public String getStreamingTmpDir(String dbName, String tableName) + throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException { + startFunction("getStreamingTmpDir", ": DB=" + dbName + ", Table="+ tableName); + boolean success = false; + try { + RawStore metastore = getMS(); + String result = metastore.getStreamingTmpDir(dbName, tableName); + success = true; + return result; + } finally { + endFunction("getStreamingTmpDir", success, null); + } + } + } public static IHMSHandler newHMSHandler(String name, HiveConf hiveConf) throws MetaException { diff --git metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java index e165b2d..14bd540 100644 --- metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java +++ metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java @@ -1399,4 +1399,73 @@ public boolean isPartitionMarkedForEvent(String db_name, String tbl_name, Map streamingStatus(String dbName, String tableName) + throws InvalidInputException, InvalidObjectException, MetaException, TException { + assert dbName != null; + assert tableName != null; + return client.streamingStatus(dbName, tableName); + } + + @Override + public void enableStreaming(String dbName, String tableName, String streamingTempDir, String partitionPath, String partitionVal) + throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, TException { + assert dbName != null; + assert tableName != null; + assert streamingTempDir != null; + assert partitionPath != null; + assert partitionVal != null; + client.enableStreaming(dbName, tableName, streamingTempDir, partitionPath, partitionVal); + } + + @Override + public void disableStreaming(String dbName, String tableName) + throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, TException { + assert dbName != null; + assert tableName != null; + client.disableStreaming(dbName, tableName); + } + + @Override + public long getNextChunkID(String dbName, String tableName) + throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, TException { + assert dbName != null; + assert tableName != null; + return client.getNextChunkID(dbName, tableName); + } + + @Override + public String getStreamingTmpDir(String dbName, String tableName) + throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, TException { + assert dbName != null; + assert tableName != null; + return client.getStreamingTmpDir(dbName, tableName); + } + + @Override + public String getCurrentStreamingPartitionPath(String dbName, String tableName) + throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, TException { + assert dbName != null; + assert tableName != null; + return client.getCurrentStreamingPartitionPath(dbName, tableName); + } + + @Override + public String rollStreamingPartition(String dbName, String tableName, String newPartitionPath, String newPartitionVal) + throws InvalidInputException, InvalidObjectException, InvalidOperationException, AlreadyExistsException, MetaException, TException { + assert dbName != null; + assert tableName != null; + assert newPartitionVal != null; + return client.rollStreamingPartition(dbName, tableName, newPartitionPath, newPartitionVal); + } + + @Override + public void updateStreamingTempLocation(String dbName, String tableName, String streamingTempDir) + throws InvalidInputException, InvalidObjectException, MetaException, TException { + assert dbName != null; + assert tableName != null; + assert streamingTempDir != null; + client.updateStreamingTempLocation(dbName, tableName, streamingTempDir); + } + } diff --git metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java index a2d6b1b..35f401f 100644 --- metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java +++ metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java @@ -506,9 +506,138 @@ public boolean isPartitionMarkedForEvent(String db_name, String tbl_name, Map partVals) - throws TException, MetaException; + public void validatePartitionNameCharacters(List partVals) throws TException, MetaException; + /** + * Check streaming related parameter values for a table + * @param dbName + * @param tableName + * @return + */ + public Map streamingStatus(String dbName, String tableName) + throws InvalidInputException, InvalidObjectException, MetaException, TException; + + /** + * Sets up the location of the streaming temp directory for the table. + * Also initializes chunkId for the table to 0. If the table is already enabled + * for streaming, only the streamingTmpDir is updated. If streaminfTmpDir is null + * it defaults to a sub folder under the specified hive.metastore.streaming.tempdir + * hive configuration. + * + * @param dbName + * @param tableName + * @param streamingTempDir can be null + * @param partitionPath location for the chunks committed into the streaming partition + * @throws NoSuchObjectException if database or table is not found + * @throws InvalidInputException if arguments are null + * @throws MetaException on other errors when interacting with Metastore + */ + public void enableStreaming(String dbName, String tableName, String streamingTempDir, String partitionPath, String partitionVal) + throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, TException; + + /** + * Clears the location of the streaming temp directory for the table. NextChunkId will be left untouched. + * + * @param dbName + * @param tableName + * @throws NoSuchObjectException if database or table is not found + * @throws InvalidInputException if arguments are null + * @throws MetaException on other errors when interacting with Metastore + * @throws TException in case of Thrift RPC errors + */ + public void disableStreaming(String dbName, String tableName) + throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, TException; + +// /** +// * Returns the directory for the current streaming partition +// * +// * @param dbName +// * @param tableName +// * @return The directory of committed chunks for the currently open streaming partition of a table +// * @throws InvalidInputException if arguments are null +// * @throws InvalidObjectException if database or table is not found +// * @throws InvalidOperationException if streaming is not enabled on the table +// * @throws MetaException on other errors when interacting with Metastore +// * @throws TException in case of Thrift RPC errors +// */ +// public String getCurrentStreamingPartitionPath(String dbName, String tableName) +// throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, TException; + + /** + * Atomically sets the streaming partition and adds the old partition path to the + * table as a standard partition + * + * @return Old parition's location + * @param dbName + * @param tableName + * @param newPartitionPath The new location for committed chunks for the streaming partition + * @throws InvalidInputException if arguments are null + * @throws InvalidObjectException if database or table is not found + * @throws InvalidOperationException if streaming is not enabled on the table + * @throws MetaException on other errors when interacting with Metastore + */ + public String rollStreamingPartition(String dbName, String tableName, String newPartitionPath, String newPartitionVal) + throws InvalidInputException, InvalidObjectException, InvalidOperationException, AlreadyExistsException, MetaException, TException; + + + /** + * Updates the + * + * @param dbName + * @param tableName + * @param streamingTempDir The location for uncommitted chunks + * @throws InvalidInputException if arguments are null + * @throws InvalidObjectException if database or table is not found + * @throws MetaException on other errors when interacting with Metastore + */ + public void updateStreamingTempLocation(String dbName, String tableName, String streamingTempDir) + throws InvalidInputException, InvalidObjectException, MetaException, TException; + + + /** + * Increments the value of the chunkId and returns the new chunk id for the streaming partition. + * + * @param dbName + * @param tableName + * @return null if streaming is not enabled. Otherwise returns the incremented chunk id value + * @throws InvalidInputException if arguments are null + * @throws InvalidObjectException if database or table is not found + * @throws InvalidOperationException if streaming is not enabled on the table + * @throws MetaException on other errors when interacting with Metastore + * @throws TException in case of Thrift RPC errors + */ + public long getNextChunkID(String dbName, String tableName) + throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, TException ; + + + /** + * Increments the value of the chunkId and returns the new chunk id for the streaming partition. + * + * @param dbName + * @param tableName + * @return The streamingTmpDir for the db/table + * @throws InvalidInputException if arguments are null + * @throws InvalidObjectException if database or table is not found + * @throws InvalidOperationException if streaming is not enabled on the table + * @throws MetaException on other errors when interacting with Metastore + * @throws TException in case of Thrift RPC errors + */ + public String getStreamingTmpDir(String dbName, String tableName) + throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, TException; + + /** + * Get the path of the current streaming parition + * @param dbName + * @param tableName + * @return + * @throws InvalidInputException + * @throws InvalidObjectException + * @throws InvalidOperationException + * @throws MetaException + * @throws TException + */ + public String getCurrentStreamingPartitionPath(String dbName, String tableName) + throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException, TException; /** * @param tbl diff --git metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java index a27243d..04ac30e 100644 --- metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java +++ metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java @@ -23,18 +23,8 @@ import java.net.URI; import java.net.URISyntaxException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; +import java.util.*; import java.util.Map.Entry; -import java.util.Properties; -import java.util.Set; -import java.util.TreeMap; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; @@ -59,35 +49,7 @@ import org.apache.hadoop.hive.common.classification.InterfaceStability; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hadoop.hive.metastore.api.BinaryColumnStatsData; -import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData; -import org.apache.hadoop.hive.metastore.api.ColumnStatistics; -import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData; -import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc; -import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj; -import org.apache.hadoop.hive.metastore.api.Database; -import org.apache.hadoop.hive.metastore.api.DoubleColumnStatsData; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege; -import org.apache.hadoop.hive.metastore.api.HiveObjectRef; -import org.apache.hadoop.hive.metastore.api.HiveObjectType; -import org.apache.hadoop.hive.metastore.api.Index; -import org.apache.hadoop.hive.metastore.api.InvalidInputException; -import org.apache.hadoop.hive.metastore.api.InvalidObjectException; -import org.apache.hadoop.hive.metastore.api.InvalidPartitionException; -import org.apache.hadoop.hive.metastore.api.LongColumnStatsData; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; -import org.apache.hadoop.hive.metastore.api.Order; -import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.api.PartitionEventType; -import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet; -import org.apache.hadoop.hive.metastore.api.PrincipalType; -import org.apache.hadoop.hive.metastore.api.PrivilegeBag; -import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo; -import org.apache.hadoop.hive.metastore.api.Role; -import org.apache.hadoop.hive.metastore.api.SerDeInfo; -import org.apache.hadoop.hive.metastore.api.SkewedInfo; +import org.apache.hadoop.hive.metastore.api.*; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.StringColumnStatsData; import org.apache.hadoop.hive.metastore.api.Table; @@ -2232,6 +2194,214 @@ public void alterPartitions(String dbname, String name, List> part_ } } + @Override + public Map streamingStatus(String dbName, String tableName) throws InvalidInputException, InvalidObjectException, MetaException { + if(dbName == null || tableName==null) + throw new InvalidInputException("dbName and tableName parameters cannot be null"); + openTransaction(); + MTable mTable = getMTable(dbName, tableName); + if(mTable==null) + throw new InvalidObjectException(dbName + "/" + tableName + " is not a valid database/table name"); + + String streamingTempDir = mTable.getStreamingTmpDir(); + String location = mTable.getCurrentStreamingPartitionPath(); + String value = mTable.getCurrentStreamingPartitionVal(); + String chunkid = (mTable.getChunkId() == null) ? "null" : mTable.getChunkId().toString(); + String streamingEnabled = mTable.isStreamingEnabled() ? "true" : "false"; + commitTransaction(); + + HashMap result = new HashMap (); + result.put("streaming_enabled", streamingEnabled); + result.put("tempDir", streamingTempDir); + result.put("location", location); + result.put("value", value); + result.put("chunkid", chunkid); + return result; + } + + @Override + public void enableStreaming(String dbName, String tableName, String streamingTempDir, String partitionPath, String partitionVal) + throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException { + boolean success = false; + if(dbName == null || tableName==null || streamingTempDir==null || partitionPath==null || partitionVal==null) + throw new InvalidInputException("All parameters are required to be non null"); + try { + openTransaction(); + MTable mTable = getMTable(dbName, tableName); + if(mTable==null) + throw new InvalidObjectException(dbName + "/" + tableName + " is not a valid database/table name"); + if(mTable.isStreamingEnabled()) + throw new InvalidOperationException("Streaming is already enabled on " + dbName + "/" + tableName); + if( mTable.getChunkId() == null ) + mTable.setChunkId(0L); + mTable.setStreamingTmpDir( streamingTempDir ); + mTable.setCurrentStreamingPartitionPath( partitionPath ); + mTable.setCurrentStreamingPartitionVal( partitionVal ); + success = commitTransaction(); + } finally { + if (!success) + rollbackTransaction(); + } + } + @Override + public void disableStreaming(String dbName, String tableName) + throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException { + boolean success = false; + if(dbName == null || tableName==null) + throw new InvalidInputException("dbName and tableName parameters cannot be null"); + try { + openTransaction(); + MTable mTable = getMTable(dbName, tableName); + if(mTable==null) + throw new InvalidObjectException(dbName + "/" + tableName + " is not a valid database/table name"); + + if(!mTable.isStreamingEnabled()) + throw new InvalidOperationException("Streaming is already disabled on " + dbName + "/" + tableName); + + mTable.setStreamingTmpDir(null); + success = commitTransaction(); + } finally { + if (!success) + rollbackTransaction(); + } + } + + @Override + public long getNextChunkID(String dbName, String tableName) + throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException { + boolean success = false; + if(dbName == null || tableName==null) + throw new InvalidInputException("dbName and tableName parameters cannot be null"); + try { + openTransaction(); + MTable mTable = getMTable(dbName, tableName); + if(mTable==null) + throw new InvalidObjectException(dbName + "/" + tableName + " is not a valid database/table name"); + if(! mTable.isStreamingEnabled()) + throw new InvalidOperationException("Cannot increment chunk id on table if streaming is disabled"); + long nextChunkId = mTable.getChunkId(); + ++nextChunkId; + mTable.setChunkId(nextChunkId); + success = commitTransaction(); + if(success) + return nextChunkId; + throw new MetaException("Transaction commit failed"); + } finally { + if (!success) + rollbackTransaction(); + } + } + + @Override + public String getStreamingTmpDir(String dbName, String tableName) + throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException { + boolean success = false; + if(dbName == null || tableName==null) + throw new InvalidInputException("dbName and tableName parameters cannot be null"); + try { + openTransaction(); + MTable mTable = getMTable(dbName, tableName); + if(mTable==null) + throw new InvalidObjectException(dbName + "/" + tableName + " is not a valid database/table name"); + if(! mTable.isStreamingEnabled()) + throw new InvalidOperationException("Cannot increment chunk id on table if streaming is disabled"); + String streamingTmpDir = mTable.getStreamingTmpDir(); + success = commitTransaction(); + if(success) + return streamingTmpDir; + throw new MetaException("Transaction commit failed"); + } finally { + if (!success) + rollbackTransaction(); + } + } + + @Override + public String getCurrentStreamingPartitionPath(String dbName, String tableName) + throws InvalidObjectException, InvalidOperationException, MetaException { + boolean success = false; + try { + openTransaction(); + MTable mTable = getMTable(dbName, tableName); + if(mTable==null) + throw new InvalidObjectException(dbName + "/" + tableName + " is not a valid database/table name"); + if(! mTable.isStreamingEnabled()) + throw new InvalidOperationException("Cannot increment chunk id on table if streaming is disabled"); + String currentStreamingPartition = mTable.getCurrentStreamingPartitionPath(); + success = commitTransaction(); + if(success) + return currentStreamingPartition; + throw new MetaException("Transaction commit failed"); + } finally { + if (!success) + rollbackTransaction(); + } + } + + @Override + public String getCurrentStreamingPartitionVal(String dbName, String tableName) + throws InvalidObjectException, InvalidOperationException, MetaException { + boolean success = false; + try { + openTransaction(); + MTable mTable = getMTable(dbName, tableName); + if(mTable==null) + throw new InvalidObjectException(dbName + "/" + tableName + " is not a valid database/table name"); + if(! mTable.isStreamingEnabled()) + throw new InvalidOperationException("Cannot increment chunk id on table if streaming is disabled"); + String currentVal = mTable.getCurrentStreamingPartitionVal(); + success = commitTransaction(); + if(success) + return currentVal; + throw new MetaException("Transaction commit failed"); + } finally { + if (!success) + rollbackTransaction(); + } + } + + @Override + public void updateCurrentStreamingPartition(String dbName, String tableName, String newPartitionPath, String newPartitionVal) + throws InvalidObjectException, InvalidOperationException, MetaException { + MTable mTable = getMTable(dbName, tableName); + if(mTable==null) + throw new InvalidObjectException(dbName + "/" + tableName + " is not a valid database/table name"); + if(! mTable.isStreamingEnabled()) + throw new InvalidOperationException("Updating current streaming partition dir is not" + + " supported if streaming is disabled on the table"); + mTable.setCurrentStreamingPartitionPath(newPartitionPath); + mTable.setCurrentStreamingPartitionVal(newPartitionVal); + } + + + @Override + public void updateStreamingTempLocation(String dbName, String tableName, String streamingTempDir) + throws InvalidInputException, InvalidObjectException, MetaException { + boolean success = false; + if(dbName == null || tableName==null || streamingTempDir==null) + throw new InvalidInputException("dbname, tableName & partitionPath parameters cannot be null"); + try { + openTransaction(); + Table table = getTable(dbName, tableName); + if(table==null) + throw new InvalidObjectException(dbName + "/" + tableName + " is not a valid database/table name"); + MTable mTable = convertToMTable(table); + if(mTable==null) + throw new InvalidObjectException(table.getTableName() + " table does not exist"); + mTable.setStreamingTmpDir(streamingTempDir); + success = commitTransaction(); + } finally { + if (!success) + rollbackTransaction(); + } + } + + + private String getDefaultStreamingTmpDir(String dbName, String tableName) { + String tempDirBase = hiveConf.get(ConfVars.STREAMING_TMPDIR.toString()); + return tempDirBase + "/" + dbName + "/" + tableName ; + } + private void copyMSD(MStorageDescriptor newSd, MStorageDescriptor oldSd) { oldSd.setLocation(newSd.getLocation()); MColumnDescriptor oldCD = oldSd.getCD(); diff --git metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java index e410c3a..0208884 100644 --- metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java +++ metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java @@ -18,29 +18,12 @@ package org.apache.hadoop.hive.metastore; +import java.io.IOException; import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configurable; -import org.apache.hadoop.hive.metastore.api.ColumnStatistics; -import org.apache.hadoop.hive.metastore.api.Database; -import org.apache.hadoop.hive.metastore.api.Index; -import org.apache.hadoop.hive.metastore.api.InvalidInputException; -import org.apache.hadoop.hive.metastore.api.InvalidObjectException; -import org.apache.hadoop.hive.metastore.api.InvalidPartitionException; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; -import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.api.PartitionEventType; -import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet; -import org.apache.hadoop.hive.metastore.api.PrincipalType; -import org.apache.hadoop.hive.metastore.api.PrivilegeBag; -import org.apache.hadoop.hive.metastore.api.Role; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.api.Type; -import org.apache.hadoop.hive.metastore.api.UnknownDBException; -import org.apache.hadoop.hive.metastore.api.UnknownPartitionException; -import org.apache.hadoop.hive.metastore.api.UnknownTableException; +import org.apache.hadoop.hive.metastore.api.*; import org.apache.hadoop.hive.metastore.model.MDBPrivilege; import org.apache.hadoop.hive.metastore.model.MGlobalPrivilege; import org.apache.hadoop.hive.metastore.model.MPartitionColumnPrivilege; @@ -417,23 +400,163 @@ public abstract boolean deleteTableColumnStatistics(String dbName, String tableN String colName) throws NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException; - public abstract long cleanupEvents(); + /** + * Check streaming status of a table + * @param dbName + * @param tableName + * @return + * @throws InvalidInputException + * @throws InvalidObjectException + * @throws MetaException + */ + public Map streamingStatus(String dbName, String tableName) + throws InvalidInputException, InvalidObjectException, MetaException; + + /** + * Sets the locations of the streaming temp directory and streaming partition for the table. + * Also initializes chunkId for the table to 0 if it was not previously initialized. + * The directories specified in streamingTmpDir & partitionPath will be created if they do not + * already exist. + * + * @param dbName + * @param tableName + * @param streamingTempDir Location of uncommitted chunks. If null, defaults to a sub folder under the specified hive.metastore.streaming.tempdir + * @param partitionPath Location of the streaming partition into which files will be committed. + * @throws InvalidObjectException if database or table is not found + * @throws InvalidInputException if arguments are null + * @throws MetaException on other errors when interacting with Metastore + */ + public void enableStreaming(String dbName, String tableName, String streamingTempDir, String partitionPath, String partitionVal) + throws InvalidInputException, InvalidObjectException, MetaException, InvalidOperationException; - public abstract boolean addToken(String tokenIdentifier, String delegationToken); + /** + * Clears the location of the streaming temp directory for the table. NextChunkId will be left untouched. + * + * @param dbName + * @param tableName + * @throws InvalidInputException if arguments are null + * @throws InvalidObjectException if database or table is not found + * @throws MetaException on other errors when interacting with Metastore + */ + public void disableStreaming(String dbName, String tableName) + throws InvalidInputException, InvalidObjectException, MetaException, InvalidOperationException; - public abstract boolean removeToken(String tokenIdentifier); + /** + * Returns the directory for the current streaming partition + * + * @param dbName + * @param tableName + * @throws InvalidInputException if arguments are null + * @throws InvalidObjectException if database or table is not found + * @throws InvalidOperationException if streaming is not enabled on the table + * @throws MetaException on other errors when interacting with Metastore + */ + public String getCurrentStreamingPartitionPath(String dbName, String tableName) + throws InvalidObjectException, InvalidOperationException, MetaException; - public abstract String getToken(String tokenIdentifier); - public abstract List getAllTokenIdentifiers(); + /** + * Returns the parition value for the current streaming partition + * + * @param dbName + * @param tableName + * @throws InvalidInputException if arguments are null + * @throws InvalidObjectException if database or table is not found + * @throws InvalidOperationException if streaming is not enabled on the table + * @throws MetaException on other errors when interacting with Metastore + */ + public String getCurrentStreamingPartitionVal(String dbName, String tableName) + throws InvalidObjectException, InvalidOperationException, MetaException; + /** + * Sets the new path and value for streaming partition. + * + * @param dbName + * @param tableName + * @param newPartitionPath The location for committed chunks for the partition + * @param newPartitionVal + * @throws InvalidInputException if arguments are null + * @throws InvalidObjectException if database or table is not found + * @throws InvalidOperationException if streaming is not enabled on the table + * @throws MetaException on other errors when interacting with Metastore + */ + public void updateCurrentStreamingPartition(String dbName, String tableName, String newPartitionPath, String newPartitionVal) + throws InvalidObjectException, InvalidOperationException, MetaException; - public abstract int addMasterKey(String key) throws MetaException; + /** + * Increments the value of the chunkId and returns the new chunk id for the streaming partition. + * + * @param dbName + * @param tableName + * @return null if streaming is not enabled. Otherwise returns the incremented chunk id value + * @throws InvalidInputException if arguments are null + * @throws InvalidObjectException if database or table is not found + * @throws InvalidOperationException if streaming is not enabled on the table + * @throws MetaException on other errors when interacting with Metastore + */ + public long getNextChunkID(String dbName, String tableName) + throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException ; + + + /** + * Returns the location of the tmp directory for uncommitted chunks for a table's streaming partition + * + * @param dbName + * @param tableName + * @return The streamingTmpDir for the db/table + * @throws InvalidInputException if arguments are null + * @throws InvalidObjectException if database or table is not found + * @throws InvalidOperationException if streaming is not enabled on the table + * @throws MetaException on other errors when interacting with Metastore + */ + public String getStreamingTmpDir(String dbName, String tableName) + throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException; + +// /** +// * Atomically sets the streaming partition to newPartitionPath and adds the old partition path to the +// * table as a standard partition. Assumes that newPartitionPath refers to a valid location. +// * +// * @param dbName +// * @param tableName +// * @param newPartitionPath The location for committed chunks for the partition +// * @throws InvalidInputException if arguments are null +// * @throws InvalidObjectException if database or table is not found +// * @throws InvalidOperationException if streaming is not enabled on the table +// * @throws MetaException on other errors when interacting with Metastore +// */ +// public void rollStreamingPartition(String dbName, String tableName, String newPartitionPath) +// throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException; + + public abstract long cleanupEvents(); - public abstract void updateMasterKey(Integer seqNo, String key) + public abstract boolean addToken(String tokenIdentifier, String delegationToken); + + public abstract boolean removeToken(String tokenIdentifier); + + public abstract String getToken(String tokenIdentifier); + + public abstract List getAllTokenIdentifiers(); + + public abstract int addMasterKey(String key) throws MetaException; + + public abstract void updateMasterKey(Integer seqNo, String key) throws NoSuchObjectException, MetaException; - public abstract boolean removeMasterKey(Integer keySeq); + public abstract boolean removeMasterKey(Integer keySeq); + + public abstract String[] getMasterKeys(); - public abstract String[] getMasterKeys(); + /** + * Updates the temp location for uncommitted chunks for the table's streaming partition + * + * @param dbName + * @param tableName + * @param streamingTempDir The location for uncommitted chunks + * @throws InvalidInputException if arguments are null + * @throws InvalidObjectException if database or table is not found + * @throws InvalidOperationException if streaming is not enabled on the table + * @throws MetaException on other errors when interacting with Metastore + */ + public void updateStreamingTempLocation(String dbName, String tableName, String streamingTempDir) + throws InvalidInputException, InvalidObjectException, MetaException; } diff --git metastore/src/model/org/apache/hadoop/hive/metastore/model/MTable.java metastore/src/model/org/apache/hadoop/hive/metastore/model/MTable.java index 2a78ce9..5d14efd 100644 --- metastore/src/model/org/apache/hadoop/hive/metastore/model/MTable.java +++ metastore/src/model/org/apache/hadoop/hive/metastore/model/MTable.java @@ -22,7 +22,7 @@ import java.util.Map; public class MTable { - + private String tableName; private MDatabase database; private MStorageDescriptor sd; @@ -35,6 +35,13 @@ private String viewOriginalText; private String viewExpandedText; private String tableType; + private String currentStreamingPartitionPath; + private String currentStreamingPartitionVal; + private String streamingTmpDir; + private Long chunkId; + + public static final String STREAMING_DATE_FORMAT = "yyyy.MM.dd_HHmm"; + public static final String STREAMING_PARTITION_NAME = "streamingpartition"; public MTable() {} @@ -237,4 +244,69 @@ public void setTableType(String tableType) { public String getTableType() { return tableType; } + + /** + * @param path the location for current Streaming Partition's committed chunks + */ + public void setCurrentStreamingPartitionPath(String path) { + this.currentStreamingPartitionPath = path; + } + + /** + * @return the location of committed chunks for the current streaming partition + */ + public String getCurrentStreamingPartitionPath() { + return currentStreamingPartitionPath; + } + + + /** + * @param val the value for the current streaming partition + */ + public void setCurrentStreamingPartitionVal(String val) { + this.currentStreamingPartitionVal = val; + } + + /** + * @@return the value for the current streaming partition + */ + public String getCurrentStreamingPartitionVal() { + return currentStreamingPartitionVal; + } + + /** + * @param streamingTmpDir the streamingTmpDir to set + */ + public void setStreamingTmpDir(String streamingTmpDir) { + this.streamingTmpDir = streamingTmpDir; + } + + /** + * @return the streamingTmpDir + */ + public String getStreamingTmpDir() { + return streamingTmpDir; + } + + /** + * @return check if streaming is enabled + */ + public boolean isStreamingEnabled() { + return streamingTmpDir!=null; + } + + + /** + * @param chunkId the chunkId value + */ + public void setChunkId(Long chunkId) { + this.chunkId = chunkId; + } + + /** + * @return the chunkId + */ + public Long getChunkId() { + return chunkId; + } } diff --git metastore/src/model/package.jdo metastore/src/model/package.jdo index c42b5b0..6d76539 100644 --- metastore/src/model/package.jdo +++ metastore/src/model/package.jdo @@ -176,6 +176,19 @@ + + + + + + + + + + + + + diff --git metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java index 8066784..53ba4db 100644 --- metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java +++ metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java @@ -24,25 +24,7 @@ import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.metastore.api.ColumnStatistics; -import org.apache.hadoop.hive.metastore.api.Database; -import org.apache.hadoop.hive.metastore.api.Index; -import org.apache.hadoop.hive.metastore.api.InvalidInputException; -import org.apache.hadoop.hive.metastore.api.InvalidObjectException; -import org.apache.hadoop.hive.metastore.api.InvalidPartitionException; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; -import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.api.PartitionEventType; -import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet; -import org.apache.hadoop.hive.metastore.api.PrincipalType; -import org.apache.hadoop.hive.metastore.api.PrivilegeBag; -import org.apache.hadoop.hive.metastore.api.Role; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.api.Type; -import org.apache.hadoop.hive.metastore.api.UnknownDBException; -import org.apache.hadoop.hive.metastore.api.UnknownPartitionException; -import org.apache.hadoop.hive.metastore.api.UnknownTableException; +import org.apache.hadoop.hive.metastore.api.*; import org.apache.hadoop.hive.metastore.model.MDBPrivilege; import org.apache.hadoop.hive.metastore.model.MGlobalPrivilege; import org.apache.hadoop.hive.metastore.model.MPartitionColumnPrivilege; @@ -498,11 +480,51 @@ public boolean deleteTableColumnStatistics(String dbName, String tableName, return objectStore.deleteTableColumnStatistics(dbName, tableName, colName); } + @Override + public void enableStreaming(String dbName, String tableName, String streamingTempDir, String partitionPath, String partitionVal) throws InvalidInputException, InvalidObjectException, MetaException { + objectStore.enableStreaming(dbName,tableName,streamingTempDir,partitionPath,partitionVal); + } + + @Override + public Map streamingStatus(String dbName, String tableName) throws InvalidInputException, InvalidObjectException, MetaException { + return objectStore.streamingStatus(dbName, tableName); + } + + @Override + public void disableStreaming(String dbName, String tableName) throws InvalidInputException, InvalidObjectException, MetaException { + objectStore.disableStreaming(dbName, tableName); + } + + @Override + public String getCurrentStreamingPartitionPath(String dbName, String tableName) throws InvalidObjectException, InvalidOperationException, MetaException { + return objectStore.getCurrentStreamingPartitionPath(dbName, tableName); + } + + @Override + public String getCurrentStreamingPartitionVal(String dbName, String tableName) throws InvalidObjectException, InvalidOperationException, MetaException { + return getCurrentStreamingPartitionVal(dbName, tableName); + } + + @Override + public void updateCurrentStreamingPartition(String dbName, String tableName, String newPartitionPath, String newPartitionVal) throws InvalidObjectException, InvalidOperationException, MetaException { + objectStore.updateCurrentStreamingPartition(dbName, tableName, newPartitionPath, newPartitionVal); + } + + @Override + public long getNextChunkID(String dbName, String tableName) throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException { + return objectStore.getNextChunkID(dbName, tableName); + } + + @Override + public String getStreamingTmpDir(String dbName, String tableName) throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException { + return objectStore.getStreamingTmpDir(dbName, tableName); + } + public boolean deletePartitionColumnStatistics(String dbName, String tableName, String partName, List partVals, String colName) throws NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException { - return objectStore.deletePartitionColumnStatistics(dbName, tableName, partName, + return objectStore.deletePartitionColumnStatistics(dbName, tableName, partName, partVals, colName); } @@ -560,4 +582,9 @@ public boolean removeMasterKey(Integer keySeq) { return new String[0]; } + @Override + public void updateStreamingTempLocation(String dbName, String tableName, String streamingTempDir) throws InvalidInputException, InvalidObjectException, MetaException { + //To change body of implemented methods use File | Settings | File Templates. + } + } diff --git metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java index 0f9b16c..61f8830 100644 --- metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java +++ metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java @@ -25,25 +25,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.api.ColumnStatistics; -import org.apache.hadoop.hive.metastore.api.Database; -import org.apache.hadoop.hive.metastore.api.Index; -import org.apache.hadoop.hive.metastore.api.InvalidInputException; -import org.apache.hadoop.hive.metastore.api.InvalidObjectException; -import org.apache.hadoop.hive.metastore.api.InvalidPartitionException; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; -import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.api.PartitionEventType; -import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet; -import org.apache.hadoop.hive.metastore.api.PrincipalType; -import org.apache.hadoop.hive.metastore.api.PrivilegeBag; -import org.apache.hadoop.hive.metastore.api.Role; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.api.Type; -import org.apache.hadoop.hive.metastore.api.UnknownDBException; -import org.apache.hadoop.hive.metastore.api.UnknownPartitionException; -import org.apache.hadoop.hive.metastore.api.UnknownTableException; +import org.apache.hadoop.hive.metastore.api.*; import org.apache.hadoop.hive.metastore.model.MDBPrivilege; import org.apache.hadoop.hive.metastore.model.MGlobalPrivilege; import org.apache.hadoop.hive.metastore.model.MPartitionColumnPrivilege; @@ -553,7 +535,12 @@ public boolean removeMasterKey(Integer keySeq) { return null; } - @Override + @Override + public void updateStreamingTempLocation(String dbName, String tableName, String streamingTempDir) throws InvalidInputException, InvalidObjectException, MetaException { + //To change body of implemented methods use File | Settings | File Templates. + } + + @Override public ColumnStatistics getTableColumnStatistics(String dbName, String tableName, String colName) throws MetaException, NoSuchObjectException { return null; @@ -567,8 +554,48 @@ public boolean deleteTableColumnStatistics(String dbName, String tableName, return false; } + @Override + public void enableStreaming(String dbName, String tableName, String streamingTempDir, String partitionPath, String partitionVal) throws InvalidInputException, InvalidObjectException, MetaException { + return; + } + + @Override + public Map streamingStatus(String dbName, String tableName) throws InvalidInputException, InvalidObjectException, MetaException { + return null; + } + + @Override + public void disableStreaming(String dbName, String tableName) throws InvalidInputException, InvalidObjectException, MetaException { + return; + } + + @Override + public String getCurrentStreamingPartitionPath(String dbName, String tableName) throws InvalidObjectException, InvalidOperationException, MetaException { + return null; + } + + @Override + public String getCurrentStreamingPartitionVal(String dbName, String tableName) throws InvalidObjectException, InvalidOperationException, MetaException { + return null; + } + + @Override + public void updateCurrentStreamingPartition(String dbName, String tableName, String newPartitionPath, String newPartitionVal) throws InvalidObjectException, InvalidOperationException, MetaException { + return; + } + + @Override + public long getNextChunkID(String dbName, String tableName) throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException { + return 0; + } + + @Override + public String getStreamingTmpDir(String dbName, String tableName) throws InvalidInputException, InvalidObjectException, InvalidOperationException, MetaException { + return null; + } + - public boolean deletePartitionColumnStatistics(String dbName, String tableName, + public boolean deletePartitionColumnStatistics(String dbName, String tableName, String partName, List partVals, String colName) throws NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException { diff --git metastore/src/test/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java metastore/src/test/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java index 80fbbac..b4bca31 100644 --- metastore/src/test/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java +++ metastore/src/test/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java @@ -2661,4 +2661,8 @@ private void createMultiPartitionTableSchema(String dbName, String tblName, createPartitions(dbName, tbl, values); } + + public void testStreamingEnable() throws Throwable { + + } } diff --git serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/test/ThriftTestObj.java serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/test/ThriftTestObj.java index dda3c5f..1b708dd 100644 --- serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/test/ThriftTestObj.java +++ serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/test/ThriftTestObj.java @@ -528,7 +528,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, ThriftTestObj struc struct.field3 = new ArrayList(_list0.size); for (int _i1 = 0; _i1 < _list0.size; ++_i1) { - InnerStruct _elem2; // optional + InnerStruct _elem2; // required _elem2 = new InnerStruct(); _elem2.read(iprot); struct.field3.add(_elem2); @@ -636,7 +636,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, ThriftTestObj struct struct.field3 = new ArrayList(_list5.size); for (int _i6 = 0; _i6 < _list5.size; ++_i6) { - InnerStruct _elem7; // optional + InnerStruct _elem7; // required _elem7 = new InnerStruct(); _elem7.read(iprot); struct.field3.add(_elem7); diff --git serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/Complex.java serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/Complex.java index aa404bf..e36a792 100644 --- serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/Complex.java +++ serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/Complex.java @@ -836,7 +836,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Complex struct) thr struct.lint = new ArrayList(_list0.size); for (int _i1 = 0; _i1 < _list0.size; ++_i1) { - int _elem2; // optional + int _elem2; // required _elem2 = iprot.readI32(); struct.lint.add(_elem2); } @@ -854,7 +854,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Complex struct) thr struct.lString = new ArrayList(_list3.size); for (int _i4 = 0; _i4 < _list3.size; ++_i4) { - String _elem5; // optional + String _elem5; // required _elem5 = iprot.readString(); struct.lString.add(_elem5); } @@ -872,7 +872,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Complex struct) thr struct.lintString = new ArrayList(_list6.size); for (int _i7 = 0; _i7 < _list6.size; ++_i7) { - IntString _elem8; // optional + IntString _elem8; // required _elem8 = new IntString(); _elem8.read(iprot); struct.lintString.add(_elem8); @@ -1074,7 +1074,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, Complex struct) thro struct.lint = new ArrayList(_list21.size); for (int _i22 = 0; _i22 < _list21.size; ++_i22) { - int _elem23; // optional + int _elem23; // required _elem23 = iprot.readI32(); struct.lint.add(_elem23); } @@ -1087,7 +1087,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, Complex struct) thro struct.lString = new ArrayList(_list24.size); for (int _i25 = 0; _i25 < _list24.size; ++_i25) { - String _elem26; // optional + String _elem26; // required _elem26 = iprot.readString(); struct.lString.add(_elem26); } @@ -1100,7 +1100,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, Complex struct) thro struct.lintString = new ArrayList(_list27.size); for (int _i28 = 0; _i28 < _list27.size; ++_i28) { - IntString _elem29; // optional + IntString _elem29; // required _elem29 = new IntString(); _elem29.read(iprot); struct.lintString.add(_elem29); diff --git serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/MegaStruct.java serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/MegaStruct.java index fba49e4..386fef9 100644 --- serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/MegaStruct.java +++ serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde2/thrift/test/MegaStruct.java @@ -2280,7 +2280,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, MegaStruct struct) _val19 = new ArrayList(_list20.size); for (int _i21 = 0; _i21 < _list20.size; ++_i21) { - String _elem22; // optional + String _elem22; // required _elem22 = iprot.readString(); _val19.add(_elem22); } @@ -2310,7 +2310,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, MegaStruct struct) _val26 = new ArrayList(_list27.size); for (int _i28 = 0; _i28 < _list27.size; ++_i28) { - MiniStruct _elem29; // optional + MiniStruct _elem29; // required _elem29 = new MiniStruct(); _elem29.read(iprot); _val26.add(_elem29); @@ -2333,7 +2333,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, MegaStruct struct) struct.my_stringlist = new ArrayList(_list30.size); for (int _i31 = 0; _i31 < _list30.size; ++_i31) { - String _elem32; // optional + String _elem32; // required _elem32 = iprot.readString(); struct.my_stringlist.add(_elem32); } @@ -2351,7 +2351,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, MegaStruct struct) struct.my_structlist = new ArrayList(_list33.size); for (int _i34 = 0; _i34 < _list33.size; ++_i34) { - MiniStruct _elem35; // optional + MiniStruct _elem35; // required _elem35 = new MiniStruct(); _elem35.read(iprot); struct.my_structlist.add(_elem35); @@ -2370,7 +2370,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, MegaStruct struct) struct.my_enumlist = new ArrayList(_list36.size); for (int _i37 = 0; _i37 < _list36.size; ++_i37) { - MyEnum _elem38; // optional + MyEnum _elem38; // required _elem38 = MyEnum.findByValue(iprot.readI32()); struct.my_enumlist.add(_elem38); } @@ -2388,7 +2388,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, MegaStruct struct) struct.my_stringset = new HashSet(2*_set39.size); for (int _i40 = 0; _i40 < _set39.size; ++_i40) { - String _elem41; // optional + String _elem41; // required _elem41 = iprot.readString(); struct.my_stringset.add(_elem41); } @@ -2406,7 +2406,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, MegaStruct struct) struct.my_enumset = new HashSet(2*_set42.size); for (int _i43 = 0; _i43 < _set42.size; ++_i43) { - MyEnum _elem44; // optional + MyEnum _elem44; // required _elem44 = MyEnum.findByValue(iprot.readI32()); struct.my_enumset.add(_elem44); } @@ -2424,7 +2424,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, MegaStruct struct) struct.my_structset = new HashSet(2*_set45.size); for (int _i46 = 0; _i46 < _set45.size; ++_i46) { - MiniStruct _elem47; // optional + MiniStruct _elem47; // required _elem47 = new MiniStruct(); _elem47.read(iprot); struct.my_structset.add(_elem47); @@ -3023,7 +3023,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, MegaStruct struct) t _val95 = new ArrayList(_list96.size); for (int _i97 = 0; _i97 < _list96.size; ++_i97) { - String _elem98; // optional + String _elem98; // required _elem98 = iprot.readString(); _val95.add(_elem98); } @@ -3047,7 +3047,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, MegaStruct struct) t _val102 = new ArrayList(_list103.size); for (int _i104 = 0; _i104 < _list103.size; ++_i104) { - MiniStruct _elem105; // optional + MiniStruct _elem105; // required _elem105 = new MiniStruct(); _elem105.read(iprot); _val102.add(_elem105); @@ -3064,7 +3064,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, MegaStruct struct) t struct.my_stringlist = new ArrayList(_list106.size); for (int _i107 = 0; _i107 < _list106.size; ++_i107) { - String _elem108; // optional + String _elem108; // required _elem108 = iprot.readString(); struct.my_stringlist.add(_elem108); } @@ -3077,7 +3077,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, MegaStruct struct) t struct.my_structlist = new ArrayList(_list109.size); for (int _i110 = 0; _i110 < _list109.size; ++_i110) { - MiniStruct _elem111; // optional + MiniStruct _elem111; // required _elem111 = new MiniStruct(); _elem111.read(iprot); struct.my_structlist.add(_elem111); @@ -3091,7 +3091,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, MegaStruct struct) t struct.my_enumlist = new ArrayList(_list112.size); for (int _i113 = 0; _i113 < _list112.size; ++_i113) { - MyEnum _elem114; // optional + MyEnum _elem114; // required _elem114 = MyEnum.findByValue(iprot.readI32()); struct.my_enumlist.add(_elem114); } @@ -3104,7 +3104,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, MegaStruct struct) t struct.my_stringset = new HashSet(2*_set115.size); for (int _i116 = 0; _i116 < _set115.size; ++_i116) { - String _elem117; // optional + String _elem117; // required _elem117 = iprot.readString(); struct.my_stringset.add(_elem117); } @@ -3117,7 +3117,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, MegaStruct struct) t struct.my_enumset = new HashSet(2*_set118.size); for (int _i119 = 0; _i119 < _set118.size; ++_i119) { - MyEnum _elem120; // optional + MyEnum _elem120; // required _elem120 = MyEnum.findByValue(iprot.readI32()); struct.my_enumset.add(_elem120); } @@ -3130,7 +3130,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, MegaStruct struct) t struct.my_structset = new HashSet(2*_set121.size); for (int _i122 = 0; _i122 < _set121.size; ++_i122) { - MiniStruct _elem123; // optional + MiniStruct _elem123; // required _elem123 = new MiniStruct(); _elem123.read(iprot); struct.my_structset.add(_elem123);