You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2018/04/26 16:39:33 UTC

[1/5] impala git commit: IMPALA-6889: Fix race around terminating processes

Repository: impala
Updated Branches:
  refs/heads/2.x 8dc867561 -> a84053ddf


IMPALA-6889: Fix race around terminating processes

A process with empty cmdline can trip up the Process class in our test
code. When a process dies it becomes a zombie until it gets reaped by
its parent.

>From 'man proc':

/proc/[pid]/cmdline
  This read-only file holds the complete command line for the
  process, unless the process is a zombie.  In the latter case,
  there is nothing in this file: that is, a read on this file
  will return 0 characters.  The command-line arguments appear
  in this file as a set of strings separated by null bytes
  ('\0'), with a further null byte after the last string.

To fix this, we take a copy of the cmdline before passing it to the
Process ctor to prevent it from changing.

I couldn't come up with a test for this.

Change-Id: Iecf630e0b71d91469650636e81f940a7bec07113
Reviewed-on: http://gerrit.cloudera.org:8080/10156
Reviewed-by: Michael Brown <mi...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/2.x
Commit: 81bed7af5e1576f59c78defee2cd64d7b0c8dbf7
Parents: 8dc8675
Author: Lars Volker <lv...@cloudera.com>
Authored: Mon Apr 23 11:22:39 2018 -0700
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Apr 26 00:46:21 2018 +0000

----------------------------------------------------------------------
 tests/common/impala_cluster.py | 19 +++++++++++++------
 1 file changed, 13 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/81bed7af/tests/common/impala_cluster.py
----------------------------------------------------------------------
diff --git a/tests/common/impala_cluster.py b/tests/common/impala_cluster.py
index 276c02b..315666b 100644
--- a/tests/common/impala_cluster.py
+++ b/tests/common/impala_cluster.py
@@ -109,12 +109,19 @@ class ImpalaCluster(object):
           if "uid not found" in str(e):
             continue
           raise
-        if process.name == 'impalad' and len(process.cmdline) >= 1:
-          impalads.append(ImpaladProcess(process.cmdline))
-        elif process.name == 'statestored' and len(process.cmdline) >= 1:
-          statestored.append(StateStoreProcess(process.cmdline))
-        elif process.name == 'catalogd' and len(process.cmdline) >= 1:
-          catalogd = CatalogdProcess(process.cmdline)
+        # IMPALA-6889: When a process shuts down and becomes a zombie its cmdline becomes
+        # empty for a brief moment, before it gets reaped by its parent (see man proc). We
+        # copy the cmdline to prevent it from changing between the following checks and
+        # the construction of the *Process objects.
+        cmdline = process.cmdline
+        if len(cmdline) == 0:
+          continue
+        if process.name == 'impalad':
+          impalads.append(ImpaladProcess(cmdline))
+        elif process.name == 'statestored':
+          statestored.append(StateStoreProcess(cmdline))
+        elif process.name == 'catalogd':
+          catalogd = CatalogdProcess(cmdline)
       except psutil.NoSuchProcess, e:
         # A process from get_pid_list() no longer exists, continue.
         LOG.info(e)


[4/5] impala git commit: IMPALA-6927: Remove invalid DCHECK from coordinator backend state

Posted by ta...@apache.org.
IMPALA-6927: Remove invalid DCHECK from coordinator backend state

An invalid DCHECK in Coordinator::BackendState::InstanceStatsToJson()
assumed that the number of fragment instances is equal to the number of
fragments. This is not true when mt_dop > 0, which results in multiple
instances of a fragment on a single node.

The fix is to remove the invalid DCHECK. It served to document the
programmers understanding and the code does not rely on the assumption
being true.

This change adds a test that would fail with the DCHECK in place.

Change-Id: I4ea4d583c6ab3fc788db8e220c0a1891918a823f
Reviewed-on: http://gerrit.cloudera.org:8080/10192
Reviewed-by: Sailesh Mukil <sa...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/2.x
Commit: a84053ddf8cc745d68bfd91bcb2a62764d29cb55
Parents: b101a66
Author: Lars Volker <lv...@cloudera.com>
Authored: Wed Apr 25 12:35:48 2018 -0700
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Apr 26 04:33:16 2018 +0000

----------------------------------------------------------------------
 be/src/runtime/coordinator-backend-state.cc |  1 -
 tests/webserver/test_web_pages.py           | 14 +++++++++++++-
 2 files changed, 13 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/a84053dd/be/src/runtime/coordinator-backend-state.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/coordinator-backend-state.cc b/be/src/runtime/coordinator-backend-state.cc
index c701f8e..303a5a3 100644
--- a/be/src/runtime/coordinator-backend-state.cc
+++ b/be/src/runtime/coordinator-backend-state.cc
@@ -616,7 +616,6 @@ void Coordinator::BackendState::InstanceStatsToJson(Value* value, Document* docu
       elem.second->ToJson(&val, document);
       instance_stats.PushBack(val, document->GetAllocator());
     }
-    DCHECK_EQ(instance_stats.Size(), fragments_.size());
   }
   value->AddMember("instance_stats", instance_stats, document->GetAllocator());
 

http://git-wip-us.apache.org/repos/asf/impala/blob/a84053dd/tests/webserver/test_web_pages.py
----------------------------------------------------------------------
diff --git a/tests/webserver/test_web_pages.py b/tests/webserver/test_web_pages.py
index c736cb3..60deca4 100644
--- a/tests/webserver/test_web_pages.py
+++ b/tests/webserver/test_web_pages.py
@@ -181,9 +181,11 @@ class TestWebPage(ImpalaTestSuite):
     self.get_and_check_status(self.TABLE_METRICS_URL +
       "?name=%s.%s" % (db_name, tbl_name), metric, ports_to_test=self.CATALOG_TEST_PORT)
 
-  def __run_query_and_get_debug_page(self, query, page_url):
+  def __run_query_and_get_debug_page(self, query, page_url, query_options=None):
     """Runs a query to obtain the content of the debug page pointed to by page_url, then
     cancels the query."""
+    if query_options:
+      self.client.set_configuration(query_options)
     query_handle =  self.client.execute_async(query)
     response_json = ""
     try:
@@ -225,6 +227,16 @@ class TestWebPage(ImpalaTestSuite):
       else:
         assert 'backend_instances' not in response_json
 
+  def test_backend_instances_mt_dop(self, unique_database):
+    """Test that accessing /query_finstances does not crash the backend when running with
+    mt_dop."""
+    # vector.get_value('exec_option')['mt_dop'] = 4
+    QUERY = "select * from tpch.lineitem where l_orderkey < 3"
+    QUERY_OPTIONS = dict(mt_dop=4)
+    response_json = self.__run_query_and_get_debug_page(QUERY, self.QUERY_FINSTANCES_URL,
+                                                        QUERY_OPTIONS)
+    assert len(response_json['backend_instances']) > 0
+
   def test_io_mgr_threads(self):
     """Test that IoMgr threads have readable names. This test assumed that all systems we
     support have a disk called 'sda'."""


[2/5] impala git commit: IMPALA-6913: Simple parser for Impala profile logs.

Posted by ta...@apache.org.
IMPALA-6913: Simple parser for Impala profile logs.

It's hard to find an example of how to parse the profiles in Impala test
logs, so I've added it to the bin/ directory. The parser's not
full-featured, but it may be useful to others. It handles three
common cases of Thrift-encoded profiles that we see.

Change-Id: Ib1174c65b002f9b71eccb0b56f875449f00eae39
Reviewed-on: http://gerrit.cloudera.org:8080/9265
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Reviewed-by: Bharath Vissapragada <bh...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/2.x
Commit: 4f811988f2408a181d34fe31907072014bbeef74
Parents: 4c285b9
Author: Philip Zeyliger <ph...@cloudera.com>
Authored: Thu Feb 8 15:11:29 2018 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Apr 26 04:33:16 2018 +0000

----------------------------------------------------------------------
 bin/parse-thrift-profile.py | 79 ++++++++++++++++++++++++++++++++++++++++
 1 file changed, 79 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/4f811988/bin/parse-thrift-profile.py
----------------------------------------------------------------------
diff --git a/bin/parse-thrift-profile.py b/bin/parse-thrift-profile.py
new file mode 100755
index 0000000..5f8485f
--- /dev/null
+++ b/bin/parse-thrift-profile.py
@@ -0,0 +1,79 @@
+#!/usr/bin/env impala-python
+#
+# 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.
+#
+# Parses a base64-encoded profile provided via stdin. It accepts
+# three common formats:
+#
+# 1. Impala profile logs of the format
+#    "<ts> <queryid> <base64encoded, compressed thrift profile>"
+# 2. Just the base64-encoded compressed thrift profile
+# 3. Base-64 encoded uncompressed thrift profile.
+#
+# In all cases, the script expects one profile per line.
+#
+# For example:
+#
+# $ cat logs/cluster_test/custom_cluster_tests/profiles/impala_profile_log \
+#      | head -n 1 | awk '{ print $3 }' | parse-profile.py
+# TRuntimeProfileTree(nodes=[TRuntimeProfileNode(info_strings_display_order=....
+#
+# or
+#
+# $ bin/parse-thrift-profile.py logs/custom_cluster_tests/profiles/impala_profile_log_1.1-1523657191158
+# 2018-04-13T15:06:34.144000 e44af7f93edb8cd6:1b1f801600000000 TRuntimeProfileTree(nodes=[TRuntimeProf...
+
+
+from thrift.protocol import TCompactProtocol
+from thrift.TSerialization import deserialize
+from RuntimeProfile.ttypes import TRuntimeProfileTree
+
+import base64
+import datetime
+import sys
+import zlib
+
+if len(sys.argv) == 1 or sys.argv[1] == "-":
+  input_data = sys.stdin
+elif len(sys.argv) == 2:
+  input_data = file(sys.argv[1])
+else:
+  print >> sys.stderr, "Usage: %s [file]" % (sys.argv[0],)
+  sys.exit(1)
+
+for line in input_data:
+  space_separated = line.split(" ")
+  if len(space_separated) == 3:
+    ts = int(space_separated[0])
+    print datetime.datetime.fromtimestamp(ts/1000.0).isoformat(), space_separated[1],
+    base64_encoded = space_separated[2]
+  elif len(space_separated) == 1:
+    base64_encoded = space_separated[0]
+  else:
+    raise Exception("Unexpected line: " + line)
+  possibly_compressed = base64.b64decode(base64_encoded)
+  # Handle both compressed and uncompressed Thrift profiles
+  try:
+    thrift = zlib.decompress(possibly_compressed)
+  except zlib.error:
+    thrift = possibly_compressed
+
+  tree = TRuntimeProfileTree()
+  deserialize(tree, thrift, protocol_factory=TCompactProtocol.TCompactProtocolFactory())
+  tree.validate()
+  print tree


[5/5] impala git commit: IMPALA-6905: support regexes with more verifiers

Posted by ta...@apache.org.
IMPALA-6905: support regexes with more verifiers

Support row_regex and other lines for the subset and superset verifiers,
which previously assumed that lines in the actual and expected had to
match exactly.

Use in test_stats_extrapolation to make the test more robust to
irrelevant changes in the explain plan.

Testing:
Manually modified a superset and a subset test to check that tests fail
as expected.

Change-Id: Ia7a28d421c8e7cd84b14d07fcb71b76449156409
Reviewed-on: http://gerrit.cloudera.org:8080/10155
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/2.x
Commit: 4c285b98eec32e854a35fc1649bd13085e7eecd7
Parents: 81bed7a
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Mon Apr 23 10:31:39 2018 -0700
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Apr 26 04:33:16 2018 +0000

----------------------------------------------------------------------
 .../queries/QueryTest/stats-extrapolation.test  | 26 +++++-----
 tests/common/test_result_verifier.py            | 52 +++++++++++++++-----
 2 files changed, 53 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/4c285b98/testdata/workloads/functional-query/queries/QueryTest/stats-extrapolation.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/stats-extrapolation.test b/testdata/workloads/functional-query/queries/QueryTest/stats-extrapolation.test
index 7da7baf..42f4d50 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/stats-extrapolation.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/stats-extrapolation.test
@@ -14,8 +14,7 @@ explain select id from alltypes;
 '     table: rows=unavailable size=unavailable'
 '     partitions: 0/12 rows=unavailable'
 '     columns: unavailable'
-'   extrapolated-rows=unavailable'
-'   mem-estimate=16.00MB mem-reservation=0B'
+row_regex:.* extrapolated-rows=unavailable.*
 '   tuple-ids=0 row-size=4B cardinality=unavailable'
 ---- TYPES
 STRING
@@ -68,8 +67,8 @@ row_regex:.*partitions=12/12 files=12 size=.*
 row_regex:.*table: rows=3650 size=.*
 '     partitions: 0/12 rows=unavailable'
 '     columns: all'
-'   extrapolated-rows=3650'
-'   mem-estimate=16.00MB mem-reservation=0B'
+row_regex:.* extrapolated-rows=3650.*
+row_regex:.*mem-estimate=.* mem-reservation=.*
 '   tuple-ids=0 row-size=4B cardinality=3650'
 ---- TYPES
 STRING
@@ -93,8 +92,8 @@ row_regex:.*partitions=3/12 files=3 size=.*
 row_regex:.*table: rows=3650 size=.*
 '     partitions: 0/3 rows=unavailable'
 '     columns: all'
-'   extrapolated-rows=904'
-'   mem-estimate=16.00MB mem-reservation=0B'
+row_regex:.* extrapolated-rows=904.*
+row_regex:.*mem-estimate=.* mem-reservation=.*
 '   tuple-ids=0 row-size=4B cardinality=904'
 ---- TYPES
 STRING
@@ -119,8 +118,8 @@ row_regex:.*partitions=12/12 files=24 size=.*
 row_regex:.*table: rows=3650 size=.*
 '     partitions: 0/12 rows=unavailable'
 '     columns: all'
-'   extrapolated-rows=7300'
-'   mem-estimate=16.00MB mem-reservation=0B'
+row_regex:.* extrapolated-rows=7300.*
+row_regex:.*mem-estimate=.* mem-reservation=.*
 '   tuple-ids=0 row-size=4B cardinality=7300'
 ---- TYPES
 STRING
@@ -146,8 +145,8 @@ row_regex:.*partitions=12/24 files=12 size=.*
 row_regex:.*table: rows=3650 size=.*
 '     partitions: 0/12 rows=unavailable'
 '     columns: all'
-'   extrapolated-rows=3651'
-'   mem-estimate=16.00MB mem-reservation=0B'
+row_regex:.* extrapolated-rows=3651.*
+row_regex:.*mem-estimate=.* mem-reservation=.*
 '   tuple-ids=0 row-size=4B cardinality=3651'
 ---- TYPES
 STRING
@@ -172,8 +171,8 @@ row_regex:.*partitions=12/24 files=12 size=.*
 row_regex:.*table: rows=10950 size=.*
 '     partitions: 0/12 rows=unavailable'
 '     columns: all'
-'   extrapolated-rows=3651'
-'   mem-estimate=16.00MB mem-reservation=0B'
+row_regex:.* extrapolated-rows=3651
+row_regex:.*mem-estimate=.* mem-reservation=.*
 '   tuple-ids=0 row-size=4B cardinality=3651'
 ---- TYPES
 STRING
@@ -187,8 +186,7 @@ explain select id from alltypes;
 '     table: rows=unavailable size=unavailable'
 '     partitions: 0/24 rows=unavailable'
 '     columns: unavailable'
-'   extrapolated-rows=unavailable'
-'   mem-estimate=16.00MB mem-reservation=0B'
+row_regex:.* extrapolated-rows=unavailable.*
 '   tuple-ids=0 row-size=4B cardinality=unavailable'
 ---- TYPES
 STRING

http://git-wip-us.apache.org/repos/asf/impala/blob/4c285b98/tests/common/test_result_verifier.py
----------------------------------------------------------------------
diff --git a/tests/common/test_result_verifier.py b/tests/common/test_result_verifier.py
index a22e0d4..2e8bbb1 100644
--- a/tests/common/test_result_verifier.py
+++ b/tests/common/test_result_verifier.py
@@ -63,6 +63,18 @@ class QueryTestResult(object):
   def __str__(self):
     return '\n'.join(['%s' % row for row in self.rows])
 
+  def separate_rows(self):
+    """Returns rows that are literal rows and rows that are not literals (e.g. regex)
+    in two lists."""
+    literal_rows = []
+    non_literal_rows = []
+    for row in self.rows:
+      if row.regex is None:
+        literal_rows.append(row)
+      else:
+        non_literal_rows.append(row)
+    return (literal_rows, non_literal_rows)
+
 
 # Represents a row in a result set
 class ResultRow(object):
@@ -207,24 +219,42 @@ def assert_args_not_none(*args):
   for arg in args:
     assert arg is not None
 
-def convert_results_to_sets(expected_results, actual_results):
-  assert_args_not_none(expected_results, actual_results)
-  expected_set = set(map(str, expected_results.rows))
-  actual_set = set(map(str, actual_results.rows))
-  return expected_set, actual_set
-
 def verify_query_result_is_subset(expected_results, actual_results):
   """Check whether the results in expected_results are a subset of the results in
   actual_results. This uses set semantics, i.e. any duplicates are ignored."""
-  expected_set, actual_set = convert_results_to_sets(expected_results, actual_results)
-  assert expected_set <= actual_set
+  expected_literals, expected_non_literals = expected_results.separate_rows()
+  expected_literal_strings = set([str(row) for row in expected_literals])
+  actual_literal_strings = set([str(row) for row in actual_results.rows])
+  # Expected literal strings must all be present in the actual strings.
+  assert expected_literal_strings <= actual_literal_strings
+  # Expected patterns must be present in the actual strings.
+  for expected_row in expected_non_literals:
+    matched = False
+    for actual_row in actual_results.rows:
+      if actual_row == expected_row:
+        matched = True
+        break
+    assert matched, "Could not find expected row {0} in actual rows:\n{1}".format(
+        str(expected_row), str(actual_results))
 
 def verify_query_result_is_superset(expected_results, actual_results):
   """Check whether the results in expected_results are a superset of the results in
   actual_results. This uses set semantics, i.e. any duplicates are ignored."""
-  expected_set, actual_set = convert_results_to_sets(expected_results, actual_results)
-  assert expected_set >= actual_set
-
+  expected_literals, expected_non_literals = expected_results.separate_rows()
+  expected_literal_strings = set([str(row) for row in expected_literals])
+  # Check that all actual rows are present in either expected_literal_strings or
+  # expected_non_literals.
+  for actual_row in actual_results.rows:
+    if str(actual_row) in expected_literal_strings:
+      # Matched to a literal string
+      continue
+    matched = False
+    for expected_row in expected_non_literals:
+      if actual_row == expected_row:
+        matched = True
+        break
+    assert matched, "Could not find actual row {0} in expected rows:\n{1}".format(
+        str(actual_row), str(expected_results))
 
 def verify_query_result_is_equal(expected_results, actual_results):
   assert_args_not_none(expected_results, actual_results)


[3/5] impala git commit: IMPALA-6906: disable test that depends on memory estimates on S3

Posted by ta...@apache.org.
IMPALA-6906: disable test that depends on memory estimates on S3

S3 divides up scan ranges into synthetic blocks smaller than the
equivalent HDFS blocks, which in turn affects the memory estimate
calculation, so the test that was tuned for HDFS does not work
in the same way as S3.

The test is exercising an admission control code path that is
independent of the filesystem, so we don't gain important coverage by
running this on S3.

ADLS can have similar block size issues, so skip that too.

Change-Id: Ida763a402203286c02ad3cbcbed5336c70abef7c
Reviewed-on: http://gerrit.cloudera.org:8080/10207
Reviewed-by: Dan Hecht <dh...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/2.x
Commit: b101a66c4450a1c2653630214dec395fc5552112
Parents: 4f81198
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Wed Apr 25 13:49:45 2018 -0700
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Apr 26 04:33:16 2018 +0000

----------------------------------------------------------------------
 tests/custom_cluster/test_admission_controller.py | 8 +++++++-
 1 file changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/b101a66c/tests/custom_cluster/test_admission_controller.py
----------------------------------------------------------------------
diff --git a/tests/custom_cluster/test_admission_controller.py b/tests/custom_cluster/test_admission_controller.py
index 85784dd..e633d44 100644
--- a/tests/custom_cluster/test_admission_controller.py
+++ b/tests/custom_cluster/test_admission_controller.py
@@ -30,6 +30,9 @@ from tests.beeswax.impala_beeswax import ImpalaBeeswaxException
 from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
 from tests.common.environ import specific_build_type_timeout, IMPALAD_BUILD
 from tests.common.impala_test_suite import ImpalaTestSuite
+from tests.common.skip import (
+    SkipIfS3,
+    SkipIfADLS)
 from tests.common.test_dimensions import (
     create_single_exec_option_dimension,
     create_uncompressed_text_dimension)
@@ -382,6 +385,8 @@ class TestAdmissionController(TestAdmissionControllerBase, HS2TestSuite):
       assert re.search("Rejected query from pool default-pool: request memory needed "
           ".* is greater than pool max mem resources 10.00 MB", str(ex))
 
+  @SkipIfS3.hdfs_block_size
+  @SkipIfADLS.hdfs_block_size
   @pytest.mark.execute_serially
   @CustomClusterTestSuite.with_args(
       impalad_args=impalad_admission_ctrl_flags(max_requests=1, max_queued=1,
@@ -389,7 +394,8 @@ class TestAdmissionController(TestAdmissionControllerBase, HS2TestSuite):
       statestored_args=_STATESTORED_ARGS)
   def test_memory_rejection(self, vector):
     """Test that rejection of queries based on reservation and estimates works as
-    expected."""
+    expected. The test depends on scanner memory estimates, which different on remote
+    filesystems with different (synthetic) block sizes."""
     # Test that the query will be rejected by admission control if:
     # a) the largest per-backend min buffer reservation is larger than the query mem limit
     # b) the largest per-backend min buffer reservation is larger than the