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 2022/10/13 16:23:28 UTC

[impala] branch master updated (1d62bddb8 -> 35dc24fbc)

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 1d62bddb8 IMPALA-11617: Pool service should be made aware of processing cost limit
     new c3d7f20a8 IMPALA-11226: Add script to simplify resolving minidumps
     new 4638a0e58 IMPALA-11646 IMPALA-11562: Fix test_unsupported_text_compression in s3
     new 35dc24fbc IMPALA-10148: Cleanup cores in TestHooksStartupFail

The 3 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:
 bin/jenkins/finalize.sh                        |  19 ++
 bin/resolve_minidumps.py                       | 376 +++++++++++++++++++++++++
 tests/authorization/test_provider.py           |  35 +--
 tests/common/custom_cluster_test_suite.py      |  37 ++-
 tests/custom_cluster/test_query_event_hooks.py |  20 +-
 tests/metadata/test_partition_metadata.py      |  20 +-
 6 files changed, 441 insertions(+), 66 deletions(-)
 create mode 100755 bin/resolve_minidumps.py


[impala] 03/03: IMPALA-10148: Cleanup cores in TestHooksStartupFail

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 35dc24fbc8bb5ea27cc8e887512672070a05bcb7
Author: Michael Smith <mi...@cloudera.com>
AuthorDate: Thu Oct 6 12:14:42 2022 -0700

    IMPALA-10148: Cleanup cores in TestHooksStartupFail
    
    Generalizes coredump cleanup and expecting startup failure from
    test_provider.py and uses it in test_query_event_hooks.py
    TestHooksStartupFail to ensure core dumps are cleaned up.
    
    Testing: ran changed tests, observed core files being created and
    cleaned up while they ran. Observed other core files already present
    were not cleaned up, as expected.
    
    Change-Id: Iec32e0acbadd65aa78264594c85ffcd574cf3458
    Reviewed-on: http://gerrit.cloudera.org:8080/19103
    Reviewed-by: Laszlo Gaal <la...@cloudera.com>
    Reviewed-by: Joe McDonnell <jo...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 tests/authorization/test_provider.py           | 35 +-----------------------
 tests/common/custom_cluster_test_suite.py      | 37 +++++++++++++++++++++++---
 tests/custom_cluster/test_query_event_hooks.py | 20 ++------------
 3 files changed, 36 insertions(+), 56 deletions(-)

diff --git a/tests/authorization/test_provider.py b/tests/authorization/test_provider.py
index c80db7849..362ca7ae9 100644
--- a/tests/authorization/test_provider.py
+++ b/tests/authorization/test_provider.py
@@ -17,17 +17,13 @@
 #
 # Client tests for SQL statement authorization
 
-import logging
 import pytest
 import os
 import tempfile
 
-from impala_py_lib.helpers import find_all_files, is_core_dump
 from tests.common.file_utils import assert_file_in_dir_contains
 from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
 
-LOG = logging.getLogger('test_provider')
-
 
 class TestAuthorizationProvider(CustomClusterTestSuite):
   """
@@ -46,6 +42,7 @@ class TestAuthorizationProvider(CustomClusterTestSuite):
 
   @pytest.mark.execute_serially
   @CustomClusterTestSuite.with_args(
+      expect_cores=True,
       impala_log_dir=LOG_DIR,
       impalad_args="--minidump_path={0}"
                    "--server-name=server1 "
@@ -63,33 +60,3 @@ class TestAuthorizationProvider(CustomClusterTestSuite):
                                 "InternalException: Could not parse "
                                 "authorization_provider flag: {0}"
                                 .format(TestAuthorizationProvider.BAD_FLAG))
-
-  def setup_method(self, method):
-    # Make a note of any core files that already exist
-    possible_cores = find_all_files('*core*')
-    self.pre_test_cores = set([f for f in possible_cores if is_core_dump(f)])
-
-    # Explicitly override CustomClusterTestSuite.setup_method() to
-    # allow it to exception, since this testsuite is for cases where
-    # startup fails
-    try:
-      super(TestAuthorizationProvider, self).setup_method(method)
-    except Exception:
-      self._stop_impala_cluster()
-
-  def teardown_method(self, method):
-    try:
-      # The core dumps expected to be generated by this test should be cleaned up
-      possible_cores = find_all_files('*core*')
-      post_test_cores = set([f for f in possible_cores if is_core_dump(f)])
-
-      for f in (post_test_cores - self.pre_test_cores):
-        LOG.info("Cleaned up {core} created by test_invalid_provider_flag".format(core=f))
-        os.remove(f)
-
-      # Explicitly override CustomClusterTestSuite.teardown_method() to
-      # allow it to exception, since it relies on setup_method() having
-      # completed successfully
-      super(TestAuthorizationProvider, self).teardown_method(method)
-    except Exception:
-      self._stop_impala_cluster()
diff --git a/tests/common/custom_cluster_test_suite.py b/tests/common/custom_cluster_test_suite.py
index 6a831bb28..2bb43e23f 100644
--- a/tests/common/custom_cluster_test_suite.py
+++ b/tests/common/custom_cluster_test_suite.py
@@ -24,6 +24,7 @@ import os.path
 import pipes
 import pytest
 import subprocess
+from impala_py_lib.helpers import find_all_files, is_core_dump
 from subprocess import check_call
 from tests.common.impala_test_suite import ImpalaTestSuite
 from tests.common.impala_cluster import ImpalaCluster
@@ -51,6 +52,7 @@ IMPALA_LOG_DIR = 'impala_log_dir'
 NUM_EXCLUSIVE_COORDINATORS = 'num_exclusive_coordinators'
 STATESTORED_TIMEOUT_S = 'statestored_timeout_s'
 IMPALAD_TIMEOUT_S = 'impalad_timeout_s'
+EXPECT_CORES = 'expect_cores'
 
 # Run with fast topic updates by default to reduce time to first query running.
 DEFAULT_STATESTORE_ARGS = '--statestore_update_frequency_ms=50 \
@@ -104,7 +106,7 @@ class CustomClusterTestSuite(ImpalaTestSuite):
       start_args=None, default_query_options=None,
       impala_log_dir=None, hive_conf_dir=None, cluster_size=None,
       num_exclusive_coordinators=None, kudu_args=None, statestored_timeout_s=None,
-      impalad_timeout_s=None):
+      impalad_timeout_s=None, expect_cores=None):
     """Records arguments to be passed to a cluster by adding them to the decorated
     method's func_dict"""
     def decorate(func):
@@ -131,6 +133,8 @@ class CustomClusterTestSuite(ImpalaTestSuite):
         func.func_dict[STATESTORED_TIMEOUT_S] = statestored_timeout_s
       if impalad_timeout_s is not None:
         func.func_dict[IMPALAD_TIMEOUT_S] = impalad_timeout_s
+      if expect_cores is not None:
+        func.func_dict[EXPECT_CORES] = expect_cores
       return func
     return decorate
 
@@ -177,13 +181,38 @@ class CustomClusterTestSuite(ImpalaTestSuite):
       kwargs["statestored_timeout_s"] = method.func_dict[STATESTORED_TIMEOUT_S]
     if IMPALAD_TIMEOUT_S in method.func_dict:
       kwargs["impalad_timeout_s"] = method.func_dict[IMPALAD_TIMEOUT_S]
-    self._start_impala_cluster(cluster_args, **kwargs)
-    super(CustomClusterTestSuite, self).setup_class()
+
+    if method.func_dict.get(EXPECT_CORES, False):
+      # Make a note of any core files that already exist
+      possible_cores = find_all_files('*core*')
+      self.pre_test_cores = set([f for f in possible_cores if is_core_dump(f)])
+
+      # Explicitly allow startup to exception, since startup is expected to fail
+      try:
+        self._start_impala_cluster(cluster_args, **kwargs)
+        pytest.fail("cluster startup should have failed")
+      except Exception:
+        self._stop_impala_cluster()
+    else:
+      self._start_impala_cluster(cluster_args, **kwargs)
+      super(CustomClusterTestSuite, self).setup_class()
 
   def teardown_method(self, method):
     if HIVE_CONF_DIR in method.func_dict:
       self._start_hive_service(None)  # Restart Hive Service using default configs
-    super(CustomClusterTestSuite, self).teardown_class()
+
+    if method.func_dict.get(EXPECT_CORES, False):
+      # The core dumps expected to be generated by this test should be cleaned up
+      possible_cores = find_all_files('*core*')
+      post_test_cores = set([f for f in possible_cores if is_core_dump(f)])
+
+      for f in (post_test_cores - self.pre_test_cores):
+        logging.info(
+            "Cleaned up {core} created by {name}".format(core=f, name=method.__name__))
+        os.remove(f)
+      # Skip teardown_class as setup was skipped.
+    else:
+      super(CustomClusterTestSuite, self).teardown_class()
 
   @classmethod
   def _stop_impala_cluster(cls):
diff --git a/tests/custom_cluster/test_query_event_hooks.py b/tests/custom_cluster/test_query_event_hooks.py
index a884a22f9..36fe36047 100644
--- a/tests/custom_cluster/test_query_event_hooks.py
+++ b/tests/custom_cluster/test_query_event_hooks.py
@@ -79,6 +79,7 @@ class TestHooksStartupFail(CustomClusterTestSuite):
 
   @pytest.mark.execute_serially
   @CustomClusterTestSuite.with_args(
+      expect_cores=True,
       cluster_size=1,
       impalad_timeout_s=5,
       impala_log_dir=LOG_DIR1,
@@ -99,6 +100,7 @@ class TestHooksStartupFail(CustomClusterTestSuite):
 
   @pytest.mark.execute_serially
   @CustomClusterTestSuite.with_args(
+      expect_cores=True,
       cluster_size=1,
       impalad_timeout_s=5,
       impala_log_dir=LOG_DIR2,
@@ -115,21 +117,3 @@ class TestHooksStartupFail(CustomClusterTestSuite):
     self.assert_impalad_log_contains("INFO",
                                      "Unable to instantiate query event hook class {0}"
                                      .format(self.NONEXIST_HOOK), expected_count=-1)
-
-  def setup_method(self, method):
-    # Explicitly override CustomClusterTestSuite.setup_method() to
-    # allow it to exception, since this test suite is for cases where
-    # startup fails
-    try:
-      super(TestHooksStartupFail, self).setup_method(method)
-    except Exception:
-      self._stop_impala_cluster()
-
-  def teardown_method(self, method):
-    # Explicitly override CustomClusterTestSuite.teardown_method() to
-    # allow it to exception, since it relies on setup_method() having
-    # completed successfully
-    try:
-      super(TestHooksStartupFail, self).teardown_method(method)
-    except Exception:
-      self._stop_impala_cluster()


[impala] 01/03: IMPALA-11226: Add script to simplify resolving minidumps

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 c3d7f20a890ad6c142aaee574da51070dee5d44e
Author: Joe McDonnell <jo...@cloudera.com>
AuthorDate: Mon Aug 22 19:57:13 2022 -0700

    IMPALA-11226: Add script to simplify resolving minidumps
    
    This adds the resolve_minidumps.py script to
    simplify resolving minidumps under ideal circumstances.
    This is designed to handle cases where the binary
    and libraries are in identical locations to when
    the minidump was created. This is true for developer
    environments and at the end of Jenkins jobs.
    
    This uses Breakpad's minidump_dump utility to get a
    list of the binaries/libraries that the minidump
    references. It uses that list to dump all the
    symbols to a temporary directory. Then it uses
    the symbols to resolve the minidump.
    
    Since it is dumping symbols for all referenced
    libraries, it resolves symbols to the maximum
    extent possible.
    
    This adds a step to bin/jenkins/finalize.sh to use
    this new script to resolve minidumps. The old method
    can be removed in a subsequent change.
    
    Testing:
     - Ran locally on a minidump generated by sending
       SIGUSR1 to local impalad
     - Tested with a Centos 7 job using Python 3.6
       and verified the minidump output
     - Tested resolving a minidump from a binary with
       compressed debug info
    
    Change-Id: I0f8fdcb8ca89d0904dc8ec69337e3d5dfdd54adf
    Reviewed-on: http://gerrit.cloudera.org:8080/18918
    Reviewed-by: Wenzhe Zhou <wz...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 bin/jenkins/finalize.sh  |  19 +++
 bin/resolve_minidumps.py | 376 +++++++++++++++++++++++++++++++++++++++++++++++
 2 files changed, 395 insertions(+)

diff --git a/bin/jenkins/finalize.sh b/bin/jenkins/finalize.sh
index 0785a9bcf..2057b7306 100755
--- a/bin/jenkins/finalize.sh
+++ b/bin/jenkins/finalize.sh
@@ -59,6 +59,25 @@ if [[ $(find $LOGS_DIR -path "*minidumps*" -name "*dmp") ]]; then
   rm -rf $SYM_DIR
 fi
 
+# Do a second pass over the minidumps with the resolve_minidump.py script.
+# This means that we now generate two JUnitXMLs per minidump. This should
+# be temporary.
+# TODO: Once we verify everything works, we can remove the first loop.
+if [[ $(find $LOGS_DIR -path "*minidumps*" -name "*dmp") ]]; then
+  for minidump in $(find $LOGS_DIR -path "*minidumps*" -name "*dmp"); do
+    # Since this is experimental, use it inside an if so that any error code doesn't
+    # abort this script.
+    if ! bin/resolve_minidumps.py --minidump_file ${minidump} \
+        --output_file ${minidump}_dumpedv2 ; then
+      echo "bin/resolve_minidumps.py failed!"
+    else
+      "${IMPALA_HOME}"/bin/generate_junitxml.py --phase finalize --step minidumpsv2 \
+          --error "Minidump generated: $minidump" \
+          --stderr "resolve_minidumps.py output:\n$(head -n 100 ${minidump}_dumpedv2)"
+    fi
+  done
+fi
+
 function check_for_asan_error {
   ERROR_LOG=${1}
   if grep -q "ERROR: AddressSanitizer:" ${ERROR_LOG} ; then
diff --git a/bin/resolve_minidumps.py b/bin/resolve_minidumps.py
new file mode 100755
index 000000000..0a0ba0c61
--- /dev/null
+++ b/bin/resolve_minidumps.py
@@ -0,0 +1,376 @@
+#!/bin/env python3
+#
+# 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.
+#
+# This script automates symbol resolution for Breakpad minidumps
+# under ideal circumstances. Specifically, it expects all the
+# binaries to be in the same locations as when the minidump
+# was taken. This is often true for minidumps on a developer
+# workstation or at the end of an Impala test job. It finds Breakpad
+# using environment variables from the Impala dev environment,
+# so it must run inside the Impala dev environment.
+# TODO: It may be possible to extend this to Docker images.
+#
+# Within this simple context, this script aims for complete
+# symbol resolution. It uses Breakpad's minidump_dump utility
+# to dump the minidump, then it parses the list of libraries
+# that were used by the binary. It gets the symbols for all
+# those libraries and resolves the minidump.
+#
+# Usage: resolve_minidump.py --minidump_file [file] --output_file [file]
+# (optional -v or --verbose for more output)
+
+import errno
+import logging
+import os
+import re
+import shutil
+import subprocess
+import sys
+import tempfile
+
+from argparse import ArgumentParser
+
+
+class ModuleInfo:
+  def __init__(self, code_file, code_id, debug_file, debug_id):
+    self.code_file = code_file
+    self.code_id = code_id
+    self.debug_file = debug_file
+    self.debug_id = debug_id
+
+
+def read_module_info(minidump_dump_contents):
+  """Read the module information out of the minidump_dump raw contents.
+  This is expecting 'minidump_dump_contents' to be the minidump_dump
+  contents for the minidump split by newlines.
+  This will return a list of ModuleInfo objects.
+  """
+  # Find the module_count
+  for idx, line in enumerate(minidump_dump_contents):
+    if line.strip().startswith("module_count"):
+      module_count = int(line.split("=")[1].strip())
+      break
+
+  # The minidump has a MDRawModule per module and it will have
+  # the same number of MDRawModule dumps as module_count.
+  module_boundaries = []
+  for idx, line in enumerate(minidump_dump_contents):
+    if line.startswith("MDRawModule"):
+      module_boundaries.append(idx)
+
+  if len(module_boundaries) != module_count:
+    logging.error("Failed to parse modules, mismatch in module count "
+                  "({0} != {1})".format(len(module_boundaries), module_count))
+    return None
+
+  # Add one more entry to module_boundaries that is the end of the file
+  # That makes this more of a list of boundaries than the list of
+  # start locations.
+  module_boundaries.append(len(minidump_dump_contents))
+
+  modules = []
+  for module_idx in range(module_count):
+    module_start = module_boundaries[module_idx]
+    module_end = module_boundaries[module_idx + 1]
+
+    # Find the code_file
+    code_file = None
+    code_identifier = None
+    debug_file = None
+    debug_identifier = None
+    for line in minidump_dump_contents[module_start:module_end]:
+      if line.find("code_file") != -1:
+        code_file = line.split("=")[1].strip().strip('"')
+      elif line.find("code_identifier") != -1:
+        code_identifier = line.split("=")[1].strip().strip('"')
+      elif line.find("debug_file") != -1:
+        debug_file = line.split("=")[1].strip().strip('"')
+      elif line.find("debug_identifier") != -1:
+        debug_identifier = line.split("=")[1].strip().strip('"')
+
+    # Important: it is ok for the fields to be the zero-length string.
+    # We just care that they are non-None (i.e. the loop above encountered
+    # them and parsed a value).
+    if code_file is None or code_identifier is None or debug_file is None or \
+       debug_identifier is None:
+      logging.error("Failed to parse dump output, missing fields for MDRawModule "
+                    "{0}".format(module_idx))
+      return None
+
+    # Jars and other files show up in this list, but they have
+    # code identifiers or debug identifiers as all zeros. Skip those,
+    # as there are no symbols to find.
+    if re.fullmatch("[0]+", code_identifier) or re.fullmatch("[0]+", debug_identifier):
+      continue
+
+    # Skip cases where the code identifier or debug identifier are null
+    if len(code_identifier) == 0 or len(debug_identifier) == 0:
+      continue
+
+    # linux-gate.so is a special case, and it is not an actual file on disk.
+    if code_file.startswith("linux-gate.so"):
+      continue
+
+    modules.append(ModuleInfo(code_file, code_identifier, debug_file, debug_identifier))
+
+  return modules
+
+
+def find_breakpad_home():
+  """Locate the Breakpad home directory.
+
+  We try to locate the package in the Impala toolchain folder.
+  """
+  toolchain_packages_home = os.environ.get('IMPALA_TOOLCHAIN_PACKAGES_HOME')
+  if not toolchain_packages_home:
+    logging.error("IMPALA_TOOLCHAIN_PACKAGES_HOME is not set")
+    return None
+
+  if not os.path.isdir(toolchain_packages_home):
+    logging.error("Could not find toolchain packages directory")
+    return None
+  breakpad_version = os.environ.get('IMPALA_BREAKPAD_VERSION')
+  if not breakpad_version:
+    logging.error("Could not determine breakpad version from toolchain")
+    return None
+  breakpad_dir = '{0}/breakpad-{1}'.format(toolchain_packages_home, breakpad_version)
+  if not os.path.isdir(breakpad_dir):
+    logging.error("Could not find breakpad directory")
+    return None
+
+  return breakpad_dir
+
+
+def find_breakpad_binary(binary_name):
+  """Locate the specified Breadpad binary"""
+  breakpad_home = find_breakpad_home()
+  if not breakpad_home:
+    return None
+
+  binary_path = os.path.join(breakpad_home, 'bin', binary_name)
+  if not os.path.isfile(binary_path):
+    logging.error("Could not find {0} executable at {1}".format(binary_name, binary_path))
+    return None
+
+  return binary_path
+
+
+def find_objcopy_binary():
+  """Locate the 'objcopy' binary from Binutils.
+
+  We try to locate the package in the Impala toolchain folder.
+  TODO: Fall back to finding objcopy in the system path.
+  """
+  toolchain_packages_home = os.environ.get('IMPALA_TOOLCHAIN_PACKAGES_HOME')
+  if not toolchain_packages_home:
+    logging.error("IMPALA_TOOLCHAIN_PACKAGES_HOME is not set")
+    return None
+
+  if not os.path.isdir(toolchain_packages_home):
+    logging.error("Could not find toolchain packages directory")
+    return None
+  binutils_version = os.environ.get('IMPALA_BINUTILS_VERSION')
+  if not binutils_version:
+    logging.error("Could not determine binutils version from toolchain")
+    return None
+  binutils_dir = "binutils-{0}".format(binutils_version)
+  objcopy = os.path.join(toolchain_packages_home, binutils_dir, 'bin', 'objcopy')
+  if not os.path.isfile(objcopy):
+    logging.error("Could not find objcopy executable at {0}".format(objcopy))
+    return None
+  return objcopy
+
+
+def ensure_dir_exists(path):
+  """Make sure the directory 'path' exists in a thread-safe way."""
+  try:
+    os.makedirs(path)
+  except OSError as e:
+    if e.errno != errno.EEXIST or not os.path.isdir(path):
+      raise e
+
+
+def dump_symbols_for_binary(dump_syms, objcopy, binary, out_dir):
+  """Dump symbols of a single binary file and move the result.
+
+  Symbols will be extracted to a temporary file and moved into place afterwards. Required
+  directories will be created if necessary.
+  """
+  logging.info("Processing binary file: {0}".format(binary))
+  ensure_dir_exists(out_dir)
+  # tmp_fd will be closed when the file object created by os.fdopen() below gets
+  # destroyed.
+  tmp_fd, tmp_file = tempfile.mkstemp(dir=out_dir, suffix='.sym')
+  try:
+    # Create a temporary directory used for decompressing debug info
+    tempdir = tempfile.mkdtemp()
+
+    # Binaries can contain compressed debug symbols. Breakpad currently
+    # does not support dumping symbols for binaries with compressed debug
+    # symbols.
+    #
+    # As a workaround, this uses objcopy to create a copy of the binary with
+    # the debug symbols decompressed. If the debug symbols are not compressed
+    # in the original binary, objcopy simply makes a copy of the binary.
+    # Breakpad is able to read symbols from the decompressed binary, and
+    # those symbols work correctly in resolving a minidump from the original
+    # compressed binary.
+    # TODO: In theory, this could work with the binary.debug_path.
+    binary_basename = os.path.basename(binary)
+    decompressed_binary = os.path.join(tempdir, binary_basename)
+    objcopy_retcode = subprocess.call([objcopy, "--decompress-debug-sections",
+                                       binary, decompressed_binary])
+
+    # Run dump_syms on the binary
+    # If objcopy failed for some reason, fall back to running dump_syms
+    # directly on the original binary. This is unlikely to work, but it is a way of
+    # guaranteeing that objcopy is not the problem.
+    args = [dump_syms, decompressed_binary]
+    if objcopy_retcode != 0:
+      sys.stderr.write('objcopy failed. Trying to run dump_sym directly.\n')
+      args = [dump_syms, binary]
+
+    # Run dump_syms on the binary.
+    proc = subprocess.Popen(args, stdout=os.fdopen(tmp_fd, 'wb'), stderr=subprocess.PIPE)
+    _, stderr = proc.communicate()
+    if proc.returncode != 0:
+      sys.stderr.write('Failed to dump symbols from %s, return code %s\n' %
+          (binary, proc.returncode))
+      sys.stderr.write(stderr.decode('utf-8'))
+      os.remove(tmp_file)
+      return False
+    # Parse the temporary file to determine the full target path.
+    with open(tmp_file, 'r') as f:
+      header = f.readline().strip()
+      # Format of header is: MODULE os arch binary_id binary
+      _, _, _, binary_id, binary = header.split(' ')
+      out_path = os.path.join(out_dir, binary, binary_id)
+      ensure_dir_exists(out_path)
+    # Move the temporary file to its final destination.
+    shutil.move(tmp_file, os.path.join(out_path, '%s.sym' % binary))
+  except Exception as e:
+    # Only need to clean up in case of errors.
+    try:
+      os.remove(tmp_file)
+    except EnvironmentError:
+      pass
+    raise e
+  finally:
+    # Cleanup temporary directory
+    shutil.rmtree(tempdir)
+  return True
+
+
+def dump_symbols_for_all_modules(dump_syms, objcopy, module_list, out_dir):
+  """Given a list of modules (ModuleInfo objects), dump symbols for
+  each library listed.
+  """
+  for module in module_list:
+    success = dump_symbols_for_binary(dump_syms, objcopy, module.code_file, out_dir)
+    if not success:
+      logging.warning("Failed to dump symbols for {0}".format(module.code_file))
+
+
+def resolve_minidump(minidump_stackwalk, minidump_path, symbol_dir, verbose, out_file):
+  with open(out_file, "w") as out_f:
+    stderr_output = None if verbose else subprocess.DEVNULL
+    subprocess.run([minidump_stackwalk, minidump_path, symbol_dir], stdout=out_f,
+                   stderr=stderr_output, check=True)
+
+
+def raw_dump_for_minidump(minidump_dump, minidump_path):
+  """Run minidump_dump on the specified minidump and split the output into lines"""
+  # minidump_dump sometimes returns an error code even though it produced usable output.
+  # So, this doesn't check the error code, and it relies on read_module_info() doing
+  # validation.
+  #
+  # Python 3.6 adjustments:
+  # 'capture_output=True' not supported: set stdout/stderr to subprocess.PIPE instead
+  # 'text=True' not supported: set 'universal_newlines=True' (the two are the same thing)
+  output = subprocess.run([minidump_dump, minidump_path], stdout=subprocess.PIPE,
+                          stderr=subprocess.PIPE, universal_newlines=True)
+  return output.stdout.split('\n')
+
+
+def parse_args():
+  """Parse command line arguments and perform sanity checks."""
+  # TODO:
+  #  - Add ability to specify Breakpad home
+  #  - Add ability to specify the symbol directory location (for reuse)
+  #  - Add ability to specify Binutils home
+  parser = ArgumentParser()
+  parser.add_argument('--minidump_file', required=True)
+  parser.add_argument('--output_file', required=True)
+  parser.add_argument('-v', '--verbose', action='store_true')
+  args = parser.parse_args()
+  return args
+
+
+def main():
+  args = parse_args()
+
+  if args.verbose:
+    logging.basicConfig(level=logging.INFO)
+  else:
+    logging.basicConfig(level=logging.WARNING)
+
+  # Step 1: Get the raw dump for the specified minidump
+  minidump_dump_bin = find_breakpad_binary("minidump_dump")
+  if not minidump_dump_bin:
+    logging.error("Could not find Breakpad minidump_dump binary")
+    sys.exit(1)
+  contents = raw_dump_for_minidump(minidump_dump_bin, args.minidump_file)
+  if not contents:
+    logging.error(
+      "minidump_dump could not get the contents of {0}".format(args.minidump_file))
+    sys.exit(1)
+
+  # Step 2: Parse the raw dump to get the list of code modules
+  # This is the list of things that have symbols we need to dump.
+  modules = read_module_info(contents)
+  if not modules:
+    logging.error("Failed to read modules for {0}".format(args.minidump_file))
+    sys.exit(1)
+
+  # Create a temporary directory to store the symbols.
+  # This automatically gets cleaned up.
+  with tempfile.TemporaryDirectory() as tmp_dir:
+    # Step 3: Dump symbols for all the modules into this temporary directory.
+    # Need both dump_syms and objcopy
+    dump_syms_bin = find_breakpad_binary("dump_syms")
+    if not dump_syms_bin:
+      logging.error("Could not find Breakpad dump_syms binary")
+      sys.exit(1)
+    objcopy_bin = find_objcopy_binary()
+    if not objcopy_bin:
+      logging.error("Could not find Binutils objcopy binary")
+      sys.exit(1)
+    dump_symbols_for_all_modules(dump_syms_bin, objcopy_bin, modules, tmp_dir)
+
+    # Step 4: Resolve the minidump with the temporary symbol directory
+    minidump_stackwalk_bin = find_breakpad_binary("minidump_stackwalk")
+    if not minidump_stackwalk_bin:
+      logging.error("Could not find Breakpad minidump_stackwalk binary")
+      sys.exit(1)
+    resolve_minidump(find_breakpad_binary("minidump_stackwalk"), args.minidump_file,
+                     tmp_dir, args.verbose, args.output_file)
+
+
+if __name__ == "__main__":
+  main()


[impala] 02/03: IMPALA-11646 IMPALA-11562: Fix test_unsupported_text_compression in s3

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 4638a0e585a76e4a0b0aebf40eb7e4c34e2944d1
Author: Michael Smith <mi...@cloudera.com>
AuthorDate: Mon Oct 10 11:56:56 2022 -0700

    IMPALA-11646 IMPALA-11562: Fix test_unsupported_text_compression in s3
    
    Fixes test_unsupported_text_compression for S3 tests where hdfs_util.py
    was changed to expect a leading slash (and the pure HDFS implementation
    removes the leading slash). Skips removing the leading slash since HDFS
    now adds it and other filesystems expect the leading slash.
    
    Change-Id: I74724b2d5af0bd879d92d5e5febe92d01c972525
    Reviewed-on: http://gerrit.cloudera.org:8080/19119
    Reviewed-by: Joe McDonnell <jo...@cloudera.com>
    Reviewed-by: Fang-Yu Rao <fa...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 tests/metadata/test_partition_metadata.py | 20 ++++++++++----------
 1 file changed, 10 insertions(+), 10 deletions(-)

diff --git a/tests/metadata/test_partition_metadata.py b/tests/metadata/test_partition_metadata.py
index 0dd913182..47024b1d1 100644
--- a/tests/metadata/test_partition_metadata.py
+++ b/tests/metadata/test_partition_metadata.py
@@ -183,7 +183,7 @@ class TestPartitionMetadataUncompressedTextOnly(ImpalaTestSuite):
     lz4_year = 2009
     lz4_month = 3
     lz4_ym_partition_loc = self.__make_ym_partition_dir(TBL_LOCATION, lz4_year, lz4_month)
-    self.filesystem_client.create_file("{0}/fake.lz4".format(lz4_ym_partition_loc)[1:],
+    self.filesystem_client.create_file("{0}/fake.lz4".format(lz4_ym_partition_loc),
         "some test data")
     self.client.execute(
         "alter table {0} add partition (year={1}, month={2}) location '{3}'".format(
@@ -195,7 +195,7 @@ class TestPartitionMetadataUncompressedTextOnly(ImpalaTestSuite):
     fake_comp_ym_partition_loc = self.__make_ym_partition_dir(
         TBL_LOCATION, fake_comp_year, fake_comp_month)
     self.filesystem_client.create_file(
-        "{0}/fake.fake_comp".format(fake_comp_ym_partition_loc)[1:], "fake compression")
+        "{0}/fake.fake_comp".format(fake_comp_ym_partition_loc), "fake compression")
     self.client.execute(
         "alter table {0} add partition (year={1}, month={2}) location '{3}'".format(
         FQ_TBL_NAME, fake_comp_year, fake_comp_month, fake_comp_ym_partition_loc))
@@ -204,7 +204,7 @@ class TestPartitionMetadataUncompressedTextOnly(ImpalaTestSuite):
     lzo_year = 2009
     lzo_month = 5
     lzo_ym_partition_loc = self.__make_ym_partition_dir(TBL_LOCATION, lzo_year, lzo_month)
-    self.filesystem_client.create_file("{0}/fake.lzo".format(lzo_ym_partition_loc)[1:],
+    self.filesystem_client.create_file("{0}/fake.lzo".format(lzo_ym_partition_loc),
         "some test data")
     self.client.execute(
         "alter table {0} add partition (year={1}, month={2}) location '{3}'".format(
@@ -228,11 +228,11 @@ class TestPartitionMetadataUncompressedTextOnly(ImpalaTestSuite):
     """Create the year/month partition directory and return the path."""
     y_partition_loc = "{0}/year={1}".format(tbl_location, year)
     ym_partition_loc = "{0}/month={1}".format(y_partition_loc, month)
-    if not self.filesystem_client.exists(tbl_location[1:]):
-      self.filesystem_client.make_dir(tbl_location[1:])
-    if not self.filesystem_client.exists(y_partition_loc[1:]):
-      self.filesystem_client.make_dir(y_partition_loc[1:])
-    if self.filesystem_client.exists(ym_partition_loc[1:]):
-      self.filesystem_client.delete_file_dir(ym_partition_loc[1:], recursive=True)
-    self.filesystem_client.make_dir(ym_partition_loc[1:])
+    if not self.filesystem_client.exists(tbl_location):
+      self.filesystem_client.make_dir(tbl_location)
+    if not self.filesystem_client.exists(y_partition_loc):
+      self.filesystem_client.make_dir(y_partition_loc)
+    if self.filesystem_client.exists(ym_partition_loc):
+      self.filesystem_client.delete_file_dir(ym_partition_loc, recursive=True)
+    self.filesystem_client.make_dir(ym_partition_loc)
     return ym_partition_loc