You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2011/12/29 23:59:03 UTC

svn commit: r1225683 [1/2] - in /hive/trunk: common/src/java/org/apache/hadoop/hive/conf/ conf/ metastore/if/ metastore/src/gen/thrift/gen-cpp/ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ metastore/src/gen/thrift/gen-php...

Author: hashutosh
Date: Thu Dec 29 22:59:02 2011
New Revision: 1225683

URL: http://svn.apache.org/viewvc?rev=1225683&view=rev
Log:
HIVE-2616 : Passing user identity from metastore client to server in non-secure mode (Ashutosh Chauhan)

Added:
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/TUGIBasedProcessor.java
    hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestSetUGIOnBothClientServer.java
    hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestSetUGIOnOnlyClient.java
    hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestSetUGIOnOnlyServer.java
    hive/trunk/shims/src/0.20S/java/org/apache/hadoop/hive/thrift/client/
    hive/trunk/shims/src/0.20S/java/org/apache/hadoop/hive/thrift/client/TUGIAssumingTransport.java
    hive/trunk/shims/src/common/java/org/apache/hadoop/hive/thrift/TFilterTransport.java
    hive/trunk/shims/src/common/java/org/apache/hadoop/hive/thrift/TUGIContainingTransport.java
Modified:
    hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
    hive/trunk/conf/hive-default.xml.template
    hive/trunk/metastore/if/hive_metastore.thrift
    hive/trunk/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
    hive/trunk/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
    hive/trunk/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
    hive/trunk/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
    hive/trunk/metastore/src/gen/thrift/gen-php/hive_metastore/ThriftHiveMetastore.php
    hive/trunk/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
    hive/trunk/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
    hive/trunk/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
    hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestRemoteHiveMetaStore.java
    hive/trunk/shims/ivy.xml
    hive/trunk/shims/src/0.20/java/org/apache/hadoop/hive/shims/Hadoop20Shims.java
    hive/trunk/shims/src/0.20S/java/org/apache/hadoop/hive/shims/Hadoop20SShims.java
    hive/trunk/shims/src/0.20S/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge20S.java
    hive/trunk/shims/src/common/java/org/apache/hadoop/hive/shims/HadoopShims.java

Modified: hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=1225683&r1=1225682&r2=1225683&view=diff
==============================================================================
--- hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Thu Dec 29 22:59:02 2011
@@ -281,6 +281,8 @@ public class HiveConf extends Configurat
     METASTORE_AUTHORIZATION_STORAGE_AUTH_CHECKS("hive.metastore.authorization.storage.checks", false),
     METASTORE_EVENT_CLEAN_FREQ("hive.metastore.event.clean.freq",0L),
     METASTORE_EVENT_EXPIRY_DURATION("hive.metastore.event.expiry.duration",0L),
+    METASTORE_EXECUTE_SET_UGI("hive.metastore.execute.setugi", false),
+
     METASTORE_MODE("hive.metastore.local",true),
     // Default parameters for creating tables
     NEWTABLEDEFAULTPARA("hive.table.parameters.default", ""),

Modified: hive/trunk/conf/hive-default.xml.template
URL: http://svn.apache.org/viewvc/hive/trunk/conf/hive-default.xml.template?rev=1225683&r1=1225682&r2=1225683&view=diff
==============================================================================
--- hive/trunk/conf/hive-default.xml.template (original)
+++ hive/trunk/conf/hive-default.xml.template Thu Dec 29 22:59:02 2011
@@ -232,6 +232,12 @@
 </property>
 
 <property>
+  <name>hive.metastore.execute.setugi</name>
+  <value>false</value>
+  <description>In unsecure mode, setting this property to true will cause the metastore to execute DFS operations using the client's reported user and group permissions. Note that this property must be set on both the client and server sides. Further note that its best effort. If client sets its to true and server sets it to false, client setting will be ignored.</description>
+</property>
+
+<property>
   <name>hive.metastore.event.listeners</name>
   <value></value>
   <description>list of comma seperated listeners for metastore events.</description>

Modified: hive/trunk/metastore/if/hive_metastore.thrift
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/if/hive_metastore.thrift?rev=1225683&r1=1225682&r2=1225683&view=diff
==============================================================================
--- hive/trunk/metastore/if/hive_metastore.thrift (original)
+++ hive/trunk/metastore/if/hive_metastore.thrift Thu Dec 29 22:59:02 2011
@@ -435,6 +435,10 @@ service ThriftHiveMetastore extends fb30
   bool grant_privileges(1:PrivilegeBag privileges) throws(1:MetaException o1)
   bool revoke_privileges(1:PrivilegeBag privileges) throws(1:MetaException o1)
   
+  // this is used by metastore client to send UGI information to metastore server immediately
+  // after setting up a connection. 
+  list<string> set_ugi(1:string user_name, 2:list<string> group_names) throws (1:MetaException o1)
+
   //Authentication (delegation token) interfaces
   
   // get metastore server delegation token for use from the map/reduce tasks to authenticate

Modified: hive/trunk/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp?rev=1225683&r1=1225682&r2=1225683&view=diff
==============================================================================
--- hive/trunk/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp (original)
+++ hive/trunk/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp Thu Dec 29 22:59:02 2011
@@ -14243,6 +14243,258 @@ uint32_t ThriftHiveMetastore_revoke_priv
   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 _size557;
+            ::apache::thrift::protocol::TType _etype560;
+            iprot->readListBegin(_etype560, _size557);
+            this->group_names.resize(_size557);
+            uint32_t _i561;
+            for (_i561 = 0; _i561 < _size557; ++_i561)
+            {
+              xfer += iprot->readString(this->group_names[_i561]);
+            }
+            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<uint32_t>(this->group_names.size()));
+    std::vector<std::string> ::const_iterator _iter562;
+    for (_iter562 = this->group_names.begin(); _iter562 != this->group_names.end(); ++_iter562)
+    {
+      xfer += oprot->writeString((*_iter562));
+    }
+    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<uint32_t>((*(this->group_names)).size()));
+    std::vector<std::string> ::const_iterator _iter563;
+    for (_iter563 = (*(this->group_names)).begin(); _iter563 != (*(this->group_names)).end(); ++_iter563)
+    {
+      xfer += oprot->writeString((*_iter563));
+    }
+    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 _size564;
+            ::apache::thrift::protocol::TType _etype567;
+            iprot->readListBegin(_etype567, _size564);
+            this->success.resize(_size564);
+            uint32_t _i568;
+            for (_i568 = 0; _i568 < _size564; ++_i568)
+            {
+              xfer += iprot->readString(this->success[_i568]);
+            }
+            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<uint32_t>(this->success.size()));
+      std::vector<std::string> ::const_iterator _iter569;
+      for (_iter569 = this->success.begin(); _iter569 != this->success.end(); ++_iter569)
+      {
+        xfer += oprot->writeString((*_iter569));
+      }
+      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 _size570;
+            ::apache::thrift::protocol::TType _etype573;
+            iprot->readListBegin(_etype573, _size570);
+            (*(this->success)).resize(_size570);
+            uint32_t _i574;
+            for (_i574 = 0; _i574 < _size570; ++_i574)
+            {
+              xfer += iprot->readString((*(this->success))[_i574]);
+            }
+            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;
@@ -18652,6 +18904,68 @@ bool ThriftHiveMetastoreClient::recv_rev
   throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "revoke_privileges failed: unknown result");
 }
 
+void ThriftHiveMetastoreClient::set_ugi(std::vector<std::string> & _return, const std::string& user_name, const std::vector<std::string> & group_names)
+{
+  send_set_ugi(user_name, group_names);
+  recv_set_ugi(_return);
+}
+
+void ThriftHiveMetastoreClient::send_set_ugi(const std::string& user_name, const std::vector<std::string> & group_names)
+{
+  int32_t cseqid = 0;
+  oprot_->writeMessageBegin("set_ugi", ::apache::thrift::protocol::T_CALL, cseqid);
+
+  ThriftHiveMetastore_set_ugi_pargs args;
+  args.user_name = &user_name;
+  args.group_names = &group_names;
+  args.write(oprot_);
+
+  oprot_->writeMessageEnd();
+  oprot_->getTransport()->writeEnd();
+  oprot_->getTransport()->flush();
+}
+
+void ThriftHiveMetastoreClient::recv_set_ugi(std::vector<std::string> & _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("set_ugi") != 0) {
+    iprot_->skip(::apache::thrift::protocol::T_STRUCT);
+    iprot_->readMessageEnd();
+    iprot_->getTransport()->readEnd();
+  }
+  ThriftHiveMetastore_set_ugi_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, "set_ugi failed: unknown result");
+}
+
 void ThriftHiveMetastoreClient::get_delegation_token(std::string& _return, const std::string& token_owner, const std::string& renewer_kerberos_principal_name)
 {
   send_get_delegation_token(token_owner, renewer_kerberos_principal_name);
@@ -22457,6 +22771,63 @@ void ThriftHiveMetastoreProcessor::proce
   }
 }
 
+void ThriftHiveMetastoreProcessor::process_set_ugi(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext)
+{
+  void* ctx = NULL;
+  if (eventHandler_.get() != NULL) {
+    ctx = eventHandler_->getContext("ThriftHiveMetastore.set_ugi", callContext);
+  }
+  ::apache::thrift::TProcessorContextFreer freer(eventHandler_.get(), ctx, "ThriftHiveMetastore.set_ugi");
+
+  if (eventHandler_.get() != NULL) {
+    eventHandler_->preRead(ctx, "ThriftHiveMetastore.set_ugi");
+  }
+
+  ThriftHiveMetastore_set_ugi_args args;
+  args.read(iprot);
+  iprot->readMessageEnd();
+  uint32_t bytes = iprot->getTransport()->readEnd();
+
+  if (eventHandler_.get() != NULL) {
+    eventHandler_->postRead(ctx, "ThriftHiveMetastore.set_ugi", bytes);
+  }
+
+  ThriftHiveMetastore_set_ugi_result result;
+  try {
+    iface_->set_ugi(result.success, args.user_name, args.group_names);
+    result.__isset.success = true;
+  } catch (MetaException &o1) {
+    result.o1 = o1;
+    result.__isset.o1 = true;
+  } catch (const std::exception& e) {
+    if (eventHandler_.get() != NULL) {
+      eventHandler_->handlerError(ctx, "ThriftHiveMetastore.set_ugi");
+    }
+
+    ::apache::thrift::TApplicationException x(e.what());
+    oprot->writeMessageBegin("set_ugi", ::apache::thrift::protocol::T_EXCEPTION, seqid);
+    x.write(oprot);
+    oprot->writeMessageEnd();
+    oprot->getTransport()->writeEnd();
+    oprot->getTransport()->flush();
+    return;
+  }
+
+  if (eventHandler_.get() != NULL) {
+    eventHandler_->preWrite(ctx, "ThriftHiveMetastore.set_ugi");
+  }
+
+  oprot->writeMessageBegin("set_ugi", ::apache::thrift::protocol::T_REPLY, seqid);
+  result.write(oprot);
+  oprot->writeMessageEnd();
+  bytes = oprot->getTransport()->writeEnd();
+  oprot->getTransport()->flush();
+
+  if (eventHandler_.get() != NULL) {
+    eventHandler_->postWrite(ctx, "ThriftHiveMetastore.set_ugi", bytes);
+  }
+}
+
 void ThriftHiveMetastoreProcessor::process_get_delegation_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext)
 {
   void* ctx = NULL;

Modified: hive/trunk/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h?rev=1225683&r1=1225682&r2=1225683&view=diff
==============================================================================
--- hive/trunk/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h (original)
+++ hive/trunk/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h Thu Dec 29 22:59:02 2011
@@ -75,6 +75,7 @@ class ThriftHiveMetastoreIf : virtual pu
   virtual void list_privileges(std::vector<HiveObjectPrivilege> & _return, const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject) = 0;
   virtual bool grant_privileges(const PrivilegeBag& privileges) = 0;
   virtual bool revoke_privileges(const PrivilegeBag& privileges) = 0;
+  virtual void set_ugi(std::vector<std::string> & _return, const std::string& user_name, const std::vector<std::string> & group_names) = 0;
   virtual void get_delegation_token(std::string& _return, const std::string& token_owner, const std::string& renewer_kerberos_principal_name) = 0;
   virtual int64_t renew_delegation_token(const std::string& token_str_form) = 0;
   virtual void cancel_delegation_token(const std::string& token_str_form) = 0;
@@ -276,6 +277,9 @@ class ThriftHiveMetastoreNull : virtual 
     bool _return = false;
     return _return;
   }
+  void set_ugi(std::vector<std::string> & /* _return */, const std::string& /* user_name */, const std::vector<std::string> & /* group_names */) {
+    return;
+  }
   void get_delegation_token(std::string& /* _return */, const std::string& /* token_owner */, const std::string& /* renewer_kerberos_principal_name */) {
     return;
   }
@@ -8677,6 +8681,133 @@ class ThriftHiveMetastore_revoke_privile
 
 };
 
+typedef struct _ThriftHiveMetastore_set_ugi_args__isset {
+  _ThriftHiveMetastore_set_ugi_args__isset() : user_name(false), group_names(false) {}
+  bool user_name;
+  bool group_names;
+} _ThriftHiveMetastore_set_ugi_args__isset;
+
+class ThriftHiveMetastore_set_ugi_args {
+ public:
+
+  ThriftHiveMetastore_set_ugi_args() : user_name("") {
+  }
+
+  virtual ~ThriftHiveMetastore_set_ugi_args() throw() {}
+
+  std::string user_name;
+  std::vector<std::string>  group_names;
+
+  _ThriftHiveMetastore_set_ugi_args__isset __isset;
+
+  void __set_user_name(const std::string& val) {
+    user_name = val;
+  }
+
+  void __set_group_names(const std::vector<std::string> & val) {
+    group_names = val;
+  }
+
+  bool operator == (const ThriftHiveMetastore_set_ugi_args & rhs) const
+  {
+    if (!(user_name == rhs.user_name))
+      return false;
+    if (!(group_names == rhs.group_names))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_set_ugi_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_set_ugi_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_set_ugi_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_set_ugi_pargs() throw() {}
+
+  const std::string* user_name;
+  const std::vector<std::string> * group_names;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_set_ugi_result__isset {
+  _ThriftHiveMetastore_set_ugi_result__isset() : success(false), o1(false) {}
+  bool success;
+  bool o1;
+} _ThriftHiveMetastore_set_ugi_result__isset;
+
+class ThriftHiveMetastore_set_ugi_result {
+ public:
+
+  ThriftHiveMetastore_set_ugi_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_set_ugi_result() throw() {}
+
+  std::vector<std::string>  success;
+  MetaException o1;
+
+  _ThriftHiveMetastore_set_ugi_result__isset __isset;
+
+  void __set_success(const std::vector<std::string> & val) {
+    success = val;
+  }
+
+  void __set_o1(const MetaException& val) {
+    o1 = val;
+  }
+
+  bool operator == (const ThriftHiveMetastore_set_ugi_result & rhs) const
+  {
+    if (!(success == rhs.success))
+      return false;
+    if (!(o1 == rhs.o1))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_set_ugi_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_set_ugi_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_set_ugi_presult__isset {
+  _ThriftHiveMetastore_set_ugi_presult__isset() : success(false), o1(false) {}
+  bool success;
+  bool o1;
+} _ThriftHiveMetastore_set_ugi_presult__isset;
+
+class ThriftHiveMetastore_set_ugi_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_set_ugi_presult() throw() {}
+
+  std::vector<std::string> * success;
+  MetaException o1;
+
+  _ThriftHiveMetastore_set_ugi_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
 typedef struct _ThriftHiveMetastore_get_delegation_token_args__isset {
   _ThriftHiveMetastore_get_delegation_token_args__isset() : token_owner(false), renewer_kerberos_principal_name(false) {}
   bool token_owner;
@@ -9222,6 +9353,9 @@ class ThriftHiveMetastoreClient : virtua
   bool revoke_privileges(const PrivilegeBag& privileges);
   void send_revoke_privileges(const PrivilegeBag& privileges);
   bool recv_revoke_privileges();
+  void set_ugi(std::vector<std::string> & _return, const std::string& user_name, const std::vector<std::string> & group_names);
+  void send_set_ugi(const std::string& user_name, const std::vector<std::string> & group_names);
+  void recv_set_ugi(std::vector<std::string> & _return);
   void get_delegation_token(std::string& _return, const std::string& token_owner, const std::string& renewer_kerberos_principal_name);
   void send_get_delegation_token(const std::string& token_owner, const std::string& renewer_kerberos_principal_name);
   void recv_get_delegation_token(std::string& _return);
@@ -9299,6 +9433,7 @@ class ThriftHiveMetastoreProcessor : vir
   void process_list_privileges(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_grant_privileges(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_revoke_privileges(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
+  void process_set_ugi(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_delegation_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_renew_delegation_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_cancel_delegation_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
@@ -9366,6 +9501,7 @@ class ThriftHiveMetastoreProcessor : vir
     processMap_["list_privileges"] = &ThriftHiveMetastoreProcessor::process_list_privileges;
     processMap_["grant_privileges"] = &ThriftHiveMetastoreProcessor::process_grant_privileges;
     processMap_["revoke_privileges"] = &ThriftHiveMetastoreProcessor::process_revoke_privileges;
+    processMap_["set_ugi"] = &ThriftHiveMetastoreProcessor::process_set_ugi;
     processMap_["get_delegation_token"] = &ThriftHiveMetastoreProcessor::process_get_delegation_token;
     processMap_["renew_delegation_token"] = &ThriftHiveMetastoreProcessor::process_renew_delegation_token;
     processMap_["cancel_delegation_token"] = &ThriftHiveMetastoreProcessor::process_cancel_delegation_token;
@@ -10049,6 +10185,18 @@ class ThriftHiveMetastoreMultiface : vir
     }
   }
 
+  void set_ugi(std::vector<std::string> & _return, const std::string& user_name, const std::vector<std::string> & group_names) {
+    size_t sz = ifaces_.size();
+    for (size_t i = 0; i < sz; ++i) {
+      if (i == sz - 1) {
+        ifaces_[i]->set_ugi(_return, user_name, group_names);
+        return;
+      } else {
+        ifaces_[i]->set_ugi(_return, user_name, group_names);
+      }
+    }
+  }
+
   void get_delegation_token(std::string& _return, const std::string& token_owner, const std::string& renewer_kerberos_principal_name) {
     size_t sz = ifaces_.size();
     for (size_t i = 0; i < sz; ++i) {

Modified: hive/trunk/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp?rev=1225683&r1=1225682&r2=1225683&view=diff
==============================================================================
--- hive/trunk/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp (original)
+++ hive/trunk/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp Thu Dec 29 22:59:02 2011
@@ -322,6 +322,11 @@ class ThriftHiveMetastoreHandler : virtu
     printf("revoke_privileges\n");
   }
 
+  void set_ugi(std::vector<std::string> & _return, const std::string& user_name, const std::vector<std::string> & group_names) {
+    // Your implementation goes here
+    printf("set_ugi\n");
+  }
+
   void get_delegation_token(std::string& _return, const std::string& token_owner, const std::string& renewer_kerberos_principal_name) {
     // Your implementation goes here
     printf("get_delegation_token\n");

Modified: hive/trunk/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java?rev=1225683&r1=1225682&r2=1225683&view=diff
==============================================================================
--- hive/trunk/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java (original)
+++ hive/trunk/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java Thu Dec 29 22:59:02 2011
@@ -147,6 +147,8 @@ public class ThriftHiveMetastore {
 
     public boolean revoke_privileges(PrivilegeBag privileges) throws MetaException, org.apache.thrift.TException;
 
+    public List<String> set_ugi(String user_name, List<String> group_names) throws MetaException, org.apache.thrift.TException;
+
     public String get_delegation_token(String token_owner, String renewer_kerberos_principal_name) throws MetaException, org.apache.thrift.TException;
 
     public long renew_delegation_token(String token_str_form) throws MetaException, org.apache.thrift.TException;
@@ -277,6 +279,8 @@ public class ThriftHiveMetastore {
 
     public void revoke_privileges(PrivilegeBag privileges, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.revoke_privileges_call> resultHandler) throws org.apache.thrift.TException;
 
+    public void set_ugi(String user_name, List<String> group_names, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.set_ugi_call> resultHandler) throws org.apache.thrift.TException;
+
     public void get_delegation_token(String token_owner, String renewer_kerberos_principal_name, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_delegation_token_call> resultHandler) throws org.apache.thrift.TException;
 
     public void renew_delegation_token(String token_str_form, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.renew_delegation_token_call> resultHandler) throws org.apache.thrift.TException;
@@ -2106,6 +2110,33 @@ public class ThriftHiveMetastore {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "revoke_privileges failed: unknown result");
     }
 
+    public List<String> set_ugi(String user_name, List<String> group_names) throws MetaException, org.apache.thrift.TException
+    {
+      send_set_ugi(user_name, group_names);
+      return recv_set_ugi();
+    }
+
+    public void send_set_ugi(String user_name, List<String> group_names) throws org.apache.thrift.TException
+    {
+      set_ugi_args args = new set_ugi_args();
+      args.setUser_name(user_name);
+      args.setGroup_names(group_names);
+      sendBase("set_ugi", args);
+    }
+
+    public List<String> recv_set_ugi() throws MetaException, org.apache.thrift.TException
+    {
+      set_ugi_result result = new set_ugi_result();
+      receiveBase(result, "set_ugi");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "set_ugi failed: unknown result");
+    }
+
     public String get_delegation_token(String token_owner, String renewer_kerberos_principal_name) throws MetaException, org.apache.thrift.TException
     {
       send_get_delegation_token(token_owner, renewer_kerberos_principal_name);
@@ -4393,6 +4424,41 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public void set_ugi(String user_name, List<String> group_names, org.apache.thrift.async.AsyncMethodCallback<set_ugi_call> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      set_ugi_call method_call = new set_ugi_call(user_name, group_names, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class set_ugi_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String user_name;
+      private List<String> group_names;
+      public set_ugi_call(String user_name, List<String> group_names, org.apache.thrift.async.AsyncMethodCallback<set_ugi_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.user_name = user_name;
+        this.group_names = group_names;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("set_ugi", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        set_ugi_args args = new set_ugi_args();
+        args.setUser_name(user_name);
+        args.setGroup_names(group_names);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public List<String> getResult() throws MetaException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_set_ugi();
+      }
+    }
+
     public void get_delegation_token(String token_owner, String renewer_kerberos_principal_name, org.apache.thrift.async.AsyncMethodCallback<get_delegation_token_call> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       get_delegation_token_call method_call = new get_delegation_token_call(token_owner, renewer_kerberos_principal_name, resultHandler, this, ___protocolFactory, ___transport);
@@ -4565,6 +4631,7 @@ public class ThriftHiveMetastore {
       processMap.put("list_privileges", new list_privileges());
       processMap.put("grant_privileges", new grant_privileges());
       processMap.put("revoke_privileges", new revoke_privileges());
+      processMap.put("set_ugi", new set_ugi());
       processMap.put("get_delegation_token", new get_delegation_token());
       processMap.put("renew_delegation_token", new renew_delegation_token());
       processMap.put("cancel_delegation_token", new cancel_delegation_token());
@@ -5904,6 +5971,26 @@ public class ThriftHiveMetastore {
       }
     }
 
+    private static class set_ugi<I extends Iface> extends org.apache.thrift.ProcessFunction<I, set_ugi_args> {
+      public set_ugi() {
+        super("set_ugi");
+      }
+
+      protected set_ugi_args getEmptyArgsInstance() {
+        return new set_ugi_args();
+      }
+
+      protected set_ugi_result getResult(I iface, set_ugi_args args) throws org.apache.thrift.TException {
+        set_ugi_result result = new set_ugi_result();
+        try {
+          result.success = iface.set_ugi(args.user_name, args.group_names);
+        } catch (MetaException o1) {
+          result.o1 = o1;
+        }
+        return result;
+      }
+    }
+
     private static class get_delegation_token<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_delegation_token_args> {
       public get_delegation_token() {
         super("get_delegation_token");
@@ -9012,7 +9099,7 @@ public class ThriftHiveMetastore {
         if (lastComparison != 0) {
           return lastComparison;
         }
-      }
+        }
       lastComparison = Boolean.valueOf(isSetO1()).compareTo(typedOther.isSetO1());
       if (lastComparison != 0) {
         return lastComparison;
@@ -60563,6 +60650,843 @@ public class ThriftHiveMetastore {
 
   }
 
+  public static class set_ugi_args implements org.apache.thrift.TBase<set_ugi_args, set_ugi_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("set_ugi_args");
+
+    private static final org.apache.thrift.protocol.TField USER_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("user_name", org.apache.thrift.protocol.TType.STRING, (short)1);
+    private static final org.apache.thrift.protocol.TField GROUP_NAMES_FIELD_DESC = new org.apache.thrift.protocol.TField("group_names", org.apache.thrift.protocol.TType.LIST, (short)2);
+
+    private String user_name; // required
+    private List<String> group_names; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      USER_NAME((short)1, "user_name"),
+      GROUP_NAMES((short)2, "group_names");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // USER_NAME
+            return USER_NAME;
+          case 2: // GROUP_NAMES
+            return GROUP_NAMES;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.USER_NAME, new org.apache.thrift.meta_data.FieldMetaData("user_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.GROUP_NAMES, new org.apache.thrift.meta_data.FieldMetaData("group_names", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(set_ugi_args.class, metaDataMap);
+    }
+
+    public set_ugi_args() {
+    }
+
+    public set_ugi_args(
+      String user_name,
+      List<String> group_names)
+    {
+      this();
+      this.user_name = user_name;
+      this.group_names = group_names;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public set_ugi_args(set_ugi_args other) {
+      if (other.isSetUser_name()) {
+        this.user_name = other.user_name;
+      }
+      if (other.isSetGroup_names()) {
+        List<String> __this__group_names = new ArrayList<String>();
+        for (String other_element : other.group_names) {
+          __this__group_names.add(other_element);
+        }
+        this.group_names = __this__group_names;
+      }
+    }
+
+    public set_ugi_args deepCopy() {
+      return new set_ugi_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.user_name = null;
+      this.group_names = null;
+    }
+
+    public String getUser_name() {
+      return this.user_name;
+    }
+
+    public void setUser_name(String user_name) {
+      this.user_name = user_name;
+    }
+
+    public void unsetUser_name() {
+      this.user_name = null;
+    }
+
+    /** Returns true if field user_name is set (has been assigned a value) and false otherwise */
+    public boolean isSetUser_name() {
+      return this.user_name != null;
+    }
+
+    public void setUser_nameIsSet(boolean value) {
+      if (!value) {
+        this.user_name = null;
+      }
+    }
+
+    public int getGroup_namesSize() {
+      return (this.group_names == null) ? 0 : this.group_names.size();
+    }
+
+    public java.util.Iterator<String> getGroup_namesIterator() {
+      return (this.group_names == null) ? null : this.group_names.iterator();
+    }
+
+    public void addToGroup_names(String elem) {
+      if (this.group_names == null) {
+        this.group_names = new ArrayList<String>();
+      }
+      this.group_names.add(elem);
+    }
+
+    public List<String> getGroup_names() {
+      return this.group_names;
+    }
+
+    public void setGroup_names(List<String> group_names) {
+      this.group_names = group_names;
+    }
+
+    public void unsetGroup_names() {
+      this.group_names = null;
+    }
+
+    /** Returns true if field group_names is set (has been assigned a value) and false otherwise */
+    public boolean isSetGroup_names() {
+      return this.group_names != null;
+    }
+
+    public void setGroup_namesIsSet(boolean value) {
+      if (!value) {
+        this.group_names = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case USER_NAME:
+        if (value == null) {
+          unsetUser_name();
+        } else {
+          setUser_name((String)value);
+        }
+        break;
+
+      case GROUP_NAMES:
+        if (value == null) {
+          unsetGroup_names();
+        } else {
+          setGroup_names((List<String>)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case USER_NAME:
+        return getUser_name();
+
+      case GROUP_NAMES:
+        return getGroup_names();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case USER_NAME:
+        return isSetUser_name();
+      case GROUP_NAMES:
+        return isSetGroup_names();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof set_ugi_args)
+        return this.equals((set_ugi_args)that);
+      return false;
+    }
+
+    public boolean equals(set_ugi_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_user_name = true && this.isSetUser_name();
+      boolean that_present_user_name = true && that.isSetUser_name();
+      if (this_present_user_name || that_present_user_name) {
+        if (!(this_present_user_name && that_present_user_name))
+          return false;
+        if (!this.user_name.equals(that.user_name))
+          return false;
+      }
+
+      boolean this_present_group_names = true && this.isSetGroup_names();
+      boolean that_present_group_names = true && that.isSetGroup_names();
+      if (this_present_group_names || that_present_group_names) {
+        if (!(this_present_group_names && that_present_group_names))
+          return false;
+        if (!this.group_names.equals(that.group_names))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      return 0;
+    }
+
+    public int compareTo(set_ugi_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      set_ugi_args typedOther = (set_ugi_args)other;
+
+      lastComparison = Boolean.valueOf(isSetUser_name()).compareTo(typedOther.isSetUser_name());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetUser_name()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.user_name, typedOther.user_name);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetGroup_names()).compareTo(typedOther.isSetGroup_names());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetGroup_names()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.group_names, typedOther.group_names);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          case 1: // USER_NAME
+            if (field.type == org.apache.thrift.protocol.TType.STRING) {
+              this.user_name = iprot.readString();
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case 2: // GROUP_NAMES
+            if (field.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list266 = iprot.readListBegin();
+                this.group_names = new ArrayList<String>(_list266.size);
+                for (int _i267 = 0; _i267 < _list266.size; ++_i267)
+                {
+                  String _elem268; // required
+                  _elem268 = iprot.readString();
+                  this.group_names.add(_elem268);
+                }
+                iprot.readListEnd();
+              }
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (this.user_name != null) {
+        oprot.writeFieldBegin(USER_NAME_FIELD_DESC);
+        oprot.writeString(this.user_name);
+        oprot.writeFieldEnd();
+      }
+      if (this.group_names != null) {
+        oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, this.group_names.size()));
+          for (String _iter269 : this.group_names)
+          {
+            oprot.writeString(_iter269);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("set_ugi_args(");
+      boolean first = true;
+
+      sb.append("user_name:");
+      if (this.user_name == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.user_name);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("group_names:");
+      if (this.group_names == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.group_names);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+  }
+
+  public static class set_ugi_result implements org.apache.thrift.TBase<set_ugi_result, set_ugi_result._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("set_ugi_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0);
+    private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private List<String> success; // required
+    private MetaException o1; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success"),
+      O1((short)1, "o1");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          case 1: // O1
+            return O1;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+      tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(set_ugi_result.class, metaDataMap);
+    }
+
+    public set_ugi_result() {
+    }
+
+    public set_ugi_result(
+      List<String> success,
+      MetaException o1)
+    {
+      this();
+      this.success = success;
+      this.o1 = o1;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public set_ugi_result(set_ugi_result other) {
+      if (other.isSetSuccess()) {
+        List<String> __this__success = new ArrayList<String>();
+        for (String other_element : other.success) {
+          __this__success.add(other_element);
+        }
+        this.success = __this__success;
+      }
+      if (other.isSetO1()) {
+        this.o1 = new MetaException(other.o1);
+      }
+    }
+
+    public set_ugi_result deepCopy() {
+      return new set_ugi_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+      this.o1 = null;
+    }
+
+    public int getSuccessSize() {
+      return (this.success == null) ? 0 : this.success.size();
+    }
+
+    public java.util.Iterator<String> getSuccessIterator() {
+      return (this.success == null) ? null : this.success.iterator();
+    }
+
+    public void addToSuccess(String elem) {
+      if (this.success == null) {
+        this.success = new ArrayList<String>();
+      }
+      this.success.add(elem);
+    }
+
+    public List<String> getSuccess() {
+      return this.success;
+    }
+
+    public void setSuccess(List<String> success) {
+      this.success = success;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public MetaException getO1() {
+      return this.o1;
+    }
+
+    public void setO1(MetaException o1) {
+      this.o1 = o1;
+    }
+
+    public void unsetO1() {
+      this.o1 = null;
+    }
+
+    /** Returns true if field o1 is set (has been assigned a value) and false otherwise */
+    public boolean isSetO1() {
+      return this.o1 != null;
+    }
+
+    public void setO1IsSet(boolean value) {
+      if (!value) {
+        this.o1 = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((List<String>)value);
+        }
+        break;
+
+      case O1:
+        if (value == null) {
+          unsetO1();
+        } else {
+          setO1((MetaException)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      case O1:
+        return getO1();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      case O1:
+        return isSetO1();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof set_ugi_result)
+        return this.equals((set_ugi_result)that);
+      return false;
+    }
+
+    public boolean equals(set_ugi_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && this.isSetSuccess();
+      boolean that_present_success = true && that.isSetSuccess();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      boolean this_present_o1 = true && this.isSetO1();
+      boolean that_present_o1 = true && that.isSetO1();
+      if (this_present_o1 || that_present_o1) {
+        if (!(this_present_o1 && that_present_o1))
+          return false;
+        if (!this.o1.equals(that.o1))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      return 0;
+    }
+
+    public int compareTo(set_ugi_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      set_ugi_result typedOther = (set_ugi_result)other;
+
+      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetO1()).compareTo(typedOther.isSetO1());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetO1()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, typedOther.o1);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          case 0: // SUCCESS
+            if (field.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list270 = iprot.readListBegin();
+                this.success = new ArrayList<String>(_list270.size);
+                for (int _i271 = 0; _i271 < _list270.size; ++_i271)
+                {
+                  String _elem272; // required
+                  _elem272 = iprot.readString();
+                  this.success.add(_elem272);
+                }
+                iprot.readListEnd();
+              }
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case 1: // O1
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.o1 = new MetaException();
+              this.o1.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      oprot.writeStructBegin(STRUCT_DESC);
+
+      if (this.isSetSuccess()) {
+        oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, this.success.size()));
+          for (String _iter273 : this.success)
+          {
+            oprot.writeString(_iter273);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      } else if (this.isSetO1()) {
+        oprot.writeFieldBegin(O1_FIELD_DESC);
+        this.o1.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("set_ugi_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("o1:");
+      if (this.o1 == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.o1);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+  }
+
   public static class get_delegation_token_args implements org.apache.thrift.TBase<get_delegation_token_args, get_delegation_token_args._Fields>, java.io.Serializable, Cloneable   {
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_delegation_token_args");
 

Modified: hive/trunk/metastore/src/gen/thrift/gen-php/hive_metastore/ThriftHiveMetastore.php
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/gen/thrift/gen-php/hive_metastore/ThriftHiveMetastore.php?rev=1225683&r1=1225682&r2=1225683&view=diff
==============================================================================
--- hive/trunk/metastore/src/gen/thrift/gen-php/hive_metastore/ThriftHiveMetastore.php (original)
+++ hive/trunk/metastore/src/gen/thrift/gen-php/hive_metastore/ThriftHiveMetastore.php Thu Dec 29 22:59:02 2011
@@ -70,6 +70,7 @@ interface ThriftHiveMetastoreIf extends 
   public function list_privileges($principal_name, $principal_type, $hiveObject);
   public function grant_privileges($privileges);
   public function revoke_privileges($privileges);
+  public function set_ugi($user_name, $group_names);
   public function get_delegation_token($token_owner, $renewer_kerberos_principal_name);
   public function renew_delegation_token($token_str_form);
   public function cancel_delegation_token($token_str_form);
@@ -3561,6 +3562,61 @@ class ThriftHiveMetastoreClient extends 
     throw new Exception("revoke_privileges failed: unknown result");
   }
 
+  public function set_ugi($user_name, $group_names)
+  {
+    $this->send_set_ugi($user_name, $group_names);
+    return $this->recv_set_ugi();
+  }
+
+  public function send_set_ugi($user_name, $group_names)
+  {
+    $args = new ThriftHiveMetastore_set_ugi_args();
+    $args->user_name = $user_name;
+    $args->group_names = $group_names;
+    $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'set_ugi', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('set_ugi', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_set_ugi()
+  {
+    $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, 'ThriftHiveMetastore_set_ugi_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
+
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
+      }
+      $result = new ThriftHiveMetastore_set_ugi_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->success !== null) {
+      return $result->success;
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
+    }
+    throw new Exception("set_ugi failed: unknown result");
+  }
+
   public function get_delegation_token($token_owner, $renewer_kerberos_principal_name)
   {
     $this->send_get_delegation_token($token_owner, $renewer_kerberos_principal_name);
@@ -17887,6 +17943,244 @@ class ThriftHiveMetastore_revoke_privile
 
 }
 
+class ThriftHiveMetastore_set_ugi_args {
+  static $_TSPEC;
+
+  public $user_name = null;
+  public $group_names = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'user_name',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'group_names',
+          'type' => TType::LST,
+          'etype' => TType::STRING,
+          'elem' => array(
+            'type' => TType::STRING,
+            ),
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['user_name'])) {
+        $this->user_name = $vals['user_name'];
+      }
+      if (isset($vals['group_names'])) {
+        $this->group_names = $vals['group_names'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_set_ugi_args';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->user_name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::LST) {
+            $this->group_names = array();
+            $_size469 = 0;
+            $_etype472 = 0;
+            $xfer += $input->readListBegin($_etype472, $_size469);
+            for ($_i473 = 0; $_i473 < $_size469; ++$_i473)
+            {
+              $elem474 = null;
+              $xfer += $input->readString($elem474);
+              $this->group_names []= $elem474;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_set_ugi_args');
+    if ($this->user_name !== null) {
+      $xfer += $output->writeFieldBegin('user_name', TType::STRING, 1);
+      $xfer += $output->writeString($this->user_name);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->group_names !== null) {
+      if (!is_array($this->group_names)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('group_names', TType::LST, 2);
+      {
+        $output->writeListBegin(TType::STRING, count($this->group_names));
+        {
+          foreach ($this->group_names as $iter475)
+          {
+            $xfer += $output->writeString($iter475);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_set_ugi_result {
+  static $_TSPEC;
+
+  public $success = null;
+  public $o1 = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        0 => array(
+          'var' => 'success',
+          'type' => TType::LST,
+          'etype' => TType::STRING,
+          'elem' => array(
+            'type' => TType::STRING,
+            ),
+          ),
+        1 => array(
+          'var' => 'o1',
+          'type' => TType::STRUCT,
+          'class' => 'MetaException',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['success'])) {
+        $this->success = $vals['success'];
+      }
+      if (isset($vals['o1'])) {
+        $this->o1 = $vals['o1'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_set_ugi_result';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 0:
+          if ($ftype == TType::LST) {
+            $this->success = array();
+            $_size476 = 0;
+            $_etype479 = 0;
+            $xfer += $input->readListBegin($_etype479, $_size476);
+            for ($_i480 = 0; $_i480 < $_size476; ++$_i480)
+            {
+              $elem481 = null;
+              $xfer += $input->readString($elem481);
+              $this->success []= $elem481;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->o1 = new MetaException();
+            $xfer += $this->o1->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_set_ugi_result');
+    if ($this->success !== null) {
+      if (!is_array($this->success)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('success', TType::LST, 0);
+      {
+        $output->writeListBegin(TType::STRING, count($this->success));
+        {
+          foreach ($this->success as $iter482)
+          {
+            $xfer += $output->writeString($iter482);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o1 !== null) {
+      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+      $xfer += $this->o1->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
 class ThriftHiveMetastore_get_delegation_token_args {
   static $_TSPEC;
 

Modified: hive/trunk/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote?rev=1225683&r1=1225682&r2=1225683&view=diff
==============================================================================
--- hive/trunk/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote (original)
+++ hive/trunk/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote Thu Dec 29 22:59:02 2011
@@ -81,6 +81,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == 
   print '   list_privileges(string principal_name, PrincipalType principal_type, HiveObjectRef hiveObject)'
   print '  bool grant_privileges(PrivilegeBag privileges)'
   print '  bool revoke_privileges(PrivilegeBag privileges)'
+  print '   set_ugi(string user_name,  group_names)'
   print '  string get_delegation_token(string token_owner, string renewer_kerberos_principal_name)'
   print '  i64 renew_delegation_token(string token_str_form)'
   print '  void cancel_delegation_token(string token_str_form)'
@@ -494,6 +495,12 @@ elif cmd == 'revoke_privileges':
     sys.exit(1)
   pp.pprint(client.revoke_privileges(eval(args[0]),))
 
+elif cmd == 'set_ugi':
+  if len(args) != 2:
+    print 'set_ugi requires 2 args'
+    sys.exit(1)
+  pp.pprint(client.set_ugi(args[0],eval(args[1]),))
+
 elif cmd == 'get_delegation_token':
   if len(args) != 2:
     print 'get_delegation_token requires 2 args'

Modified: hive/trunk/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py?rev=1225683&r1=1225682&r2=1225683&view=diff
==============================================================================
--- hive/trunk/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py (original)
+++ hive/trunk/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py Thu Dec 29 22:59:02 2011
@@ -525,6 +525,14 @@ class Iface(fb303.FacebookService.Iface)
     """
     pass
 
+  def set_ugi(self, user_name, group_names):
+    """
+    Parameters:
+     - user_name
+     - group_names
+    """
+    pass
+
   def get_delegation_token(self, token_owner, renewer_kerberos_principal_name):
     """
     Parameters:
@@ -2751,6 +2759,40 @@ class Client(fb303.FacebookService.Clien
       raise result.o1
     raise TApplicationException(TApplicationException.MISSING_RESULT, "revoke_privileges failed: unknown result");
 
+  def set_ugi(self, user_name, group_names):
+    """
+    Parameters:
+     - user_name
+     - group_names
+    """
+    self.send_set_ugi(user_name, group_names)
+    return self.recv_set_ugi()
+
+  def send_set_ugi(self, user_name, group_names):
+    self._oprot.writeMessageBegin('set_ugi', TMessageType.CALL, self._seqid)
+    args = set_ugi_args()
+    args.user_name = user_name
+    args.group_names = group_names
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_set_ugi(self, ):
+    (fname, mtype, rseqid) = self._iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(self._iprot)
+      self._iprot.readMessageEnd()
+      raise x
+    result = set_ugi_result()
+    result.read(self._iprot)
+    self._iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    if result.o1 is not None:
+      raise result.o1
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "set_ugi failed: unknown result");
+
   def get_delegation_token(self, token_owner, renewer_kerberos_principal_name):
     """
     Parameters:
@@ -2911,6 +2953,7 @@ class Processor(fb303.FacebookService.Pr
     self._processMap["list_privileges"] = Processor.process_list_privileges
     self._processMap["grant_privileges"] = Processor.process_grant_privileges
     self._processMap["revoke_privileges"] = Processor.process_revoke_privileges
+    self._processMap["set_ugi"] = Processor.process_set_ugi
     self._processMap["get_delegation_token"] = Processor.process_get_delegation_token
     self._processMap["renew_delegation_token"] = Processor.process_renew_delegation_token
     self._processMap["cancel_delegation_token"] = Processor.process_cancel_delegation_token
@@ -3890,6 +3933,20 @@ class Processor(fb303.FacebookService.Pr
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
+  def process_set_ugi(self, seqid, iprot, oprot):
+    args = set_ugi_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = set_ugi_result()
+    try:
+      result.success = self._handler.set_ugi(args.user_name, args.group_names)
+    except MetaException, o1:
+      result.o1 = o1
+    oprot.writeMessageBegin("set_ugi", TMessageType.REPLY, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
   def process_get_delegation_token(self, seqid, iprot, oprot):
     args = get_delegation_token_args()
     args.read(iprot)
@@ -13983,6 +14040,166 @@ class revoke_privileges_result:
   def __ne__(self, other):
     return not (self == other)
 
+class set_ugi_args:
+  """
+  Attributes:
+   - user_name
+   - group_names
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'user_name', None, None, ), # 1
+    (2, TType.LIST, 'group_names', (TType.STRING,None), None, ), # 2
+  )
+
+  def __init__(self, user_name=None, group_names=None,):
+    self.user_name = user_name
+    self.group_names = group_names
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.user_name = iprot.readString();
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.LIST:
+          self.group_names = []
+          (_etype472, _size469) = iprot.readListBegin()
+          for _i473 in xrange(_size469):
+            _elem474 = iprot.readString();
+            self.group_names.append(_elem474)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('set_ugi_args')
+    if self.user_name is not None:
+      oprot.writeFieldBegin('user_name', TType.STRING, 1)
+      oprot.writeString(self.user_name)
+      oprot.writeFieldEnd()
+    if self.group_names is not None:
+      oprot.writeFieldBegin('group_names', TType.LIST, 2)
+      oprot.writeListBegin(TType.STRING, len(self.group_names))
+      for iter475 in self.group_names:
+        oprot.writeString(iter475)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class set_ugi_result:
+  """
+  Attributes:
+   - success
+   - o1
+  """
+
+  thrift_spec = (
+    (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0
+    (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, success=None, o1=None,):
+    self.success = success
+    self.o1 = o1
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 0:
+        if ftype == TType.LIST:
+          self.success = []
+          (_etype479, _size476) = iprot.readListBegin()
+          for _i480 in xrange(_size476):
+            _elem481 = iprot.readString();
+            self.success.append(_elem481)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 1:
+        if ftype == TType.STRUCT:
+          self.o1 = MetaException()
+          self.o1.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('set_ugi_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.LIST, 0)
+      oprot.writeListBegin(TType.STRING, len(self.success))
+      for iter482 in self.success:
+        oprot.writeString(iter482)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.o1 is not None:
+      oprot.writeFieldBegin('o1', TType.STRUCT, 1)
+      self.o1.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class get_delegation_token_args:
   """
   Attributes:

Modified: hive/trunk/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb?rev=1225683&r1=1225682&r2=1225683&view=diff
==============================================================================
--- hive/trunk/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb (original)
+++ hive/trunk/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb Thu Dec 29 22:59:02 2011
@@ -1022,6 +1022,22 @@ module ThriftHiveMetastore
       raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'revoke_privileges failed: unknown result')
     end
 
+    def set_ugi(user_name, group_names)
+      send_set_ugi(user_name, group_names)
+      return recv_set_ugi()
+    end
+
+    def send_set_ugi(user_name, group_names)
+      send_message('set_ugi', Set_ugi_args, :user_name => user_name, :group_names => group_names)
+    end
+
+    def recv_set_ugi()
+      result = receive_message(Set_ugi_result)
+      return result.success unless result.success.nil?
+      raise result.o1 unless result.o1.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'set_ugi failed: unknown result')
+    end
+
     def get_delegation_token(token_owner, renewer_kerberos_principal_name)
       send_get_delegation_token(token_owner, renewer_kerberos_principal_name)
       return recv_get_delegation_token()
@@ -1854,6 +1870,17 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'revoke_privileges', seqid)
     end
 
+    def process_set_ugi(seqid, iprot, oprot)
+      args = read_args(iprot, Set_ugi_args)
+      result = Set_ugi_result.new()
+      begin
+        result.success = @handler.set_ugi(args.user_name, args.group_names)
+      rescue MetaException => o1
+        result.o1 = o1
+      end
+      write_result(result, oprot, 'set_ugi', seqid)
+    end
+
     def process_get_delegation_token(seqid, iprot, oprot)
       args = read_args(iprot, Get_delegation_token_args)
       result = Get_delegation_token_result.new()
@@ -4236,6 +4263,42 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
+  class Set_ugi_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    USER_NAME = 1
+    GROUP_NAMES = 2
+
+    FIELDS = {
+      USER_NAME => {:type => ::Thrift::Types::STRING, :name => 'user_name'},
+      GROUP_NAMES => {:type => ::Thrift::Types::LIST, :name => 'group_names', :element => {:type => ::Thrift::Types::STRING}}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Set_ugi_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+    O1 = 1
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::LIST, :name => 'success', :element => {:type => ::Thrift::Types::STRING}},
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
   class Get_delegation_token_args
     include ::Thrift::Struct, ::Thrift::Struct_Union
     TOKEN_OWNER = 1