You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by jo...@apache.org on 2019/06/11 20:01:57 UTC

[impala] branch master updated (d753600 -> 666c1be)

This is an automated email from the ASF dual-hosted git repository.

joemcdonnell pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git.


    from d753600  IMPALA-8629: (part 1) Add temp KuduStorageHandler
     new 2f3780e  Add option to set driver in ImpalaJdbcDriver
     new 564def2  IMPALA-8623: Expose HS2 HTTP port in containers
     new f1967a0  Bump CDP_BUILD_NUMBER to 1153860
     new 270deda  IMPALA-8642: krpc should depend on thrift-deps in CMake
     new 666c1be  IMPALA-8551: Make the grant/revoke error messages to be more user friendly

The 5 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 be/src/kudu/rpc/CMakeLists.txt                     |  2 +
 bin/impala-config.sh                               | 10 ++--
 docker/coord_exec/Dockerfile                       |  2 +
 docker/coordinator/Dockerfile                      |  2 +
 .../ranger/RangerCatalogdAuthorizationManager.java | 12 ++++-
 .../apache/impala/testutil/ImpalaJdbcClient.java   | 40 ++++++++++------
 tests/authorization/test_ranger.py                 | 54 ++++++++++++++++++++++
 7 files changed, 101 insertions(+), 21 deletions(-)


[impala] 01/05: Add option to set driver in ImpalaJdbcDriver

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joemcdonnell pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit 2f3780e846d8c0870e73788837af67019481ae23
Author: Thomas Tauber-Marshall <tm...@cloudera.com>
AuthorDate: Mon Jun 3 11:41:06 2019 -0700

    Add option to set driver in ImpalaJdbcDriver
    
    ImpalaJdbcDriver is a simple class that provides a wrapper around the
    Hive Jdbc driver for use in testing. This patch adds a '-d' parameter
    that takes a driver class and executes queries with that driver
    instead of the Hive driver, if its in the classpath.
    
    This makes it easy to use ./bin/run-jdbc-client.sh to test other
    Jdbc drivers, such as the proprietary Cloudera Impala driver.
    
    Change-Id: Ie6c8cec2f61e76b7c8321c954eef830778d8a8ee
    Reviewed-on: http://gerrit.cloudera.org:8080/13514
    Reviewed-by: Thomas Marshall <tm...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 .../apache/impala/testutil/ImpalaJdbcClient.java   | 40 ++++++++++++++--------
 1 file changed, 26 insertions(+), 14 deletions(-)

diff --git a/fe/src/test/java/org/apache/impala/testutil/ImpalaJdbcClient.java b/fe/src/test/java/org/apache/impala/testutil/ImpalaJdbcClient.java
index 01a4b2f..c52399a 100644
--- a/fe/src/test/java/org/apache/impala/testutil/ImpalaJdbcClient.java
+++ b/fe/src/test/java/org/apache/impala/testutil/ImpalaJdbcClient.java
@@ -103,7 +103,7 @@ public class ImpalaJdbcClient {
     // Make sure the driver can be found, throws a ClassNotFoundException if
     // it is not available.
     Class.forName(driverName_);
-    conn_ = DriverManager.getConnection(connString_, "", "");
+    conn_ = DriverManager.getConnection(connString_);
     stmt_ = conn_.createStatement();
   }
 
@@ -145,17 +145,20 @@ public class ImpalaJdbcClient {
     return stmt_;
   }
 
+  public static ImpalaJdbcClient createClient(String driver, String connString) {
+    return new ImpalaJdbcClient(driver, connString);
+  }
+
   public static ImpalaJdbcClient createClientUsingHiveJdbcDriver() {
-    return new ImpalaJdbcClient(
-        HIVE_SERVER2_DRIVER_NAME, getNoAuthConnectionStr("binary"));
+    return createClient(HIVE_SERVER2_DRIVER_NAME, getNoAuthConnectionStr("binary"));
   }
 
   public static ImpalaJdbcClient createClientUsingHiveJdbcDriver(String connString) {
-    return new ImpalaJdbcClient(HIVE_SERVER2_DRIVER_NAME, connString);
+    return createClient(HIVE_SERVER2_DRIVER_NAME, connString);
   }
 
   public static ImpalaJdbcClient createHttpClientUsingHiveJdbcDriver() {
-    return new ImpalaJdbcClient(HIVE_SERVER2_DRIVER_NAME, getNoAuthConnectionStr("http"));
+    return createClient(HIVE_SERVER2_DRIVER_NAME, getNoAuthConnectionStr("http"));
   }
 
   public static String getNoAuthConnectionStr(String connType) {
@@ -183,10 +186,12 @@ public class ImpalaJdbcClient {
   private static class ClientExecOptions {
     private final String connStr;
     private final String query;
+    private final String driver;
 
-    public ClientExecOptions(String connStr, String query) {
+    public ClientExecOptions(String connStr, String query, String driver) {
       this.connStr = connStr;
       this.query = query;
+      this.driver = driver;
     }
 
     public String getQuery() {
@@ -196,6 +201,10 @@ public class ImpalaJdbcClient {
     public String getConnStr() {
       return connStr;
     }
+
+    public String getDriver() {
+      return driver;
+    }
   }
 
   /**
@@ -208,6 +217,7 @@ public class ImpalaJdbcClient {
         "Full connection string to use. Overrides host/port value");
     options.addOption("t", true, "SASL/NOSASL, whether to use SASL transport or not");
     options.addOption("q", true, "Query String");
+    options.addOption("d", true, "Driver name, default: org.apache.hive.jdbc.HiveDriver");
     options.addOption("help", false, "Help");
 
     BasicParser optionParser = new BasicParser();
@@ -235,12 +245,12 @@ public class ImpalaJdbcClient {
     if (connStr == null) {
       String hostPort = cmdArgs.getOptionValue("i", "localhost:21050");
       connStr = "jdbc:hive2://" + hostPort + "/";
-    }
-    // Append appropriate auth option to connection string.
-    if (useSasl) {
-      connStr = connStr + SASL_AUTH_SPEC;
-    } else {
-      connStr = connStr + NOSASL_AUTH_SPEC;
+      // Append appropriate auth option to connection string.
+      if (useSasl) {
+        connStr = connStr + SASL_AUTH_SPEC;
+      } else {
+        connStr = connStr + NOSASL_AUTH_SPEC;
+      }
     }
 
     String query = cmdArgs.getOptionValue("q");
@@ -250,7 +260,9 @@ public class ImpalaJdbcClient {
       System.exit(1);
     }
 
-    return new ClientExecOptions(connStr, query);
+    String driver = cmdArgs.getOptionValue("d", HIVE_SERVER2_DRIVER_NAME);
+
+    return new ClientExecOptions(connStr, query, driver);
   }
 
   private static String formatColumnValue(String colVal, String columnType)
@@ -338,7 +350,7 @@ public class ImpalaJdbcClient {
     ClientExecOptions execOptions = parseOptions(args);
 
     ImpalaJdbcClient client =
-      ImpalaJdbcClient.createClientUsingHiveJdbcDriver(execOptions.getConnStr());
+      ImpalaJdbcClient.createClient(execOptions.getDriver(), execOptions.getConnStr());
 
     try {
       client.connect();


[impala] 03/05: Bump CDP_BUILD_NUMBER to 1153860

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joemcdonnell pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit f1967a0d597f034e1300d351a339a06a3d5700ec
Author: Fredy Wijaya <fw...@cloudera.com>
AuthorDate: Mon Jun 10 08:58:21 2019 -0700

    Bump CDP_BUILD_NUMBER to 1153860
    
    This patch bumps the CDP_BUILD_NUMBER to 1153860 to pull RANGER-2437 fix
    that provides better error messages when granting/revoking with invalid
    user/group.
    
    Testing:
    - Ran core tests
    - Ran the build with USE_CDP_HIVE=true
    
    Change-Id: Ief3458c7dbf2ee973b586e2d0d54cc0739b6ae3b
    Reviewed-on: http://gerrit.cloudera.org:8080/13574
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 bin/impala-config.sh | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)

diff --git a/bin/impala-config.sh b/bin/impala-config.sh
index 2b01c7b..2386bbd 100755
--- a/bin/impala-config.sh
+++ b/bin/impala-config.sh
@@ -163,19 +163,19 @@ fi
 export IMPALA_TOOLCHAIN_HOST
 export CDH_MAJOR_VERSION=6
 export CDH_BUILD_NUMBER=1137441
-export CDP_BUILD_NUMBER=1056671
+export CDP_BUILD_NUMBER=1153860
 export CDH_HADOOP_VERSION=3.0.0-cdh6.x-SNAPSHOT
-export CDP_HADOOP_VERSION=3.1.1.6.0.99.0-147
+export CDP_HADOOP_VERSION=3.1.1.7.0.0.0-107
 export IMPALA_HBASE_VERSION=2.1.0-cdh6.x-SNAPSHOT
 export IMPALA_SENTRY_VERSION=2.1.0-cdh6.x-SNAPSHOT
-export IMPALA_RANGER_VERSION=1.2.0.6.0.99.0-147
+export IMPALA_RANGER_VERSION=1.2.0.7.0.0.0-107
 export IMPALA_PARQUET_VERSION=1.9.0-cdh6.x-SNAPSHOT
 export IMPALA_AVRO_JAVA_VERSION=1.8.2-cdh6.x-SNAPSHOT
 export IMPALA_LLAMA_MINIKDC_VERSION=1.0.0
 export IMPALA_KITE_VERSION=1.0.0-cdh6.x-SNAPSHOT
 export IMPALA_KUDU_JAVA_VERSION=1.10.0-cdh6.x-SNAPSHOT
 export CDH_HIVE_VERSION=2.1.1-cdh6.x-SNAPSHOT
-export CDP_HIVE_VERSION=3.1.0.6.0.99.0-147
+export CDP_HIVE_VERSION=3.1.0.7.0.0.0-107
 
 # When IMPALA_(CDH_COMPONENT)_URL are overridden, they may contain '$(platform_label)'
 # which will be substituted for the CDH platform label in bootstrap_toolchain.py
@@ -202,7 +202,7 @@ if $USE_CDP_HIVE; then
   # When USE_CDP_HIVE is set we use the CDP hive version to build as well as deploy in
   # the minicluster
   export IMPALA_HIVE_VERSION=${CDP_HIVE_VERSION}
-  export IMPALA_TEZ_VERSION=0.9.1.6.0.99.0-147
+  export IMPALA_TEZ_VERSION=0.9.1.7.0.0.0-107
   export IMPALA_HADOOP_VERSION=${CDP_HADOOP_VERSION}
   export HADOOP_HOME="$CDP_COMPONENTS_HOME/hadoop-${CDP_HADOOP_VERSION}/"
 else


[impala] 04/05: IMPALA-8642: krpc should depend on thrift-deps in CMake

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joemcdonnell pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit 270deda049f4f8551742475017e860f1dab3754b
Author: stiga-huang <hu...@gmail.com>
AuthorDate: Sun Jun 9 18:03:24 2019 -0700

    IMPALA-8642: krpc should depend on thrift-deps in CMake
    
    When build with IMPALA_BUILD_THREADS=1, the compiling of krpc will fail
    since kudu_version.cc includes "gen-cpp/Status_types.h" which will be
    built later. We should add dependency on thrift-deps for krpc to avoid
    such a race condition in compiling.
    
    Change-Id: Ieaf9d992340f07fdf88b35fb967553fa2e35b628
    Reviewed-on: http://gerrit.cloudera.org:8080/13569
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/kudu/rpc/CMakeLists.txt | 2 ++
 1 file changed, 2 insertions(+)

diff --git a/be/src/kudu/rpc/CMakeLists.txt b/be/src/kudu/rpc/CMakeLists.txt
index 91dd484..3b31ec5 100644
--- a/be/src/kudu/rpc/CMakeLists.txt
+++ b/be/src/kudu/rpc/CMakeLists.txt
@@ -96,6 +96,8 @@ add_executable(protoc-gen-krpc protoc-gen-krpc.cc
   # TODO: Consider either removing code that depends on these flags, or namespacing them
   # somehow.
   ${CMAKE_CURRENT_SOURCE_DIR}/../../common/global-flags.cc)
+# IMPALA-8642: kudu_version.cc depends on gen-cpp/Status_types.h in target thrift-deps
+add_dependencies(protoc-gen-krpc thrift-deps)
 target_link_libraries(protoc-gen-krpc
     ${KUDU_BASE_LIBS}
     rpc_header_proto


[impala] 05/05: IMPALA-8551: Make the grant/revoke error messages to be more user friendly

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joemcdonnell pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit 666c1be66420468936b6b34f6d1fb8f695a72d7d
Author: Fredy Wijaya <fw...@cloudera.com>
AuthorDate: Wed Jun 5 17:42:54 2019 -0700

    IMPALA-8551: Make the grant/revoke error messages to be more user friendly
    
    This patch updates the grant/revoke error messages to be more user
    friendly, especially when granting/revoking with an invalid principal.
    
    Testing:
    - Added E2E test to test grant/revoke with invalid principal
    
    Change-Id: I8995f5dc88b211cd3af415713802cfeac44fe576
    Reviewed-on: http://gerrit.cloudera.org:8080/13525
    Reviewed-by: Fredy Wijaya <fw...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 .../ranger/RangerCatalogdAuthorizationManager.java | 12 ++++-
 tests/authorization/test_ranger.py                 | 54 ++++++++++++++++++++++
 2 files changed, 64 insertions(+), 2 deletions(-)

diff --git a/fe/src/main/java/org/apache/impala/authorization/ranger/RangerCatalogdAuthorizationManager.java b/fe/src/main/java/org/apache/impala/authorization/ranger/RangerCatalogdAuthorizationManager.java
index 7bb6aaf..0b296d0 100644
--- a/fe/src/main/java/org/apache/impala/authorization/ranger/RangerCatalogdAuthorizationManager.java
+++ b/fe/src/main/java/org/apache/impala/authorization/ranger/RangerCatalogdAuthorizationManager.java
@@ -40,6 +40,8 @@ import org.apache.impala.thrift.TShowRolesParams;
 import org.apache.impala.thrift.TShowRolesResult;
 import org.apache.impala.util.ClassUtil;
 import org.apache.ranger.plugin.util.GrantRevokeRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
 import java.util.Collections;
@@ -57,6 +59,8 @@ import java.util.function.Supplier;
  * Operations not supported by Ranger will throw an {@link UnsupportedFeatureException}.
  */
 public class RangerCatalogdAuthorizationManager implements AuthorizationManager {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      RangerCatalogdAuthorizationManager.class);
   private static final String AUTHZ_CACHE_INVALIDATION_MARKER = "ranger";
 
   private final Supplier<RangerImpalaPlugin> plugin_;
@@ -171,7 +175,9 @@ public class RangerCatalogdAuthorizationManager implements AuthorizationManager
         }
       }
     } catch (Exception e) {
-      throw new InternalException(e.getMessage());
+      LOG.error("Error granting a privilege in Ranger: ", e);
+      throw new InternalException("Error granting a privilege in Ranger. " +
+          "Ranger error message: " + e.getMessage());
     }
   }
 
@@ -184,7 +190,9 @@ public class RangerCatalogdAuthorizationManager implements AuthorizationManager
         }
       }
     } catch (Exception e) {
-      throw new InternalException(e.getMessage());
+      LOG.error("Error revoking a privilege in Ranger: ", e);
+      throw new InternalException("Error revoking a privilege in Ranger. " +
+          "Ranger error message: " + e.getMessage());
     }
   }
 
diff --git a/tests/authorization/test_ranger.py b/tests/authorization/test_ranger.py
index 83ca5e2..bb1019a 100644
--- a/tests/authorization/test_ranger.py
+++ b/tests/authorization/test_ranger.py
@@ -548,3 +548,57 @@ class TestRanger(CustomClusterTestSuite):
                        error_msg.format("SHOW ROLE GRANT GROUP"))]:
       result = self.execute_query_expect_failure(impala_client, statement[0], user=user)
       assert statement[1] in str(result)
+
+  @CustomClusterTestSuite.with_args(
+    impalad_args=IMPALAD_ARGS, catalogd_args=CATALOGD_ARGS)
+  def test_grant_revoke_invalid_principal(self):
+    """Tests grant/revoke to/from invalid principal should return more readable
+       error messages."""
+    valid_user = "admin"
+    invalid_user = "invalid_user"
+    invalid_group = "invalid_group"
+    # TODO(IMPALA-8640): Create two different Impala clients because the users to
+    # workaround the bug.
+    invalid_impala_client = self.create_impala_client()
+    valid_impala_client = self.create_impala_client()
+    for statement in ["grant select on table functional.alltypes to user {0}"
+                      .format(getuser()),
+                      "revoke select on table functional.alltypes from user {0}"
+                      .format(getuser())]:
+      result = self.execute_query_expect_failure(invalid_impala_client,
+                                                 statement,
+                                                 user=invalid_user)
+      if "grant" in statement:
+        assert "Error granting a privilege in Ranger. Ranger error message: " \
+               "HTTP 403 Error: Grantor user invalid_user doesn't exist" in str(result)
+      else:
+        assert "Error revoking a privilege in Ranger. Ranger error message: " \
+               "HTTP 403 Error: Grantor user invalid_user doesn't exist" in str(result)
+
+    for statement in ["grant select on table functional.alltypes to user {0}"
+                      .format(invalid_user),
+                      "revoke select on table functional.alltypes from user {0}"
+                      .format(invalid_user)]:
+      result = self.execute_query_expect_failure(valid_impala_client,
+                                                 statement,
+                                                 user=valid_user)
+      if "grant" in statement:
+        assert "Error granting a privilege in Ranger. Ranger error message: " \
+               "HTTP 403 Error: Grantee user invalid_user doesn't exist" in str(result)
+      else:
+        assert "Error revoking a privilege in Ranger. Ranger error message: " \
+               "HTTP 403 Error: Grantee user invalid_user doesn't exist" in str(result)
+
+    for statement in ["grant select on table functional.alltypes to group {0}"
+                      .format(invalid_group),
+                      "revoke select on table functional.alltypes from group {0}"
+                      .format(invalid_group)]:
+      result = self.execute_query_expect_failure(valid_impala_client,
+                                                 statement,
+                                                 user=valid_user)
+      if "grant" in statement:
+        assert "Error granting a privilege in Ranger. Ranger error message: " \
+               "HTTP 403 Error: Grantee group invalid_group doesn't exist" in str(result)
+      else:
+        assert "Error revoking a privilege in Ranger. Ranger error message: " \
+               "HTTP 403 Error: Grantee group invalid_group doesn't exist" in str(result)


[impala] 02/05: IMPALA-8623: Expose HS2 HTTP port in containers

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joemcdonnell pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit 564def2dabec5209525514f4c2ef77992550e42f
Author: Tim Armstrong <ta...@cloudera.com>
AuthorDate: Wed Jun 5 12:06:01 2019 -0700

    IMPALA-8623: Expose HS2 HTTP port in containers
    
    Testing:
    Ran dockerised test cluster locally, checked
    that ports were mapped as expected.
    
    Change-Id: Iece20bc134fa5867f18b166cee2a2f75b21f9f36
    Reviewed-on: http://gerrit.cloudera.org:8080/13520
    Reviewed-by: Tim Armstrong <ta...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 docker/coord_exec/Dockerfile  | 2 ++
 docker/coordinator/Dockerfile | 2 ++
 2 files changed, 4 insertions(+)

diff --git a/docker/coord_exec/Dockerfile b/docker/coord_exec/Dockerfile
index d1893b9..eeb1163 100644
--- a/docker/coord_exec/Dockerfile
+++ b/docker/coord_exec/Dockerfile
@@ -22,6 +22,8 @@ FROM impala_base
 EXPOSE 21000
 # HS2
 EXPOSE 21050
+# HS2 over HTTP
+EXPOSE 28000
 # Debug webserver
 EXPOSE 25000
 
diff --git a/docker/coordinator/Dockerfile b/docker/coordinator/Dockerfile
index f4744c2..22ea7d7 100644
--- a/docker/coordinator/Dockerfile
+++ b/docker/coordinator/Dockerfile
@@ -22,6 +22,8 @@ FROM impala_base
 EXPOSE 21000
 # HS2
 EXPOSE 21050
+# HS2 over HTTP
+EXPOSE 28000
 # Debug webserver
 EXPOSE 25000