You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by mi...@apache.org on 2017/07/26 13:28:12 UTC

[1/2] incubator-impala git commit: IMPALA-5489: Improve Sentry authorization for Kudu tables

Repository: incubator-impala
Updated Branches:
  refs/heads/master 7ccbfe47f -> c5a9b43db


IMPALA-5489: Improve Sentry authorization for Kudu tables

IMPALA-4000 added basic authorization support for Kudu
tables, but it had several limitations:
* Only the ALL privilege level can be granted to Kudu tables.
  (Finer-grained levels such as only SELECT or only INSERT are
  not supported.)
* Column level permissions on Kudu tables are not supported.
* Only users with ALL privileges on SERVER may create external
  Kudu tables.

This patch relaxes the restrictions to allow:
* Allow column-level permissions
* Allow fine grained privileges SELECT and INSERT for those
  statement types.

DELETE/UPDATE/UPSERT privileges now require ALL privileges
because Sentry will eventually get fine grained privilege
actions, and at that point Impala should support the more
specific actions (IMPALA-3840). The assumption is that the
Kudu table authorization support is currently so limited
that most users are not using this functionality yet, but
this is a behavior change that needs to be clearly stated in
the Impala release notes.

Testing: Adds FE and EE tests.

Change-Id: Ib12d2b32fa3e142e69bd8b0f24f53f9e5cbf7460
Reviewed-on: http://gerrit.cloudera.org:8080/7307
Reviewed-by: Matthew Jacobs <mj...@cloudera.com>
Tested-by: Impala Public Jenkins


Project: http://git-wip-us.apache.org/repos/asf/incubator-impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-impala/commit/1aa3a5c6
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/1aa3a5c6
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/1aa3a5c6

Branch: refs/heads/master
Commit: 1aa3a5c616ec058ab50e2185472beb9aff306b1e
Parents: 7ccbfe4
Author: Matthew Jacobs <mj...@cloudera.com>
Authored: Tue Jun 27 14:08:01 2017 -0700
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Wed Jul 26 05:43:01 2017 +0000

----------------------------------------------------------------------
 .../org/apache/impala/analysis/InsertStmt.java  |  7 +-
 .../org/apache/impala/analysis/ModifyStmt.java  |  6 +-
 .../apache/impala/analysis/PrivilegeSpec.java   | 11 ---
 .../impala/analysis/AnalyzeAuthStmtsTest.java   | 19 ++---
 .../apache/impala/analysis/AuditingTest.java    |  8 +-
 .../org/apache/impala/analysis/ParserTest.java  |  4 +
 .../queries/QueryTest/grant_revoke_kudu.test    | 78 ++++++++++++++++++++
 7 files changed, 102 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1aa3a5c6/fe/src/main/java/org/apache/impala/analysis/InsertStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/InsertStmt.java b/fe/src/main/java/org/apache/impala/analysis/InsertStmt.java
index c23145d..39a64fa 100644
--- a/fe/src/main/java/org/apache/impala/analysis/InsertStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/InsertStmt.java
@@ -404,6 +404,9 @@ public class InsertStmt extends StatementBase {
    * Adds table_ to the analyzer's descriptor table if analysis succeeds.
    */
   private void analyzeTargetTable(Analyzer analyzer) throws AnalysisException {
+    // Fine-grained privileges for UPSERT do not exist yet, so they require ALL for now.
+    Privilege privilegeRequired = isUpsert_ ? Privilege.ALL : Privilege.INSERT;
+
     // If the table has not yet been set, load it from the Catalog. This allows for
     // callers to set a table to analyze that may not actually be created in the Catalog.
     // One example use case is CREATE TABLE AS SELECT which must run analysis on the
@@ -413,12 +416,12 @@ public class InsertStmt extends StatementBase {
         targetTableName_ =
             new TableName(analyzer.getDefaultDb(), targetTableName_.getTbl());
       }
-      table_ = analyzer.getTable(targetTableName_, Privilege.INSERT);
+      table_ = analyzer.getTable(targetTableName_, privilegeRequired);
     } else {
       targetTableName_ = new TableName(table_.getDb().getName(), table_.getName());
       PrivilegeRequestBuilder pb = new PrivilegeRequestBuilder();
       analyzer.registerPrivReq(pb.onTable(table_.getDb().getName(), table_.getName())
-          .allOf(Privilege.INSERT).toRequest());
+          .allOf(privilegeRequired).toRequest());
     }
 
     // We do not support (in|up)serting into views.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1aa3a5c6/fe/src/main/java/org/apache/impala/analysis/ModifyStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/ModifyStmt.java b/fe/src/main/java/org/apache/impala/analysis/ModifyStmt.java
index c538968..5bac75c 100644
--- a/fe/src/main/java/org/apache/impala/analysis/ModifyStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/ModifyStmt.java
@@ -144,9 +144,9 @@ public abstract class ModifyStmt extends StatementBase {
     }
     table_ = (KuduTable) dstTbl;
 
-    // Make sure that the user is allowed to modify the target table, since no
-    // UPDATE / DELETE privilege exists, we reuse the INSERT one.
-    analyzer.registerAuthAndAuditEvent(dstTbl, Privilege.INSERT);
+    // Make sure that the user is allowed to modify the target table. Use ALL because no
+    // UPDATE / DELETE privilege exists yet (IMPALA-3840).
+    analyzer.registerAuthAndAuditEvent(dstTbl, Privilege.ALL);
 
     // Validates the assignments_ and creates the sourceStmt_.
     if (sourceStmt_ == null) createSourceStmt(analyzer);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1aa3a5c6/fe/src/main/java/org/apache/impala/analysis/PrivilegeSpec.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/PrivilegeSpec.java b/fe/src/main/java/org/apache/impala/analysis/PrivilegeSpec.java
index b999cda..9e8731a 100644
--- a/fe/src/main/java/org/apache/impala/analysis/PrivilegeSpec.java
+++ b/fe/src/main/java/org/apache/impala/analysis/PrivilegeSpec.java
@@ -283,17 +283,6 @@ public class PrivilegeSpec implements ParseNode {
           "to issue a GRANT/REVOKE statement.", tableName_.toString()));
     }
     Preconditions.checkNotNull(table);
-    if (table instanceof KuduTable) {
-      // We only support the ALL privilege on Kudu tables since many of the finer-grained
-      // levels (DELETE/UPDATE) are not available. See IMPALA-4000 for details.
-      if (privilegeLevel_ != TPrivilegeLevel.ALL) {
-        throw new AnalysisException("Kudu tables only support the ALL privilege level.");
-      }
-      if (scope_ == TPrivilegeScope.COLUMN) {
-        throw new AnalysisException("Column-level privileges on Kudu " +
-            "tables are not supported.");
-      }
-    }
     return table;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1aa3a5c6/fe/src/test/java/org/apache/impala/analysis/AnalyzeAuthStmtsTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/AnalyzeAuthStmtsTest.java b/fe/src/test/java/org/apache/impala/analysis/AnalyzeAuthStmtsTest.java
index 07a1c0a..1fe11bc 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AnalyzeAuthStmtsTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AnalyzeAuthStmtsTest.java
@@ -129,6 +129,8 @@ public class AnalyzeAuthStmtsTest extends AnalyzerTest {
           createAnalyzer("functional"));
       AnalyzesOk(String.format("%s ALL ON TABLE functional.alltypes %s myrole",
           formatArgs));
+      AnalyzesOk(String.format("%s ALL ON TABLE functional_kudu.alltypes %s myrole",
+          formatArgs));
       AnalyzesOk(String.format("%s ALL ON DATABASE functional %s myrole", formatArgs));
       AnalyzesOk(String.format("%s ALL ON SERVER %s myrole", formatArgs));
       AnalyzesOk(String.format("%s ALL ON SERVER server1 %s myrole", formatArgs));
@@ -151,6 +153,8 @@ public class AnalyzeAuthStmtsTest extends AnalyzerTest {
       // INSERT privilege
       AnalyzesOk(String.format("%s INSERT ON TABLE alltypesagg %s myrole", formatArgs),
           createAnalyzer("functional"));
+      AnalyzesOk(String.format(
+          "%s INSERT ON TABLE functional_kudu.alltypessmall %s myrole", formatArgs));
       AnalyzesOk(String.format("%s INSERT ON TABLE functional.alltypesagg %s myrole",
           formatArgs));
       AnalyzesOk(String.format("%s INSERT ON DATABASE functional %s myrole",
@@ -160,19 +164,14 @@ public class AnalyzeAuthStmtsTest extends AnalyzerTest {
       AnalysisError(String.format("%s INSERT ON URI 'hdfs:////abc//123' %s myrole",
           formatArgs), "Only 'ALL' privilege may be applied at URI scope in privilege " +
           "spec.");
-      // IMPALA-4000: Insert privilege on a Kudu table
-      AnalysisError(String.format(
-          "%s SELECT ON TABLE functional_kudu.alltypessmall %s myrole", formatArgs),
-          "Kudu tables only support the ALL privilege level.");
-      AnalysisError(String.format(
-          "%s INSERT ON TABLE functional_kudu.alltypessmall %s myrole", formatArgs),
-          "Kudu tables only support the ALL privilege level.");
 
       // SELECT privilege
       AnalyzesOk(String.format("%s SELECT ON TABLE alltypessmall %s myrole", formatArgs),
           createAnalyzer("functional"));
       AnalyzesOk(String.format("%s SELECT ON TABLE functional.alltypessmall %s myrole",
           formatArgs));
+      AnalyzesOk(String.format(
+          "%s SELECT ON TABLE functional_kudu.alltypessmall %s myrole", formatArgs));
       AnalyzesOk(String.format("%s SELECT ON DATABASE functional %s myrole",
           formatArgs));
       AnalysisError(String.format("%s SELECT ON SERVER %s myrole", formatArgs),
@@ -189,6 +188,8 @@ public class AnalyzeAuthStmtsTest extends AnalyzerTest {
       // SELECT privilege on both regular and partition columns
       AnalyzesOk(String.format("%s SELECT (id, int_col, year, month) ON TABLE " +
           "alltypes %s myrole", formatArgs), createAnalyzer("functional"));
+      AnalyzesOk(String.format("%s SELECT (id, bool_col) ON TABLE " +
+          "functional_kudu.alltypessmall %s myrole", formatArgs));
       // Empty column list
       AnalysisError(String.format("%s SELECT () ON TABLE functional.alltypes " +
           "%s myrole", formatArgs), "Empty column list in column privilege spec.");
@@ -203,10 +204,6 @@ public class AnalyzeAuthStmtsTest extends AnalyzerTest {
       AnalysisError(String.format("%s SELECT (id, bool_col) ON TABLE " +
           "functional.alltypes_hive_view %s myrole", formatArgs), "Column-level " +
           "privileges on views are not supported.");
-      // IMPALA-4000: Column-level privileges on a KUDU table
-      AnalysisError(String.format("%s SELECT (id, bool_col) ON TABLE " +
-          "functional_kudu.alltypessmall %s myrole", formatArgs),
-          "Kudu tables only support the ALL privilege level.");
       // Columns/table that don't exist
       AnalysisError(String.format("%s SELECT (invalid_col) ON TABLE " +
           "functional.alltypes %s myrole", formatArgs), "Error setting column-level " +

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1aa3a5c6/fe/src/test/java/org/apache/impala/analysis/AuditingTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/AuditingTest.java b/fe/src/test/java/org/apache/impala/analysis/AuditingTest.java
index 60d31c8..a14fbc9 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AuditingTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AuditingTest.java
@@ -406,14 +406,14 @@ public class AuditingTest extends AnalyzerTest {
         "functional_kudu.alltypes");
     Assert.assertEquals(accessEvents, Sets.newHashSet(
         new TAccessEvent("functional_kudu.alltypes", TCatalogObjectType.TABLE, "SELECT"),
-        new TAccessEvent("functional_kudu.testtbl", TCatalogObjectType.TABLE, "INSERT")));
+        new TAccessEvent("functional_kudu.testtbl", TCatalogObjectType.TABLE, "ALL")));
 
     // Delete
     accessEvents = AnalyzeAccessEvents(
         "delete from functional_kudu.testtbl where id = 1");
     Assert.assertEquals(accessEvents, Sets.newHashSet(
         new TAccessEvent("functional_kudu.testtbl", TCatalogObjectType.TABLE, "SELECT"),
-        new TAccessEvent("functional_kudu.testtbl", TCatalogObjectType.TABLE, "INSERT")));
+        new TAccessEvent("functional_kudu.testtbl", TCatalogObjectType.TABLE, "ALL")));
 
     // Delete using a complex query
     accessEvents = AnalyzeAccessEvents(
@@ -422,14 +422,14 @@ public class AuditingTest extends AnalyzerTest {
     Assert.assertEquals(accessEvents, Sets.newHashSet(
         new TAccessEvent("functional_kudu.testtbl", TCatalogObjectType.TABLE, "SELECT"),
         new TAccessEvent("functional_kudu.alltypes", TCatalogObjectType.TABLE, "SELECT"),
-        new TAccessEvent("functional_kudu.testtbl", TCatalogObjectType.TABLE, "INSERT")));
+        new TAccessEvent("functional_kudu.testtbl", TCatalogObjectType.TABLE, "ALL")));
 
     // Update
     accessEvents = AnalyzeAccessEvents(
         "update functional_kudu.testtbl set name = 'test' where id < 10");
     Assert.assertEquals(accessEvents, Sets.newHashSet(
         new TAccessEvent("functional_kudu.testtbl", TCatalogObjectType.TABLE, "SELECT"),
-        new TAccessEvent("functional_kudu.testtbl", TCatalogObjectType.TABLE, "INSERT")));
+        new TAccessEvent("functional_kudu.testtbl", TCatalogObjectType.TABLE, "ALL")));
 
     // Drop table
     accessEvents = AnalyzeAccessEvents("drop table functional_kudu.testtbl");

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1aa3a5c6/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java b/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
index 3447fe6..01ca9f5 100644
--- a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
@@ -3520,6 +3520,10 @@ public class ParserTest extends FrontendTestBase {
 
     ParserError("GRANT ALL ON TABLE foo FROM myrole");
     ParserError("REVOKE ALL ON TABLE foo TO myrole");
+
+    ParserError("GRANT UPDATE ON TABLE foo TO myRole");
+    ParserError("GRANT DELETE ON TABLE foo TO myRole");
+    ParserError("GRANT UPSERT ON TABLE foo TO myRole");
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1aa3a5c6/testdata/workloads/functional-query/queries/QueryTest/grant_revoke_kudu.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/grant_revoke_kudu.test b/testdata/workloads/functional-query/queries/QueryTest/grant_revoke_kudu.test
index 84e8d2a..c552773 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/grant_revoke_kudu.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/grant_revoke_kudu.test
@@ -99,11 +99,89 @@ alter table grant_rev_db.kudu_tbl set tblproperties('EXTERNAL'='TRUE');
 alter table grant_rev_db.kudu_tbl set tblproperties('EXTERNAL'='FALSE');
 ====
 ---- QUERY
+create role grant_revoke_test_KUDU
+====
+---- QUERY
+grant role grant_revoke_test_KUDU to group $GROUP_NAME;
+====
+---- QUERY
+revoke role grant_revoke_test_ALL_SERVER from group $GROUP_NAME
+====
+---- QUERY
+revoke role grant_revoke_test_ALL_TEST_DB from group $GROUP_NAME
+====
+---- QUERY
+insert into grant_rev_db.kudu_tbl values (1, "foo");
+---- CATCH
+does not have privileges to execute 'INSERT' on: grant_rev_db.kudu_tbl
+====
+---- QUERY
+grant insert on table grant_rev_db.kudu_tbl to grant_revoke_test_KUDU
+====
+---- QUERY
+insert into grant_rev_db.kudu_tbl values (1, "foo");
+====
+---- QUERY
+# UPSERT requires ALL
+upsert into grant_rev_db.kudu_tbl values (1, "bar");
+---- CATCH
+does not have privileges to access: grant_rev_db.kudu_tbl
+====
+---- QUERY
+select * from grant_rev_db.kudu_tbl
+---- CATCH
+does not have privileges to execute 'SELECT' on: grant_rev_db.kudu_tbl
+====
+---- QUERY
+grant select(i) on table grant_rev_db.kudu_tbl to grant_revoke_test_KUDU
+====
+---- QUERY
+select i from grant_rev_db.kudu_tbl
+---- RESULTS
+1
+---- TYPES
+INT
+====
+---- QUERY
+# UPDATE/DELETE requires ALL privileges
+update grant_rev_db.kudu_tbl set a = "zzz"
+---- CATCH
+does not have privileges to access: grant_rev_db.kudu_tbl
+====
+---- QUERY
+delete from grant_rev_db.kudu_tbl
+---- CATCH
+does not have privileges to access: grant_rev_db.kudu_tbl
+====
+---- QUERY
+grant select(a) on table grant_rev_db.kudu_tbl to grant_revoke_test_KUDU
+---- RESULTS
+====
+---- QUERY
+grant ALL on table grant_rev_db.kudu_tbl to grant_revoke_test_KUDU
+====
+---- QUERY
+update grant_rev_db.kudu_tbl set a = "zzz"
+---- RESULTS
+====
+---- QUERY
+upsert into grant_rev_db.kudu_tbl values (1, "mom");
+---- RESULTS
+====
+---- QUERY
+select * from grant_rev_db.kudu_tbl
+---- RESULTS
+1,'mom'
+---- TYPES
+INT, STRING
+====
+---- QUERY
 drop table grant_rev_db.kudu_tbl
 ====
 ---- QUERY
 # Cleanup test roles
 drop role grant_revoke_test_ALL_SERVER;
 drop role grant_revoke_test_ALL_TEST_DB;
+drop role grant_revoke_test_KUDU;
 ---- RESULTS
 ====


[2/2] incubator-impala git commit: IMPALA-3496: stress test: print version info

Posted by mi...@apache.org.
IMPALA-3496: stress test: print version info

Print the version info of each impalad that's used in a stress test run,
sorted by host name.

Testing done:
$ tests/stress/concurrent_select.py [redacted cluster options] --tpcds-db null --max-queries 0
Cluster Impalad Version Info:
host2.redacted: impalad version 2.10.0-SNAPSHOT RELEASE (build e862385281c74baa6f1a4d10d44f3411a4303abc)
Built on Tue Jul 25 07:06:27 PDT 2017
host3.redacted: impalad version 2.10.0-SNAPSHOT RELEASE (build e862385281c74baa6f1a4d10d44f3411a4303abc)
Built on Tue Jul 25 07:06:27 PDT 2017
host4.redacted: impalad version 2.10.0-SNAPSHOT RELEASE (build e862385281c74baa6f1a4d10d44f3411a4303abc)
Built on Tue Jul 25 07:06:27 PDT 2017
host5.redacted: impalad version 2.10.0-SNAPSHOT RELEASE (build e862385281c74baa6f1a4d10d44f3411a4303abc)
Built on Tue Jul 25 07:06:27 PDT 2017
host6.redacted: impalad version 2.10.0-SNAPSHOT RELEASE (build e862385281c74baa6f1a4d10d44f3411a4303abc)
Built on Tue Jul 25 07:06:27 PDT 2017
2017-07-25 12:38:52,732 12793 Thread-1 INFO:cluster[691]:Finding impalad binary location
...

Change-Id: Ie4b40783ddae6b1bfb2bb4e28c0e3bf97ab944c5
Reviewed-on: http://gerrit.cloudera.org:8080/7501
Reviewed-by: Michael Brown <mi...@cloudera.com>
Tested-by: Michael Brown <mi...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/incubator-impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-impala/commit/c5a9b43d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/c5a9b43d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/c5a9b43d

Branch: refs/heads/master
Commit: c5a9b43db4315f4728a3454f20fa1b265a0de0fc
Parents: 1aa3a5c
Author: Michael Brown <mi...@cloudera.com>
Authored: Tue Jul 25 11:01:48 2017 -0700
Committer: Michael Brown <mi...@cloudera.com>
Committed: Wed Jul 26 13:16:08 2017 +0000

----------------------------------------------------------------------
 tests/comparison/cluster.py       |  8 ++++++++
 tests/stress/concurrent_select.py | 14 ++++++++++++++
 2 files changed, 22 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/c5a9b43d/tests/comparison/cluster.py
----------------------------------------------------------------------
diff --git a/tests/comparison/cluster.py b/tests/comparison/cluster.py
index 761bc57..1cf9516 100644
--- a/tests/comparison/cluster.py
+++ b/tests/comparison/cluster.py
@@ -563,6 +563,9 @@ class Impala(Service):
   def cancel_queries(self):
     self.for_each_impalad(lambda i: i.cancel_queries())
 
+  def get_version_info(self):
+    return self.for_each_impalad(lambda i: i.get_version_info(), as_dict=True)
+
   def queries_are_running(self):
     return any(self.for_each_impalad(lambda i: i.queries_are_running()))
 
@@ -676,6 +679,11 @@ class Impalad(object):
       # TODO: Handle losing the race
       raise e
 
+  def get_version_info(self):
+    with self.cluster.impala.cursor(impalad=self) as cursor:
+      cursor.execute("SELECT version()")
+      return ''.join(cursor.fetchone()).strip()
+
   def shell(self, cmd, timeout_secs=DEFAULT_TIMEOUT):
     return self.cluster.shell(cmd, self.host_name, timeout_secs=timeout_secs)
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/c5a9b43d/tests/stress/concurrent_select.py
----------------------------------------------------------------------
diff --git a/tests/stress/concurrent_select.py b/tests/stress/concurrent_select.py
index 2b45c9c..d77e306 100755
--- a/tests/stress/concurrent_select.py
+++ b/tests/stress/concurrent_select.py
@@ -1654,6 +1654,19 @@ def populate_all_queries(queries, impala, args, runtime_info_path,
   return result
 
 
+def print_version(cluster):
+  """
+  Print the cluster impalad version info to the console sorted by hostname.
+  """
+  def _sorter(i1, i2):
+    return cmp(i1.host_name, i2.host_name)
+
+  version_info = cluster.impala.get_version_info()
+  print("Cluster Impalad Version Info:")
+  for impalad in sorted(version_info.keys(), cmp=_sorter):
+    print("{0}: {1}".format(impalad.host_name, version_info[impalad]))
+
+
 def main():
   from argparse import ArgumentDefaultsHelpFormatter, ArgumentParser
   from random import shuffle
@@ -1851,6 +1864,7 @@ def main():
   impala = cluster.impala
   if impala.find_stopped_impalads():
     impala.restart()
+  print_version(cluster)
   impala.find_and_set_path_to_running_impalad_binary()
   if args.cancel_current_queries and impala.queries_are_running():
     impala.cancel_queries()