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 2014/09/02 18:22:09 UTC

svn commit: r1622056 - in /hive/trunk: common/src/java/org/apache/hadoop/hive/conf/ itests/hive-unit/src/test/java/org/apache/hive/service/auth/ service/src/java/org/apache/hive/service/auth/

Author: hashutosh
Date: Tue Sep  2 16:22:08 2014
New Revision: 1622056

URL: http://svn.apache.org/r1622056
Log:
HIVE-7543 : Cleanup of org.apache.hive.service.auth package (Lars Francke via Ashutosh Chauhan)

Modified:
    hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
    hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/service/auth/TestCustomAuthentication.java
    hive/trunk/service/src/java/org/apache/hive/service/auth/AnonymousAuthenticationProviderImpl.java
    hive/trunk/service/src/java/org/apache/hive/service/auth/AuthenticationProviderFactory.java
    hive/trunk/service/src/java/org/apache/hive/service/auth/CustomAuthenticationProviderImpl.java
    hive/trunk/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java
    hive/trunk/service/src/java/org/apache/hive/service/auth/HttpAuthUtils.java
    hive/trunk/service/src/java/org/apache/hive/service/auth/HttpAuthenticationException.java
    hive/trunk/service/src/java/org/apache/hive/service/auth/HttpCLIServiceUGIProcessor.java
    hive/trunk/service/src/java/org/apache/hive/service/auth/KerberosSaslHelper.java
    hive/trunk/service/src/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java
    hive/trunk/service/src/java/org/apache/hive/service/auth/PamAuthenticationProviderImpl.java
    hive/trunk/service/src/java/org/apache/hive/service/auth/PasswdAuthenticationProvider.java
    hive/trunk/service/src/java/org/apache/hive/service/auth/PlainSaslHelper.java
    hive/trunk/service/src/java/org/apache/hive/service/auth/PlainSaslServer.java
    hive/trunk/service/src/java/org/apache/hive/service/auth/SaslQOP.java
    hive/trunk/service/src/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java
    hive/trunk/service/src/java/org/apache/hive/service/auth/TSubjectAssumingTransport.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=1622056&r1=1622055&r2=1622056&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 Tue Sep  2 16:22:08 2014
@@ -1587,8 +1587,8 @@ public class HiveConf extends Configurat
         "must be a proper implementation of the interface\n" +
         "org.apache.hive.service.auth.PasswdAuthenticationProvider. HiveServer2\n" +
         "will call its Authenticate(user, passed) method to authenticate requests.\n" +
-        "The implementation may optionally extend Hadoop's\n" +
-        "org.apache.hadoop.conf.Configured class to grab Hive's Configuration object."),
+        "The implementation may optionally implement Hadoop's\n" +
+        "org.apache.hadoop.conf.Configurable class to grab Hive's Configuration object."),
     HIVE_SERVER2_PAM_SERVICES("hive.server2.authentication.pam.services", null,
       "List of the underlying pam services that should be used when auth type is PAM\n" +
       "A file with the same name must exist in /etc/pam.d"),

Modified: hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/service/auth/TestCustomAuthentication.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/service/auth/TestCustomAuthentication.java?rev=1622056&r1=1622055&r2=1622056&view=diff
==============================================================================
--- hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/service/auth/TestCustomAuthentication.java (original)
+++ hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/service/auth/TestCustomAuthentication.java Tue Sep  2 16:22:08 2014
@@ -18,7 +18,6 @@
 package org.apache.hive.service.auth;
 
 import junit.framework.Assert;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hive.service.server.HiveServer2;
 import org.junit.AfterClass;

Modified: hive/trunk/service/src/java/org/apache/hive/service/auth/AnonymousAuthenticationProviderImpl.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/AnonymousAuthenticationProviderImpl.java?rev=1622056&r1=1622055&r2=1622056&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/AnonymousAuthenticationProviderImpl.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/AnonymousAuthenticationProviderImpl.java Tue Sep  2 16:22:08 2014
@@ -20,12 +20,14 @@ package org.apache.hive.service.auth;
 
 import javax.security.sasl.AuthenticationException;
 
+/**
+ * This authentication provider allows any combination of username and password.
+ */
 public class AnonymousAuthenticationProviderImpl implements PasswdAuthenticationProvider {
 
   @Override
   public void Authenticate(String user, String password) throws AuthenticationException {
     // no-op authentication
-    return;
   }
 
 }

Modified: hive/trunk/service/src/java/org/apache/hive/service/auth/AuthenticationProviderFactory.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/AuthenticationProviderFactory.java?rev=1622056&r1=1622055&r2=1622056&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/AuthenticationProviderFactory.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/AuthenticationProviderFactory.java Tue Sep  2 16:22:08 2014
@@ -19,15 +19,18 @@ package org.apache.hive.service.auth;
 
 import javax.security.sasl.AuthenticationException;
 
-public class AuthenticationProviderFactory {
+/**
+ * This class helps select a {@link PasswdAuthenticationProvider} for a given {@code AuthMethod}.
+ */
+public final class AuthenticationProviderFactory {
 
-  public static enum AuthMethods {
+  public enum AuthMethods {
     LDAP("LDAP"),
     PAM("PAM"),
     CUSTOM("CUSTOM"),
     NONE("NONE");
 
-    String authMethod;
+    private final String authMethod;
 
     AuthMethods(String authMethod) {
       this.authMethod = authMethod;
@@ -37,7 +40,8 @@ public class AuthenticationProviderFacto
       return authMethod;
     }
 
-    public static AuthMethods getValidAuthMethod(String authMethodStr) throws AuthenticationException {
+    public static AuthMethods getValidAuthMethod(String authMethodStr)
+      throws AuthenticationException {
       for (AuthMethods auth : AuthMethods.values()) {
         if (authMethodStr.equals(auth.getAuthMethod())) {
           return auth;
@@ -47,24 +51,20 @@ public class AuthenticationProviderFacto
     }
   }
 
-  private AuthenticationProviderFactory () {
+  private AuthenticationProviderFactory() {
   }
 
   public static PasswdAuthenticationProvider getAuthenticationProvider(AuthMethods authMethod)
-      throws AuthenticationException {
-    if (authMethod.equals(AuthMethods.LDAP)) {
+    throws AuthenticationException {
+    if (authMethod == AuthMethods.LDAP) {
       return new LdapAuthenticationProviderImpl();
-    }
-    else if (authMethod.equals(AuthMethods.PAM)) {
+    } else if (authMethod == AuthMethods.PAM) {
       return new PamAuthenticationProviderImpl();
-    }
-    else if (authMethod.equals(AuthMethods.CUSTOM)) {
+    } else if (authMethod == AuthMethods.CUSTOM) {
       return new CustomAuthenticationProviderImpl();
-    }
-    else if (authMethod.equals(AuthMethods.NONE)) {
+    } else if (authMethod == AuthMethods.NONE) {
       return new AnonymousAuthenticationProviderImpl();
-    }
-    else {
+    } else {
       throw new AuthenticationException("Unsupported authentication method");
     }
   }

Modified: hive/trunk/service/src/java/org/apache/hive/service/auth/CustomAuthenticationProviderImpl.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/CustomAuthenticationProviderImpl.java?rev=1622056&r1=1622055&r2=1622056&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/CustomAuthenticationProviderImpl.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/CustomAuthenticationProviderImpl.java Tue Sep  2 16:22:08 2014
@@ -22,27 +22,29 @@ import javax.security.sasl.Authenticatio
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.util.ReflectionUtils;
 
-public class CustomAuthenticationProviderImpl
-  implements PasswdAuthenticationProvider {
+/**
+ * This authentication provider implements the {@code CUSTOM} authentication. It allows a {@link
+ * PasswdAuthenticationProvider} to be specified at configuration time which may additionally
+ * implement {@link org.apache.hadoop.conf.Configurable Configurable} to grab Hive's {@link
+ * org.apache.hadoop.conf.Configuration Configuration}.
+ */
+public class CustomAuthenticationProviderImpl implements PasswdAuthenticationProvider {
 
-  Class<? extends PasswdAuthenticationProvider> customHandlerClass;
-  PasswdAuthenticationProvider customProvider;
+  private final PasswdAuthenticationProvider customProvider;
 
   @SuppressWarnings("unchecked")
-  CustomAuthenticationProviderImpl () {
+  CustomAuthenticationProviderImpl() {
     HiveConf conf = new HiveConf();
-    this.customHandlerClass = (Class<? extends PasswdAuthenticationProvider>)
-        conf.getClass(
-            HiveConf.ConfVars.HIVE_SERVER2_CUSTOM_AUTHENTICATION_CLASS.varname,
-            PasswdAuthenticationProvider.class);
-    this.customProvider =
-        ReflectionUtils.newInstance(this.customHandlerClass, conf);
+    Class<? extends PasswdAuthenticationProvider> customHandlerClass =
+      (Class<? extends PasswdAuthenticationProvider>) conf.getClass(
+        HiveConf.ConfVars.HIVE_SERVER2_CUSTOM_AUTHENTICATION_CLASS.varname,
+        PasswdAuthenticationProvider.class);
+    customProvider = ReflectionUtils.newInstance(customHandlerClass, conf);
   }
 
   @Override
-  public void Authenticate(String user, String  password)
-      throws AuthenticationException {
-    this.customProvider.Authenticate(user, password);
+  public void Authenticate(String user, String password) throws AuthenticationException {
+    customProvider.Authenticate(user, password);
   }
 
 }

Modified: hive/trunk/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java?rev=1622056&r1=1622055&r2=1622056&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java Tue Sep  2 16:22:08 2014
@@ -23,7 +23,6 @@ import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.util.HashMap;
 import java.util.Map;
-
 import javax.security.auth.login.LoginException;
 import javax.security.sasl.Sasl;
 
@@ -41,13 +40,14 @@ import org.apache.thrift.transport.TSock
 import org.apache.thrift.transport.TTransport;
 import org.apache.thrift.transport.TTransportException;
 import org.apache.thrift.transport.TTransportFactory;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
+/**
+ * This class helps in some aspects of authentication. It creates the proper Thrift classes for the
+ * given configuration as well as helps with authenticating requests.
+ */
 public class HiveAuthFactory {
-  private static final Logger LOG = LoggerFactory.getLogger(HiveAuthFactory.class);
 
-  public static enum AuthTypes {
+  public enum AuthTypes {
     NOSASL("NOSASL"),
     NONE("NONE"),
     LDAP("LDAP"),
@@ -55,7 +55,7 @@ public class HiveAuthFactory {
     CUSTOM("CUSTOM"),
     PAM("PAM");
 
-    private String authType;
+    private final String authType;
 
     AuthTypes(String authType) {
       this.authType = authType;
@@ -65,11 +65,11 @@ public class HiveAuthFactory {
       return authType;
     }
 
-  };
+  }
 
-  private HadoopThriftAuthBridge.Server saslServer = null;
+  private HadoopThriftAuthBridge.Server saslServer;
   private String authTypeStr;
-  private String transportMode;
+  private final String transportMode;
   private final HiveConf conf;
 
   public static final String HS2_PROXY_USER = "hive.server2.proxy.user";
@@ -81,21 +81,19 @@ public class HiveAuthFactory {
     authTypeStr = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_AUTHENTICATION);
 
     // In http mode we use NOSASL as the default auth type
-    if (transportMode.equalsIgnoreCase("http")) {
+    if ("http".equalsIgnoreCase(transportMode)) {
       if (authTypeStr == null) {
         authTypeStr = AuthTypes.NOSASL.getAuthName();
       }
-    }
-    else {
+    } else {
       if (authTypeStr == null) {
         authTypeStr = AuthTypes.NONE.getAuthName();
       }
       if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())
           && ShimLoader.getHadoopShims().isSecureShimImpl()) {
-        saslServer = ShimLoader.getHadoopThriftAuthBridge().createServer(
-            conf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB),
-            conf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL)
-            );
+        saslServer = ShimLoader.getHadoopThriftAuthBridge()
+          .createServer(conf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB),
+                        conf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL));
         // start delegation token manager
         try {
           saslServer.startDelegationTokenSecretManager(conf, null);
@@ -108,8 +106,7 @@ public class HiveAuthFactory {
 
   public Map<String, String> getSaslProperties() {
     Map<String, String> saslProps = new HashMap<String, String>();
-    SaslQOP saslQOP =
-        SaslQOP.fromString(conf.getVar(ConfVars.HIVE_SERVER2_THRIFT_SASL_QOP));
+    SaslQOP saslQOP = SaslQOP.fromString(conf.getVar(ConfVars.HIVE_SERVER2_THRIFT_SASL_QOP));
     saslProps.put(Sasl.QOP, saslQOP.toString());
     saslProps.put(Sasl.SERVER_AUTH, "true");
     return saslProps;
@@ -139,12 +136,10 @@ public class HiveAuthFactory {
     return transportFactory;
   }
 
-  public TProcessorFactory getAuthProcFactory(ThriftCLIService service)
-      throws LoginException {
-    if (transportMode.equalsIgnoreCase("http")) {
+  public TProcessorFactory getAuthProcFactory(ThriftCLIService service) throws LoginException {
+    if ("http".equalsIgnoreCase(transportMode)) {
       return HttpAuthUtils.getAuthProcFactory(service);
-    }
-    else {
+    } else {
       if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())) {
         return KerberosSaslHelper.getKerberosProcessorFactory(saslServer, service);
       } else {
@@ -154,18 +149,14 @@ public class HiveAuthFactory {
   }
 
   public String getRemoteUser() {
-    if (saslServer != null) {
-      return saslServer.getRemoteUser();
-    } else {
-      return null;
-    }
+    return saslServer == null ? null : saslServer.getRemoteUser();
   }
 
   public String getIpAddress() {
-    if(saslServer != null && saslServer.getRemoteAddress() != null) {
-      return saslServer.getRemoteAddress().getHostAddress();
-    } else {
+    if (saslServer == null || saslServer.getRemoteAddress() == null) {
       return null;
+    } else {
+      return saslServer.getRemoteAddress().getHostAddress();
     }
   }
 
@@ -173,62 +164,58 @@ public class HiveAuthFactory {
   public static void loginFromKeytab(HiveConf hiveConf) throws IOException {
     String principal = hiveConf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL);
     String keyTabFile = hiveConf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB);
-    if (!principal.isEmpty() && !keyTabFile.isEmpty()) {
-      ShimLoader.getHadoopShims().loginUserFromKeytab(principal, keyTabFile);
+    if (principal.isEmpty() || keyTabFile.isEmpty()) {
+      throw new IOException("HiveServer2 Kerberos principal or keytab is not correctly configured");
     } else {
-      throw new IOException ("HiveServer2 kerberos principal or keytab " +
-          "is not correctly configured");
+      ShimLoader.getHadoopShims().loginUserFromKeytab(principal, keyTabFile);
     }
   }
 
-  // Perform spnego login using the hadoop shim API if the configuration is available
-  public static UserGroupInformation loginFromSpnegoKeytabAndReturnUGI(
-      HiveConf hiveConf) throws IOException {
+  // Perform SPNEGO login using the hadoop shim API if the configuration is available
+  public static UserGroupInformation loginFromSpnegoKeytabAndReturnUGI(HiveConf hiveConf)
+    throws IOException {
     String principal = hiveConf.getVar(ConfVars.HIVE_SERVER2_SPNEGO_PRINCIPAL);
     String keyTabFile = hiveConf.getVar(ConfVars.HIVE_SERVER2_SPNEGO_KEYTAB);
-    if (!principal.isEmpty() && !keyTabFile.isEmpty()) {
-      return ShimLoader.getHadoopShims().loginUserFromKeytabAndReturnUGI(
-          principal, keyTabFile);
+    if (principal.isEmpty() || keyTabFile.isEmpty()) {
+      throw new IOException("HiveServer2 SPNEGO principal or keytab is not correctly configured");
     } else {
-      throw new IOException ("HiveServer2 SPNego principal or keytab " +
-          "is not correctly configured");
+      return ShimLoader.getHadoopShims().loginUserFromKeytabAndReturnUGI(principal, keyTabFile);
     }
   }
 
-  public static TTransport getSocketTransport(String host, int port, int loginTimeout)
-      throws TTransportException {
+  public static TTransport getSocketTransport(String host, int port, int loginTimeout) {
     return new TSocket(host, port, loginTimeout);
   }
 
   public static TTransport getSSLSocket(String host, int port, int loginTimeout)
-      throws TTransportException {
+    throws TTransportException {
     return TSSLTransportFactory.getClientSocket(host, port, loginTimeout);
   }
 
   public static TTransport getSSLSocket(String host, int port, int loginTimeout,
-      String trustStorePath, String trustStorePassWord) throws TTransportException {
+    String trustStorePath, String trustStorePassWord) throws TTransportException {
     TSSLTransportFactory.TSSLTransportParameters params =
-        new TSSLTransportFactory.TSSLTransportParameters();
+      new TSSLTransportFactory.TSSLTransportParameters();
     params.setTrustStore(trustStorePath, trustStorePassWord);
     params.requireClientAuth(true);
     return TSSLTransportFactory.getClientSocket(host, port, loginTimeout, params);
   }
 
   public static TServerSocket getServerSocket(String hiveHost, int portNum)
-      throws TTransportException {
-    InetSocketAddress serverAddress = null;
-    if (hiveHost != null && !hiveHost.isEmpty()) {
-      serverAddress = new InetSocketAddress(hiveHost, portNum);
+    throws TTransportException {
+    InetSocketAddress serverAddress;
+    if (hiveHost == null || hiveHost.isEmpty()) {
+      serverAddress = new InetSocketAddress(portNum);
     } else {
-      serverAddress = new  InetSocketAddress(portNum);
+      serverAddress = new InetSocketAddress(hiveHost, portNum);
     }
-    return new TServerSocket(serverAddress );
+    return new TServerSocket(serverAddress);
   }
 
-  public static TServerSocket getServerSSLSocket(String hiveHost, int portNum,
-      String keyStorePath, String keyStorePassWord) throws TTransportException, UnknownHostException {
+  public static TServerSocket getServerSSLSocket(String hiveHost, int portNum, String keyStorePath,
+    String keyStorePassWord) throws TTransportException, UnknownHostException {
     TSSLTransportFactory.TSSLTransportParameters params =
-        new TSSLTransportFactory.TSSLTransportParameters();
+      new TSSLTransportFactory.TSSLTransportParameters();
     params.setKeyStore(keyStorePath, keyStorePassWord);
 
     InetAddress serverAddress;
@@ -243,8 +230,7 @@ public class HiveAuthFactory {
   // retrieve delegation token for the given user
   public String getDelegationToken(String owner, String renewer) throws HiveSQLException {
     if (saslServer == null) {
-      throw new HiveSQLException(
-          "Delegation token only supported over kerberos authentication");
+      throw new HiveSQLException("Delegation token only supported over kerberos authentication");
     }
 
     try {
@@ -263,8 +249,7 @@ public class HiveAuthFactory {
   // cancel given delegation token
   public void cancelDelegationToken(String delegationToken) throws HiveSQLException {
     if (saslServer == null) {
-      throw new HiveSQLException(
-          "Delegation token only supported over kerberos authentication");
+      throw new HiveSQLException("Delegation token only supported over kerberos authentication");
     }
     try {
       saslServer.cancelDelegationToken(delegationToken);
@@ -275,8 +260,7 @@ public class HiveAuthFactory {
 
   public void renewDelegationToken(String delegationToken) throws HiveSQLException {
     if (saslServer == null) {
-      throw new HiveSQLException(
-          "Delegation token only supported over kerberos authentication");
+      throw new HiveSQLException("Delegation token only supported over kerberos authentication");
     }
     try {
       saslServer.renewDelegationToken(delegationToken);
@@ -287,21 +271,21 @@ public class HiveAuthFactory {
 
   public String getUserFromToken(String delegationToken) throws HiveSQLException {
     if (saslServer == null) {
-      throw new HiveSQLException(
-          "Delegation token only supported over kerberos authentication");
+      throw new HiveSQLException("Delegation token only supported over kerberos authentication");
     }
     try {
       return saslServer.getUserFromToken(delegationToken);
     } catch (IOException e) {
-      throw new HiveSQLException("Error extracting user from delegation token " + delegationToken, e);
+      throw new HiveSQLException("Error extracting user from delegation token " + delegationToken,
+                                 e);
     }
   }
 
   public static void verifyProxyAccess(String realUser, String proxyUser, String ipAddress,
-      HiveConf hiveConf) throws HiveSQLException {
-    UserGroupInformation sessionUgi;
+    HiveConf hiveConf) throws HiveSQLException {
 
     try {
+      UserGroupInformation sessionUgi;
       if (ShimLoader.getHadoopShims().isSecurityEnabled()) {
         sessionUgi = ShimLoader.getHadoopShims().createProxyUser(realUser);
       } else {
@@ -309,11 +293,11 @@ public class HiveAuthFactory {
       }
       if (!proxyUser.equalsIgnoreCase(realUser)) {
         ShimLoader.getHadoopShims().
-        authorizeProxyAccess(proxyUser, sessionUgi, ipAddress, hiveConf);
+          authorizeProxyAccess(proxyUser, sessionUgi, ipAddress, hiveConf);
       }
     } catch (IOException e) {
-      throw new HiveSQLException("Failed to validate proxy privilage of " + realUser +
-          " for " + proxyUser, e);
+      throw new HiveSQLException(
+        "Failed to validate proxy privilege of " + realUser + " for " + proxyUser, e);
     }
   }
 

Modified: hive/trunk/service/src/java/org/apache/hive/service/auth/HttpAuthUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/HttpAuthUtils.java?rev=1622056&r1=1622055&r2=1622056&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/HttpAuthUtils.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/HttpAuthUtils.java Tue Sep  2 16:22:08 2014
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.hive.service.auth;
 
 import java.io.IOException;
@@ -36,110 +35,95 @@ import org.apache.thrift.TProcessorFacto
 import org.apache.thrift.transport.TTransport;
 import org.ietf.jgss.GSSContext;
 import org.ietf.jgss.GSSCredential;
-import org.ietf.jgss.GSSException;
 import org.ietf.jgss.GSSManager;
 import org.ietf.jgss.GSSName;
 import org.ietf.jgss.Oid;
 
 /**
- *
- * Utility functions for http mode authentication
- *
+ * Utility functions for HTTP mode authentication.
  */
-public class HttpAuthUtils {
+public final class HttpAuthUtils {
 
   public static final String WWW_AUTHENTICATE = "WWW-Authenticate";
   public static final String AUTHORIZATION = "Authorization";
   public static final String BASIC = "Basic";
   public static final String NEGOTIATE = "Negotiate";
 
-  public static class HttpCLIServiceProcessorFactory extends TProcessorFactory {
-    private final ThriftCLIService service;
-    private final HiveConf hiveConf;
-    private final boolean isDoAsEnabled;
-
-    public HttpCLIServiceProcessorFactory(ThriftCLIService service) {
-      super(null);
-      this.service = service;
-      this.hiveConf = service.getHiveConf();
-      this.isDoAsEnabled = hiveConf.getBoolVar(
-          HiveConf.ConfVars.HIVE_SERVER2_ENABLE_DOAS);
-    }
-
-    @Override
-    public TProcessor getProcessor(TTransport trans) {
-      TProcessor baseProcessor = new TCLIService.Processor<Iface>(service);
-      return isDoAsEnabled ? new HttpCLIServiceUGIProcessor(baseProcessor) :
-        baseProcessor;
-    }
-  }
-
   public static TProcessorFactory getAuthProcFactory(ThriftCLIService service) {
     return new HttpCLIServiceProcessorFactory(service);
   }
 
   /**
-   *
    * @return Stringified Base64 encoded kerberosAuthHeader on success
-   * @throws GSSException
-   * @throws IOException
-   * @throws InterruptedException
    */
-  public static String getKerberosServiceTicket(String principal,
-      String host, String serverHttpUrl)
-          throws GSSException, IOException, InterruptedException {
+  public static String getKerberosServiceTicket(String principal, String host, String serverHttpUrl)
+    throws IOException, InterruptedException {
     UserGroupInformation clientUGI = getClientUGI("kerberos");
     String serverPrincipal = getServerPrincipal(principal, host);
     // Uses the Ticket Granting Ticket in the UserGroupInformation
-    return clientUGI.doAs(new HttpKerberosClientAction(serverPrincipal,
-        clientUGI.getShortUserName(), serverHttpUrl));
+    return clientUGI.doAs(
+      new HttpKerberosClientAction(serverPrincipal, clientUGI.getShortUserName(), serverHttpUrl));
   }
 
   /**
-   * Get server pricipal and verify that hostname is present
-   * @return
-   * @throws IOException
+   * Get server principal and verify that hostname is present.
    */
-  private static String getServerPrincipal(String principal, String host)
-      throws IOException {
-    return ShimLoader.getHadoopThriftAuthBridge().getServerPrincipal(
-        principal, host);
+  private static String getServerPrincipal(String principal, String host) throws IOException {
+    return ShimLoader.getHadoopThriftAuthBridge().getServerPrincipal(principal, host);
   }
 
   /**
    * JAAS login to setup the client UserGroupInformation.
-   * Sets up the kerberos Ticket Granting Ticket,
-   * in the client UserGroupInformation object
+   * Sets up the Kerberos Ticket Granting Ticket,
+   * in the client UserGroupInformation object.
+   *
    * @return Client's UserGroupInformation
-   * @throws IOException
    */
-  public static UserGroupInformation getClientUGI(String authType)
-      throws IOException {
+  public static UserGroupInformation getClientUGI(String authType) throws IOException {
     return ShimLoader.getHadoopThriftAuthBridge().getCurrentUGIWithConf(authType);
   }
 
-  /**
-   *
-   * HttpKerberosClientAction
-   *
-   */
-  public static class HttpKerberosClientAction implements
-  PrivilegedExceptionAction<String> {
-    String serverPrincipal;
-    String clientUserName;
-    String serverHttpUrl;
-    private final Base64 base64codec;
+  private HttpAuthUtils() {
+    throw new UnsupportedOperationException("Can't initialize class");
+  }
+
+  public static class HttpCLIServiceProcessorFactory extends TProcessorFactory {
+
+    private final ThriftCLIService service;
+    private final HiveConf hiveConf;
+    private final boolean isDoAsEnabled;
+
+    public HttpCLIServiceProcessorFactory(ThriftCLIService service) {
+      super(null);
+      this.service = service;
+      hiveConf = service.getHiveConf();
+      isDoAsEnabled = hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_ENABLE_DOAS);
+    }
+
+    @Override
+    public TProcessor getProcessor(TTransport trans) {
+      TProcessor baseProcessor = new TCLIService.Processor<Iface>(service);
+      return isDoAsEnabled ? new HttpCLIServiceUGIProcessor(baseProcessor) : baseProcessor;
+    }
+  }
+
+  public static class HttpKerberosClientAction implements PrivilegedExceptionAction<String> {
+
     public static final String HTTP_RESPONSE = "HTTP_RESPONSE";
     public static final String SERVER_HTTP_URL = "SERVER_HTTP_URL";
+    private final String serverPrincipal;
+    private final String clientUserName;
+    private final String serverHttpUrl;
+    private final Base64 base64codec;
     private final HttpContext httpContext;
 
-    public HttpKerberosClientAction(String serverPrincipal,
-        String clientUserName, String serverHttpUrl) {
+    public HttpKerberosClientAction(String serverPrincipal, String clientUserName,
+      String serverHttpUrl) {
       this.serverPrincipal = serverPrincipal;
       this.clientUserName = clientUserName;
       this.serverHttpUrl = serverHttpUrl;
-      this.base64codec = new Base64(0);
-      this.httpContext = new BasicHttpContext();
+      base64codec = new Base64(0);
+      httpContext = new BasicHttpContext();
       httpContext.setAttribute(SERVER_HTTP_URL, serverHttpUrl);
     }
 
@@ -158,8 +142,8 @@ public class HttpAuthUtils {
       GSSName serverName = manager.createName(serverPrincipal, krb5PrincipalOid);
 
       // GSS credentials for client
-      GSSCredential clientCreds = manager.createCredential(clientName,
-          GSSCredential.DEFAULT_LIFETIME, mechOid,
+      GSSCredential clientCreds =
+        manager.createCredential(clientName, GSSCredential.DEFAULT_LIFETIME, mechOid,
           GSSCredential.INITIATE_ONLY);
 
       /*
@@ -170,22 +154,20 @@ public class HttpAuthUtils {
        *      use. The client chooses the mechanism to use.
        *    - clientCreds are the client credentials
        */
-      GSSContext gssContext = manager.createContext(serverName,
-          mechOid, clientCreds, GSSContext.DEFAULT_LIFETIME);
+      GSSContext gssContext =
+        manager.createContext(serverName, mechOid, clientCreds, GSSContext.DEFAULT_LIFETIME);
 
       // Mutual authentication not r
       gssContext.requestMutualAuth(false);
 
-      // Estabilish context
+      // Establish context
       byte[] inToken = new byte[0];
-      byte[] outToken;
 
-      outToken = gssContext.initSecContext(inToken, 0, inToken.length);
+      byte[] outToken = gssContext.initSecContext(inToken, 0, inToken.length);
 
       gssContext.dispose();
       // Base64 encoded and stringified token for server
-      String authHeaderBase64String = new String(base64codec.encode(outToken));
-      return authHeaderBase64String;
+      return new String(base64codec.encode(outToken));
     }
   }
 }

Modified: hive/trunk/service/src/java/org/apache/hive/service/auth/HttpAuthenticationException.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/HttpAuthenticationException.java?rev=1622056&r1=1622055&r2=1622056&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/HttpAuthenticationException.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/HttpAuthenticationException.java Tue Sep  2 16:22:08 2014
@@ -14,26 +14,27 @@
 
 package org.apache.hive.service.auth;
 
-public class HttpAuthenticationException extends Exception{
-  static final long serialVersionUID = 0;
+public class HttpAuthenticationException extends Exception {
+
+  private static final long serialVersionUID = 0;
 
   /**
-   * @param cause original exception.
+   * @param cause original exception
    */
   public HttpAuthenticationException(Throwable cause) {
     super(cause);
   }
 
   /**
-   * @param msg exception message.
+   * @param msg exception message
    */
   public HttpAuthenticationException(String msg) {
     super(msg);
   }
 
   /**
-   * @param msg exception message.
-   * @param cause original exception.
+   * @param msg   exception message
+   * @param cause original exception
    */
   public HttpAuthenticationException(String msg, Throwable cause) {
     super(msg, cause);

Modified: hive/trunk/service/src/java/org/apache/hive/service/auth/HttpCLIServiceUGIProcessor.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/HttpCLIServiceUGIProcessor.java?rev=1622056&r1=1622055&r2=1622056&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/HttpCLIServiceUGIProcessor.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/HttpCLIServiceUGIProcessor.java Tue Sep  2 16:22:08 2014
@@ -31,14 +31,11 @@ import org.apache.thrift.TProcessor;
 import org.apache.thrift.protocol.TProtocol;
 
 /**
- *
- * Wraps the underlying thrift processor's process call,
+ * Wraps the underlying Thrift processor's process call,
  * to assume the client user's UGI/Subject for the doAs calls.
- * Gets the client's username from a threadlocal in SessionManager which is
+ * Gets the client's username from a ThreadLocal in SessionManager which is
  * set in the ThriftHttpServlet, and constructs a client UGI object from that.
- *
  */
-
 public class HttpCLIServiceUGIProcessor implements TProcessor {
 
   private final TProcessor underlyingProcessor;
@@ -46,18 +43,18 @@ public class HttpCLIServiceUGIProcessor 
 
   public HttpCLIServiceUGIProcessor(TProcessor underlyingProcessor) {
     this.underlyingProcessor = underlyingProcessor;
-    this.shim = ShimLoader.getHadoopShims();
+    shim = ShimLoader.getHadoopShims();
   }
 
   @Override
   public boolean process(final TProtocol in, final TProtocol out) throws TException {
-    /**
-     * Build the client UGI from threadlocal username [SessionManager.getUserName()].
-     * The threadlocal username is set in the ThriftHttpServlet.
+    /*
+     * Build the client UGI from ThreadLocal username [SessionManager.getUserName()].
+     * The ThreadLocal username is set in the ThriftHttpServlet.
      */
-    UserGroupInformation clientUgi = null;
     try {
-      clientUgi = shim.createRemoteUser(SessionManager.getUserName(), new ArrayList<String>());
+      UserGroupInformation clientUgi =
+        shim.createRemoteUser(SessionManager.getUserName(), new ArrayList<String>());
       return shim.doAs(clientUgi, new PrivilegedExceptionAction<Boolean>() {
         @Override
         public Boolean run() {
@@ -68,10 +65,9 @@ public class HttpCLIServiceUGIProcessor 
           }
         }
       });
-    }
-    catch (RuntimeException rte) {
+    } catch (RuntimeException rte) {
       if (rte.getCause() instanceof TException) {
-        throw (TException)rte.getCause();
+        throw (TException) rte.getCause();
       }
       throw rte;
     } catch (InterruptedException ie) {

Modified: 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=1622056&r1=1622055&r2=1622056&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/KerberosSaslHelper.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/KerberosSaslHelper.java Tue Sep  2 16:22:08 2014
@@ -19,7 +19,6 @@ package org.apache.hive.service.auth;
 
 import java.io.IOException;
 import java.util.Map;
-
 import javax.security.sasl.SaslException;
 
 import org.apache.hadoop.hive.shims.ShimLoader;
@@ -33,37 +32,20 @@ import org.apache.thrift.TProcessorFacto
 import org.apache.thrift.transport.TSaslClientTransport;
 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 final class KerberosSaslHelper {
 
   public static TProcessorFactory getKerberosProcessorFactory(Server saslServer,
-      ThriftCLIService service) {
-    return new CLIServiceProcessorFactory (saslServer, service);
+    ThriftCLIService service) {
+    return new CLIServiceProcessorFactory(saslServer, service);
   }
 
   public static TTransport getKerberosTransport(String principal, String host,
-      final TTransport underlyingTransport, Map<String, String> saslProps, boolean assumeSubject) throws SaslException {
+    TTransport underlyingTransport, Map<String, String> saslProps, boolean assumeSubject)
+    throws SaslException {
     try {
-      final String names[] = principal.split("[/@]");
+      String[] names = principal.split("[/@]");
       if (names.length != 3) {
-        throw new IllegalArgumentException("Kerberos principal should have 3 parts: "
-            + principal);
+        throw new IllegalArgumentException("Kerberos principal should have 3 parts: " + principal);
       }
 
       if (assumeSubject) {
@@ -71,20 +53,21 @@ public class KerberosSaslHelper {
       } else {
         HadoopThriftAuthBridge.Client authBridge =
           ShimLoader.getHadoopThriftAuthBridge().createClientWithConf("kerberos");
-        return authBridge.createClientTransport(principal, host,
-          "KERBEROS", null, underlyingTransport, saslProps);
+        return authBridge.createClientTransport(principal, host, "KERBEROS", null,
+                                                underlyingTransport, saslProps);
       }
     } catch (IOException e) {
       throw new SaslException("Failed to open client transport", e);
     }
   }
 
-  public static TTransport createSubjectAssumedTransport(String principal, 
-		  TTransport underlyingTransport, Map<String, String> saslProps) throws IOException {
-    TTransport saslTransport = null;
-    final String names[] = principal.split("[/@]");
+  public static TTransport createSubjectAssumedTransport(String principal,
+    TTransport underlyingTransport, Map<String, String> saslProps) throws IOException {
+    String[] names = principal.split("[/@]");
     try {
-      saslTransport = new TSaslClientTransport("GSSAPI", null, names[0], names[1], saslProps, null, underlyingTransport);
+      TTransport saslTransport =
+        new TSaslClientTransport("GSSAPI", null, names[0], names[1], saslProps, null,
+          underlyingTransport);
       return new TSubjectAssumingTransport(saslTransport);
     } catch (SaslException se) {
       throw new IOException("Could not instantiate SASL transport", se);
@@ -92,15 +75,37 @@ public class KerberosSaslHelper {
   }
 
   public static TTransport getTokenTransport(String tokenStr, String host,
-      final TTransport underlyingTransport, Map<String, String> saslProps) throws SaslException {
+    TTransport underlyingTransport, Map<String, String> saslProps) throws SaslException {
     HadoopThriftAuthBridge.Client authBridge =
       ShimLoader.getHadoopThriftAuthBridge().createClientWithConf("kerberos");
 
     try {
-      return authBridge.createClientTransport(null, host,
-          "DIGEST", tokenStr, underlyingTransport, saslProps);
+      return authBridge.createClientTransport(null, host, "DIGEST", tokenStr, underlyingTransport,
+                                              saslProps);
     } catch (IOException e) {
       throw new SaslException("Failed to open client transport", e);
     }
   }
+
+  private KerberosSaslHelper() {
+    throw new UnsupportedOperationException("Can't initialize class");
+  }
+
+  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);
+    }
+  }
 }

Modified: 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=1622056&r1=1622055&r2=1622056&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java Tue Sep  2 16:22:08 2014
@@ -18,10 +18,8 @@
 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;
 
@@ -33,16 +31,15 @@ public class LdapAuthenticationProviderI
   private final String baseDN;
   private final String ldapDomain;
 
-  LdapAuthenticationProviderImpl () {
+  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);
-    this.ldapDomain = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_DOMAIN);
+    ldapURL = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_URL);
+    baseDN = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_BASEDN);
+    ldapDomain = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_DOMAIN);
   }
 
   @Override
-  public void Authenticate(String user, String  password)
-      throws AuthenticationException {
+  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");
@@ -51,15 +48,15 @@ public class LdapAuthenticationProviderI
     //  If the domain is supplied, then append it. LDAP providers like Active Directory
     // use a fully qualified user name like foo@bar.com.
     if (ldapDomain != null) {
-      user  = user + "@" + ldapDomain;
+      user = user + "@" + ldapDomain;
     }
 
     // setup the security principal
     String bindDN;
-    if (baseDN != null) {
-      bindDN = "uid=" + user + "," + baseDN;
-    } else {
+    if (baseDN == null) {
       bindDN = user;
+    } else {
+      bindDN = "uid=" + user + "," + baseDN;
     }
     env.put(Context.SECURITY_AUTHENTICATION, "simple");
     env.put(Context.SECURITY_PRINCIPAL, bindDN);
@@ -67,12 +64,11 @@ public class LdapAuthenticationProviderI
 
     try {
       // Create initial context
-      DirContext ctx = new InitialDirContext(env);
+      Context ctx = new InitialDirContext(env);
       ctx.close();
     } catch (NamingException e) {
       throw new AuthenticationException("Error validating LDAP user", e);
     }
-  return;
   }
 
 }

Modified: hive/trunk/service/src/java/org/apache/hive/service/auth/PamAuthenticationProviderImpl.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/PamAuthenticationProviderImpl.java?rev=1622056&r1=1622055&r2=1622056&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/PamAuthenticationProviderImpl.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/PamAuthenticationProviderImpl.java Tue Sep  2 16:22:08 2014
@@ -20,33 +20,32 @@ package org.apache.hive.service.auth;
 import javax.security.sasl.AuthenticationException;
 
 import net.sf.jpam.Pam;
-
 import org.apache.hadoop.hive.conf.HiveConf;
 
 public class PamAuthenticationProviderImpl implements PasswdAuthenticationProvider {
 
   private final String pamServiceNames;
 
-  PamAuthenticationProviderImpl () {
+  PamAuthenticationProviderImpl() {
     HiveConf conf = new HiveConf();
-    this.pamServiceNames = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PAM_SERVICES);
+    pamServiceNames = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PAM_SERVICES);
   }
 
   @Override
-  public void Authenticate(String user, String  password)
-      throws AuthenticationException {
+  public void Authenticate(String user, String password) throws AuthenticationException {
 
     if (pamServiceNames == null || pamServiceNames.trim().isEmpty()) {
       throw new AuthenticationException("No PAM services are set.");
     }
 
-    String pamServices[] = pamServiceNames.split(",");
+    String[] pamServices = pamServiceNames.split(",");
     for (String pamService : pamServices) {
       Pam pam = new Pam(pamService);
       boolean isAuthenticated = pam.authenticateSuccessful(user, password);
       if (!isAuthenticated) {
-        throw new AuthenticationException("Error authenticating with the PAM service: " + pamService);
+        throw new AuthenticationException(
+          "Error authenticating with the PAM service: " + pamService);
       }
     }
   }
-}
\ No newline at end of file
+}

Modified: 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=1622056&r1=1622055&r2=1622056&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/PasswdAuthenticationProvider.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/PasswdAuthenticationProvider.java Tue Sep  2 16:22:08 2014
@@ -20,18 +20,20 @@ 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}.
-   *
+   * <p/>
    * 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
+   * @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;
 }

Modified: 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=1622056&r1=1622055&r2=1622056&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/PlainSaslHelper.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/PlainSaslHelper.java Tue Sep  2 16:22:08 2014
@@ -18,8 +18,8 @@
 package org.apache.hive.service.auth;
 
 import java.io.IOException;
+import java.security.Security;
 import java.util.HashMap;
-
 import javax.security.auth.callback.Callback;
 import javax.security.auth.callback.CallbackHandler;
 import javax.security.auth.callback.NameCallback;
@@ -30,10 +30,8 @@ import javax.security.sasl.Authenticatio
 import javax.security.sasl.AuthorizeCallback;
 import javax.security.sasl.SaslException;
 
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hive.service.auth.PlainSaslServer.SaslPlainProvider;
 import org.apache.hive.service.auth.AuthenticationProviderFactory.AuthMethods;
-import org.apache.hive.service.cli.thrift.TCLIService;
+import org.apache.hive.service.auth.PlainSaslServer.SaslPlainProvider;
 import org.apache.hive.service.cli.thrift.TCLIService.Iface;
 import org.apache.hive.service.cli.thrift.ThriftCLIService;
 import org.apache.thrift.TProcessor;
@@ -43,78 +41,108 @@ import org.apache.thrift.transport.TSasl
 import org.apache.thrift.transport.TTransport;
 import org.apache.thrift.transport.TTransportFactory;
 
-public class PlainSaslHelper {
+public final class PlainSaslHelper {
+
+  public static TProcessorFactory getPlainProcessorFactory(ThriftCLIService service) {
+    return new SQLPlainProcessorFactory(service);
+  }
+
+  // Register Plain SASL server provider
+  static {
+    Security.addProvider(new SaslPlainProvider());
+  }
+
+  public static TTransportFactory getPlainTransportFactory(String authTypeStr)
+    throws LoginException {
+    TSaslServerTransport.Factory saslFactory = new TSaslServerTransport.Factory();
+    try {
+      saslFactory.addServerDefinition("PLAIN", authTypeStr, null, new HashMap<String, String>(),
+        new PlainServerCallbackHandler(authTypeStr));
+    } catch (AuthenticationException e) {
+      throw new LoginException("Error setting callback handler" + e);
+    }
+    return saslFactory;
+  }
+
+  public static TTransport getPlainTransport(String username, String password,
+    TTransport underlyingTransport) throws SaslException {
+    return new TSaslClientTransport("PLAIN", null, null, null, new HashMap<String, String>(),
+      new PlainCallbackHandler(username, password), underlyingTransport);
+  }
+
+  private PlainSaslHelper() {
+    throw new UnsupportedOperationException("Can't initialize class");
+  }
+
+  private static final class PlainServerCallbackHandler implements CallbackHandler {
 
-  private static class PlainServerCallbackHandler implements CallbackHandler {
     private final AuthMethods authMethod;
-    public PlainServerCallbackHandler(String authMethodStr) throws AuthenticationException {
+
+    PlainServerCallbackHandler(String authMethodStr) throws AuthenticationException {
       authMethod = AuthMethods.getValidAuthMethod(authMethodStr);
     }
 
     @Override
     public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
-      String userName = null;
-      String passWord = null;
+      String username = null;
+      String password = null;
       AuthorizeCallback ac = null;
 
-      for (int i = 0; i < callbacks.length; i++) {
-        if (callbacks[i] instanceof NameCallback) {
-          NameCallback nc = (NameCallback)callbacks[i];
-          userName = nc.getName();
-        } else if (callbacks[i] instanceof PasswordCallback) {
-          PasswordCallback pc = (PasswordCallback)callbacks[i];
-          passWord = new String(pc.getPassword());
-        } else if (callbacks[i] instanceof AuthorizeCallback) {
-          ac = (AuthorizeCallback) callbacks[i];
+      for (Callback callback : callbacks) {
+        if (callback instanceof NameCallback) {
+          NameCallback nc = (NameCallback) callback;
+          username = nc.getName();
+        } else if (callback instanceof PasswordCallback) {
+          PasswordCallback pc = (PasswordCallback) callback;
+          password = new String(pc.getPassword());
+        } else if (callback instanceof AuthorizeCallback) {
+          ac = (AuthorizeCallback) callback;
         } else {
-          throw new UnsupportedCallbackException(callbacks[i]);
+          throw new UnsupportedCallbackException(callback);
         }
       }
       PasswdAuthenticationProvider provider =
-            AuthenticationProviderFactory.getAuthenticationProvider(authMethod);
-      provider.Authenticate(userName, passWord);
+        AuthenticationProviderFactory.getAuthenticationProvider(authMethod);
+      provider.Authenticate(username, password);
       if (ac != null) {
         ac.setAuthorized(true);
       }
     }
   }
 
-  public static class PlainClientbackHandler implements CallbackHandler {
+  public static class PlainCallbackHandler implements CallbackHandler {
 
-    private final String userName;
-    private final String passWord;
+    private final String username;
+    private final String password;
 
-    public PlainClientbackHandler (String userName, String passWord) {
-      this.userName = userName;
-      this.passWord = passWord;
+    public PlainCallbackHandler(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());
+    public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+      for (Callback callback : callbacks) {
+        if (callback instanceof NameCallback) {
+          NameCallback nameCallback = (NameCallback) callback;
+          nameCallback.setName(username);
+        } else if (callback instanceof PasswordCallback) {
+          PasswordCallback passCallback = (PasswordCallback) callback;
+          passCallback.setPassword(password.toCharArray());
         } else {
-          throw new UnsupportedCallbackException(callbacks[i]);
+          throw new UnsupportedCallbackException(callback);
         }
       }
     }
   }
 
-  private static class SQLPlainProcessorFactory extends TProcessorFactory {
+  private static final class SQLPlainProcessorFactory extends TProcessorFactory {
+
     private final ThriftCLIService service;
-    private final HiveConf conf;
 
-    public SQLPlainProcessorFactory(ThriftCLIService service) {
+    SQLPlainProcessorFactory(ThriftCLIService service) {
       super(null);
       this.service = service;
-      this.conf = service.getHiveConf();
     }
 
     @Override
@@ -123,33 +151,4 @@ public class PlainSaslHelper {
     }
   }
 
-  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)
-      throws LoginException {
-    TSaslServerTransport.Factory saslFactory = new TSaslServerTransport.Factory();
-    try {
-      saslFactory.addServerDefinition("PLAIN",
-          authTypeStr, null, new HashMap<String, String>(),
-          new PlainServerCallbackHandler(authTypeStr));
-    } catch (AuthenticationException e) {
-      throw new LoginException ("Error setting callback handler" + e);
-    }
-    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);
-  }
-
 }

Modified: 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=1622056&r1=1622055&r2=1622056&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/PlainSaslServer.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/PlainSaslServer.java Tue Sep  2 16:22:08 2014
@@ -18,10 +18,10 @@
 package org.apache.hive.service.auth;
 
 import java.io.IOException;
+import java.security.Provider;
 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.NameCallback;
@@ -35,27 +35,26 @@ import javax.security.sasl.SaslServerFac
 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)
+ * Sun JDK only provides a PLAIN client and no 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;
+public class PlainSaslServer implements SaslServer {
+
+  public static final String PLAIN_METHOD = "PLAIN";
   private String user;
-  private String passwd;
-  private String authzId;
   private final CallbackHandler handler;
 
   PlainSaslServer(CallbackHandler handler, String authMethodStr) throws SaslException {
     this.handler = handler;
-    this.authMethod = AuthMethods.getValidAuthMethod(authMethodStr);
+    AuthMethods.getValidAuthMethod(authMethodStr);
   }
 
+  @Override
   public String getMechanismName() {
-    return "PLAIN";
+    return PLAIN_METHOD;
   }
 
+  @Override
   public byte[] evaluateResponse(byte[] response) throws SaslException {
     try {
       // parse the response
@@ -68,28 +67,29 @@ public class PlainSaslServer implements 
           tokenList.addLast(messageToken.toString());
           messageToken = new StringBuilder();
         } else {
-          messageToken.append((char)b);
+          messageToken.append((char) b);
         }
       }
       tokenList.addLast(messageToken.toString());
 
       // validate response
-      if ((tokenList.size() < 2) || (tokenList.size() > 3)) {
+      if (tokenList.size() < 2 || tokenList.size() > 3) {
         throw new SaslException("Invalid message format");
       }
-      passwd = tokenList.removeLast();
+      String passwd = tokenList.removeLast();
       user = tokenList.removeLast();
       // optional authzid
-      if (!tokenList.isEmpty()) {
-        authzId = tokenList.removeLast();
-      } else {
+      String authzId;
+      if (tokenList.isEmpty()) {
         authzId = user;
+      } else {
+        authzId = tokenList.removeLast();
       }
       if (user == null || user.isEmpty()) {
-        throw new SaslException("No user name provide");
+        throw new SaslException("No user name provided");
       }
       if (passwd == null || passwd.isEmpty()) {
-        throw new SaslException("No password name provide");
+        throw new SaslException("No password name provided");
       }
 
       NameCallback nameCallback = new NameCallback("User");
@@ -98,7 +98,7 @@ public class PlainSaslServer implements 
       pcCallback.setPassword(passwd.toCharArray());
       AuthorizeCallback acCallback = new AuthorizeCallback(user, authzId);
 
-      Callback[] cbList = new Callback[] {nameCallback, pcCallback, acCallback};
+      Callback[] cbList = {nameCallback, pcCallback, acCallback};
       handler.handle(cbList);
       if (!acCallback.isAuthorized()) {
         throw new SaslException("Authentication failed");
@@ -113,49 +113,62 @@ public class PlainSaslServer implements 
     return null;
   }
 
+  @Override
   public boolean isComplete() {
     return user != null;
   }
 
+  @Override
   public String getAuthorizationID() {
     return user;
   }
 
+  @Override
   public byte[] unwrap(byte[] incoming, int offset, int len) {
-      throw new UnsupportedOperationException();
+    throw new UnsupportedOperationException();
   }
 
+  @Override
   public byte[] wrap(byte[] outgoing, int offset, int len) {
     throw new UnsupportedOperationException();
   }
 
+  @Override
   public Object getNegotiatedProperty(String propName) {
     return null;
   }
 
+  @Override
   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)) {
+    @Override
+    public SaslServer createSaslServer(String mechanism, String protocol, String serverName,
+      Map<String, ?> props, CallbackHandler cbh) {
+      if (PLAIN_METHOD.equals(mechanism)) {
         try {
           return new PlainSaslServer(cbh, protocol);
         } catch (SaslException e) {
+          /* This is to fulfill the contract of the interface which states that an exception shall
+             be thrown when a SaslServer cannot be created due to an error but null should be
+             returned when a Server can't be created due to the parameters supplied. And the only
+             thing PlainSaslServer can fail on is a non-supported authentication mechanism.
+             That's why we return null instead of throwing the Exception */
           return null;
         }
       }
       return null;
     }
 
+    @Override
     public String[] getMechanismNames(Map<String, ?> props) {
-      return new String[] { "PLAIN" };
+      return new String[] {PLAIN_METHOD};
     }
   }
 
-  public static class SaslPlainProvider extends java.security.Provider {
+  public static class SaslPlainProvider extends Provider {
+
     public SaslPlainProvider() {
       super("HiveSaslPlain", 1.0, "Hive Plain SASL provider");
       put("SaslServerFactory.PLAIN", SaslPlainServerFactory.class.getName());

Modified: hive/trunk/service/src/java/org/apache/hive/service/auth/SaslQOP.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/SaslQOP.java?rev=1622056&r1=1622055&r2=1622056&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/SaslQOP.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/SaslQOP.java Tue Sep  2 16:22:08 2014
@@ -22,7 +22,7 @@ import java.util.HashMap;
 import java.util.Map;
 
 /**
- * Possible values of  SASL quality-of-protection value.
+ * Possible values of SASL quality-of-protection value.
  */
 public enum SaslQOP {
   AUTH("auth"), // Authentication only.
@@ -32,14 +32,15 @@ public enum SaslQOP {
 
   public final String saslQop;
 
-  private static final Map<String, SaslQOP> strToEnum
-          = new HashMap<String, SaslQOP>();
+  private static final Map<String, SaslQOP> STR_TO_ENUM = new HashMap<String, SaslQOP>();
+
   static {
-    for (SaslQOP SaslQOP : values())
-      strToEnum.put(SaslQOP.toString(), SaslQOP);
+    for (SaslQOP saslQop : values()) {
+      STR_TO_ENUM.put(saslQop.toString(), saslQop);
+    }
   }
 
-  private SaslQOP(final String saslQop) {
+  SaslQOP(String saslQop) {
     this.saslQop = saslQop;
   }
 
@@ -48,13 +49,13 @@ public enum SaslQOP {
   }
 
   public static SaslQOP fromString(String str) {
-    if(str != null) {
+    if (str != null) {
       str = str.toLowerCase();
     }
-    SaslQOP saslQOP = strToEnum.get(str);
-    if(saslQOP == null) {
-      throw new IllegalArgumentException("Unknown auth type: " + str + " Allowed values are: "
-              + strToEnum.keySet());
+    SaslQOP saslQOP = STR_TO_ENUM.get(str);
+    if (saslQOP == null) {
+      throw new IllegalArgumentException(
+        "Unknown auth type: " + str + " Allowed values are: " + STR_TO_ENUM.keySet());
     }
     return saslQOP;
   }

Modified: 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=1622056&r1=1622055&r2=1622056&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java Tue Sep  2 16:22:08 2014
@@ -34,10 +34,12 @@ 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>
+ * <li>IP address is only set for operations that calls listeners with hookContext</li>
+ * <li>IP address is only set if the underlying transport mechanism is socket</li>
  * </ul>
  * </p>
+ *
+ * @see org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext
  */
 public class TSetIpAddressProcessor<I extends Iface> extends TCLIService.Processor<Iface> {
 
@@ -54,26 +56,26 @@ public class TSetIpAddressProcessor<I ex
     try {
       return super.process(in, out);
     } finally {
-      threadLocalUserName.remove();
-      threadLocalIpAddress.remove();
+      THREAD_LOCAL_USER_NAME.remove();
+      THREAD_LOCAL_IP_ADDRESS.remove();
     }
   }
 
   private void setUserName(final TProtocol in) {
     TTransport transport = in.getTransport();
     if (transport instanceof TSaslServerTransport) {
-      String userName = ((TSaslServerTransport)transport).getSaslServer().getAuthorizationID();
-      threadLocalUserName.set(userName);
+      String userName = ((TSaslServerTransport) transport).getSaslServer().getAuthorizationID();
+      THREAD_LOCAL_USER_NAME.set(userName);
     }
   }
 
   protected void setIpAddress(final TProtocol in) {
     TTransport transport = in.getTransport();
     TSocket tSocket = getUnderlyingSocketFromTransport(transport);
-    if (tSocket != null) {
-      threadLocalIpAddress.set(tSocket.getSocket().getInetAddress().toString());
-    } else {
+    if (tSocket == null) {
       LOGGER.warn("Unknown Transport, cannot determine ipAddress");
+    } else {
+      THREAD_LOCAL_IP_ADDRESS.set(tSocket.getSocket().getInetAddress().toString());
     }
   }
 
@@ -92,14 +94,14 @@ public class TSetIpAddressProcessor<I ex
     return null;
   }
 
-  private static ThreadLocal<String> threadLocalIpAddress = new ThreadLocal<String>() {
+  private static final ThreadLocal<String> THREAD_LOCAL_IP_ADDRESS = new ThreadLocal<String>() {
     @Override
     protected synchronized String initialValue() {
       return null;
     }
   };
 
-  private static ThreadLocal<String> threadLocalUserName = new ThreadLocal<String>(){
+  private static final ThreadLocal<String> THREAD_LOCAL_USER_NAME = new ThreadLocal<String>() {
     @Override
     protected synchronized String initialValue() {
       return null;
@@ -107,10 +109,10 @@ public class TSetIpAddressProcessor<I ex
   };
 
   public static String getUserIpAddress() {
-    return threadLocalIpAddress.get();
+    return THREAD_LOCAL_IP_ADDRESS.get();
   }
 
   public static String getUserName() {
-    return threadLocalUserName.get();
+    return THREAD_LOCAL_USER_NAME.get();
   }
-}
\ No newline at end of file
+}

Modified: hive/trunk/service/src/java/org/apache/hive/service/auth/TSubjectAssumingTransport.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/TSubjectAssumingTransport.java?rev=1622056&r1=1622055&r2=1622056&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/TSubjectAssumingTransport.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/TSubjectAssumingTransport.java Tue Sep  2 16:22:08 2014
@@ -22,7 +22,6 @@ import java.security.AccessControlContex
 import java.security.AccessController;
 import java.security.PrivilegedActionException;
 import java.security.PrivilegedExceptionAction;
-
 import javax.security.auth.Subject;
 
 import org.apache.hadoop.hive.thrift.TFilterTransport;
@@ -30,43 +29,42 @@ import org.apache.thrift.transport.TTran
 import org.apache.thrift.transport.TTransportException;
 
 /**
-  *
-  * This is used on the client side, where the API explicitly opens a transport to
-  * the server using the Subject.doAs()
-  */
- public class TSubjectAssumingTransport extends TFilterTransport {
-
-   public TSubjectAssumingTransport(TTransport wrapped) {
-     super(wrapped);
-   }
-
-   @Override
-   public void open() throws TTransportException {
-     try {
-       AccessControlContext context = AccessController.getContext();
-       Subject subject = Subject.getSubject(context);
-       Subject.doAs(subject, new PrivilegedExceptionAction<Void>() {
-         public Void run() {
-           try {
-             wrapped.open();
-           } catch (TTransportException tte) {
-             // Wrap the transport exception in an RTE, since Subject.doAs() then goes
-             // and unwraps this for us out of the doAs block. We then unwrap one
-             // more time in our catch clause to get back the TTE. (ugh)
-             throw new RuntimeException(tte);
-           }
-           return null;
-         }
-       });
-     } catch (PrivilegedActionException ioe) {
-       throw new RuntimeException("Received an ioe we never threw!", ioe);
-     } catch (RuntimeException rte) {
-       if (rte.getCause() instanceof TTransportException) {
-         throw (TTransportException)rte.getCause();
-       } else {
-         throw rte;
-       }
-     }
-   }
+ * This is used on the client side, where the API explicitly opens a transport to
+ * the server using the Subject.doAs().
+ */
+public class TSubjectAssumingTransport extends TFilterTransport {
+
+  public TSubjectAssumingTransport(TTransport wrapped) {
+    super(wrapped);
+  }
+
+  @Override
+  public void open() throws TTransportException {
+    try {
+      AccessControlContext context = AccessController.getContext();
+      Subject subject = Subject.getSubject(context);
+      Subject.doAs(subject, new PrivilegedExceptionAction<Void>() {
+        public Void run() {
+          try {
+            wrapped.open();
+          } catch (TTransportException tte) {
+            // Wrap the transport exception in an RTE, since Subject.doAs() then goes
+            // and unwraps this for us out of the doAs block. We then unwrap one
+            // more time in our catch clause to get back the TTE. (ugh)
+            throw new RuntimeException(tte);
+          }
+          return null;
+        }
+      });
+    } catch (PrivilegedActionException ioe) {
+      throw new RuntimeException("Received an ioe we never threw!", ioe);
+    } catch (RuntimeException rte) {
+      if (rte.getCause() instanceof TTransportException) {
+        throw (TTransportException) rte.getCause();
+      } else {
+        throw rte;
+      }
+    }
+  }
 
- }
+}