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::vectorreadMessageEnd();
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