You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by th...@apache.org on 2016/04/15 02:43:35 UTC

hive git commit: HIVE-13418 : HiveServer2 HTTP mode should support X-Forwarded-Host header for authorization/audits (Thejas Nair, reviewed by Vaibhav Gumashta)

Repository: hive
Updated Branches:
  refs/heads/master cc2d0f036 -> 833a7d158


HIVE-13418 : HiveServer2 HTTP mode should support X-Forwarded-Host header for authorization/audits (Thejas Nair, reviewed by Vaibhav Gumashta)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/833a7d15
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/833a7d15
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/833a7d15

Branch: refs/heads/master
Commit: 833a7d158b3a8e45f492e7c82640c1a367d79b30
Parents: cc2d0f0
Author: Thejas Nair <th...@hortonworks.com>
Authored: Thu Apr 14 17:43:18 2016 -0700
Committer: Thejas Nair <th...@hortonworks.com>
Committed: Thu Apr 14 17:43:18 2016 -0700

----------------------------------------------------------------------
 .../cli/thrift/TestThriftBinaryCLIService.java  |  92 -------
 .../cli/thrift/TestThriftHttpCLIService.java    | 241 -------------------
 .../java/org/apache/hadoop/hive/ql/Driver.java  |   1 +
 .../hadoop/hive/ql/processors/CommandUtil.java  |   1 +
 .../AuthorizationMetaStoreFilterHook.java       |   1 +
 .../authorization/plugin/QueryContext.java      |  19 +-
 .../hadoop/hive/ql/session/SessionState.java    |  10 +
 .../cli/operation/MetadataOperation.java        |   1 +
 .../service/cli/session/HiveSessionImpl.java    |   2 +
 .../service/cli/session/SessionManager.java     |  21 +-
 .../service/cli/thrift/ThriftHttpServlet.java   |  17 ++
 .../cli/thrift/ThriftCLIServiceTest.java        |   4 +-
 12 files changed, 68 insertions(+), 342 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/833a7d15/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftBinaryCLIService.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftBinaryCLIService.java b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftBinaryCLIService.java
deleted file mode 100644
index de31699..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftBinaryCLIService.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/**
- * 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.thrift;
-
-import static org.junit.Assert.assertNotNull;
-
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hive.service.auth.HiveAuthFactory.AuthTypes;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-
-
-/**
- *
- * TestThriftBinaryCLIService.
- * This tests ThriftCLIService started in binary mode.
- *
- */
-
-public class TestThriftBinaryCLIService extends ThriftCLIServiceTest {
-
-  private static String transportMode = "binary";
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    // Set up the base class
-    ThriftCLIServiceTest.setUpBeforeClass();
-
-    assertNotNull(port);
-    assertNotNull(hiveServer2);
-    assertNotNull(hiveConf);
-
-    hiveConf.setBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS, false);
-    hiveConf.setVar(ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST, host);
-    hiveConf.setIntVar(ConfVars.HIVE_SERVER2_THRIFT_PORT, port);
-    hiveConf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, AuthTypes.NONE.toString());
-    hiveConf.setVar(ConfVars.HIVE_SERVER2_TRANSPORT_MODE, transportMode);
-
-    startHiveServer2WithConf(hiveConf);
-
-    client = getServiceClientInternal();
-  }
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    ThriftCLIServiceTest.tearDownAfterClass();
-  }
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @Override
-  @Before
-  public void setUp() throws Exception {
-
-  }
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @Override
-  @After
-  public void tearDown() throws Exception {
-
-  }
-
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/833a7d15/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftHttpCLIService.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftHttpCLIService.java b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftHttpCLIService.java
deleted file mode 100644
index 3ed6dd8..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftHttpCLIService.java
+++ /dev/null
@@ -1,241 +0,0 @@
-/**
- * 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.thrift;
-
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hive.jdbc.HttpBasicAuthInterceptor;
-import org.apache.hive.service.auth.HiveAuthFactory;
-import org.apache.hive.service.auth.HiveAuthFactory.AuthTypes;
-import org.apache.hive.service.rpc.thrift.TCLIService;
-import org.apache.hive.service.rpc.thrift.TOpenSessionReq;
-import org.apache.http.HttpException;
-import org.apache.http.HttpRequest;
-import org.apache.http.client.CookieStore;
-import org.apache.http.impl.client.DefaultHttpClient;
-import org.apache.http.protocol.HttpContext;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.protocol.TProtocol;
-import org.apache.thrift.transport.THttpClient;
-import org.apache.thrift.transport.TTransport;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-/**
- *
- * TestThriftHttpCLIService.
- * This tests ThriftCLIService started in http mode.
- *
- */
-
-public class TestThriftHttpCLIService extends ThriftCLIServiceTest {
-
-  private static String transportMode = "http";
-  private static String thriftHttpPath = "cliservice";
-
-  /**
-   *  HttpBasicAuthInterceptorWithLogging
-   *  This adds httpRequestHeaders to the BasicAuthInterceptor
-   */
-  public class HttpBasicAuthInterceptorWithLogging extends HttpBasicAuthInterceptor {
-
-   ArrayList<String> requestHeaders;
-
-   public HttpBasicAuthInterceptorWithLogging(String username,
-      String password, CookieStore cookieStore, String cn, boolean isSSL,
-      Map<String, String> additionalHeaders) {
-      super(username, password, cookieStore, cn, isSSL, additionalHeaders);
-      requestHeaders = new ArrayList<String>();
-    }
-
-    @Override
-    public void process(HttpRequest httpRequest, HttpContext httpContext)
-      throws HttpException, IOException {
-      super.process(httpRequest, httpContext);
-
-      String currHeaders = "";
-
-      for (org.apache.http.Header h : httpRequest.getAllHeaders()) {
-        currHeaders += h.getName() + ":" + h.getValue() + " ";
-      }
-      requestHeaders.add(currHeaders);
-    }
-
-    public ArrayList<String>  getRequestHeaders() {
-      return requestHeaders;
-    }
-  }
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    // Set up the base class
-    ThriftCLIServiceTest.setUpBeforeClass();
-
-    assertNotNull(port);
-    assertNotNull(hiveServer2);
-    assertNotNull(hiveConf);
-
-    hiveConf.setBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS, false);
-    hiveConf.setVar(ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST, host);
-    hiveConf.setIntVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT, port);
-    hiveConf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, AuthTypes.NOSASL.toString());
-    hiveConf.setVar(ConfVars.HIVE_SERVER2_TRANSPORT_MODE, transportMode);
-    hiveConf.setVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PATH, thriftHttpPath);
-
-    startHiveServer2WithConf(hiveConf);
-
-    client = getServiceClientInternal();
-  }
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    ThriftCLIServiceTest.tearDownAfterClass();
-  }
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @Override
-  @Before
-  public void setUp() throws Exception {
-
-  }
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @Override
-  @After
-  public void tearDown() throws Exception {
-
-  }
-
-  @Test
-  /**
-   * Tests calls from a raw (NOSASL) binary client,
-   * to a HiveServer2 running in http mode.
-   * This should throw an expected exception due to incompatibility.
-   * @throws Exception
-   */
-  public void testBinaryClientHttpServer() throws Exception {
-    TTransport transport = getRawBinaryTransport();
-    TCLIService.Client rawBinaryClient = getClient(transport);
-
-    // This will throw an expected exception since client-server modes are incompatible
-    testOpenSessionExpectedException(rawBinaryClient);
-  }
-
-  /**
-   * Configure a wrong service endpoint for the client transport,
-   * and test for error.
-   * @throws Exception
-   */
-  @Test
-  public void testIncorrectHttpPath() throws Exception {
-    thriftHttpPath = "wrongPath";
-    TTransport transport = getHttpTransport();
-    TCLIService.Client httpClient = getClient(transport);
-
-    // This will throw an expected exception since
-    // client is communicating with the wrong http service endpoint
-    testOpenSessionExpectedException(httpClient);
-
-    // Reset to correct http path
-    thriftHttpPath = "cliservice";
-  }
-
-  private void testOpenSessionExpectedException(TCLIService.Client client) {
-    boolean caughtEx = false;
-    // Create a new open session request object
-    TOpenSessionReq openReq = new TOpenSessionReq();
-    try {
-      client.OpenSession(openReq).getSessionHandle();
-    } catch (Exception e) {
-      caughtEx = true;
-      System.out.println("Exception expected: " + e.toString());
-    }
-    assertTrue("Exception expected", caughtEx);
-  }
-
-  private TCLIService.Client getClient(TTransport transport) throws Exception {
-    // Create the corresponding client
-    TProtocol protocol = new TBinaryProtocol(transport);
-    return new TCLIService.Client(protocol);
-  }
-
-  private TTransport getRawBinaryTransport() throws Exception {
-    return HiveAuthFactory.getSocketTransport(host, port, 0);
-  }
-
-  private static TTransport getHttpTransport() throws Exception {
-    DefaultHttpClient httpClient = new DefaultHttpClient();
-    String httpUrl = transportMode + "://" + host + ":" + port +
-        "/" + thriftHttpPath + "/";
-    httpClient.addRequestInterceptor(
-				     new HttpBasicAuthInterceptor(USERNAME, PASSWORD, null, null, false, null));
-    return new THttpClient(httpUrl, httpClient);
-  }
-
-  /**
-   * Test additional http headers passed to request interceptor.
-   * @throws Exception
-   */
-  @Test
-  public void testAdditionalHttpHeaders() throws Exception {
-    TTransport transport;
-    DefaultHttpClient hClient = new DefaultHttpClient();
-    String httpUrl = transportMode + "://" + host + ":" + port +
-        "/" + thriftHttpPath + "/";
-    Map<String, String> additionalHeaders = new HashMap<String, String>();
-    additionalHeaders.put("key1", "value1");
-    additionalHeaders.put("key2", "value2");
-    HttpBasicAuthInterceptorWithLogging authInt =
-      new HttpBasicAuthInterceptorWithLogging(USERNAME, PASSWORD, null, null,
-      false, additionalHeaders);
-    hClient.addRequestInterceptor(authInt);
-    transport = new THttpClient(httpUrl, hClient);
-    TCLIService.Client httpClient = getClient(transport);
-
-    // Create a new open session request object
-    TOpenSessionReq openReq = new TOpenSessionReq();
-    httpClient.OpenSession(openReq).getSessionHandle();
-    ArrayList<String> headers = authInt.getRequestHeaders();
-
-    for (String h : headers) {
-      assertTrue(h.contains("key1:value1"));
-      assertTrue(h.contains("key2:value2"));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/833a7d15/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
index bd510d6..92c2c76 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -848,6 +848,7 @@ public class Driver implements CommandProcessor {
      */
 
     QueryContext.Builder authzContextBuilder = new QueryContext.Builder();
+    authzContextBuilder.setForwardedAddresses(ss.getForwardedAddresses());
     authzContextBuilder.setCommandString(command);
 
     HiveOperationType hiveOpType = getHiveOperationType(op);

http://git-wip-us.apache.org/repos/asf/hive/blob/833a7d15/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandUtil.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandUtil.java
index 7971dab..9288ee2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandUtil.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandUtil.java
@@ -82,6 +82,7 @@ class CommandUtil {
     HivePrivilegeObject commandObj = HivePrivilegeObject.createHivePrivilegeObject(command);
     QueryContext.Builder ctxBuilder = new QueryContext.Builder();
     ctxBuilder.setCommandString(Joiner.on(' ').join(command));
+    ctxBuilder.setForwardedAddresses(ss.getForwardedAddresses());
     ss.getAuthorizerV2().checkPrivileges(type, Arrays.asList(commandObj), null, ctxBuilder.build());
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/833a7d15/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/AuthorizationMetaStoreFilterHook.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/AuthorizationMetaStoreFilterHook.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/AuthorizationMetaStoreFilterHook.java
index a9ad015..20367da 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/AuthorizationMetaStoreFilterHook.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/AuthorizationMetaStoreFilterHook.java
@@ -74,6 +74,7 @@ public class AuthorizationMetaStoreFilterHook extends DefaultMetaStoreFilterHook
   private List<HivePrivilegeObject> getFilteredObjects(List<HivePrivilegeObject> listObjs) throws MetaException {
     SessionState ss = SessionState.get();
     QueryContext.Builder authzContextBuilder = new QueryContext.Builder();
+    authzContextBuilder.setForwardedAddresses(ss.getForwardedAddresses());
     try {
       return ss.getAuthorizerV2().filterListCmdObjects(listObjs, authzContextBuilder.build());
     } catch (HiveAuthzPluginException e) {

http://git-wip-us.apache.org/repos/asf/hive/blob/833a7d15/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/QueryContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/QueryContext.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/QueryContext.java
index 318343c..17f8913 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/QueryContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/QueryContext.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hive.ql.security.authorization.plugin;
 
+import java.util.List;
+
 import org.apache.hadoop.hive.common.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.hive.common.classification.InterfaceStability.Evolving;
 
@@ -31,6 +33,7 @@ public final class QueryContext {
 
   public static class Builder {
     private String commandString;
+    private List<String> forwardedAddresses;
 
     public String getCommandString() {
       return commandString;
@@ -38,24 +41,38 @@ public final class QueryContext {
     public void setCommandString(String commandString) {
       this.commandString = commandString;
     }
+
+    public List<String> getForwardedAddresses() {
+      return forwardedAddresses;
+    }
+    public void setForwardedAddresses(List<String> forwardedAddresses) {
+      this.forwardedAddresses = forwardedAddresses;
+    }
+
     public QueryContext build(){
       return new QueryContext(this);
     }
   }
 
   private final String commandString;
+  private final List<String> forwardedAddresses;
 
   private QueryContext(Builder builder) {
     this.commandString = builder.commandString;
+    this.forwardedAddresses = builder.forwardedAddresses;
   }
 
   public String getCommandString() {
     return commandString;
   }
 
+  public List<String> getForwardedAddresses() {
+    return forwardedAddresses;
+  }
+
   @Override
   public String toString() {
-    return "QueryContext [commandString=" + commandString + "]";
+    return "QueryContext [commandString=" + commandString + ", forwardedAddresses=" + forwardedAddresses + "]";
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/833a7d15/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
index ca18247..2b15c23 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
@@ -275,6 +275,8 @@ public class SessionState {
 
   private final ResourceDownloader resourceDownloader;
 
+  private List<String> forwardedAddresses;
+
   /**
    * Get the lineage state stored in this session.
    *
@@ -1677,6 +1679,14 @@ public class SessionState {
   public ResourceDownloader getResourceDownloader() {
     return resourceDownloader;
   }
+
+  public void setForwardedAddresses(List<String> forwardedAddresses) {
+    this.forwardedAddresses = forwardedAddresses;
+  }
+
+  public List<String> getForwardedAddresses() {
+    return forwardedAddresses;
+  }
 }
 
 class ResourceMaps {

http://git-wip-us.apache.org/repos/asf/hive/blob/833a7d15/service/src/java/org/apache/hive/service/cli/operation/MetadataOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/MetadataOperation.java b/service/src/java/org/apache/hive/service/cli/operation/MetadataOperation.java
index c4a7e69..77228fa 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/MetadataOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/MetadataOperation.java
@@ -135,6 +135,7 @@ public abstract class MetadataOperation extends Operation {
       String cmdString) throws HiveSQLException {
     SessionState ss = SessionState.get();
     QueryContext.Builder ctxBuilder = new QueryContext.Builder();
+    ctxBuilder.setForwardedAddresses(ss.getForwardedAddresses());
     ctxBuilder.setCommandString(cmdString);
     try {
       ss.getAuthorizerV2().checkPrivileges(opType, inpObjs, null,

http://git-wip-us.apache.org/repos/asf/hive/blob/833a7d15/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java b/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
index 80a1844..0f36cd6 100644
--- a/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
+++ b/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
@@ -159,6 +159,7 @@ public class HiveSessionImpl implements HiveSession {
     sessionState = new SessionState(hiveConf, username);
     sessionState.setUserIpAddress(ipAddress);
     sessionState.setIsHiveServerQuery(true);
+    sessionState.setForwardedAddresses(SessionManager.getForwardedAddresses());
     SessionState.start(sessionState);
     try {
       sessionState.reloadAuxJars();
@@ -326,6 +327,7 @@ public class HiveSessionImpl implements HiveSession {
     // Need to make sure that the this HiveServer2's session's SessionState is
     // stored in the thread local for the handler thread.
     SessionState.setCurrentSessionState(sessionState);
+    sessionState.setForwardedAddresses(SessionManager.getForwardedAddresses());
     if (userAccess) {
       lastAccessTime = System.currentTimeMillis();
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/833a7d15/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/session/SessionManager.java b/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
index 79eddd9..ad8678e 100644
--- a/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
+++ b/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
@@ -395,12 +395,7 @@ public class SessionManager extends CompositeService {
     return operationManager;
   }
 
-  private static ThreadLocal<String> threadLocalIpAddress = new ThreadLocal<String>() {
-    @Override
-    protected String initialValue() {
-      return null;
-    }
-  };
+  private static ThreadLocal<String> threadLocalIpAddress = new ThreadLocal<String>();
 
   public static void setIpAddress(String ipAddress) {
     threadLocalIpAddress.set(ipAddress);
@@ -414,6 +409,20 @@ public class SessionManager extends CompositeService {
     return threadLocalIpAddress.get();
   }
 
+  private static ThreadLocal<List<String>> threadLocalForwardedAddresses = new ThreadLocal<List<String>>();
+
+  public static void setForwardedAddresses(List<String> ipAddress) {
+    threadLocalForwardedAddresses.set(ipAddress);
+  }
+
+  public static void clearForwardedAddresses() {
+    threadLocalForwardedAddresses.remove();
+  }
+
+  public static List<String> getForwardedAddresses() {
+    return threadLocalForwardedAddresses.get();
+  }
+
   private static ThreadLocal<String> threadLocalUserName = new ThreadLocal<String>(){
     @Override
     protected String initialValue() {

http://git-wip-us.apache.org/repos/asf/hive/blob/833a7d15/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java b/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java
index 7e12fae..74d73b7 100644
--- a/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java
+++ b/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java
@@ -20,7 +20,11 @@ package org.apache.hive.service.cli.thrift;
 
 import java.io.IOException;
 import java.io.UnsupportedEncodingException;
+import java.net.InetAddress;
 import java.security.PrivilegedExceptionAction;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 import java.util.Random;
 import java.util.Set;
@@ -87,6 +91,7 @@ public class ThriftHttpServlet extends TServlet {
   private boolean isHttpOnlyCookie;
   private final HiveAuthFactory hiveAuthFactory;
   private static final String HIVE_DELEGATION_TOKEN_HEADER =  "X-Hive-Delegation-Token";
+  private static final String X_FORWARDED_FOR = "X-Forwarded-For";
 
   public ThriftHttpServlet(TProcessor processor, TProtocolFactory protocolFactory,
       String authType, UserGroupInformation serviceUGI, UserGroupInformation httpUGI,
@@ -166,6 +171,17 @@ public class ThriftHttpServlet extends TServlet {
       LOG.debug("Client IP Address: " + clientIpAddress);
       // Set the thread local ip address
       SessionManager.setIpAddress(clientIpAddress);
+
+      // get forwarded hosts address
+      String forwarded_for = request.getHeader(X_FORWARDED_FOR);
+      if (forwarded_for != null) {
+        LOG.debug("{}:{}", X_FORWARDED_FOR, forwarded_for);
+        List<String> forwardedAddresses = Arrays.asList(forwarded_for.split(","));
+        SessionManager.setForwardedAddresses(forwardedAddresses);
+      } else {
+        SessionManager.setForwardedAddresses(Collections.<String>emptyList());
+      }
+
       // Generate new cookie and add it to the response
       if (requireNewCookie &&
           !authType.equalsIgnoreCase(HiveAuthFactory.AuthTypes.NOSASL.toString())) {
@@ -195,6 +211,7 @@ public class ThriftHttpServlet extends TServlet {
       SessionManager.clearUserName();
       SessionManager.clearIpAddress();
       SessionManager.clearProxyUserName();
+      SessionManager.clearForwardedAddresses();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/833a7d15/service/src/test/org/apache/hive/service/cli/thrift/ThriftCLIServiceTest.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/cli/thrift/ThriftCLIServiceTest.java b/service/src/test/org/apache/hive/service/cli/thrift/ThriftCLIServiceTest.java
index 630cfc9..1740079 100644
--- a/service/src/test/org/apache/hive/service/cli/thrift/ThriftCLIServiceTest.java
+++ b/service/src/test/org/apache/hive/service/cli/thrift/ThriftCLIServiceTest.java
@@ -73,7 +73,7 @@ public abstract class ThriftCLIServiceTest {
     stopHiveServer2();
   }
 
-  protected static void startHiveServer2WithConf(HiveConf hiveConf) throws Exception {
+   static void startHiveServer2WithConf(HiveConf hiveConf) throws Exception {
     hiveServer2.init(hiveConf);
     // Start HiveServer2 with given config
     // Fail if server doesn't start
@@ -94,7 +94,7 @@ public abstract class ThriftCLIServiceTest {
     }
   }
 
-  protected static ThriftCLIServiceClient getServiceClientInternal() {
+  static ThriftCLIServiceClient getServiceClientInternal() {
     for (Service service : hiveServer2.getServices()) {
       if (service instanceof ThriftBinaryCLIService) {
         return new ThriftCLIServiceClient((ThriftBinaryCLIService) service);