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 2013/03/12 19:22:03 UTC

svn commit: r1455659 [8/11] - in /hive/trunk: ./ ant/src/org/apache/hadoop/hive/ant/ bin/ bin/ext/ cli/ common/ common/src/gen/ common/src/gen/org/ common/src/gen/org/apache/ common/src/gen/org/apache/hive/ common/src/gen/org/apache/hive/common/ common...

Added: hive/trunk/service/src/java/org/apache/hive/service/auth/KerberosSaslHelper.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/KerberosSaslHelper.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/KerberosSaslHelper.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/KerberosSaslHelper.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,77 @@
+/**
+ * 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.service.auth;
+
+import java.io.IOException;
+
+import javax.security.sasl.SaslException;
+
+import org.apache.hadoop.hive.shims.ShimLoader;
+import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge;
+import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge.Server;
+import org.apache.hive.service.cli.thrift.TCLIService;
+import org.apache.hive.service.cli.thrift.TCLIService.Iface;
+import org.apache.hive.service.cli.thrift.ThriftCLIService;
+import org.apache.thrift.TProcessor;
+import org.apache.thrift.TProcessorFactory;
+import org.apache.thrift.transport.TTransport;
+
+public class KerberosSaslHelper {
+
+  private static class CLIServiceProcessorFactory extends TProcessorFactory {
+    private final ThriftCLIService service;
+    private final Server saslServer;
+
+    public CLIServiceProcessorFactory(Server saslServer, ThriftCLIService service) {
+      super(null);
+      this.service = service;
+      this.saslServer = saslServer;
+    }
+
+    @Override
+    public TProcessor getProcessor(TTransport trans) {
+      TProcessor sqlProcessor = new TCLIService.Processor<Iface>(service);
+      return saslServer.wrapNonAssumingProcessor(sqlProcessor);
+    }
+  }
+
+  public static TProcessorFactory getKerberosProcessorFactory(Server saslServer,
+      ThriftCLIService service) {
+    return new CLIServiceProcessorFactory (saslServer, service);
+  }
+
+  public static TTransport getKerberosTransport(String principal, String host,
+      final TTransport underlyingTransport) throws SaslException {
+    try {
+      final String names[] = principal.split("[/@]");
+      if (names.length != 3) {
+        throw new IllegalArgumentException("Kerberos principal should have 3 parts: "
+            + principal);
+      }
+
+      HadoopThriftAuthBridge.Client authBridge =
+        ShimLoader.getHadoopThriftAuthBridge().createClientWithConf("kerberos");
+      return authBridge.createClientTransport(principal, host,
+          "KERBEROS", null, underlyingTransport);
+    } catch (IOException e) {
+      throw new SaslException("Failed to open client transport", e);
+    }
+  }
+
+
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,70 @@
+/**
+ * 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.service.auth;
+
+import java.util.Hashtable;
+
+import javax.naming.Context;
+import javax.naming.NamingException;
+import javax.naming.directory.DirContext;
+import javax.naming.directory.InitialDirContext;
+import javax.security.sasl.AuthenticationException;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+
+public class LdapAuthenticationProviderImpl implements PasswdAuthenticationProvider {
+
+  String ldapURL;
+  String baseDN;
+
+  LdapAuthenticationProviderImpl () {
+    HiveConf conf = new HiveConf();
+    this.ldapURL = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_URL);
+    this.baseDN = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_BASEDN);
+  }
+
+  @Override
+  public void Authenticate(String user, String  password)
+      throws AuthenticationException {
+
+    Hashtable<String, Object> env = new Hashtable<String, Object>();
+    env.put(Context.INITIAL_CONTEXT_FACTORY, "com.sun.jndi.ldap.LdapCtxFactory");
+    env.put(Context.PROVIDER_URL, ldapURL);
+
+    // setup the security principal
+    String bindDN;
+    if (baseDN != null) {
+      bindDN = "uid=" + user + "," + baseDN;
+    } else {
+      bindDN = user;
+    }
+    env.put(Context.SECURITY_AUTHENTICATION, "simple");
+    env.put(Context.SECURITY_PRINCIPAL, bindDN);
+    env.put(Context.SECURITY_CREDENTIALS, password);
+
+    try {
+      // Create initial context
+      DirContext ctx = new InitialDirContext(env);
+      ctx.close();
+    } catch (NamingException e) {
+      throw new AuthenticationException("Error validating LDAP user");
+    }
+  return;
+  }
+
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/auth/PasswdAuthenticationProvider.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/PasswdAuthenticationProvider.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/PasswdAuthenticationProvider.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/PasswdAuthenticationProvider.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hive.service.auth;
+
+import javax.security.sasl.AuthenticationException;
+
+public interface PasswdAuthenticationProvider {
+  /**
+   * The Authenticate method is called by the HiveServer2 authentication layer
+   * to authenticate users for their requests.
+   * If a user is to be granted, return nothing/throw nothing.
+   * When a user is to be disallowed, throw an appropriate {@link AuthenticationException}.
+   *
+   * For an example implementation, see {@link LdapAuthenticationProviderImpl}.
+   *
+   * @param user - The username received over the connection request
+   * @param password - The password received over the connection request
+   * @throws AuthenticationException - When a user is found to be
+   * invalid by the implementation
+   */
+  void Authenticate(String user, String password) throws AuthenticationException;
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/auth/PlainSaslHelper.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/PlainSaslHelper.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/PlainSaslHelper.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/PlainSaslHelper.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,141 @@
+/**
+ * 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.service.auth;
+
+import java.io.IOException;
+import java.util.HashMap;
+
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.SaslException;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hive.service.auth.PlainSaslServer.ExternalAuthenticationCallback;
+import org.apache.hive.service.auth.PlainSaslServer.SaslPlainProvider;
+import org.apache.hive.service.cli.thrift.TCLIService;
+import org.apache.hive.service.cli.thrift.TCLIService.Iface;
+import org.apache.hive.service.cli.thrift.ThriftCLIService;
+import org.apache.thrift.TProcessor;
+import org.apache.thrift.TProcessorFactory;
+import org.apache.thrift.transport.TSaslClientTransport;
+import org.apache.thrift.transport.TSaslServerTransport;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportFactory;
+
+public class PlainSaslHelper {
+
+  private static class PlainServerCallbackHandler implements CallbackHandler {
+
+    @Override
+    public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+      ExternalAuthenticationCallback ac = null;
+      for (int i = 0; i < callbacks.length; i++) {
+        if (callbacks[i] instanceof ExternalAuthenticationCallback) {
+          ac = (ExternalAuthenticationCallback) callbacks[i];
+          break;
+        } else {
+          throw new UnsupportedCallbackException(callbacks[i]);
+        }
+      }
+
+      if (ac != null) {
+        PasswdAuthenticationProvider provider =
+            AuthenticationProviderFactory.getAuthenticationProvider(ac.getAuthMethod());
+        provider.Authenticate(ac.getUserName(), ac.getPasswd());
+        ac.setAuthenticated(true);
+      }
+    }
+  }
+
+  public static class PlainClientbackHandler implements CallbackHandler {
+
+    private final String userName;
+    private final String passWord;
+
+    public PlainClientbackHandler (String userName, String passWord) {
+      this.userName = userName;
+      this.passWord = passWord;
+    }
+
+    @Override
+    public void handle(Callback[] callbacks)
+          throws IOException, UnsupportedCallbackException {
+      AuthorizeCallback ac = null;
+      for (int i = 0; i < callbacks.length; i++) {
+        if (callbacks[i] instanceof NameCallback) {
+          NameCallback nameCallback = (NameCallback)callbacks[i];
+          nameCallback.setName(userName);
+        } else if (callbacks[i] instanceof PasswordCallback) {
+          PasswordCallback passCallback = (PasswordCallback) callbacks[i];
+          passCallback.setPassword(passWord.toCharArray());
+        } else {
+          throw new UnsupportedCallbackException(callbacks[i]);
+        }
+      }
+    }
+  }
+
+  private static class SQLPlainProcessorFactory extends TProcessorFactory {
+    private final ThriftCLIService service;
+    private final HiveConf conf;
+    private final boolean doAsEnabled;
+
+    public SQLPlainProcessorFactory(ThriftCLIService service) {
+      super(null);
+      this.service = service;
+      this.conf = service.getHiveConf();
+      this.doAsEnabled = conf.getBoolean("hive.server2.enable.doAs", false);
+    }
+
+    @Override
+    public TProcessor getProcessor(TTransport trans) {
+      TProcessor baseProcessor =  new TCLIService.Processor<Iface>(service);
+      return doAsEnabled ? new TUGIContainingProcessor(baseProcessor, conf) :
+            new TSetIpAddressProcessor<Iface>(service);
+    }
+  }
+
+  public static TProcessorFactory getPlainProcessorFactory(ThriftCLIService service) {
+    return new SQLPlainProcessorFactory(service);
+  }
+
+  // Register Plain SASL server provider
+  static {
+    java.security.Security.addProvider(new SaslPlainProvider());
+  }
+
+  public static TTransportFactory getPlainTransportFactory(String authTypeStr) {
+    TSaslServerTransport.Factory saslFactory = new TSaslServerTransport.Factory();
+    saslFactory.addServerDefinition("PLAIN",
+        authTypeStr, null, new HashMap<String, String>(),
+        new PlainServerCallbackHandler());
+    return saslFactory;
+  }
+
+  public static TTransport getPlainTransport(String userName, String passwd,
+      final TTransport underlyingTransport) throws SaslException {
+    return new TSaslClientTransport("PLAIN", null,
+        null, null, new HashMap<String, String>(),
+        new PlainClientbackHandler(userName, passwd), underlyingTransport);
+  }
+
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/auth/PlainSaslServer.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/PlainSaslServer.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/PlainSaslServer.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/PlainSaslServer.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,189 @@
+/**
+ * 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.service.auth;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Deque;
+import java.util.Map;
+
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+import javax.security.sasl.SaslServerFactory;
+
+import org.apache.hive.service.auth.AuthenticationProviderFactory.AuthMethods;
+
+/**
+ *
+ * PlainSaslServer.
+ * Sun JDK only provides PLAIN client and not server. This class implements the Plain SASL server
+ * conforming to RFC #4616 (http://www.ietf.org/rfc/rfc4616.txt)
+ */
+public class PlainSaslServer implements SaslServer  {
+  private final AuthMethods authMethod;
+  private String user;
+  private String passwd;
+  private final CallbackHandler handler;
+
+  // Callback for external authentication
+  // The authMethod indicates the type of authentication (LDAP, Unix, Windows)
+  public static class ExternalAuthenticationCallback implements Callback {
+    private final AuthMethods authMethod;
+    private final String userName;
+    private final String passwd;
+    private boolean authenticated;
+
+    public ExternalAuthenticationCallback(AuthMethods authMethod, String userName, String passwd) {
+      this.authMethod = authMethod;
+      this.userName = userName;
+      this.passwd = passwd;
+      authenticated = false;
+    }
+
+    public AuthMethods getAuthMethod() {
+      return authMethod;
+    }
+
+    public String getUserName() {
+      return userName;
+    }
+
+    public String getPasswd() {
+      return passwd;
+    }
+
+    public void setAuthenticated (boolean authenticated) {
+      this.authenticated = authenticated;
+    }
+
+    public boolean isAuthenticated () {
+      return authenticated;
+    }
+  }
+
+
+  PlainSaslServer(CallbackHandler handler, String authMethodStr) throws SaslException {
+    this.handler = handler;
+    this.authMethod = AuthMethods.getValidAuthMethod(authMethodStr);
+  }
+
+  public String getMechanismName() {
+    return "PLAIN";
+  }
+
+  public byte[] evaluateResponse(byte[] response) throws SaslException {
+    try {
+      // parse the response
+      // message   = [authzid] UTF8NUL authcid UTF8NUL passwd'
+
+      Deque<String> tokenList = new ArrayDeque<String>();
+      StringBuilder messageToken = new StringBuilder();
+      for (byte b : response) {
+        if (b == 0) {
+          tokenList.addLast(messageToken.toString());
+          messageToken = new StringBuilder();
+        } else {
+          messageToken.append((char)b);
+        }
+      }
+      tokenList.addLast(messageToken.toString());
+
+      // validate response
+      if ((tokenList.size() < 2) || (tokenList.size() > 3)) {
+        throw new SaslException("Invalid message format");
+      }
+      passwd = tokenList.removeLast();
+      user = tokenList.removeLast();
+      if (user == null || user.isEmpty()) {
+        throw new SaslException("No user name provide");
+      }
+      if (passwd == null || passwd.isEmpty()) {
+        throw new SaslException("No password name provide");
+      }
+
+      // pass the user and passwd via AuthorizeCallback
+      // the caller needs to authenticate
+      ExternalAuthenticationCallback exAuth = new
+          ExternalAuthenticationCallback(authMethod, user, passwd);
+      Callback[] cbList = new Callback[] {exAuth};
+      handler.handle(cbList);
+      if (!exAuth.isAuthenticated()) {
+        throw new SaslException("Authentication failed");
+      }
+    } catch (IllegalStateException eL) {
+      throw new SaslException("Invalid message format", eL);
+    } catch (IOException eI) {
+      throw new SaslException("Error validating the login", eI);
+    } catch (UnsupportedCallbackException eU) {
+      throw new SaslException("Error validating the login", eU);
+    }
+    return null;
+  }
+
+  public boolean isComplete() {
+    return user != null;
+  }
+
+  public String getAuthorizationID() {
+    return user;
+  }
+
+  public byte[] unwrap(byte[] incoming, int offset, int len) {
+      throw new UnsupportedOperationException();
+  }
+
+  public byte[] wrap(byte[] outgoing, int offset, int len) {
+    throw new UnsupportedOperationException();
+  }
+
+  public Object getNegotiatedProperty(String propName) {
+    return null;
+  }
+
+  public void dispose() {}
+
+  public static class SaslPlainServerFactory implements SaslServerFactory {
+
+    public SaslServer createSaslServer(
+      String mechanism, String protocol, String serverName, Map<String,?> props, CallbackHandler cbh)
+    {
+      if ("PLAIN".equals(mechanism)) {
+        try {
+          return new PlainSaslServer(cbh, protocol);
+        } catch (SaslException e) {
+          return null;
+        }
+      }
+      return null;
+    }
+
+    public String[] getMechanismNames(Map<String, ?> props) {
+      return new String[] { "PLAIN" };
+    }
+  }
+
+  public static class SaslPlainProvider extends java.security.Provider {
+    public SaslPlainProvider() {
+      super("HiveSaslPlain", 1.0, "Hive Plain SASL provider");
+      put("SaslServerFactory.PLAIN", SaslPlainServerFactory.class.getName());
+    }
+  }
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,78 @@
+package org.apache.hive.service.auth;
+
+import java.net.Socket;
+
+import org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Processor;
+import org.apache.hive.service.cli.session.SessionManager;
+import org.apache.hive.service.cli.thrift.TCLIService;
+import org.apache.hive.service.cli.thrift.TCLIService.Iface;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TSaslClientTransport;
+import org.apache.thrift.transport.TSaslServerTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class is responsible for setting the ipAddress for operations executed via HiveServer2.
+ * <p>
+ * <ul>
+ * <li>Ipaddress is only set for operations that calls listeners with hookContext @see ExecuteWithHookContext.</li>
+ * <li>Ipaddress is only set if the underlying transport mechanism is socket. </li>
+ * </ul>
+ * </p>
+ */
+public class TSetIpAddressProcessor<I extends Iface> extends TCLIService.Processor<Iface> {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName());
+
+  public TSetIpAddressProcessor(Iface iface) {
+    super(iface);
+  }
+
+  @Override
+  public boolean process(final TProtocol in, final TProtocol out) throws TException {
+    setIpAddress(in);
+    setUserName(in);
+    return super.process(in, out);
+  }
+
+  private void setUserName(final TProtocol in) {
+    TTransport transport = in.getTransport();
+    if (transport instanceof TSaslServerTransport) {
+      String userName = ((TSaslServerTransport)transport).getSaslServer().getAuthorizationID();
+      SessionManager.setUserName(userName);
+    }
+  }
+
+  protected void setIpAddress(final TProtocol in) {
+    TTransport transport = in.getTransport();
+    TSocket tSocket = getUnderlyingSocketFromTransport(transport);
+    if (tSocket != null) {
+     setIpAddress(tSocket.getSocket());
+    } else {
+      LOGGER.warn("Unknown Transport, cannot determine ipAddress");
+    }
+  }
+
+  private void setIpAddress(Socket socket) {
+    SessionManager.setIpAddress(socket.getInetAddress().toString());
+  }
+
+  private TSocket getUnderlyingSocketFromTransport(TTransport transport) {
+    while (transport != null) {
+      if (transport instanceof TSaslServerTransport) {
+        transport = ((TSaslServerTransport) transport).getUnderlyingTransport();
+      }
+      if (transport instanceof TSaslClientTransport) {
+        transport = ((TSaslClientTransport) transport).getUnderlyingTransport();
+      }
+      if (transport instanceof TSocket) {
+        return (TSocket) transport;
+      }
+    }
+    return null;
+  }
+}
\ No newline at end of file

Added: hive/trunk/service/src/java/org/apache/hive/service/auth/TUGIContainingProcessor.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/TUGIContainingProcessor.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/TUGIContainingProcessor.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/TUGIContainingProcessor.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,65 @@
+package org.apache.hive.service.auth;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hive.shims.HadoopShims;
+import org.apache.hadoop.hive.shims.ShimLoader;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.thrift.TException;
+import org.apache.thrift.TProcessor;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TSaslServerTransport;
+
+public class TUGIContainingProcessor implements TProcessor{
+
+  private final TProcessor wrapped;
+  private final HadoopShims shim;
+  private final boolean isFsCacheDisabled;
+
+  public TUGIContainingProcessor(TProcessor wrapped, Configuration conf) {
+    this.wrapped = wrapped;
+    this.isFsCacheDisabled = conf.getBoolean(String.format("fs.%s.impl.disable.cache",
+      FileSystem.getDefaultUri(conf).getScheme()), false);
+    this.shim = ShimLoader.getHadoopShims();
+  }
+
+  @Override
+  public boolean process(final TProtocol in, final TProtocol out) throws TException {
+    UserGroupInformation clientUgi = null;
+
+    try {
+      clientUgi = shim.createRemoteUser(((TSaslServerTransport)in.getTransport()).
+          getSaslServer().getAuthorizationID(), new ArrayList<String>());
+      return shim.doAs(clientUgi, new PrivilegedExceptionAction<Boolean>() {
+        public Boolean run() {
+          try {
+            return wrapped.process(in, out);
+          } catch (TException te) {
+            throw new RuntimeException(te);
+          }
+        }
+      });
+    }
+    catch (RuntimeException rte) {
+      if (rte.getCause() instanceof TException) {
+        throw (TException)rte.getCause();
+      }
+      throw rte;
+    } catch (InterruptedException ie) {
+      throw new RuntimeException(ie); // unexpected!
+    } catch (IOException ioe) {
+      throw new RuntimeException(ioe); // unexpected!
+    }
+    finally {
+      // cleanup the filesystem handles at the end if they are cached
+      // clientUgi will be null if createRemoteUser() fails
+      if (clientUgi != null && !isFsCacheDisabled) {
+        shim.closeAllForUGI(clientUgi);
+      }
+    }
+  }
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/cli/CLIService.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/CLIService.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/CLIService.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/CLIService.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,328 @@
+/**
+ * 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.service.cli;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+import javax.security.auth.login.LoginException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.shims.ShimLoader;
+import org.apache.hive.service.CompositeService;
+import org.apache.hive.service.ServiceException;
+import org.apache.hive.service.auth.HiveAuthFactory;
+import org.apache.hive.service.cli.session.SessionManager;
+
+/**
+ * CLIService.
+ *
+ */
+public class CLIService extends CompositeService implements ICLIService {
+
+  private final Log LOG = LogFactory.getLog(CLIService.class.getName());
+
+  private HiveConf hiveConf;
+  private SessionManager sessionManager;
+  private IMetaStoreClient metastoreClient;
+  private String serverUserName = null;
+
+
+  public CLIService() {
+    super("CLIService");
+  }
+
+  @Override
+  public synchronized void init(HiveConf hiveConf) {
+    this.hiveConf = hiveConf;
+
+    sessionManager = new SessionManager();
+    addService(sessionManager);
+    try {
+      HiveAuthFactory.loginFromKeytab(hiveConf);
+      serverUserName = ShimLoader.getHadoopShims().
+          getShortUserName(ShimLoader.getHadoopShims().getUGIForConf(hiveConf));
+    } catch (IOException e) {
+      throw new ServiceException("Unable to login to kerberos with given principal/keytab", e);
+    } catch (LoginException e) {
+      throw new ServiceException("Unable to login to kerberos with given principal/keytab", e);
+    }
+    super.init(hiveConf);
+  }
+
+  @Override
+  public synchronized void start() {
+    super.start();
+
+    // Initialize and test a connection to the metastore
+    try {
+      metastoreClient = new HiveMetaStoreClient(hiveConf);
+      metastoreClient.getDatabases("default");
+    } catch (Exception e) {
+      throw new ServiceException("Unable to connect to MetaStore!", e);
+    }
+  }
+
+  @Override
+  public synchronized void stop() {
+    if (metastoreClient != null) {
+      metastoreClient.close();
+    }
+    super.stop();
+  }
+
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#openSession(java.lang.String, java.lang.String, java.util.Map)
+   */
+  @Override
+  public SessionHandle openSession(String username, String password, Map<String, String> configuration)
+      throws HiveSQLException {
+    SessionHandle sessionHandle = sessionManager.openSession(username, password, configuration, false, null);
+    LOG.info(sessionHandle + ": openSession()");
+    return sessionHandle;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#openSession(java.lang.String, java.lang.String, java.util.Map)
+   */
+  @Override
+  public SessionHandle openSessionWithImpersonation(String username, String password, Map<String, String> configuration,
+       String delegationToken) throws HiveSQLException {
+    SessionHandle sessionHandle = sessionManager.openSession(username, password, configuration,
+          true, delegationToken);
+    LOG.info(sessionHandle + ": openSession()");
+    return sessionHandle;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#closeSession(org.apache.hive.service.cli.SessionHandle)
+   */
+  @Override
+  public void closeSession(SessionHandle sessionHandle)
+      throws HiveSQLException {
+    sessionManager.closeSession(sessionHandle);
+    LOG.info(sessionHandle + ": closeSession()");
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#getInfo(org.apache.hive.service.cli.SessionHandle, java.util.List)
+   */
+  @Override
+  public GetInfoValue getInfo(SessionHandle sessionHandle, GetInfoType getInfoType)
+      throws HiveSQLException {
+    GetInfoValue infoValue = sessionManager.getSession(sessionHandle).getInfo(getInfoType);
+    LOG.info(sessionHandle + ": getInfo()");
+    return infoValue;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#executeStatement(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.util.Map)
+   */
+  @Override
+  public OperationHandle executeStatement(SessionHandle sessionHandle, String statement, Map<String, String> confOverlay)
+      throws HiveSQLException {
+    OperationHandle opHandle = sessionManager.getSession(sessionHandle)
+        .executeStatement(statement, confOverlay);
+    LOG.info(sessionHandle + ": executeStatement()");
+    return opHandle;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#getTypeInfo(org.apache.hive.service.cli.SessionHandle)
+   */
+  @Override
+  public OperationHandle getTypeInfo(SessionHandle sessionHandle)
+      throws HiveSQLException {
+    OperationHandle opHandle = sessionManager.getSession(sessionHandle).getTypeInfo();
+    LOG.info(sessionHandle + ": getTypeInfo()");
+    return opHandle;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#getCatalogs(org.apache.hive.service.cli.SessionHandle)
+   */
+  @Override
+  public OperationHandle getCatalogs(SessionHandle sessionHandle)
+      throws HiveSQLException {
+    OperationHandle opHandle = sessionManager.getSession(sessionHandle).getCatalogs();
+    LOG.info(sessionHandle + ": getCatalogs()");
+    return opHandle;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#getSchemas(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String)
+   */
+  @Override
+  public OperationHandle getSchemas(SessionHandle sessionHandle,
+      String catalogName, String schemaName)
+      throws HiveSQLException {
+    OperationHandle opHandle = sessionManager.getSession(sessionHandle)
+        .getSchemas(catalogName, schemaName);
+    LOG.info(sessionHandle + ": getSchemas()");
+    return opHandle;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#getTables(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String, java.lang.String, java.util.List)
+   */
+  @Override
+  public OperationHandle getTables(SessionHandle sessionHandle,
+      String catalogName, String schemaName, String tableName, List<String> tableTypes)
+      throws HiveSQLException {
+    OperationHandle opHandle = sessionManager
+        .getSession(sessionHandle).getTables(catalogName, schemaName, tableName, tableTypes);
+    LOG.info(sessionHandle + ": getTables()");
+    return opHandle;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#getTableTypes(org.apache.hive.service.cli.SessionHandle)
+   */
+  @Override
+  public OperationHandle getTableTypes(SessionHandle sessionHandle)
+      throws HiveSQLException {
+    OperationHandle opHandle = sessionManager.getSession(sessionHandle).getTableTypes();
+    LOG.info(sessionHandle + ": getTableTypes()");
+    return opHandle;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#getColumns(org.apache.hive.service.cli.SessionHandle)
+   */
+  @Override
+  public OperationHandle getColumns(SessionHandle sessionHandle,
+      String catalogName, String schemaName, String tableName, String columnName)
+      throws HiveSQLException {
+    OperationHandle opHandle = sessionManager.getSession(sessionHandle)
+        .getColumns(catalogName, schemaName, tableName, columnName);
+    LOG.info(sessionHandle + ": getColumns()");
+    return opHandle;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#getFunctions(org.apache.hive.service.cli.SessionHandle)
+   */
+  @Override
+  public OperationHandle getFunctions(SessionHandle sessionHandle,
+      String catalogName, String schemaName, String functionName)
+      throws HiveSQLException {
+    OperationHandle opHandle = sessionManager.getSession(sessionHandle)
+        .getFunctions(catalogName, schemaName, functionName);
+    LOG.info(sessionHandle + ": getFunctions()");
+    return opHandle;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#getOperationStatus(org.apache.hive.service.cli.OperationHandle)
+   */
+  @Override
+  public OperationState getOperationStatus(OperationHandle opHandle)
+      throws HiveSQLException {
+    OperationState opState = sessionManager.getOperationManager().getOperationState(opHandle);
+    LOG.info(opHandle + ": getOperationStatus()");
+    return opState;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#cancelOperation(org.apache.hive.service.cli.OperationHandle)
+   */
+  @Override
+  public void cancelOperation(OperationHandle opHandle)
+      throws HiveSQLException {
+    sessionManager.getOperationManager().getOperation(opHandle).
+        getParentSession().cancelOperation(opHandle);
+    LOG.info(opHandle + ": cancelOperation()");
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#closeOperation(org.apache.hive.service.cli.OperationHandle)
+   */
+  @Override
+  public void closeOperation(OperationHandle opHandle)
+      throws HiveSQLException {
+    sessionManager.getOperationManager().getOperation(opHandle).
+        getParentSession().closeOperation(opHandle);
+    LOG.info(opHandle + ": closeOperation");
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#getResultSetMetadata(org.apache.hive.service.cli.OperationHandle)
+   */
+  @Override
+  public TableSchema getResultSetMetadata(OperationHandle opHandle)
+      throws HiveSQLException {
+    TableSchema tableSchema = sessionManager.getOperationManager().getOperation(opHandle).
+        getParentSession().getResultSetMetadata(opHandle);
+    LOG.info(opHandle + ": getResultSetMetadata()");
+    return tableSchema;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#fetchResults(org.apache.hive.service.cli.OperationHandle, org.apache.hive.service.cli.FetchOrientation, long)
+   */
+  @Override
+  public RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, long maxRows)
+      throws HiveSQLException {
+    RowSet rowSet = sessionManager.getOperationManager().getOperation(opHandle).
+        getParentSession().fetchResults(opHandle, orientation, maxRows);
+    LOG.info(opHandle + ": fetchResults()");
+    return rowSet;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#fetchResults(org.apache.hive.service.cli.OperationHandle)
+   */
+  @Override
+  public RowSet fetchResults(OperationHandle opHandle)
+      throws HiveSQLException {
+    RowSet rowSet = sessionManager.getOperationManager().getOperation(opHandle).
+        getParentSession().fetchResults(opHandle);
+    LOG.info(opHandle + ": fetchResults()");
+    return rowSet;
+  }
+
+  // obtain delegation token for the give user from metastore
+  public synchronized String getDelegationTokenFromMetaStore(String owner)
+      throws HiveSQLException, UnsupportedOperationException, LoginException, IOException {
+    if (!hiveConf.getBoolVar(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL) ||
+        !hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_KERBEROS_IMPERSONATION)) {
+      throw new UnsupportedOperationException(
+        "delegation token is can only be obtained for a secure remote metastore");
+    }
+
+    try {
+      Hive.closeCurrent();
+      return Hive.get(hiveConf).getDelegationToken(owner, owner);
+    } catch (HiveException e) {
+      if (e.getCause() instanceof UnsupportedOperationException) {
+        throw (UnsupportedOperationException)e.getCause();
+      } else {
+        throw new HiveSQLException("Error connect metastore to setup impersonation", e);
+      }
+    }
+  }
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/cli/CLIServiceClient.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/CLIServiceClient.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/CLIServiceClient.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/CLIServiceClient.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,152 @@
+/**
+ * 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.service.cli;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * CLIServiceClient.
+ *
+ */
+public abstract class CLIServiceClient implements ICLIService {
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#openSession(java.lang.String, java.lang.String, java.util.Map)
+   */
+  @Override
+  public abstract SessionHandle openSession(String username, String password,
+      Map<String, String> configuration) throws HiveSQLException;
+
+
+  public SessionHandle openSession(String username, String password)
+      throws HiveSQLException {
+    return openSession(username, password, Collections.<String, String>emptyMap());
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#closeSession(org.apache.hive.service.cli.SessionHandle)
+   */
+  @Override
+  public abstract void closeSession(SessionHandle sessionHandle) throws HiveSQLException;
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#getInfo(org.apache.hive.service.cli.SessionHandle, java.util.List)
+   */
+  @Override
+  public abstract GetInfoValue getInfo(SessionHandle sessionHandle, GetInfoType getInfoType)
+      throws HiveSQLException;
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#executeStatement(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.util.Map)
+   */
+  @Override
+  public abstract OperationHandle executeStatement(SessionHandle sessionHandle, String statement,
+      Map<String, String> confOverlay) throws HiveSQLException;
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#getTypeInfo(org.apache.hive.service.cli.SessionHandle)
+   */
+  @Override
+  public abstract OperationHandle getTypeInfo(SessionHandle sessionHandle) throws HiveSQLException;
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#getCatalogs(org.apache.hive.service.cli.SessionHandle)
+   */
+  @Override
+  public abstract OperationHandle getCatalogs(SessionHandle sessionHandle) throws HiveSQLException;
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#getSchemas(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String)
+   */
+  @Override
+  public abstract OperationHandle getSchemas(SessionHandle sessionHandle, String catalogName,
+      String schemaName) throws HiveSQLException;
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#getTables(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String, java.lang.String, java.util.List)
+   */
+  @Override
+  public abstract OperationHandle getTables(SessionHandle sessionHandle, String catalogName,
+      String schemaName, String tableName, List<String> tableTypes) throws HiveSQLException;
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#getTableTypes(org.apache.hive.service.cli.SessionHandle)
+   */
+  @Override
+  public abstract OperationHandle getTableTypes(SessionHandle sessionHandle) throws HiveSQLException;
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#getColumns(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String, java.lang.String, java.lang.String)
+   */
+  @Override
+  public abstract OperationHandle getColumns(SessionHandle sessionHandle, String catalogName,
+      String schemaName, String tableName, String columnName) throws HiveSQLException;
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#getFunctions(org.apache.hive.service.cli.SessionHandle, java.lang.String)
+   */
+  @Override
+  public abstract OperationHandle getFunctions(SessionHandle sessionHandle,
+      String catalogName, String schemaName, String functionName)
+      throws HiveSQLException;
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#getOperationStatus(org.apache.hive.service.cli.OperationHandle)
+   */
+  @Override
+  public abstract OperationState getOperationStatus(OperationHandle opHandle) throws HiveSQLException;
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#cancelOperation(org.apache.hive.service.cli.OperationHandle)
+   */
+  @Override
+  public abstract void cancelOperation(OperationHandle opHandle) throws HiveSQLException;
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#closeOperation(org.apache.hive.service.cli.OperationHandle)
+   */
+  @Override
+  public abstract void closeOperation(OperationHandle opHandle) throws HiveSQLException;
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#getResultSetMetadata(org.apache.hive.service.cli.OperationHandle)
+   */
+  @Override
+  public abstract TableSchema getResultSetMetadata(OperationHandle opHandle) throws HiveSQLException;
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#fetchResults(org.apache.hive.service.cli.OperationHandle, org.apache.hive.service.cli.FetchOrientation, long)
+   */
+  @Override
+  public abstract RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, long maxRows)
+      throws HiveSQLException;
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.ICLIService#fetchResults(org.apache.hive.service.cli.OperationHandle)
+   */
+  @Override
+  public RowSet fetchResults(OperationHandle opHandle) throws HiveSQLException {
+    // TODO: provide STATIC default value
+    return fetchResults(opHandle, FetchOrientation.FETCH_NEXT, 1000);
+  }
+
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/cli/CLIServiceUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/CLIServiceUtils.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/CLIServiceUtils.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/CLIServiceUtils.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,69 @@
+/**
+ * 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.service.cli;
+
+/**
+ * CLIServiceUtils.
+ *
+ */
+public class CLIServiceUtils {
+
+
+  private static final char SEARCH_STRING_ESCAPE = '\\';
+
+  /**
+   * Convert a SQL search pattern into an equivalent Java Regex.
+   *
+   * @param pattern input which may contain '%' or '_' wildcard characters, or
+   * these characters escaped using {@link #getSearchStringEscape()}.
+   * @return replace %/_ with regex search characters, also handle escaped
+   * characters.
+   */
+  public static String patternToRegex(String pattern) {
+    if (pattern == null) {
+      return ".*";
+    } else {
+      StringBuilder result = new StringBuilder(pattern.length());
+
+      boolean escaped = false;
+      for (int i = 0, len = pattern.length(); i < len; i++) {
+        char c = pattern.charAt(i);
+        if (escaped) {
+          if (c != SEARCH_STRING_ESCAPE) {
+            escaped = false;
+          }
+          result.append(c);
+        } else {
+          if (c == SEARCH_STRING_ESCAPE) {
+            escaped = true;
+            continue;
+          } else if (c == '%') {
+            result.append(".*");
+          } else if (c == '_') {
+            result.append('.');
+          } else {
+            result.append(Character.toLowerCase(c));
+          }
+        }
+      }
+      return result.toString();
+    }
+  }
+
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/cli/ColumnDescriptor.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/ColumnDescriptor.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/ColumnDescriptor.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/ColumnDescriptor.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,97 @@
+/**
+ * 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.service.cli;
+
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hive.service.cli.thrift.TColumnDesc;
+
+
+/**
+ * ColumnDescriptor.
+ *
+ */
+public class ColumnDescriptor {
+  private final String name;
+  private final String comment;
+  private final TypeDescriptor type;
+  // ordinal position of this column in the schema
+  private final int position;
+
+  public ColumnDescriptor(String name, String comment, TypeDescriptor type, int position) {
+    this.name = name;
+    this.comment = comment;
+    this.type = type;
+    this.position = position;
+  }
+
+  public ColumnDescriptor(TColumnDesc tColumnDesc) {
+    name = tColumnDesc.getColumnName();
+    comment = tColumnDesc.getComment();
+    type = new TypeDescriptor(tColumnDesc.getTypeDesc());
+    position = tColumnDesc.getPosition();
+  }
+
+  public ColumnDescriptor(FieldSchema column, int position) {
+    name = column.getName();
+    comment = column.getComment();
+    type = new TypeDescriptor(column.getType());
+    this.position = position;
+  }
+
+  public static ColumnDescriptor newPrimitiveColumnDescriptor(String name, String comment, Type type, int position) {
+    return new ColumnDescriptor(name, comment, new TypeDescriptor(type), position);
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public String getComment() {
+    return comment;
+  }
+
+  public TypeDescriptor getTypeDescriptor() {
+    return type;
+  }
+
+  public int getOrdinalPosition() {
+    return position;
+  }
+
+  public TColumnDesc toTColumnDesc() {
+    TColumnDesc tColumnDesc = new TColumnDesc();
+    tColumnDesc.setColumnName(name);
+    tColumnDesc.setComment(comment);
+    tColumnDesc.setTypeDesc(type.toTTypeDesc());
+    tColumnDesc.setPosition(position);
+    return tColumnDesc;
+  }
+
+  public Type getType() {
+    return type.getType();
+  }
+
+  public boolean isPrimitive() {
+    return type.getType().isPrimitiveType();
+  }
+
+  public String getTypeName() {
+    return type.getTypeName();
+  }
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/cli/ColumnValue.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/ColumnValue.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/ColumnValue.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/ColumnValue.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,168 @@
+/**
+ * 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.service.cli;
+
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+
+import org.apache.hive.service.cli.thrift.TBoolValue;
+import org.apache.hive.service.cli.thrift.TByteValue;
+import org.apache.hive.service.cli.thrift.TColumnValue;
+import org.apache.hive.service.cli.thrift.TDoubleValue;
+import org.apache.hive.service.cli.thrift.TI16Value;
+import org.apache.hive.service.cli.thrift.TI32Value;
+import org.apache.hive.service.cli.thrift.TI64Value;
+import org.apache.hive.service.cli.thrift.TStringValue;
+
+/**
+ * ColumnValue.
+ *
+ */
+public class ColumnValue {
+
+  // TODO: replace this with a non-Thrift implementation
+  private final TColumnValue tColumnValue;
+
+  public ColumnValue(TColumnValue tColumnValue) {
+    this.tColumnValue = new TColumnValue(tColumnValue);
+  }
+
+  private static boolean isNull(Object value) {
+    return (value == null);
+  }
+
+  public static ColumnValue booleanValue(Boolean value) {
+    TBoolValue tBoolValue = new TBoolValue();
+    if (value != null) {
+      tBoolValue.setValue(value);
+    }
+    return new ColumnValue(TColumnValue.boolVal(tBoolValue));
+  }
+
+  public static ColumnValue byteValue(Byte value) {
+    TByteValue tByteValue = new TByteValue();
+    if (value != null) {
+      tByteValue.setValue(value);
+    }
+    return new ColumnValue(TColumnValue.byteVal(tByteValue));
+  }
+
+  public static ColumnValue shortValue(Short value) {
+    TI16Value tI16Value = new TI16Value();
+    if (value != null) {
+      tI16Value.setValue(value);
+    }
+    return new ColumnValue(TColumnValue.i16Val(tI16Value));
+  }
+
+  public static ColumnValue intValue(Integer value) {
+    TI32Value tI32Value = new TI32Value();
+    if (value != null) {
+      tI32Value.setValue(value);
+    }
+    return new ColumnValue(TColumnValue.i32Val(tI32Value));
+  }
+
+  public static ColumnValue longValue(Long value) {
+    TI64Value tI64Value = new TI64Value();
+    if (value != null) {
+      tI64Value.setValue(value);
+    }
+    return new ColumnValue(TColumnValue.i64Val(tI64Value));
+  }
+
+  public static ColumnValue floatValue(Float value) {
+    TDoubleValue tDoubleValue = new TDoubleValue();
+    if (value != null) {
+      tDoubleValue.setValue(value);
+    }
+    return new ColumnValue(TColumnValue.doubleVal(tDoubleValue));
+  }
+
+  public static ColumnValue doubleValue(Double value) {
+    TDoubleValue tDoubleValue = new TDoubleValue();
+    if (value != null) {
+      tDoubleValue.setValue(value);
+    }
+    return new ColumnValue(TColumnValue.doubleVal(tDoubleValue));
+  }
+
+  public static ColumnValue stringValue(String value) {
+    TStringValue tStringValue = new TStringValue();
+    if (value != null) {
+      tStringValue.setValue(value);
+    }
+    return new ColumnValue(TColumnValue.stringVal(tStringValue));
+  }
+
+  public static ColumnValue timestampValue(Timestamp value) {
+    TStringValue tStringValue = new TStringValue();
+    if (value != null) {
+      tStringValue.setValue(value.toString());
+    }
+    return new ColumnValue(TColumnValue.stringVal(tStringValue));
+  }
+
+  public static ColumnValue stringValue(BigDecimal value) {
+    TStringValue tStrValue = new TStringValue();
+    if (value != null) {
+      tStrValue.setValue(((BigDecimal)value).toString());
+    }
+    return new ColumnValue(TColumnValue.stringVal(tStrValue));
+  }
+
+  public static ColumnValue newColumnValue(Type type, Object value) {
+    switch (type) {
+    case BOOLEAN_TYPE:
+      return booleanValue((Boolean)value);
+    case TINYINT_TYPE:
+      return byteValue((Byte)value);
+    case SMALLINT_TYPE:
+      return shortValue((Short)value);
+    case INT_TYPE:
+      return intValue((Integer)value);
+    case BIGINT_TYPE:
+      return longValue((Long)value);
+    case FLOAT_TYPE:
+      return floatValue((Float)value);
+    case DOUBLE_TYPE:
+      return doubleValue((Double)value);
+    case STRING_TYPE:
+      return stringValue((String)value);
+    case TIMESTAMP_TYPE:
+      return timestampValue((Timestamp)value);
+    case DECIMAL_TYPE:
+      return stringValue(((BigDecimal)value));
+    case BINARY_TYPE:
+    case ARRAY_TYPE:
+    case MAP_TYPE:
+    case STRUCT_TYPE:
+    case UNION_TYPE:
+    case USER_DEFINED_TYPE:
+      return stringValue((String)value);
+    default:
+      return null;
+    }
+  }
+
+  public TColumnValue toTColumnValue() {
+    return new TColumnValue(tColumnValue);
+  }
+
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,179 @@
+/**
+ * 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.service.cli;
+
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * EmbeddedCLIServiceClient.
+ *
+ */
+public class EmbeddedCLIServiceClient extends CLIServiceClient {
+  private final ICLIService cliService;
+
+  public EmbeddedCLIServiceClient(ICLIService cliService) {
+    this.cliService = cliService;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.CLIServiceClient#openSession(java.lang.String, java.lang.String, java.util.Map)
+   */
+  @Override
+  public SessionHandle openSession(String username, String password,
+      Map<String, String> configuration) throws HiveSQLException {
+    return cliService.openSession(username, password, configuration);
+  }
+
+  @Override
+  public SessionHandle openSessionWithImpersonation(String username, String password,
+      Map<String, String> configuration, String delegationToken) throws HiveSQLException {
+    throw new HiveSQLException("Impersonated session is not supported in the embedded mode");
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.CLIServiceClient#closeSession(org.apache.hive.service.cli.SessionHandle)
+   */
+  @Override
+  public void closeSession(SessionHandle sessionHandle) throws HiveSQLException {
+    cliService.closeSession(sessionHandle);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.CLIServiceClient#getInfo(org.apache.hive.service.cli.SessionHandle, java.util.List)
+   */
+  @Override
+  public GetInfoValue getInfo(SessionHandle sessionHandle, GetInfoType getInfoType)
+      throws HiveSQLException {
+    return cliService.getInfo(sessionHandle, getInfoType);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.CLIServiceClient#executeStatement(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.util.Map)
+   */
+  @Override
+  public OperationHandle executeStatement(SessionHandle sessionHandle, String statement,
+      Map<String, String> confOverlay) throws HiveSQLException {
+    return cliService.executeStatement(sessionHandle, statement, confOverlay);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.CLIServiceClient#getTypeInfo(org.apache.hive.service.cli.SessionHandle)
+   */
+  @Override
+  public OperationHandle getTypeInfo(SessionHandle sessionHandle) throws HiveSQLException {
+    return cliService.getTypeInfo(sessionHandle);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.CLIServiceClient#getCatalogs(org.apache.hive.service.cli.SessionHandle)
+   */
+  @Override
+  public OperationHandle getCatalogs(SessionHandle sessionHandle) throws HiveSQLException {
+    return cliService.getCatalogs(sessionHandle);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.CLIServiceClient#getSchemas(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String)
+   */
+  @Override
+  public OperationHandle getSchemas(SessionHandle sessionHandle, String catalogName,
+      String schemaName) throws HiveSQLException {
+    return cliService.getSchemas(sessionHandle, catalogName, schemaName);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.CLIServiceClient#getTables(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String, java.lang.String, java.util.List)
+   */
+  @Override
+  public OperationHandle getTables(SessionHandle sessionHandle, String catalogName,
+      String schemaName, String tableName, List<String> tableTypes) throws HiveSQLException {
+    return cliService.getTables(sessionHandle, catalogName, schemaName, tableName, tableTypes);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.CLIServiceClient#getTableTypes(org.apache.hive.service.cli.SessionHandle)
+   */
+  @Override
+  public OperationHandle getTableTypes(SessionHandle sessionHandle) throws HiveSQLException {
+    return cliService.getTableTypes(sessionHandle);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.CLIServiceClient#getColumns(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String, java.lang.String, java.lang.String)
+   */
+  @Override
+  public OperationHandle getColumns(SessionHandle sessionHandle, String catalogName,
+      String schemaName, String tableName, String columnName) throws HiveSQLException {
+    return cliService.getColumns(sessionHandle, catalogName, schemaName, tableName, columnName);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.CLIServiceClient#getFunctions(org.apache.hive.service.cli.SessionHandle, java.lang.String)
+   */
+  @Override
+  public OperationHandle getFunctions(SessionHandle sessionHandle,
+      String catalogName, String schemaName, String functionName)
+      throws HiveSQLException {
+    return cliService.getFunctions(sessionHandle, catalogName, schemaName, functionName);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.CLIServiceClient#getOperationStatus(org.apache.hive.service.cli.OperationHandle)
+   */
+  @Override
+  public OperationState getOperationStatus(OperationHandle opHandle) throws HiveSQLException {
+    return cliService.getOperationStatus(opHandle);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.CLIServiceClient#cancelOperation(org.apache.hive.service.cli.OperationHandle)
+   */
+  @Override
+  public void cancelOperation(OperationHandle opHandle) throws HiveSQLException {
+    cliService.cancelOperation(opHandle);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.CLIServiceClient#closeOperation(org.apache.hive.service.cli.OperationHandle)
+   */
+  @Override
+  public void closeOperation(OperationHandle opHandle) throws HiveSQLException {
+    cliService.closeOperation(opHandle);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.CLIServiceClient#getResultSetMetadata(org.apache.hive.service.cli.OperationHandle)
+   */
+  @Override
+  public TableSchema getResultSetMetadata(OperationHandle opHandle) throws HiveSQLException {
+    return cliService.getResultSetMetadata(opHandle);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.CLIServiceClient#fetchResults(org.apache.hive.service.cli.OperationHandle, org.apache.hive.service.cli.FetchOrientation, long)
+   */
+  @Override
+  public RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, long maxRows)
+      throws HiveSQLException {
+    return cliService.fetchResults(opHandle, orientation, maxRows);
+  }
+
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/cli/FetchOrientation.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/FetchOrientation.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/FetchOrientation.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/FetchOrientation.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,54 @@
+/**
+ * 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.service.cli;
+
+import org.apache.hive.service.cli.thrift.TFetchOrientation;
+
+/**
+ * FetchOrientation.
+ *
+ */
+public enum FetchOrientation {
+  FETCH_NEXT(TFetchOrientation.FETCH_NEXT),
+  FETCH_PRIOR(TFetchOrientation.FETCH_PRIOR),
+  FETCH_RELATIVE(TFetchOrientation.FETCH_RELATIVE),
+  FETCH_ABSOLUTE(TFetchOrientation.FETCH_ABSOLUTE),
+  FETCH_FIRST(TFetchOrientation.FETCH_FIRST),
+  FETCH_LAST(TFetchOrientation.FETCH_LAST);
+
+  private TFetchOrientation tFetchOrientation;
+
+  FetchOrientation(TFetchOrientation tFetchOrientation) {
+    this.tFetchOrientation = tFetchOrientation;
+  }
+
+  public static FetchOrientation getFetchOrientation(TFetchOrientation tFetchOrientation) {
+    for (FetchOrientation fetchOrientation : values()) {
+      if (tFetchOrientation.equals(fetchOrientation.toTFetchOrientation())) {
+        return fetchOrientation;
+      }
+    }
+    // TODO: Should this really default to FETCH_NEXT?
+    return FETCH_NEXT;
+  }
+
+  public TFetchOrientation toTFetchOrientation() {
+    return tFetchOrientation;
+  }
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/cli/GetInfoType.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/GetInfoType.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/GetInfoType.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/GetInfoType.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,96 @@
+/**
+ * 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.service.cli;
+
+import org.apache.hive.service.cli.thrift.TGetInfoType;
+
+/**
+ * GetInfoType.
+ *
+ */
+public enum GetInfoType {
+  CLI_MAX_DRIVER_CONNECTIONS(TGetInfoType.CLI_MAX_DRIVER_CONNECTIONS),
+  CLI_MAX_CONCURRENT_ACTIVITIES(TGetInfoType.CLI_MAX_CONCURRENT_ACTIVITIES),
+  CLI_DATA_SOURCE_NAME(TGetInfoType.CLI_DATA_SOURCE_NAME),
+  CLI_FETCH_DIRECTION(TGetInfoType.CLI_FETCH_DIRECTION),
+  CLI_SERVER_NAME(TGetInfoType.CLI_SERVER_NAME),
+  CLI_SEARCH_PATTERN_ESCAPE(TGetInfoType.CLI_SEARCH_PATTERN_ESCAPE),
+  CLI_DBMS_NAME(TGetInfoType.CLI_DBMS_NAME),
+  CLI_DBMS_VER(TGetInfoType.CLI_DBMS_VER),
+  CLI_ACCESSIBLE_TABLES(TGetInfoType.CLI_ACCESSIBLE_TABLES),
+  CLI_ACCESSIBLE_PROCEDURES(TGetInfoType.CLI_ACCESSIBLE_PROCEDURES),
+  CLI_CURSOR_COMMIT_BEHAVIOR(TGetInfoType.CLI_CURSOR_COMMIT_BEHAVIOR),
+  CLI_DATA_SOURCE_READ_ONLY(TGetInfoType.CLI_DATA_SOURCE_READ_ONLY),
+  CLI_DEFAULT_TXN_ISOLATION(TGetInfoType.CLI_DEFAULT_TXN_ISOLATION),
+  CLI_IDENTIFIER_CASE(TGetInfoType.CLI_IDENTIFIER_CASE),
+  CLI_IDENTIFIER_QUOTE_CHAR(TGetInfoType.CLI_IDENTIFIER_QUOTE_CHAR),
+  CLI_MAX_COLUMN_NAME_LEN(TGetInfoType.CLI_MAX_COLUMN_NAME_LEN),
+  CLI_MAX_CURSOR_NAME_LEN(TGetInfoType.CLI_MAX_CURSOR_NAME_LEN),
+  CLI_MAX_SCHEMA_NAME_LEN(TGetInfoType.CLI_MAX_SCHEMA_NAME_LEN),
+  CLI_MAX_CATALOG_NAME_LEN(TGetInfoType.CLI_MAX_CATALOG_NAME_LEN),
+  CLI_MAX_TABLE_NAME_LEN(TGetInfoType.CLI_MAX_TABLE_NAME_LEN),
+  CLI_SCROLL_CONCURRENCY(TGetInfoType.CLI_SCROLL_CONCURRENCY),
+  CLI_TXN_CAPABLE(TGetInfoType.CLI_TXN_CAPABLE),
+  CLI_USER_NAME(TGetInfoType.CLI_USER_NAME),
+  CLI_TXN_ISOLATION_OPTION(TGetInfoType.CLI_TXN_ISOLATION_OPTION),
+  CLI_INTEGRITY(TGetInfoType.CLI_INTEGRITY),
+  CLI_GETDATA_EXTENSIONS(TGetInfoType.CLI_GETDATA_EXTENSIONS),
+  CLI_NULL_COLLATION(TGetInfoType.CLI_NULL_COLLATION),
+  CLI_ALTER_TABLE(TGetInfoType.CLI_ALTER_TABLE),
+  CLI_ORDER_BY_COLUMNS_IN_SELECT(TGetInfoType.CLI_ORDER_BY_COLUMNS_IN_SELECT),
+  CLI_SPECIAL_CHARACTERS(TGetInfoType.CLI_SPECIAL_CHARACTERS),
+  CLI_MAX_COLUMNS_IN_GROUP_BY(TGetInfoType.CLI_MAX_COLUMNS_IN_GROUP_BY),
+  CLI_MAX_COLUMNS_IN_INDEX(TGetInfoType.CLI_MAX_COLUMNS_IN_INDEX),
+  CLI_MAX_COLUMNS_IN_ORDER_BY(TGetInfoType.CLI_MAX_COLUMNS_IN_ORDER_BY),
+  CLI_MAX_COLUMNS_IN_SELECT(TGetInfoType.CLI_MAX_COLUMNS_IN_SELECT),
+  CLI_MAX_COLUMNS_IN_TABLE(TGetInfoType.CLI_MAX_COLUMNS_IN_TABLE),
+  CLI_MAX_INDEX_SIZE(TGetInfoType.CLI_MAX_INDEX_SIZE),
+  CLI_MAX_ROW_SIZE(TGetInfoType.CLI_MAX_ROW_SIZE),
+  CLI_MAX_STATEMENT_LEN(TGetInfoType.CLI_MAX_STATEMENT_LEN),
+  CLI_MAX_TABLES_IN_SELECT(TGetInfoType.CLI_MAX_TABLES_IN_SELECT),
+  CLI_MAX_USER_NAME_LEN(TGetInfoType.CLI_MAX_USER_NAME_LEN),
+  CLI_OJ_CAPABILITIES(TGetInfoType.CLI_OJ_CAPABILITIES),
+
+  CLI_XOPEN_CLI_YEAR(TGetInfoType.CLI_XOPEN_CLI_YEAR),
+  CLI_CURSOR_SENSITIVITY(TGetInfoType.CLI_CURSOR_SENSITIVITY),
+  CLI_DESCRIBE_PARAMETER(TGetInfoType.CLI_DESCRIBE_PARAMETER),
+  CLI_CATALOG_NAME(TGetInfoType.CLI_CATALOG_NAME),
+  CLI_COLLATION_SEQ(TGetInfoType.CLI_COLLATION_SEQ),
+  CLI_MAX_IDENTIFIER_LEN(TGetInfoType.CLI_MAX_IDENTIFIER_LEN);
+
+  private final TGetInfoType tInfoType;
+
+  GetInfoType(TGetInfoType tInfoType) {
+    this.tInfoType = tInfoType;
+  }
+
+  public static GetInfoType getGetInfoType(TGetInfoType tGetInfoType) {
+    for (GetInfoType infoType : values()) {
+      if (tGetInfoType.equals(infoType.tInfoType)) {
+        return infoType;
+      }
+    }
+    throw new IllegalArgumentException("Unrecognized Thrift TGetInfoType value: " + tGetInfoType);
+  }
+
+  public TGetInfoType toTGetInfoType() {
+    return tInfoType;
+  }
+
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/cli/GetInfoValue.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/GetInfoValue.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/GetInfoValue.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/GetInfoValue.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,82 @@
+/**
+ * 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.service.cli;
+
+import org.apache.hive.service.cli.thrift.TGetInfoValue;
+
+/**
+ * GetInfoValue.
+ *
+ */
+public class GetInfoValue {
+  private String stringValue = null;
+  private short shortValue;
+  private int intValue;
+  private long longValue;
+
+  public GetInfoValue(String stringValue) {
+    this.stringValue = stringValue;
+  }
+
+  public GetInfoValue(short shortValue) {
+    this.shortValue = shortValue;
+  }
+
+  public GetInfoValue(int intValue) {
+    this.intValue = intValue;
+  }
+
+  public GetInfoValue(long longValue) {
+    this.longValue = longValue;
+  }
+
+  public GetInfoValue(TGetInfoValue tGetInfoValue) {
+    switch (tGetInfoValue.getSetField()) {
+    case STRING_VALUE:
+      stringValue = tGetInfoValue.getStringValue();
+      break;
+    default:
+      throw new IllegalArgumentException("Unreconigzed TGetInfoValue");
+    }
+  }
+
+  public TGetInfoValue toTGetInfoValue() {
+    TGetInfoValue tInfoValue = new TGetInfoValue();
+    if (stringValue != null) {
+      tInfoValue.setStringValue(stringValue);
+    }
+    return tInfoValue;
+  }
+
+  public String getStringValue() {
+    return stringValue;
+  }
+
+  public short getShortValue() {
+    return shortValue;
+  }
+
+  public int getIntValue() {
+    return intValue;
+  }
+
+  public long getLongValue() {
+    return longValue;
+  }
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/cli/Handle.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/Handle.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/Handle.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/Handle.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,78 @@
+/**
+ * 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.service.cli;
+
+import org.apache.hive.service.cli.thrift.THandleIdentifier;
+
+
+
+
+public abstract class Handle {
+
+  private final HandleIdentifier handleId;
+
+  public Handle() {
+    handleId = new HandleIdentifier();
+  }
+
+  public Handle(HandleIdentifier handleId) {
+    this.handleId = handleId;
+  }
+
+  public Handle(THandleIdentifier tHandleIdentifier) {
+    this.handleId = new HandleIdentifier(tHandleIdentifier);
+  }
+
+  public HandleIdentifier getHandleIdentifier() {
+    return handleId;
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((handleId == null) ? 0 : handleId.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (!(obj instanceof Handle)) {
+      return false;
+    }
+    Handle other = (Handle) obj;
+    if (handleId == null) {
+      if (other.handleId != null) {
+        return false;
+      }
+    } else if (!handleId.equals(other.handleId)) {
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public abstract String toString();
+
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/cli/HandleIdentifier.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/HandleIdentifier.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/HandleIdentifier.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/HandleIdentifier.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,113 @@
+/**
+ * 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.service.cli;
+
+import java.nio.ByteBuffer;
+import java.util.UUID;
+
+import org.apache.hive.service.cli.thrift.THandleIdentifier;
+
+/**
+ * HandleIdentifier.
+ *
+ */
+public class HandleIdentifier {
+  private final UUID publicId;
+  private final UUID secretId;
+
+  public HandleIdentifier() {
+    publicId = UUID.randomUUID();
+    secretId = UUID.randomUUID();
+  }
+
+  public HandleIdentifier(UUID publicId, UUID secretId) {
+    this.publicId = publicId;
+    this.secretId = secretId;
+  }
+
+  public HandleIdentifier(THandleIdentifier tHandleId) {
+    ByteBuffer bb = ByteBuffer.wrap(tHandleId.getGuid());
+    this.publicId = new UUID(bb.getLong(), bb.getLong());
+    bb = ByteBuffer.wrap(tHandleId.getSecret());
+    this.secretId = new UUID(bb.getLong(), bb.getLong());
+  }
+
+  public UUID getPublicId() {
+    return publicId;
+  }
+
+  public UUID getSecretId() {
+    return secretId;
+  }
+
+  public THandleIdentifier toTHandleIdentifier() {
+    byte[] guid = new byte[16];
+    byte[] secret = new byte[16];
+    ByteBuffer guidBB = ByteBuffer.wrap(guid);
+    ByteBuffer secretBB = ByteBuffer.wrap(secret);
+    guidBB.putLong(publicId.getMostSignificantBits());
+    guidBB.putLong(publicId.getLeastSignificantBits());
+    secretBB.putLong(secretId.getMostSignificantBits());
+    secretBB.putLong(secretId.getLeastSignificantBits());
+    return new THandleIdentifier(ByteBuffer.wrap(guid), ByteBuffer.wrap(secret));
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((publicId == null) ? 0 : publicId.hashCode());
+    result = prime * result + ((secretId == null) ? 0 : secretId.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (!(obj instanceof HandleIdentifier)) {
+      return false;
+    }
+    HandleIdentifier other = (HandleIdentifier) obj;
+    if (publicId == null) {
+      if (other.publicId != null) {
+        return false;
+      }
+    } else if (!publicId.equals(other.publicId)) {
+      return false;
+    }
+    if (secretId == null) {
+      if (other.secretId != null) {
+        return false;
+      }
+    } else if (!secretId.equals(other.secretId)) {
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    return publicId.toString();
+  }
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/cli/HiveSQLException.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/HiveSQLException.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/HiveSQLException.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/HiveSQLException.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,124 @@
+/**
+ * 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.service.cli;
+
+import java.sql.SQLException;
+
+import org.apache.hive.service.cli.thrift.TStatus;
+import org.apache.hive.service.cli.thrift.TStatusCode;
+
+/**
+ * HiveSQLException.
+ *
+ */
+public class HiveSQLException extends SQLException {
+
+  /**
+   *
+   */
+  private static final long serialVersionUID = -6095254671958748094L;
+
+  /**
+   *
+   */
+  public HiveSQLException() {
+    super();
+  }
+
+  /**
+   * @param reason
+   */
+  public HiveSQLException(String reason) {
+    super(reason);
+  }
+
+  /**
+   * @param cause
+   */
+  public HiveSQLException(Throwable cause) {
+    super(cause);
+  }
+
+  /**
+   * @param reason
+   * @param sqlState
+   */
+  public HiveSQLException(String reason, String sqlState) {
+    super(reason, sqlState);
+  }
+
+  /**
+   * @param reason
+   * @param cause
+   */
+  public HiveSQLException(String reason, Throwable cause) {
+    super(reason, cause);
+  }
+
+  /**
+   * @param reason
+   * @param sqlState
+   * @param vendorCode
+   */
+  public HiveSQLException(String reason, String sqlState, int vendorCode) {
+    super(reason, sqlState, vendorCode);
+  }
+
+  /**
+   * @param reason
+   * @param sqlState
+   * @param cause
+   */
+  public HiveSQLException(String reason, String sqlState, Throwable cause) {
+    super(reason, sqlState, cause);
+  }
+
+  /**
+   * @param reason
+   * @param sqlState
+   * @param vendorCode
+   * @param cause
+   */
+  public HiveSQLException(String reason, String sqlState, int vendorCode, Throwable cause) {
+    super(reason, sqlState, vendorCode, cause);
+  }
+
+  public HiveSQLException(TStatus status) {
+    // TODO: set correct vendorCode field
+    super(status.getErrorMessage(), status.getSqlState(), 1);
+  }
+
+  public TStatus toTStatus() {
+    // TODO: convert sqlState, etc.
+    TStatus tStatus = new TStatus(TStatusCode.ERROR_STATUS);
+    tStatus.setSqlState(getSQLState());
+    tStatus.setErrorCode(getErrorCode());
+    tStatus.setErrorMessage(getMessage());
+    return tStatus;
+  }
+
+  public static TStatus toTStatus(Exception e) {
+    if (e instanceof HiveSQLException) {
+      return ((HiveSQLException)e).toTStatus();
+    }
+    TStatus tStatus = new TStatus(TStatusCode.ERROR_STATUS);
+    return tStatus;
+  }
+
+}