[impala] 02/04: IMPALA-10427: Remove SkipIfS3.eventually_consistent pytest marker

2021-01-08 Thread tarmstrong
This is an automated email from the ASF dual-hosted git repository.

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

commit 35bae939abc5534098931221813cb5d500b68993
Author: Joe McDonnell 
AuthorDate: Wed Jan 6 13:58:14 2021 -0800

IMPALA-10427: Remove SkipIfS3.eventually_consistent pytest marker

These tests were disabled due to S3's eventually consistent
behavior. Now that S3 is strongly consistent, these tests do
not need to be disabled.

Testing:
 - Ran s3 core job

Change-Id: Ie9041f530bf3a818f8954b31a3d01d9f6753d7d4
Reviewed-on: http://gerrit.cloudera.org:8080/16931
Reviewed-by: Tim Armstrong 
Tested-by: Impala Public Jenkins 
---
 tests/common/skip.py|  2 --
 tests/metadata/test_compute_stats.py| 10 --
 tests/query_test/test_aggregation.py|  1 -
 tests/query_test/test_insert.py |  1 -
 tests/query_test/test_insert_parquet.py |  1 -
 tests/query_test/test_insert_permutation.py |  1 -
 tests/query_test/test_nested_types.py   |  2 --
 7 files changed, 18 deletions(-)

diff --git a/tests/common/skip.py b/tests/common/skip.py
index 21bbfdd..92cf18c 100644
--- a/tests/common/skip.py
+++ b/tests/common/skip.py
@@ -59,8 +59,6 @@ class SkipIfS3:
   hbase = pytest.mark.skipif(IS_S3, reason="HBase not started with S3")
   qualified_path = pytest.mark.skipif(IS_S3,
   reason="Tests rely on HDFS qualified paths, IMPALA-1872")
-  eventually_consistent = pytest.mark.skipif(IS_S3,
-  reason="Flakiness on account of S3 eventual consistency.")
   iceberg = pytest.mark.skipif(IS_S3,
   reason="Currently Iceberg is only supported on HDFS.")
   variable_listing_times = pytest.mark.skipif(IS_S3,
diff --git a/tests/metadata/test_compute_stats.py 
b/tests/metadata/test_compute_stats.py
index 5455d60..58754a9 100644
--- a/tests/metadata/test_compute_stats.py
+++ b/tests/metadata/test_compute_stats.py
@@ -51,12 +51,10 @@ class TestComputeStats(ImpalaTestSuite):
 create_uncompressed_text_dimension(cls.get_workload()))
 
   @SkipIfLocal.hdfs_blocks
-  @SkipIfS3.eventually_consistent
   def test_compute_stats(self, vector, unique_database):
 self.run_test_case('QueryTest/compute-stats', vector, unique_database)
 
   @SkipIfLocal.hdfs_blocks
-  @SkipIfS3.eventually_consistent
   def test_compute_stats_avro(self, vector, unique_database, 
cluster_properties):
 if cluster_properties.is_catalog_v2_cluster():
   # IMPALA-7308: changed behaviour of various Avro edge cases 
significantly in the
@@ -67,29 +65,24 @@ class TestComputeStats(ImpalaTestSuite):
   self.run_test_case('QueryTest/compute-stats-avro', vector, 
unique_database)
 
   @SkipIfLocal.hdfs_blocks
-  @SkipIfS3.eventually_consistent
   def test_compute_stats_decimal(self, vector, unique_database):
 # Test compute stats on decimal columns separately so we can vary between 
platforms
 # with and without write support for decimals (Hive < 0.11 and >= 0.11).
 self.run_test_case('QueryTest/compute-stats-decimal', vector, 
unique_database)
 
   @SkipIfLocal.hdfs_blocks
-  @SkipIfS3.eventually_consistent
   def test_compute_stats_date(self, vector, unique_database):
 # Test compute stats on date columns separately.
 self.run_test_case('QueryTest/compute-stats-date', vector, unique_database)
 
-  @SkipIfS3.eventually_consistent
   def test_compute_stats_incremental(self, vector, unique_database):
 self.run_test_case('QueryTest/compute-stats-incremental', vector, 
unique_database)
 
-  @SkipIfS3.eventually_consistent
   def test_compute_stats_complextype_warning(self, vector, unique_database):
 self.run_test_case('QueryTest/compute-stats-complextype-warning', vector,
 unique_database)
 
   @pytest.mark.execute_serially
-  @SkipIfS3.eventually_consistent
   def test_compute_stats_many_partitions(self, vector):
 # To cut down on test execution time, only run the compute stats test 
against many
 # partitions if performing an exhaustive test run.
@@ -97,7 +90,6 @@ class TestComputeStats(ImpalaTestSuite):
 self.run_test_case('QueryTest/compute-stats-many-partitions', vector)
 
   @pytest.mark.execute_serially
-  @SkipIfS3.eventually_consistent
   def test_compute_stats_keywords(self, vector):
 """IMPALA-1055: Tests compute stats with a db/table name that are 
keywords."""
 self.execute_query("drop database if exists `parquet` cascade")
@@ -109,7 +101,6 @@ class TestComputeStats(ImpalaTestSuite):
 finally:
   self.cleanup_db("parquet")
 
-  @SkipIfS3.eventually_consistent
   def test_compute_stats_compression_codec(self, vector, unique_database):
 """IMPALA-8254: Tests that running compute stats with compression_codec set
 should not throw an error."""
@@ -292,7 +283,6 @@ class TestComputeStats(ImpalaTestSuite):
 self.create_load_test_corrupt_stats(self, unique_database, 
create_load_stmts,
  

[impala] branch master updated (799bc22 -> ab6b796)

2021-01-08 Thread tarmstrong
This is an automated email from the ASF dual-hosted git repository.

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


from 799bc22  IMPALA-10424: Fix race on not_admitted_reason in 
AdmissionController
 new 44bade8  IMPALA-10091: [DOCS] add REFRESH_UPDATED_HMS_PARTITIONS query 
option
 new 35bae93  IMPALA-10427: Remove SkipIfS3.eventually_consistent pytest 
marker
 new 425e424  IMPALA-9687 Improve estimates for number of hosts in Kudu 
plans
 new ab6b796  IMPALA-10027: configurable default anonymous user

The 4 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/common/global-flags.cc  |  4 +-
 be/src/service/impala-beeswax-server.cc| 11 ++-
 be/src/service/impala-hs2-server.cc|  5 +-
 docs/impala.ditamap|  1 +
 docs/topics/impala_refresh_updated_hms.xml | 72 +++
 .../org/apache/impala/planner/KuduScanNode.java| 82 --
 .../java/org/apache/impala/service/JdbcTest.java   |  6 +-
 .../queries/PlannerTest/kudu-selectivity.test  |  2 +-
 .../queries/PlannerTest/tpch-kudu.test | 20 +++---
 tests/common/skip.py   |  2 -
 tests/custom_cluster/test_admission_controller.py  |  2 +-
 tests/metadata/test_compute_stats.py   | 10 ---
 tests/query_test/test_aggregation.py   |  1 -
 tests/query_test/test_insert.py|  1 -
 tests/query_test/test_insert_parquet.py|  1 -
 tests/query_test/test_insert_permutation.py|  1 -
 tests/query_test/test_nested_types.py  |  2 -
 tests/shell/test_shell_commandline.py  | 24 +++
 18 files changed, 203 insertions(+), 44 deletions(-)
 create mode 100644 docs/topics/impala_refresh_updated_hms.xml



[impala] 04/04: IMPALA-10027: configurable default anonymous user

2021-01-08 Thread tarmstrong
This is an automated email from the ASF dual-hosted git repository.

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

commit ab6b7960db3181096134b8aa46cb90baff6df006
Author: Tim Armstrong 
AuthorDate: Wed Dec 23 14:13:38 2020 -0800

IMPALA-10027: configurable default anonymous user

A username can be determined for a session via two mechanisms:
* In a secure env, the user is authenticated by LDAP or Kerberos
* In an unsecure env, the client specifies the user name, either
  as a parameter to the OpenSession API (HS2) or as a parameter
  to the first query run (beeswax)

This patch affects what happens if neither of the above mechanisms
is used. Previously we would end up with the username being an
empty string, but this makes Ranger unhappy. Hive uses the name
"anonymous" in this situation, so we change Impala's behaviour too.

This is configurable by -anonymous_user_name. -anonymous_user_name=
reverts to the old behaviour.

Test
* Add an end-to-end test that exercises this via impala-shell for
  HS2, HS2-HTTP and beeswax protocols.
* Tweak a couple of existing tests that depended on the previous
  behavior.

Change-Id: I6db491231fa22484aed476062b8fe4c8f69130b0
Reviewed-on: http://gerrit.cloudera.org:8080/16902
Reviewed-by: Impala Public Jenkins 
Tested-by: Impala Public Jenkins 
---
 be/src/common/global-flags.cc  |  4 +++-
 be/src/service/impala-beeswax-server.cc| 11 --
 be/src/service/impala-hs2-server.cc|  5 -
 .../java/org/apache/impala/service/JdbcTest.java   |  6 +++---
 tests/custom_cluster/test_admission_controller.py  |  2 +-
 tests/shell/test_shell_commandline.py  | 24 ++
 6 files changed, 44 insertions(+), 8 deletions(-)

diff --git a/be/src/common/global-flags.cc b/be/src/common/global-flags.cc
index 1b8c16f..794c14d 100644
--- a/be/src/common/global-flags.cc
+++ b/be/src/common/global-flags.cc
@@ -66,7 +66,9 @@ DEFINE_bool(skip_external_kerberos_auth, false,
 "(Advanced) skip kerberos authentication for incoming external connections 
to "
 "this daemon, e.g. clients connecting to the HS2 interface. Only has an 
effect "
 "if --principal is set, i.e. Kerberos is enabled.");
-
+DEFINE_string(anonymous_user_name, "anonymous",
+"Default username used when a client connects to an unsecured impala 
daemon and "
+"does not specify a username.");
 
 static const string mem_limit_help_msg = "Limit on process memory consumption. 
"
 "Includes the JVM's memory consumption only if --mem_limit_includes_jvm is 
true. "
diff --git a/be/src/service/impala-beeswax-server.cc 
b/be/src/service/impala-beeswax-server.cc
index 4d14903..7eb0094 100644
--- a/be/src/service/impala-beeswax-server.cc
+++ b/be/src/service/impala-beeswax-server.cc
@@ -50,6 +50,7 @@ using namespace beeswax;
   } while (false)
 
 DECLARE_bool(ping_expose_webserver_url);
+DECLARE_string(anonymous_user_name);
 
 namespace impala {
 
@@ -108,7 +109,10 @@ void ImpalaServer::executeAndWait(beeswax::QueryHandle& 
beeswax_handle,
 // transport, the username may be known at that time. If the username 
hasn't been set
 // yet, set it now.
 lock_guard l(session->lock);
-if (session->connected_user.empty()) session->connected_user = 
query.hadoop_user;
+if (session->connected_user.empty()) {
+  session->connected_user = query.hadoop_user.empty() ?
+  FLAGS_anonymous_user_name : query.hadoop_user;
+}
   }
 
   // raise Syntax error or access violation; it's likely to be syntax/analysis 
error
@@ -509,7 +513,10 @@ Status ImpalaServer::QueryToTQueryContext(const Query& 
query,
   // transport, the username may be known at that time. If the username 
hasn't been
   // set yet, set it now.
   lock_guard l(session->lock);
-  if (session->connected_user.empty()) session->connected_user = 
query.hadoop_user;
+  if (session->connected_user.empty()) {
+session->connected_user = query.hadoop_user.empty() ?
+FLAGS_anonymous_user_name : query.hadoop_user;
+  }
   query_ctx->client_request.query_options = session->QueryOptions();
   set_query_options_mask = session->set_query_options_mask;
 }
diff --git a/be/src/service/impala-hs2-server.cc 
b/be/src/service/impala-hs2-server.cc
index 9cd55bb..b38a254 100644
--- a/be/src/service/impala-hs2-server.cc
+++ b/be/src/service/impala-hs2-server.cc
@@ -94,6 +94,7 @@ DECLARE_int32(webserver_port);
 DECLARE_int32(idle_session_timeout);
 DECLARE_int32(disconnected_session_timeout);
 DECLARE_bool(ping_expose_webserver_url);
+DECLARE_string(anonymous_user_name);
 
 namespace impala {
 
@@ -329,8 +330,10 @@ void ImpalaServer::OpenSession(TOpenSessionResp& 
return_val,
   Status status = AuthorizeProxyUser(state->connected_user, 
state->do_as_user);
   

[impala] 01/04: IMPALA-10091: [DOCS] add REFRESH_UPDATED_HMS_PARTITIONS query option

2021-01-08 Thread tarmstrong
This is an automated email from the ASF dual-hosted git repository.

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

commit 44bade8e7f14d22b6a16650d54dc9bee62caeb50
Author: Shajini Thayasingh 
AuthorDate: Tue Jan 5 12:49:21 2021 -0800

IMPALA-10091: [DOCS] add REFRESH_UPDATED_HMS_PARTITIONS query option

remove trailing spaces
added this new query option for Impala 4.0

Change-Id: I95b31b33f99073c57752e66eaf0f34facf511fc6
Reviewed-on: http://gerrit.cloudera.org:8080/16925
Reviewed-by: Vihang Karajgaonkar 
Tested-by: Impala Public Jenkins 
---
 docs/impala.ditamap|  1 +
 docs/topics/impala_refresh_updated_hms.xml | 72 ++
 2 files changed, 73 insertions(+)

diff --git a/docs/impala.ditamap b/docs/impala.ditamap
index d6aeedd..cf7e3b6 100644
--- a/docs/impala.ditamap
+++ b/docs/impala.ditamap
@@ -230,6 +230,7 @@ under the License.
   
   
   
+  
   
   
   
diff --git a/docs/topics/impala_refresh_updated_hms.xml 
b/docs/topics/impala_refresh_updated_hms.xml
new file mode 100644
index 000..779c7d2
--- /dev/null
+++ b/docs/topics/impala_refresh_updated_hms.xml
@@ -0,0 +1,72 @@
+
+
+
+
+
+ REFRESH_UPDATED_HMS_PARTITIONS Query Option
+
+ 
+
+  REFRESH_UPDATED_HMS_PARTITIONS
+
+ 
+
+ 
+  
+   
+   
+   
+   
+   
+  
+ 
+
+ 
+
+  
+   As the name implies the query option 
REFRESH_UPDATED_HMS_PARTITIONS is used
+   to refresh any updated HMS partitions.
+  
+
+  This option is disabled by default so that the performance is not 
compromised when refreshing a
+   table. However, for certain corner case scenarios refresh table command 
does not detect changed
+   partitions. In case of the default refresh, catalogd detects, adds any new 
partitions and removes
+   any partitions which are not present in HMS anymore. However, it does not 
update any partitions
+   that changed (eg. change of location). When this query option is enabled, 
the refresh table
+   command will detect certain changes to the partitions and update them 
accordingly. Currently,
+   catalogd will update the partitions if any of the following 
StorageDescriptor properties have
+   been modified. 1. Partition Location 2. Partition Fileformat. 3 SerdeInfo. 
4. Partition schema
+   changes. 5. Partition bucketInfo.
+
+  
+   Type:BOOLEAN; recognized values are 1 and 0, or 
true and false; any other
+   value will be interpreted as false.
+  
+
+  
+   Default:FALSE (shown as 0 in output of SET 
statement).
+  
+
+  Added in:
+   
+
+  
+
+



[impala] 03/04: IMPALA-9687 Improve estimates for number of hosts in Kudu plans

2021-01-08 Thread tarmstrong
This is an automated email from the ASF dual-hosted git repository.

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

commit 425e424b37f081ba1262f4f39298c6c13910aba8
Author: Akos Kovacs 
AuthorDate: Tue Dec 8 23:51:09 2020 +0100

IMPALA-9687 Improve estimates for number of hosts in Kudu plans

In some cases Kudu plans could contain more hosts than the actual number of 
executors.
This commit fixes it by capping the number of hosts at the number of 
executors,
and determining which executors have local scan ranges.

Testing:
 - Ran core tests

Updated Kudu planner tests where the memory estimates changed.

Change-Id: I72e341597e980fb6a7e3792905b942ddf5797d03
Reviewed-on: http://gerrit.cloudera.org:8080/16880
Reviewed-by: Impala Public Jenkins 
Tested-by: Impala Public Jenkins 
---
 .../org/apache/impala/planner/KuduScanNode.java| 82 --
 .../queries/PlannerTest/kudu-selectivity.test  |  2 +-
 .../queries/PlannerTest/tpch-kudu.test | 20 +++---
 3 files changed, 86 insertions(+), 18 deletions(-)

diff --git a/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java 
b/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
index 613c40b..e4a2d0c 100644
--- a/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
@@ -19,9 +19,11 @@ package org.apache.impala.planner;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.ListIterator;
+import java.util.Map;
 import java.util.Set;
 
 import org.apache.impala.analysis.Analyzer;
@@ -54,6 +56,7 @@ import org.apache.impala.thrift.TScanRangeLocation;
 import org.apache.impala.thrift.TScanRangeLocationList;
 import org.apache.impala.thrift.TScanRangeSpec;
 import org.apache.impala.util.KuduUtil;
+import org.apache.impala.util.ExecutorMembershipSnapshot;
 import org.apache.kudu.ColumnSchema;
 import org.apache.kudu.Schema;
 import org.apache.kudu.client.KuduClient;
@@ -277,16 +280,81 @@ public class KuduScanNode extends ScanNode {
 return computeCombinedSelectivity(allConjuncts);
   }
 
+  /**
+   * Estimate the number of impalad nodes that this scan node will execute on 
(which is
+   * ultimately determined by the scheduling done by the backend's Scheduler).
+   * Assume that scan ranges that can be scheduled locally will be, and that 
scan
+   * ranges that cannot will be round-robined across the cluster.
+   */
+  protected void computeNumNodes(Analyzer analyzer) {
+ExecutorMembershipSnapshot cluster = 
ExecutorMembershipSnapshot.getCluster();
+final int maxInstancesPerNode = getMaxInstancesPerNode(analyzer);
+final int maxPossibleInstances = cluster.numExecutors() * 
maxInstancesPerNode;
+int totalNodes = 0;
+int totalInstances = 0;
+int numLocalRanges = 0;
+int numRemoteRanges = 0;
+// Counts the number of local ranges, capped at maxInstancesPerNode.
+Map localRangeCounts = new HashMap<>();
+// Sum of the counter values in localRangeCounts.
+int totalLocalParallelism = 0;
+if (scanRangeSpecs_.isSetConcrete_ranges()) {
+  for (TScanRangeLocationList range : scanRangeSpecs_.concrete_ranges) {
+boolean anyLocal = false;
+if (range.isSetLocations()) {
+  for (TScanRangeLocation loc : range.locations) {
+TNetworkAddress address =
+analyzer.getHostIndex().getEntry(loc.getHost_idx());
+if (cluster.contains(address)) {
+  anyLocal = true;
+  // Use the full tserver address (including port) to account for 
the test
+  // minicluster where there are multiple tservers and impalads on 
a single
+  // host.  This assumes that when an impalad is colocated with a 
tserver,
+  // there are the same number of impalads as tservers on this 
host in this
+  // cluster.
+  int count = localRangeCounts.getOrDefault(address, 0);
+  if (count < maxInstancesPerNode) {
+++totalLocalParallelism;
+localRangeCounts.put(address, count + 1);
+  }
+}
+  }
+}
+// This range has at least one replica with a colocated impalad, so 
assume it
+// will be scheduled on one of those nodes.
+if (anyLocal) {
+  ++numLocalRanges;
+} else {
+  ++numRemoteRanges;
+}
+// Approximate the number of nodes that will execute locally assigned 
ranges to
+// be the smaller of the number of locally assigned ranges and the 
number of
+// hosts that hold replica for those ranges.
+int numLocalNodes = Math.min(numLocalRanges, localRangeCounts.size());
+// The remote ranges are round-robined across all the