From e405ad1694e3c503a5ce3793c80697cda158e6c6 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 5 Oct 2023 07:53:33 +0000 Subject: [PATCH] DWARF input format --- contrib/llvm-project-cmake/CMakeLists.txt | 4 +- docs/en/interfaces/formats.md | 48 ++ src/Common/CurrentMetrics.cpp | 2 + src/Common/Elf.cpp | 20 +- src/Common/Elf.h | 22 +- src/Common/config.h.in | 1 + src/Formats/FormatFactory.cpp | 7 +- src/Formats/FormatFactory.h | 3 +- src/Formats/registerFormats.cpp | 4 + .../Formats/Impl/DWARFBlockInputFormat.cpp | 792 ++++++++++++++++++ .../Formats/Impl/DWARFBlockInputFormat.h | 110 +++ src/Storages/StorageFile.cpp | 4 +- src/Storages/StorageS3.cpp | 5 +- src/Storages/StorageURL.cpp | 5 +- src/configure_config.cmake | 3 +- .../0_stateless/02896_dwarf_format.reference | 25 + .../queries/0_stateless/02896_dwarf_format.sh | 14 + tests/queries/0_stateless/data_dwarf/a.out | Bin 0 -> 64416 bytes .../aspell-ignore/en/aspell-dict.txt | 2 + 19 files changed, 1047 insertions(+), 24 deletions(-) create mode 100644 src/Processors/Formats/Impl/DWARFBlockInputFormat.cpp create mode 100644 src/Processors/Formats/Impl/DWARFBlockInputFormat.h create mode 100644 tests/queries/0_stateless/02896_dwarf_format.reference create mode 100755 tests/queries/0_stateless/02896_dwarf_format.sh create mode 100644 tests/queries/0_stateless/data_dwarf/a.out diff --git a/contrib/llvm-project-cmake/CMakeLists.txt b/contrib/llvm-project-cmake/CMakeLists.txt index ce82a10d3eb6..05464b65d93b 100644 --- a/contrib/llvm-project-cmake/CMakeLists.txt +++ b/contrib/llvm-project-cmake/CMakeLists.txt @@ -6,7 +6,9 @@ endif() option (ENABLE_EMBEDDED_COMPILER "Enable support for JIT compilation during query execution" ${ENABLE_EMBEDDED_COMPILER_DEFAULT}) -if (NOT ENABLE_EMBEDDED_COMPILER) +option (ENABLE_DWARF_PARSER "Enable support for DWARF input format" ON) + +if (NOT ENABLE_EMBEDDED_COMPILER AND NOT ENABLE_DWARF_PARSER) message(STATUS "Not using LLVM") return() endif() diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index e98f19b2a65d..52690895c443 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -87,6 +87,7 @@ The supported formats are: | [RawBLOB](#rawblob) | ✔ | ✔ | | [MsgPack](#msgpack) | ✔ | ✔ | | [MySQLDump](#mysqldump) | ✔ | ✗ | +| [DWARF](#dwarf) | ✔ | ✗ | | [Markdown](#markdown) | ✗ | ✔ | @@ -2711,6 +2712,53 @@ FROM file(dump.sql, MySQLDump) └───┘ ``` +## DWARF {#dwarf} + +Parses DWARF debug symbols from an ELF file (executable, library, or object file). Similar to `dwarfdump`, but much faster (hundreds of MB/s) and with SQL. Produces one row for each Debug Information Entry (DIE) in the `.debug_info` section. Includes "null" entries that the DWARF encoding uses to terminate lists of children in the tree. + +Quick background: `.debug_info` consists of *units*, corresponding to compilation units. Each unit is a tree of *DIE*s, with a `compile_unit` DIE as its root. Each DIE has a *tag* and a list of *attributes*. Each attribute has a *name* and a *value* (and also a *form*, which specifies how the value is encoded). The DIEs represent things from the source code, and their *tag* tells what kind of thing it is. E.g. there are functions (tag = `subprogram`), classes/structs/enums (`class_type`/`structure_type`/`enumeration_type`), variables (`variable`), function arguments (`formal_parameter`). The tree structure mirrors the corresponding source code. E.g. a `class_type` DIE can contain `subprogram` DIEs representing methods of the class. + +Outputs the following columns: + - `offset` - position of the DIE in the `.debug_info` section + - `size` - number of bytes in the encoded DIE (including attributes) + - `tag` - type of the DIE; the conventional "DW_TAG_" prefix is omitted + - `unit_name` - name of the compilation unit containing this DIE + - `unit_offset` - position of the compilation unit containing this DIE in the `.debug_info` section + - `ancestor_tags` - array of tags of the ancestors of the current DIE in the tree, in order from innermost to outermost + - `ancestor_offsets` - offsets of ancestors, parallel to `ancestor_tags` + - a few common attributes duplicated from the attributes array for convenience: + - `name` + - `linkage_name` - mangled fully-qualified name; typically only functions have it (but not all functions) + - `decl_file` - name of the source code file where this entity was declared + - `decl_line` - line number in the source code where this entity was declared + - parallel arrays describing attributes: + - `attr_name` - name of the attribute; the conventional "DW_AT_" prefix is omitted + - `attr_form` - how the attribute is encoded and interpreted; the conventional DW_FORM_ prefix is omitted + - `attr_int` - integer value of the attribute; 0 if the attribute doesn't have a numeric value + - `attr_str` - string value of the attribute; empty if the attribute doesn't have a string value + +Example: find compilation units that have the most function definitions (including template instantiations and functions from included header files): +```sql +SELECT + unit_name, + count() AS c +FROM file('programs/clickhouse', DWARF) +WHERE tag = 'subprogram' AND NOT has(attr_name, 'declaration') +GROUP BY unit_name +ORDER BY c DESC +LIMIT 3 +``` +```text +┌─unit_name──────────────────────────────────────────────────┬─────c─┐ +│ ./src/Core/Settings.cpp │ 28939 │ +│ ./src/AggregateFunctions/AggregateFunctionSumMap.cpp │ 23327 │ +│ ./src/AggregateFunctions/AggregateFunctionUniqCombined.cpp │ 22649 │ +└────────────────────────────────────────────────────────────┴───────┘ + +3 rows in set. Elapsed: 1.487 sec. Processed 139.76 million rows, 1.12 GB (93.97 million rows/s., 752.77 MB/s.) +Peak memory usage: 271.92 MiB. +``` + ## Markdown {#markdown} You can export results using [Markdown](https://en.wikipedia.org/wiki/Markdown) format to generate output ready to be pasted into your `.md` files: diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 065719e477d3..b6d61966973a 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -153,6 +153,8 @@ M(ParquetDecoderThreadsActive, "Number of threads in the ParquetBlockInputFormat thread pool running a task.") \ M(ParquetEncoderThreads, "Number of threads in ParquetBlockOutputFormat thread pool.") \ M(ParquetEncoderThreadsActive, "Number of threads in ParquetBlockOutputFormat thread pool running a task.") \ + M(DWARFReaderThreads, "Number of threads in the DWARFBlockInputFormat thread pool.") \ + M(DWARFReaderThreadsActive, "Number of threads in the DWARFBlockInputFormat thread pool running a task.") \ M(OutdatedPartsLoadingThreads, "Number of threads in the threadpool for loading Outdated data parts.") \ M(OutdatedPartsLoadingThreadsActive, "Number of active threads in the threadpool for loading Outdated data parts.") \ M(DistributedBytesToInsert, "Number of pending bytes to process for asynchronous insertion into Distributed tables. Number of bytes for every shard is summed.") \ diff --git a/src/Common/Elf.cpp b/src/Common/Elf.cpp index b83718af2b14..5599df63948b 100644 --- a/src/Common/Elf.cpp +++ b/src/Common/Elf.cpp @@ -16,15 +16,27 @@ namespace ErrorCodes } -Elf::Elf(const std::string & path) - : in(path, 0) +Elf::Elf(const std::string & path_) { + in.emplace(path_, 0); + init(in->buffer().begin(), in->buffer().size(), path_); +} + +Elf::Elf(const char * data, size_t size, const std::string & path_) +{ + init(data, size, path_); +} + +void Elf::init(const char * data, size_t size, const std::string & path_) +{ + path = path_; + mapped = data; + elf_size = size; + /// Check if it's an elf. - elf_size = in.buffer().size(); if (elf_size < sizeof(ElfEhdr)) throw Exception(ErrorCodes::CANNOT_PARSE_ELF, "The size of supposedly ELF file '{}' is too small", path); - mapped = in.buffer().begin(); header = reinterpret_cast(mapped); if (memcmp(header->e_ident, "\x7F""ELF", 4) != 0) diff --git a/src/Common/Elf.h b/src/Common/Elf.h index 5a6bd9e302da..a1a63cf00387 100644 --- a/src/Common/Elf.h +++ b/src/Common/Elf.h @@ -9,16 +9,14 @@ #include #include -#include -using ElfAddr = ElfW(Addr); -using ElfEhdr = ElfW(Ehdr); -using ElfOff = ElfW(Off); -using ElfPhdr = ElfW(Phdr); -using ElfShdr = ElfW(Shdr); -using ElfNhdr = ElfW(Nhdr); -using ElfSym = ElfW(Sym); +using ElfEhdr = Elf64_Ehdr; +using ElfOff = Elf64_Off; +using ElfPhdr = Elf64_Phdr; +using ElfShdr = Elf64_Shdr; +using ElfNhdr = Elf64_Nhdr; +using ElfSym = Elf64_Sym; namespace DB @@ -44,7 +42,8 @@ class Elf final const Elf & elf; }; - explicit Elf(const std::string & path); + explicit Elf(const std::string & path_); + Elf(const char * data, size_t size, const std::string & path_); bool iterateSections(std::function && pred) const; std::optional
findSection(std::function && pred) const; @@ -64,13 +63,16 @@ class Elf final String getStoredBinaryHash() const; private: - MMapReadBufferFromFile in; + std::string path; // just for error messages + std::optional in; size_t elf_size; const char * mapped; const ElfEhdr * header; const ElfShdr * section_headers; const ElfPhdr * program_headers; const char * section_names = nullptr; + + void init(const char * data, size_t size, const std::string & path_); }; } diff --git a/src/Common/config.h.in b/src/Common/config.h.in index 72932dda55d1..dbd8572f7b97 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -43,6 +43,7 @@ #cmakedefine01 USE_AMQPCPP #cmakedefine01 USE_NATSIO #cmakedefine01 USE_EMBEDDED_COMPILER +#cmakedefine01 USE_DWARF_PARSER #cmakedefine01 USE_LDAP #cmakedefine01 USE_ROCKSDB #cmakedefine01 USE_LIBPQXX diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 862e51aa088e..30fac6e555d3 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -256,7 +256,8 @@ InputFormatPtr FormatFactory::getInput( std::optional _max_parsing_threads, std::optional _max_download_threads, bool is_remote_fs, - CompressionMethod compression) const + CompressionMethod compression, + bool need_only_count) const { const auto& creators = getCreators(name); if (!creators.input_creator && !creators.random_access_input_creator) @@ -284,7 +285,9 @@ InputFormatPtr FormatFactory::getInput( // Decide whether to use ParallelParsingInputFormat. - bool parallel_parsing = max_parsing_threads > 1 && settings.input_format_parallel_parsing && creators.file_segmentation_engine && !creators.random_access_input_creator; + bool parallel_parsing = + max_parsing_threads > 1 && settings.input_format_parallel_parsing && creators.file_segmentation_engine && + !creators.random_access_input_creator && !need_only_count; if (settings.max_memory_usage && settings.min_chunk_bytes_for_parallel_parsing * max_parsing_threads * 2 > settings.max_memory_usage) parallel_parsing = false; diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index 48a1869d5633..d1e8c350391a 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -167,7 +167,8 @@ class FormatFactory final : private boost::noncopyable bool is_remote_fs = false, // allows to do: buf -> parallel read -> decompression, // because parallel read after decompression is not possible - CompressionMethod compression = CompressionMethod::None) const; + CompressionMethod compression = CompressionMethod::None, + bool need_only_count = false) const; /// Checks all preconditions. Returns ordinary format if parallel formatting cannot be done. OutputFormatPtr getOutputFormatParallelIfPossible( diff --git a/src/Formats/registerFormats.cpp b/src/Formats/registerFormats.cpp index 580db61edde4..ddeb1a345129 100644 --- a/src/Formats/registerFormats.cpp +++ b/src/Formats/registerFormats.cpp @@ -101,6 +101,7 @@ void registerInputFormatJSONAsObject(FormatFactory & factory); void registerInputFormatLineAsString(FormatFactory & factory); void registerInputFormatMySQLDump(FormatFactory & factory); void registerInputFormatParquetMetadata(FormatFactory & factory); +void registerInputFormatDWARF(FormatFactory & factory); void registerInputFormatOne(FormatFactory & factory); #if USE_HIVE @@ -143,6 +144,7 @@ void registerTemplateSchemaReader(FormatFactory & factory); void registerMySQLSchemaReader(FormatFactory & factory); void registerBSONEachRowSchemaReader(FormatFactory & factory); void registerParquetMetadataSchemaReader(FormatFactory & factory); +void registerDWARFSchemaReader(FormatFactory & factory); void registerOneSchemaReader(FormatFactory & factory); void registerFileExtensions(FormatFactory & factory); @@ -245,6 +247,7 @@ void registerFormats() registerInputFormatMySQLDump(factory); registerInputFormatParquetMetadata(factory); + registerInputFormatDWARF(factory); registerInputFormatOne(factory); registerNonTrivialPrefixAndSuffixCheckerJSONEachRow(factory); @@ -282,6 +285,7 @@ void registerFormats() registerMySQLSchemaReader(factory); registerBSONEachRowSchemaReader(factory); registerParquetMetadataSchemaReader(factory); + registerDWARFSchemaReader(factory); registerOneSchemaReader(factory); } diff --git a/src/Processors/Formats/Impl/DWARFBlockInputFormat.cpp b/src/Processors/Formats/Impl/DWARFBlockInputFormat.cpp new file mode 100644 index 000000000000..90ff4f9106c4 --- /dev/null +++ b/src/Processors/Formats/Impl/DWARFBlockInputFormat.cpp @@ -0,0 +1,792 @@ +#include "DWARFBlockInputFormat.h" +#if USE_DWARF_PARSER && defined(__ELF__) && !defined(OS_FREEBSD) + +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace CurrentMetrics +{ + extern const Metric DWARFReaderThreads; + extern const Metric DWARFReaderThreadsActive; +} + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int CANNOT_PARSE_ELF; + extern const int CANNOT_PARSE_DWARF; +} + +enum DwarfColumn +{ + COL_OFFSET, + COL_SIZE, + COL_TAG, + COL_UNIT_NAME, + COL_UNIT_OFFSET, + + COL_ANCESTOR_TAGS, + COL_ANCESTOR_OFFSETS, + + /// A few very common attributes get their own columns, just for convenience. + /// We put their values *both* in the dedicated columns and in the attr_str/attr_int arrays. + /// This duplication wastes considerable time and space (tens of percent), but I can't think of + /// an alternative that wouldn't be really inconvenient or confusing: + /// * omitting these attributes from the arrays would make collecting attribute stats inconvenient, + /// and would lose information about the form of the attribute, + /// * using empty value for the attribute would be confusing and error-prone, e.g. when collecting stats + /// about all attribute values the user would need to add these columns too, somehow, + /// * not having these dedicated columns would make it inconvenient to look up entry name/file/line. + /// (But maybe that's fine? I.e. maybe it's not very commonly used and maybe the array lookup is not that inconvenient? Idk.) + + COL_NAME, + COL_LINKAGE_NAME, + COL_DECL_FILE, + COL_DECL_LINE, + /// TODO: Dedicated column for ranges (DW_AT_ranges, DW_AT_low_pc, DW_AT_high_pc). + /// In practice there are often many incorrect ranges/range-lists that start at zero. I'm guessing they're caused by LTO. + /// We'd want to check for that and exclude those ranges/range-lists from the dedicated column. + + COL_ATTR_NAME, + COL_ATTR_FORM, + COL_ATTR_INT, + COL_ATTR_STR, + + COL_COUNT, +}; + +static NamesAndTypesList getHeaderForDWARF() +{ + std::vector cols(COL_COUNT); + cols[COL_OFFSET] = {"offset", std::make_shared()}; + cols[COL_SIZE] = {"size", std::make_shared()}; + cols[COL_TAG] = {"tag", std::make_shared(std::make_shared())}; + cols[COL_UNIT_NAME] = {"unit_name", std::make_shared(std::make_shared())}; + cols[COL_UNIT_OFFSET] = {"unit_offset", std::make_shared(std::make_shared())}; + cols[COL_ANCESTOR_TAGS] = {"ancestor_tags", std::make_shared(std::make_shared(std::make_shared()))}; + cols[COL_ANCESTOR_OFFSETS] = {"ancestor_offsets", std::make_shared(std::make_shared())}; + cols[COL_NAME] = {"name", std::make_shared()}; + cols[COL_LINKAGE_NAME] = {"linkage_name", std::make_shared()}; + cols[COL_DECL_FILE] = {"decl_file", std::make_shared(std::make_shared())}; + cols[COL_DECL_LINE] = {"decl_line", std::make_shared()}; + cols[COL_ATTR_NAME] = {"attr_name", std::make_shared(std::make_shared(std::make_shared()))}; + cols[COL_ATTR_FORM] = {"attr_form", std::make_shared(std::make_shared(std::make_shared()))}; + cols[COL_ATTR_INT] = {"attr_int", std::make_shared(std::make_shared())}; + cols[COL_ATTR_STR] = {"attr_str", std::make_shared(std::make_shared(std::make_shared()))}; + return NamesAndTypesList(cols.begin(), cols.end()); +} + +static const std::unordered_map & getColumnNameToIdx() +{ + static std::once_flag once; + static std::unordered_map name_to_idx; + std::call_once(once, [&] { + size_t i = 0; + for (const auto & c : getHeaderForDWARF()) + { + name_to_idx.emplace(c.name, i); + ++i; + } + }); + return name_to_idx; +} + +DWARFBlockInputFormat::UnitState::UnitState(llvm::DWARFUnit * u) + : dwarf_unit(u), end_offset(dwarf_unit->getNextUnitOffset()) + , offset(dwarf_unit->getOffset() + dwarf_unit->getHeaderSize()) +{ + /// This call is not thread safe, so we do it during initialization. + abbrevs = dwarf_unit->getAbbreviations(); + if (abbrevs == nullptr) + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Couldn't find abbreviation set for unit at offset {}", dwarf_unit->getOffset()); + + /// This call initializes some data structures inside DWARFUnit that are needed for parsing attributes. + auto err = u->tryExtractDIEsIfNeeded(/*CUDieOnly*/ true); + if (err) + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Failed to parse compilation unit entry: {}", llvm::toString(std::move(err))); +} + +static llvm::StringRef removePrefix(llvm::StringRef s, size_t prefix_len) +{ + if (s.size() >= prefix_len) + s = llvm::StringRef(s.data() + prefix_len, s.size() - prefix_len); + return s; +} + +template +static void append(C & col, llvm::StringRef s) +{ + col->insertData(s.data(), s.size()); +} + +DWARFBlockInputFormat::DWARFBlockInputFormat(ReadBuffer & in_, Block header_, const FormatSettings & format_settings_, size_t num_threads_) + : IInputFormat(std::move(header_), &in_), format_settings(format_settings_), num_threads(num_threads_) +{ + auto tag_names = ColumnString::create(); + /// Note: TagString() returns empty string for tags that don't exist, and tag 0 doesn't exist. + for (uint32_t tag = 0; tag <= UINT16_MAX; ++tag) + append(tag_names, removePrefix(llvm::dwarf::TagString(tag), strlen("DW_TAG_"))); + tag_dict_column = ColumnUnique::create(std::move(tag_names), /*is_nullable*/ false); + + auto attr_names = ColumnString::create(); + for (uint32_t attr = 0; attr <= UINT16_MAX; ++attr) + append(attr_names, removePrefix(llvm::dwarf::AttributeString(attr), strlen("DW_AT_"))); + attr_name_dict_column = ColumnUnique::create(std::move(attr_names), /*is_nullable*/ false); + + auto attr_forms = ColumnString::create(); + for (uint32_t form = 0; form <= UINT16_MAX; ++form) + append(attr_forms, removePrefix(llvm::dwarf::FormEncodingString(form), strlen("DW_FORM_"))); + attr_form_dict_column = ColumnUnique::create(std::move(attr_forms), /*is_nullable*/ false); +} + +DWARFBlockInputFormat::~DWARFBlockInputFormat() +{ + stopThreads(); +} + +void DWARFBlockInputFormat::initELF() +{ + /// If it's a local file, mmap it. + if (ReadBufferFromFileBase * file_in = dynamic_cast(in)) + { + size_t offset = 0; + if (file_in->isRegularLocalFile(&offset) && offset == 0) + { + elf.emplace(file_in->getFileName()); + return; + } + } + + /// If can't mmap, read the entire file into memory. + /// We could read just the .debug_* sections, but typically they take up most of the binary anyway (60% for clickhouse debug build). + { + WriteBufferFromVector buf(file_contents); + copyData(*in, buf, is_stopped); + buf.finalize(); + } + elf.emplace(file_contents.data(), file_contents.size(), ""); +} + +void DWARFBlockInputFormat::initializeIfNeeded() +{ + if (elf.has_value()) + return; + + LOG_DEBUG(&Poco::Logger::get("DWARF"), "Opening ELF"); + initELF(); + if (is_stopped) + return; + + auto info_section = elf->findSectionByName(".debug_info"); + if (!info_section.has_value()) + throw Exception(ErrorCodes::CANNOT_PARSE_ELF, "No .debug_info section"); + auto abbrev_section = elf->findSectionByName(".debug_abbrev"); + if (!abbrev_section.has_value()) + throw Exception(ErrorCodes::CANNOT_PARSE_ELF, "No .debug_abbrev section"); + LOG_DEBUG(&Poco::Logger::get("DWARF"), ".debug_abbrev is {:.3f} MiB, .debug_info is {:.3f} MiB", abbrev_section->size() * 1. / (1 << 20), info_section->size() * 1. / (1 << 20)); + + extractor.emplace(llvm::StringRef(info_section->begin(), info_section->size()), /*IsLittleEndian*/ true, /*AddressSize*/ 8); + + auto line_section = elf->findSectionByName(".debug_line"); + if (line_section.has_value()) + debug_line_extractor.emplace(llvm::StringRef(line_section->begin(), line_section->size()), /*IsLittleEndian*/ true, /*AddressSize*/ 8); + + llvm::StringMap> sections; + elf->iterateSections([&](const Elf::Section & section, size_t /*idx*/) + { + std::string name = section.name(); + std::string name_without_dot = name.starts_with(".") ? name.substr(1) : name; + sections.try_emplace(name_without_dot, llvm::MemoryBuffer::getMemBuffer( + llvm::StringRef(section.begin(), section.size()), /*BufferName*/ name, /*RequiresNullTerminator*/ false)); + return false; + }); + dwarf_context = llvm::DWARFContext::create(sections, /*AddrSize*/ 8); + + for (std::unique_ptr & unit : dwarf_context->info_section_units()) + units_queue.emplace_back(unit.get()); + + LOG_DEBUG(&Poco::Logger::get("DWARF"), "{} units, reading in {} threads", units_queue.size(), num_threads); + + pool.emplace(CurrentMetrics::DWARFReaderThreads, CurrentMetrics::DWARFReaderThreadsActive, num_threads); + for (size_t i = 0; i < num_threads; ++i) + pool->scheduleOrThrowOnError( + [this, thread_group = CurrentThread::getGroup()]() + { + if (thread_group) + CurrentThread::attachToGroupIfDetached(thread_group); + SCOPE_EXIT_SAFE(if (thread_group) CurrentThread::detachFromGroupIfNotDetached();); + try + { + setThreadName("DWARFDecoder"); + + std::unique_lock lock(mutex); + while (!units_queue.empty() && !is_stopped) + { + if (delivery_queue.size() > num_threads) + { + wake_up_threads.wait(lock); + continue; + } + UnitState unit = std::move(units_queue.front()); + units_queue.pop_front(); + ++units_in_progress; + + lock.unlock(); + + size_t offset_before = unit.offset; + Chunk chunk = parseEntries(unit); + size_t offset_after = unit.offset; + + lock.lock(); + + --units_in_progress; + if (chunk) + { + delivery_queue.emplace_back(std::move(chunk), offset_after - offset_before); + deliver_chunk.notify_one(); + } + if (!unit.eof()) + units_queue.push_front(std::move(unit)); + } + } + catch (...) + { + std::lock_guard lock(mutex); + background_exception = std::current_exception(); + deliver_chunk.notify_all(); + } + }); +} + +void DWARFBlockInputFormat::stopThreads() +{ + { + std::unique_lock lock(mutex); // required even if is_stopped is atomic + is_stopped = true; + } + wake_up_threads.notify_all(); + if (pool) + pool->wait(); +} + +static inline void throwIfError(llvm::Error & e, const char * what) +{ + if (!e) + return; + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Failed to parse {}: {}", what, llvm::toString(std::move(e))); +} + +Chunk DWARFBlockInputFormat::parseEntries(UnitState & unit) +{ + const auto & header = getPort().getHeader(); + const auto & column_name_to_idx = getColumnNameToIdx(); + std::array need{}; + for (const std::string & name : header.getNames()) + need[column_name_to_idx.at(name)] = true; + auto form_params = unit.dwarf_unit->getFormParams(); + + /// For parallel arrays, we nominate one of them to be responsible for populating the offsets vector. + if (need[COL_ATTR_FORM] || need[COL_ATTR_INT] || need[COL_ATTR_STR]) + need[COL_ATTR_NAME] = true; + if (need[COL_ANCESTOR_OFFSETS]) + need[COL_ANCESTOR_TAGS] = true; + + auto col_offset = ColumnVector::create(); + auto col_size = ColumnVector::create(); + auto col_tag = ColumnVector::create(); + auto col_ancestor_tags = ColumnVector::create(); + auto col_ancestor_dwarf_offsets = ColumnVector::create(); + auto col_ancestor_array_offsets = ColumnVector::create(); + auto col_name = ColumnString::create(); + auto col_linkage_name = ColumnString::create(); + ColumnLowCardinality::Index col_decl_file; + auto col_decl_line = ColumnVector::create(); + auto col_attr_name = ColumnVector::create(); + auto col_attr_form = ColumnVector::create(); + auto col_attr_int = ColumnVector::create(); + auto col_attr_str = ColumnLowCardinality::create(MutableColumnPtr(ColumnUnique::create(ColumnString::create()->cloneResized(1), /*is_nullable*/ false)), MutableColumnPtr(ColumnVector::create())); + auto col_attr_offsets = ColumnVector::create(); + size_t num_rows = 0; + auto err = llvm::Error::success(); + + while (num_rows < 65536) + { + ++num_rows; + uint64_t die_offset = unit.offset; + if (need[COL_OFFSET]) + col_offset->insertValue(die_offset); + if (need[COL_ANCESTOR_TAGS]) + { + for (size_t i = unit.stack.size() - 1; i != UINT64_MAX; --i) + { + col_ancestor_tags->insertValue(unit.stack[i].tag); + if (need[COL_ANCESTOR_OFFSETS]) + col_ancestor_dwarf_offsets->insertValue(unit.stack[i].offset); + } + col_ancestor_array_offsets->insertValue(col_ancestor_tags->size()); + } + + uint64_t abbrev_code = extractor->getULEB128(&unit.offset, &err); + throwIfError(err, "DIE header"); + + if (abbrev_code == 0) + { + if (need[COL_SIZE]) + col_size->insertValue(static_cast(unit.offset - die_offset)); + if (need[COL_TAG]) + col_tag->insertValue(0); // "null" + + if (need[COL_NAME]) col_name->insertDefault(); + if (need[COL_LINKAGE_NAME]) col_linkage_name->insertDefault(); + if (need[COL_DECL_FILE]) col_decl_file.insertPosition(0); + if (need[COL_DECL_LINE]) col_decl_line->insertDefault(); + if (need[COL_ATTR_NAME]) col_attr_offsets->insertValue(col_attr_name->size()); + + if (unit.stack.empty()) + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Stack underflow"); + unit.stack.pop_back(); + } + else + { + const llvm::DWARFAbbreviationDeclaration * abbrev = unit.abbrevs->getAbbreviationDeclaration(static_cast(abbrev_code)); + if (abbrev == nullptr || abbrev_code > UINT32_MAX) + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Abbrev code in DIE header is out of bounds: {}, offset {}", abbrev_code, unit.offset); + + auto tag = abbrev->getTag(); + if (need[COL_TAG]) + col_tag->insertValue(tag); + + bool need_name = need[COL_NAME]; + bool need_linkage_name = need[COL_LINKAGE_NAME]; + bool need_decl_file = need[COL_DECL_FILE]; + bool need_decl_line = need[COL_DECL_LINE]; + + for (auto attr : abbrev->attributes()) + { + auto val = llvm::DWARFFormValue::createFromSValue(attr.Form, attr.isImplicitConst() ? attr.getImplicitConstValue() : 0); + /// This is relatively slow, maybe we should reimplement it. + if (!val.extractValue(*extractor, &unit.offset, form_params, unit.dwarf_unit)) + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Failed to parse attribute {} of form {} at offset {}", + llvm::dwarf::AttributeString(attr.Attr), attr.Form, unit.offset); + + if (need[COL_ATTR_NAME]) + col_attr_name->insertValue(attr.Attr); + /// Note that in case of DW_FORM_implicit_const val.getForm() is different from attr.Form. + /// Not sure which one would be more useful in the attr_form column. Guessing attr.Form for now. + if (need[COL_ATTR_FORM]) + col_attr_form->insertValue(attr.Form); + + if (attr.Attr == llvm::dwarf::DW_AT_stmt_list && unit.filename_table == nullptr) + { + /// We expect that this attribute appears before any attributes that point into the filename table. + auto offset = val.getAsSectionOffset(); + if (offset.has_value()) + parseFilenameTable(unit, offset.value()); + } + + switch (val.getForm()) // (may be different from attr.Form because of DW_FORM_indirect) + { + /// A 64-bit value. + case llvm::dwarf::DW_FORM_data2: + case llvm::dwarf::DW_FORM_data4: + case llvm::dwarf::DW_FORM_data8: + case llvm::dwarf::DW_FORM_data1: + case llvm::dwarf::DW_FORM_sdata: + case llvm::dwarf::DW_FORM_udata: + case llvm::dwarf::DW_FORM_data16: + case llvm::dwarf::DW_FORM_flag: + case llvm::dwarf::DW_FORM_flag_present: + case llvm::dwarf::DW_FORM_loclistx: // points to .debug_loclists + case llvm::dwarf::DW_FORM_rnglistx: // points to .debug_rnglists + case llvm::dwarf::DW_FORM_sec_offset: // points to some other section, depending on attr.Attr + case llvm::dwarf::DW_FORM_implicit_const: + if (need[COL_ATTR_INT]) col_attr_int->insertValue(val.getRawUValue()); + + if (attr.Attr == llvm::dwarf::DW_AT_decl_line && std::exchange(need_decl_line, false)) + col_decl_line->insertValue(static_cast(val.getRawUValue())); + + /// Some attribute values are indices into lookup tables that we can stringify usefully. + if ((attr.Attr == llvm::dwarf::DW_AT_decl_file || attr.Attr == llvm::dwarf::DW_AT_call_file) && + val.getRawUValue() < unit.filename_table_size) // filename + { + UInt64 idx = val.getRawUValue() + 1; + if (attr.Attr == llvm::dwarf::DW_AT_decl_file && std::exchange(need_decl_file, false)) + col_decl_file.insertPosition(idx); + + if (need[COL_ATTR_STR]) + { + auto data = unit.filename_table->getDataAt(idx); + col_attr_str->insertData(data.data, data.size); + } + } + else if (need[COL_ATTR_STR]) + { + if (attr.Attr == llvm::dwarf::DW_AT_language) // programming language + append(col_attr_str, removePrefix(llvm::dwarf::LanguageString(static_cast(val.getRawUValue())), + strlen("DW_LANG_"))); + else if (attr.Attr == llvm::dwarf::DW_AT_encoding) // primitive type + append(col_attr_str, removePrefix(llvm::dwarf::AttributeEncodingString(static_cast(val.getRawUValue())), + strlen("DW_ATE_"))); + else + col_attr_str->insertDefault(); + } + break; + + /// An address, i.e. just a 64-bit value. + /// May have indirection to .debug_addr section. + case llvm::dwarf::DW_FORM_addr: + case llvm::dwarf::DW_FORM_addrx: + case llvm::dwarf::DW_FORM_addrx1: + case llvm::dwarf::DW_FORM_addrx2: + case llvm::dwarf::DW_FORM_addrx3: + case llvm::dwarf::DW_FORM_addrx4: + case llvm::dwarf::DW_FORM_GNU_addr_index: + case llvm::dwarf::DW_FORM_LLVM_addrx_offset: + if (need[COL_ATTR_INT]) col_attr_int->insertValue(val.getAsAddress().value_or(0)); + if (need[COL_ATTR_STR]) col_attr_str->insertDefault(); + break; + + /// A byte string. + case llvm::dwarf::DW_FORM_block2: + case llvm::dwarf::DW_FORM_block4: + case llvm::dwarf::DW_FORM_block: + case llvm::dwarf::DW_FORM_block1: + case llvm::dwarf::DW_FORM_exprloc: // DWARF expression + { + auto slice = val.getAsBlock().value_or(llvm::ArrayRef()); + if (need[COL_ATTR_STR]) col_attr_str->insertData(reinterpret_cast(slice.data()), slice.size()); + if (need[COL_ATTR_INT]) col_attr_int->insertDefault(); + break; + } + + /// A text string. + /// May have indirection to .debug_str or .debug_line_str. + case llvm::dwarf::DW_FORM_string: + case llvm::dwarf::DW_FORM_strp: + case llvm::dwarf::DW_FORM_strx: + case llvm::dwarf::DW_FORM_strp_sup: + case llvm::dwarf::DW_FORM_line_strp: + case llvm::dwarf::DW_FORM_strx1: + case llvm::dwarf::DW_FORM_strx2: + case llvm::dwarf::DW_FORM_strx3: + case llvm::dwarf::DW_FORM_strx4: + case llvm::dwarf::DW_FORM_GNU_str_index: + case llvm::dwarf::DW_FORM_GNU_strp_alt: + { + auto res = val.getAsCString(); + if (auto e = res.takeError()) + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, + "Error parsing string attribute: {}", llvm::toString(std::move(e))); + size_t len = strlen(*res); + + if (attr.Attr == llvm::dwarf::DW_AT_name) + { + if (std::exchange(need_name, false)) + col_name->insertData(*res, len); + if (tag == llvm::dwarf::DW_TAG_compile_unit) + unit.unit_name = *res; + } + if (attr.Attr == llvm::dwarf::DW_AT_linkage_name && std::exchange(need_linkage_name, false)) + col_linkage_name->insertData(*res, len); + + if (need[COL_ATTR_STR]) col_attr_str->insertData(*res, len); + if (need[COL_ATTR_INT]) col_attr_int->insertDefault(); + break; + } + + /// Offset of another entry in .debug_info. + case llvm::dwarf::DW_FORM_ref_addr: + case llvm::dwarf::DW_FORM_ref1: + case llvm::dwarf::DW_FORM_ref2: + case llvm::dwarf::DW_FORM_ref4: + case llvm::dwarf::DW_FORM_ref8: + case llvm::dwarf::DW_FORM_ref_udata: + case llvm::dwarf::DW_FORM_ref_sup4: + case llvm::dwarf::DW_FORM_ref_sig8: + case llvm::dwarf::DW_FORM_ref_sup8: + case llvm::dwarf::DW_FORM_GNU_ref_alt: + // If the offset is relative to the current unit, we convert it to be relative to the .debug_info + // section start. This seems more convenient for the user (e.g. for JOINs), but it's + // also confusing to see e.g. DW_FORM_ref4 (unit-relative reference) next to an absolute offset. + if (need[COL_ATTR_INT]) col_attr_int->insertValue(val.getAsReference().value_or(0)); + if (need[COL_ATTR_STR]) col_attr_str->insertDefault(); + break; + + default: + if (need[COL_ATTR_INT]) col_attr_int->insertDefault(); + if (need[COL_ATTR_STR]) col_attr_str->insertDefault(); + } + } + + if (need[COL_SIZE]) + col_size->insertValue(static_cast(unit.offset - die_offset)); + if (need[COL_ATTR_NAME]) + col_attr_offsets->insertValue(col_attr_name->size()); + + if (need_name) col_name->insertDefault(); + if (need_linkage_name) col_linkage_name->insertDefault(); + if (need_decl_file) col_decl_file.insertPosition(0); + if (need_decl_line) col_decl_line->insertDefault(); + + if (abbrev->hasChildren()) + unit.stack.push_back(StackEntry{.offset = die_offset, .tag = tag}); + } + + if (unit.stack.empty()) + { + if (!unit.eof()) + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Unexpected end of DIE tree at offset {} != {}", unit.offset, unit.end_offset); + break; + } + } + + ColumnPtr immutable_attr_offsets = std::move(col_attr_offsets); + ColumnPtr immutable_ancestor_array_offsets = std::move(col_ancestor_array_offsets); + + Columns cols; + for (const std::string & name : header.getNames()) + { + switch (column_name_to_idx.at(name)) + { + case COL_OFFSET: + cols.push_back(std::exchange(col_offset, nullptr)); + break; + case COL_SIZE: + cols.push_back(std::exchange(col_size, nullptr)); + break; + case COL_TAG: + cols.push_back(ColumnLowCardinality::create(tag_dict_column, std::exchange(col_tag, nullptr), /*is_shared*/ true)); + break; + case COL_UNIT_NAME: + { + auto dict = ColumnString::create(); + dict->insertDefault(); + dict->insertData(unit.unit_name.data(), unit.unit_name.size()); + auto index = ColumnVector::create(); + index->insert(1); + auto indices = index->replicate({num_rows}); + cols.push_back(ColumnLowCardinality::create(ColumnUnique::create( + std::move(dict), /*is_nullable*/ false), indices)); + break; + } + case COL_UNIT_OFFSET: + { + auto dict = ColumnVector::create(); + dict->insertDefault(); + dict->insertValue(unit.dwarf_unit->getOffset()); + auto index = ColumnVector::create(); + index->insert(1); + auto indices = index->replicate({num_rows}); + cols.push_back(ColumnLowCardinality::create(ColumnUnique>::create( + std::move(dict), /*is_nullable*/ false), indices)); + break; + } + case COL_ANCESTOR_TAGS: + cols.push_back(ColumnArray::create(ColumnLowCardinality::create( + tag_dict_column, std::exchange(col_ancestor_tags, nullptr), /*is_shared*/ true), immutable_ancestor_array_offsets)); + break; + case COL_ANCESTOR_OFFSETS: + cols.push_back(ColumnArray::create(std::exchange(col_ancestor_dwarf_offsets, nullptr), immutable_ancestor_array_offsets)); + break; + case COL_NAME: + cols.push_back(std::exchange(col_name, nullptr)); + break; + case COL_LINKAGE_NAME: + cols.push_back(std::exchange(col_linkage_name, nullptr)); + break; + case COL_DECL_FILE: + cols.push_back(ColumnLowCardinality::create(unit.filename_table, col_decl_file.detachPositions(), /*is_shared*/ true)); + break; + case COL_DECL_LINE: + cols.push_back(std::exchange(col_decl_line, nullptr)); + break; + case COL_ATTR_NAME: + cols.push_back(ColumnArray::create(ColumnLowCardinality::create( + attr_name_dict_column, std::exchange(col_attr_name, nullptr), /*is_shared*/ true), immutable_attr_offsets)); + break; + case COL_ATTR_FORM: + cols.push_back(ColumnArray::create(ColumnLowCardinality::create( + attr_form_dict_column, std::exchange(col_attr_form, nullptr), /*is_shared*/ true), immutable_attr_offsets)); + break; + case COL_ATTR_INT: + cols.push_back(ColumnArray::create(std::exchange(col_attr_int, nullptr), immutable_attr_offsets)); + break; + case COL_ATTR_STR: + cols.push_back(ColumnArray::create(std::exchange(col_attr_str, nullptr), immutable_attr_offsets)); + break; + + default: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected column index"); + } + } + return Chunk(std::move(cols), num_rows); +} + +void DWARFBlockInputFormat::parseFilenameTable(UnitState & unit, uint64_t offset) +{ + if (!debug_line_extractor.has_value()) + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "There are DW_AT_stmt_list but no .debug_line section"); + + llvm::DWARFDebugLine::Prologue prologue; + auto error = prologue.parse(*debug_line_extractor, &offset, /*RecoverableErrorHandler*/ [&](auto e) + { + if (++seen_debug_line_warnings < 10) + LOG_INFO(&Poco::Logger::get("DWARF"), "{}", llvm::toString(std::move(e))); + }, *dwarf_context, unit.dwarf_unit); + + if (error) + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Failed to parse .debug_line unit prologue: {}", llvm::toString(std::move(error))); + + auto col = ColumnString::create(); + col->insertDefault(); + /// DWARF v5 changed file indexes from 1-based to 0-based. + if (prologue.getVersion() <= 4) + col->insertDefault(); + for (const auto & entry : prologue.FileNames) + { + auto val = entry.Name.getAsCString(); + const char * c_str; + if (llvm::Error e = val.takeError()) + { + c_str = ""; + llvm::consumeError(std::move(e)); + } + else + c_str = *val; + col->insertData(c_str, strlen(c_str)); + } + unit.filename_table_size = col->size() - 1; + unit.filename_table = ColumnUnique::create(std::move(col), /*is_nullable*/ false); +} + +Chunk DWARFBlockInputFormat::generate() +{ + initializeIfNeeded(); + + std::unique_lock lock(mutex); + bool ok = false; + SCOPE_EXIT({ + if (!ok) + { + is_stopped = true; + wake_up_threads.notify_all(); + } + }); + + while (true) + { + if (is_stopped) + return {}; + if (background_exception) + std::rethrow_exception(background_exception); + + if (!delivery_queue.empty()) + { + Chunk chunk = std::move(delivery_queue.front().first); + approx_bytes_read_for_chunk = delivery_queue.front().second; + delivery_queue.pop_front(); + wake_up_threads.notify_one(); + ok = true; + return chunk; + } + + if (units_queue.empty() && units_in_progress == 0) + return {}; + + deliver_chunk.wait(lock); + } +} + +void DWARFBlockInputFormat::resetParser() +{ + stopThreads(); + + pool.reset(); + background_exception = nullptr; + is_stopped = false; + units_queue.clear(); + delivery_queue.clear(); + units_in_progress = 0; + elf.reset(); + extractor.reset(); + + IInputFormat::resetParser(); +} + +DWARFSchemaReader::DWARFSchemaReader(ReadBuffer & in_) + : ISchemaReader(in_) +{ +} + +NamesAndTypesList DWARFSchemaReader::readSchema() +{ + return getHeaderForDWARF(); +} + +void registerDWARFSchemaReader(FormatFactory & factory) +{ + factory.registerSchemaReader( + "DWARF", + [](ReadBuffer & buf, const FormatSettings &) + { + return std::make_shared(buf); + } + ); +} + +void registerInputFormatDWARF(FormatFactory & factory) +{ + factory.registerRandomAccessInputFormat( + "DWARF", + [](ReadBuffer & buf, + const Block & sample, + const FormatSettings & settings, + const ReadSettings &, + bool /* is_remote_fs */, + size_t /* max_download_threads */, + size_t max_parsing_threads) + { + return std::make_shared( + buf, + sample, + settings, + max_parsing_threads); + }); + factory.markFormatSupportsSubsetOfColumns("DWARF"); +} + +} + +#else + +namespace DB +{ +class FormatFactory; +void registerInputFormatDWARF(FormatFactory &) +{ +} + +void registerDWARFSchemaReader(FormatFactory &) {} +} + +#endif diff --git a/src/Processors/Formats/Impl/DWARFBlockInputFormat.h b/src/Processors/Formats/Impl/DWARFBlockInputFormat.h new file mode 100644 index 000000000000..0049949289a6 --- /dev/null +++ b/src/Processors/Formats/Impl/DWARFBlockInputFormat.h @@ -0,0 +1,110 @@ +#pragma once +#include "config.h" +#if USE_DWARF_PARSER && defined(__ELF__) && !defined(OS_FREEBSD) + +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +namespace DB +{ + +class DWARFBlockInputFormat : public IInputFormat +{ +public: + DWARFBlockInputFormat(ReadBuffer & in_, Block header_, const FormatSettings & format_settings_, size_t num_threads_); + ~DWARFBlockInputFormat() override; + + String getName() const override { return "DWARFBlockInputFormat"; } + + void resetParser() override; + + size_t getApproxBytesReadForChunk() const override { return approx_bytes_read_for_chunk; } + +protected: + Chunk generate() override; + + void onCancel() override + { + is_stopped = 1; + } + +private: + struct StackEntry + { + uint64_t offset; + llvm::dwarf::Tag tag; + }; + + struct UnitState + { + const llvm::DWARFUnit * dwarf_unit; + const llvm::DWARFAbbreviationDeclarationSet * abbrevs; + uint64_t end_offset; + + std::string unit_name; + ColumnPtr filename_table; // from .debug_line + size_t filename_table_size = 0; + + uint64_t offset = 0; + std::vector stack; + + bool eof() const { return offset == end_offset; } + + explicit UnitState(llvm::DWARFUnit * u); + }; + + const FormatSettings format_settings; + size_t num_threads; + + /// Dictionary columns shared among all LowCardinality columns we produce. + ColumnPtr tag_dict_column; + ColumnPtr attr_name_dict_column; + ColumnPtr attr_form_dict_column; + + std::exception_ptr background_exception = nullptr; + std::atomic is_stopped{0}; + size_t approx_bytes_read_for_chunk = 0; + + std::optional pool; + std::mutex mutex; + std::condition_variable deliver_chunk; + std::condition_variable wake_up_threads; + std::deque units_queue; + std::deque> delivery_queue; + size_t units_in_progress = 0; + + std::optional elf; + PODArray file_contents; // if we couldn't mmap it + + std::unique_ptr dwarf_context; + std::optional extractor; + std::optional debug_line_extractor; + + std::atomic seen_debug_line_warnings {0}; + + void initializeIfNeeded(); + void initELF(); + void stopThreads(); + void parseFilenameTable(UnitState & unit, uint64_t offset); + Chunk parseEntries(UnitState & unit); +}; + +class DWARFSchemaReader : public ISchemaReader +{ +public: + DWARFSchemaReader(ReadBuffer & in_); + + NamesAndTypesList readSchema() override; +}; + +} + +#endif diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index cbafff50bf5c..856c1f21d27f 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1237,7 +1237,9 @@ class StorageFileSource : public ISource chassert(file_num > 0); const auto max_parsing_threads = std::max(settings.max_threads / file_num, 1UL); - input_format = context->getInputFormat(storage->format_name, *read_buf, block_for_format, max_block_size, storage->format_settings, need_only_count ? 1 : max_parsing_threads); + input_format = FormatFactory::instance().getInput( + storage->format_name, *read_buf, block_for_format, context, max_block_size, storage->format_settings, + max_parsing_threads, std::nullopt, /*is_remote_fs*/ false, CompressionMethod::None, need_only_count); input_format->setQueryInfo(query_info, context); if (need_only_count) input_format->needOnlyCount(); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 95f0dc308a62..b678fa7b969d 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -604,10 +604,11 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() getContext(), max_block_size, format_settings, - need_only_count ? 1 : max_parsing_threads, + max_parsing_threads, /* max_download_threads= */ std::nullopt, /* is_remote_fs */ true, - compression_method); + compression_method, + need_only_count); if (query_info.has_value()) input_format->setQueryInfo(query_info.value(), getContext()); diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 4d313b9e4320..58f01312399e 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -353,10 +353,11 @@ StorageURLSource::StorageURLSource( getContext(), max_block_size, format_settings, - need_only_count ? 1 : max_parsing_threads, + max_parsing_threads, /*max_download_threads*/ std::nullopt, /* is_remote_ fs */ true, - compression_method); + compression_method, + need_only_count); input_format->setQueryInfo(query_info, getContext()); if (need_only_count) diff --git a/src/configure_config.cmake b/src/configure_config.cmake index 6e636e580c2b..5b4615f0793d 100644 --- a/src/configure_config.cmake +++ b/src/configure_config.cmake @@ -101,7 +101,8 @@ if (TARGET ch_contrib::ulid) set(USE_ULID 1) endif() if (TARGET ch_contrib::llvm) - set(USE_EMBEDDED_COMPILER 1) + set(USE_EMBEDDED_COMPILER ${ENABLE_EMBEDDED_COMPILER}) + set(USE_DWARF_PARSER ${ENABLE_DWARF_PARSER}) endif() if (TARGET ch_contrib::unixodbc) set(USE_ODBC 1) diff --git a/tests/queries/0_stateless/02896_dwarf_format.reference b/tests/queries/0_stateless/02896_dwarf_format.reference new file mode 100644 index 000000000000..e4a29204c621 --- /dev/null +++ b/tests/queries/0_stateless/02896_dwarf_format.reference @@ -0,0 +1,25 @@ +13857 1 null a.cpp 0 ['subprogram','compile_unit'] [13827,12] 0 [] [] [] [] +2355 5 formal_parameter a.cpp 0 ['subprogram','class_type','namespace','compile_unit'] [2349,2076,43,12] 0 ['type','artificial'] ['ref4','flag_present'] [7415,1] ['',''] +9267 7 imported_declaration a.cpp 0 ['namespace','compile_unit'] [9241,12] cstdlib 200 ['decl_file','decl_line','import'] ['data1','data1','ref4'] [38,200,11139] ['cstdlib','',''] +8880 5 formal_parameter a.cpp 0 ['subprogram','compile_unit'] [8860,12] 0 ['type'] ['ref4'] [8821] [''] +2941 1 null a.cpp 0 ['subprogram','class_type','namespace','compile_unit'] [2915,2782,43,12] 0 [] [] [] [] +4165 5 subprogram a.cpp 0 ['structure_type','namespace','compile_unit'] [4093,43,12] _Tuple_impl tuple 434 ['name','decl_file','decl_line','declaration','explicit'] ['strx1','data1','data2','flag_present','flag_present'] [0,11,434,1,1] ['_Tuple_impl','tuple','','',''] +3087 1 null a.cpp 0 ['subprogram','structure_type','namespace','compile_unit'] [3067,2943,43,12] 0 [] [] [] [] +73 7 structure_type a.cpp 0 ['namespace','namespace','compile_unit'] [62,43,12] duration > chrono 459 ['calling_convention','name','byte_size','decl_file','decl_line'] ['data1','strx1','data1','data1','data2'] [5,0,8,5,459] ['','duration >','','chrono',''] +9151 1 null a.cpp 0 ['subprogram','compile_unit'] [9131,12] 0 [] [] [] [] +11619 5 formal_parameter a.cpp 0 ['subprogram','compile_unit'] [11603,12] 0 ['type'] ['ref4'] [11491] [''] +offset 16023471350623073201 13857 +size 5435489408010792888 18 +tag 17905751708343214984 template_type_parameter +unit_name 8657192746161260862 a.cpp +unit_offset 14905628884015248742 0 +ancestor_tags 830906299496007322 ['structure_type','compile_unit'] +ancestor_offsets 18099428250664153593 [10047,12] +name 13953303733802802013 tuple_element<0UL, std::tuple > > +linkage_name 2553262765459789953 _ZNSt6chrono15duration_valuesIlE3maxEv +decl_file 9506789155169696987 ctype.h +decl_line 5651408086801131471 540 +attr_name 8508813696511238777 ['linkage_name','name','decl_file','decl_line','type','declaration','external','accessibility','explicit'] +attr_form 2204414726958745669 ['strx1','data1','data2','flag_present','flag_present'] +attr_int 3798582667264880443 [4432,11,0,0,1] +attr_str 13716946669142575194 ['putwc','wchar.h','','','',''] diff --git a/tests/queries/0_stateless/02896_dwarf_format.sh b/tests/queries/0_stateless/02896_dwarf_format.sh new file mode 100755 index 000000000000..93fb9e0442be --- /dev/null +++ b/tests/queries/0_stateless/02896_dwarf_format.sh @@ -0,0 +1,14 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_dwarf/a.out', DWARF) order by cityHash64(offset) limit 10" + +# Select each column individually to make sure we didn't mess up any of the `if (need[COL_WHATEVER])` checks in the code. +for c in `$CLICKHOUSE_LOCAL -q "desc file('$CURDIR/data_dwarf/a.out', DWARF)" | cut -f1` +do + $CLICKHOUSE_LOCAL -q "select '$c', sum(cityHash64($c) as h), argMin($c, h) as random_value_to_sanity_check_visually from file('$CURDIR/data_dwarf/a.out', DWARF)" +done diff --git a/tests/queries/0_stateless/data_dwarf/a.out b/tests/queries/0_stateless/data_dwarf/a.out new file mode 100644 index 0000000000000000000000000000000000000000..963718c522f2a5d5ccfa334a61578057a0d55d96 GIT binary patch literal 64416 zcmeFa2Y8fK_CNmKck&L&%#efBQ4ZEyjS#?);^ZT50?>qC(kWl?U&-ed4J5SDg?mhS1 zbI(1um-o$@oPrs;5Ssb3X_GbbE$t>L*(zY?H_8E&t&P*V;yFw^O|!!v#xxn8EfGTH z?_)g}hby`bnBw&zM+Z1ok~I^X8WN^>o&A1t$q*B3(b7-x400-eD4rB4`Alqv`_qe; z;aUiO!YSz{w#Fmd6ei2?!bu@m<$ICBW;`+%#k)(zyGzA0akR3hiA{TwO$f#*d7(HS zVf2gVr>2F*GxZ3?CclYI+t2^PS9UV#6{>heN?<6CRrxcqncf+QM|L^hpIOZFRC=A2 z!)}$HiK)ElH_BU4GA47Bw=CIPQB~iR+%#@X@|et&y6Tj44kx`R4XRT!=Pc0lwb~w{ zneokV@>4kdsAyDjjQhb);WA$r!KGfS+6BLu3Z8x;C9d>!!* z=tNE*$mxhbtrL1)C-fOewrV9F z*NL2xPUxR?qCeHGctGgyt4`#A+#&zSPUvrf-ckAf2Z9ENCQ5#{ds$_5mAlSYTOJlnUv1u;`ED~JLNd!d<;C@0A9V71 ze4f1dY3`hy!Wj(>Bz3-TJanwAs6r|YGL>-^)phQY;yO=eUR8x}dRh+X?$V}WHxjFE zlnSP$5~-=Uq@p1$9qzgccd56yuFmaSS>th6RFzjlI7cdh8LHf+#lF(zh+5_?RH+1K zaC*8wm13{A8tfjor>WFa$QiFpUiWLuHm$ z*ZVa0f~v-fsxtRHPhEW_3Y0Th4*Pj4&L=NA$5K@|mBkfRT2*mXb)DDasZp%RSyKhX zxIMMCRn>0JE9DXK>U?FTBSuh1I)=QJe%G>6h$LSc(Mu(L=Co<D!axrLg}^{>+yx+* zS2hCjxzYES?XfU%B}V)%S|e!m6OcGVg=fS6{sSv8BpBKnMe_~63PWv}cA4_m{52y^_-R*iU(nkSg_Le*(44WXNj zz9xjeS^1kn=-E9_ray5yBOW`=F_S2JEOavsqQ7RL!=V1(ehb~plRPl4fuC%^P%wER^yXVnwER<9-rR6 zKezei@R!&;?W6w!a{JKF@r)Xn4Hx;BQ`o@+?f4D-9r>wCI=G+x?~z|8{~q?gL4N9* z4(?|Ei{z)}(ZQYUe~SFn6&>8n{)fm_0rQuc2rKXvH` z3)z1a`Kc>En9csn$WL9=!BqCIBR_Rb2jkg)KKZFjIvCCVTT(Qwd^7y~Tadfqi`-3< z8o<=H8{xUFlU@Kr?xxYDAmwg~nMENp5OVfQWFO*hdsTU-OZ%q%BX7gI=j3kqOK$V$ zhYO2x()Ogi4w>&Ff%ceLkoJ9fO4Ps>P7m}57HsH*#{cAtLD7t%BvCXG?FXU;t|dk! z#u)scNPXTTH2lB08xH5b@YU(LFZ^W76?=2v{jaYl1Q<#{c>96!sDaSSj8EmicG6Y^ zYxN@*0WAfUB@_IcZ@DE@;WFs1DvDu?v7PRc>*ut@E0kXix>BwFHKcR^QJ-kL6txj@_JHMy zv^^VMY&#P%O-&E9T}Q#qFBd}3)>W@rb#0qWj5KYqQb}%GnUwi)+X}E8sefvS{vWs9 zsv_iWGIkFqfn9Q&Uqv&9*5mwIOKZU}W_oQKaz1Zs`i1K8sSga zXO<`FN0TL42AntwvMn3E%2D|{%a5un$ZBwUlt1c8I?f-}t;1~-DIAHDKAp+8Fr`?&27GV}mEZEZC&gd1*Kv-Ipo9JI+F+vfkP zy?w(AX?vRg)qZKt^HCQ^k~s!|*gP&QYRewbF3Wj7ScezT&7mmJutWM6f8T=qw9Y=9 zTN;tOAv`c1SnG3Jf20#6w(ZHE0xia@ukn8Sv{L3^733#d+5q(o*{1rH*`6Sx9@_(* z;-OQZ{#f;ym~$A}VLre8Re9vT3K^Pa%}#KAm+gzv`>CxImJOsw-9R8c)aSNarR7Oe z+40{3f^3jf%>G7sFUSG&S<{p4=XxoJD~ya4;KsOPsgEbZpACB+M*oj$17osHWrZn# z0bpQs_f)0Gi5jwMe_JNgUjX^F+~KG&IaH-6V+Yz^ry!Z9Y^&`8_Au?+8f4q!52x|d zT869_hECLh9aIOYw)^uziXCWcVLIviaqjrnqISphGLVA<~Y_)7Ewm5u|zI&wfdjSAhTb7@>13_GoWxpNM<} zqH%xekA}*1{QCLZmvub26{Qq19?<;Hq7SMVRNeri_BQJ-8MS$w7S%GC%NRklA5D`2 zHg{@UGQ2F+tl5WEBa&^9I{r6k3KBg3@Yjc~3lbo%`2`Gb9rX`Z{Vn5%Us$02lRcrk ze?D`dZHUzP0BZ`vwoPTi>^3ANl?&8A>-a@mi7+5mxZF}_=v zE#+yI*U|WA+N-nv3E`pr58I=&@yD88$Kxdm1orGKzS0M2TKg->{~O98dgwrf@OO24 zFuMt=Z(LreVpQ*&e?m3mSuti}xvgE=3so@%>W;ad@YnCo^gGr39V@;+|CaTle}#zr z|L^zzTMN+sb6tIHLq&t9%%Lr>&;q9oX=(CY#$8ca*U!i=@;wqb3`@i% zzzM)RfGfyMPmb_W@4_{uKB& zN)OA`GWZLCTY$a5w*hYiei?Wd@aMq$fWu%9b8l9hKTi`)(W1YrOZa9CzeK0I9{ay) zZ%+i)rg!0O_XId;F4pvq_V&X>kBpuX*(X2B*%-c7JH6k;q>O~YM5g#l0NIF-v=deI!w*L`JXIr**Y2L&ES<_!hu&)UUn3{&4*TW}xs#0DVx8$n6h5GZ6m+6;3-R z{_y32@b5t$hWHb#;q`&=o;3YH_yBA8;y`$&3g`0%Q~$z1c##TEvc~rW!hHz87V?K! z!_NtXZ$bE7M+tuf;ZGpE*cxA7!UD)9A0Rv&?S_w>O#KQ1^1nYucu%w^ieD5Yf39CX z^$)oS=W;dWPYcK|Lijj@cT_%f=HgS~w8Zn551hom8DS$4H`N-pA|RX2-KftPYYndr zgzrZ<^(7spO|-)Z|K%v*(bOlPZgy5SsR-YJa9TF{bzc}rzfgq_vxb)j!fOz|3-Je9 z!_NwYZ$|j1M@fG-!Ve)l$aZ;w`1@7-&h$Hc4EfRM!$^K-@l%fxUU-ynD*GB0UTDof z+*CHyuf}71>MZSD#|Yo2!aK9gAqqcAyKubJ4LVt~Z`7AU$P||yBPj9E9>-7MdzAR5S^1iR_) zG?qzmEN=(GmmielL73CRw>p^bW$-P3^-mUghe#gy%8BnsJpB2fK`@Yy zo)Dq^P072-!WSYh3w(VrpQLsX_PB-bH-2%Pk23J3V!p}xbs`V3wf1)5q7EZA!=ug3 zWPdje_Yx)QhA~YWHyrNkyNa+r#xV10E&IE5O%C#hhEF!m2?~!k z;zRs?d7R+?9{=YoVBW_u@86jBZA?YYdoI~MC1t6?=Dit{uUow7l9;MK{phX{{Y?LY z5I@~jvWC;$BKo-`@?WgM0UQ5PdD5wA<{gwmb>D{jba0>x;a@+6&HF?&e$tQUWWfLY zZ+kWQn=szf4_!b1Xw5}Gg|wY2oW~T7MF32`3kfu`@H4oip3!|x+$k%Vyx`Y`9?g%>Ud7z9p z=GwX#UU2^%rXQT7vOfGoyfh6ABD4bALBid2cOMFPkv}!`x=@a zxfp(O>eA^thyDGGo+>D+f2Cf7HNuW1vo2DgU6s#t^H2 zNP=-%K!_GKG{H!b;gr6smW7-)%t*H~4IgHVKY}S~gfS_=q|_OiY)n6bm@>+kWg#|j z8mVc_(to2HVYV;8VSf*WbGi{>i_x!ax|XP9Xn1O_(+z_?-AU3cy*m|SfzG~u z(l=Yzx)^5!N;>@avZT*IeF-V)DV>+}W{6iMO-|QV_H#*dP}Ec^>6<`}41`BbqmrhC zB5x!9sOePF83BI|m9)h_gG$;jM2nh9CC%Y1RF-rum9&K^k4id}DW6K(&m`52nnfiY zN-Us~4iN8VMQ2kA!#@ywRM+DbsrXm*8)sQu~L%emy|@34iReylH!(P?l50B+T&KkaV8vS zVLe2cLkystbQJ>$MT+>S>A+##goy9XuAZ3t4l*VQW6j{KA*3chv;=ZaHr7E(wnYjH zi(I?q@1NQ_ki>LtXxLfd5t9?=$6_`TQKOw^EDjeDXCxNUhH%7uZP?<7g(C7*XrT>X z9I;qBT@kn5xI|;hW#dDt8$JwE8=?TbiK<#XGPqg{J$sv$R)86 zI}gBRWlBCfHsTfwkvUV&aYx*%xF}3ZURoUS09jl_9wdKCX<5X>l0i%Hlt(W zXJ>+>4GR0ai1<4EQH<3G6B`Ic{HSI8tuDeYhM+=3K1mTX)Afk&M)%f-Od%iBTu-LK zIcB1srAO$@I(mXW_9fC@F}%(5ym5W>h&7@+MeL=I?~N{>>?&hVNQY`+DSC9o>)j{8 z*-M{D6fJ9nPTq(&!zYc@ll6%Ag&wUx2}*{N{@IupbgQaywfvuaB2Zg4b|bj zaL2ddBsMFeIs!M~5*w}oa{Q!aKkb+)MC6Z{=DL2Nxo-9|j#5R?oX^HO>V$ySRu1QL zv5r#jt4|6hp^Ns``&UqEMzT_$-Moj<0&PKa-mIq=V8aoW|k;< zZR&ov<1LdyxzVOQ?fAeXBoblzyRnWBt$AW} zd5)=(-%_Ec*48=ZM*c`;D|q2BDD0_l$9~Bc&Xym`y36OpzeBJ#^9jfMk!@4}|DZUz zPsTbv_!ce;E&UGZAq!G*J|6B^YZaQ<@JEC=Ue|`*;n*%B-yk_~qNS44#nF-`N%La0L-}-RIc#iJzz?-+!OuK1xGG z{*fa+u-kF}C`Hg6Op%o!*wS`17WI3c#sIr9kI^u!XATD zu~l)^y(QN9Un_MBQK#PKc4iogkD_P=*}AiDnBY<2bY0ArwUCm-YX)K0hC6jJhuVIV z7NqCIMKtN2^%|{2ZBd(>oi0&>IIi<4V$_w*&M0vv`PZ=js;iyd#Rh-4og!~>hKs1J zS36_G<$lJlmpXfk$ZILi+)JJPrPIZN$kZCbPg6t|TzE~abA+HYIg(5bOv#Gr28uLw zTdVULp-T?d4;kAioI6#<1TrRV&#xWZqSiM%?-EqhuJIH<>XK&Xy@JwrO(s9&-!CF_ z$q6MNl1`VY-!sb3tc4q5oqGgHVSa!7y^3iz37L9Xt1}*}3X&*OL_@q-IOj;a3$D&9 zW#dXbpEPw|q|Lm*Sr}=j=y1(vmsA1Xh0+_XLeG@me#(1}^tPbnTwAI5a@RFGFXZfI z5=(ANvvUpSmIXkU3xzA=j0;^Ekpn3lgo}hLf8vW2H*xhnB&PtxwZc`9x+d0D5Sair z*Kp?5<{7JmD|e7_MY!{6Rgc1{w#)oziRY8Y`4#2j*~obm2d+Kr;&LbNOVV4cLSL5N z70Uaj^jh+>so8muD`P#ev@sT;$e#&f5g+>^WB z7p~2}sf%%|m5UnMjp45_7rB|sZbvV>Co)!aFvFw9k2;G+-UJ68pG3JAV1;utXf)mihEXW^ThVn+#KNrbp5Xg%Vt+g}JGnpZIyXW) z)A&Gi&5syld?dR5xG-zc-iH*^W}QXmux0#_8Y#=Xgvc3xqDIO>H=MrwD@FuHpaqh0qgy8jJEyALpW=qE-G4;V!89?4+zXfC72&Svy@ zEu$wcVf5tFjGlUv(bHcrdZwMxvwa7Xl;@^1dVVpZ7uGU*X$Pa1A7k{&yNvc8V)UvV zi_X#>uk~j1dOD*wW;1$oIir2ejNZD2(cAYjdS?%#{U0)V_iv2eqf=^652x6LQzcIi zmw1hFS8Diot)*n+CO=N) z7MHz`iBmpeH1+R{rs2CO2%8?mC}$|68K*OvIhRqco6+3Wj0$fBT6URO-YT@x_|eZ1 zF@DU8jK=wbozUwqbcTZt-?>t7kDj6MaX7ufyjJ|(_(LZZZ!%Gu?c!hmG{hrak296}+ zzyHp@pAGi?l1XS-Ngkupd5k>E8I_;UXxU|qmS4}PVi%)xpJBA(Ek@q&8C6Clld!6x zjH)Lws+rH|yb4COS1_u(hmr4PM)iMU)X>hTF*=2$G|@|BrNdUvWVCu7qcv_uYik)@ zbSa~C*D-3ohf&K@j4pnM(fUsrUGfd1OKqb_=7u4RE}O`xbpfMIZbnyBGunJHqbqkZ zy6Q1TPrl3OsjnD4?Mx+M&kSMo>@-3n=PhM4zm8E6j|3wZUdX;PH!@nhgVAqxF?CX<>ic)H85Jfo>9d$Kox!UbM1RI znvYG_bM#R&XbyHIFzx%T(63y6l|Cw;HU!~bdbK`k7H#ywox4RJHJh5Vb&HG+5imGG zR<53>M-1c@JnczdLmt}9p>4%&rbp{-Ox8tek{CV7+)mNFr^(6T3f-$aR~h*@?;ruU zkcS4{ezf#DSNG3;w8~*JLakn>M`cxkH!6#!hR8?F6`;@%UtOc~bm~0XmC?LOGDd^m zs5^@dnlh;*iVd2`m`TV<;P`r7>p5D+Sfj7io#z>I14^A|(3;VV5wsxGdXAB?n)Mb+ zhbG|ubZFIO#*)+j(e=7qbZI@aWW0;@^^{ssC#lgYjn)cgtVuG~WqK>6M#~p}YP8-` zsj0;i<*GM!xkcA&!sYxtJP!qz#pgqlyKW0d=t65K?S^LA>B*sScVhFiT2F20ZB*&P zS!l3$oX+y@T9%z=rjwoX3@vUy7}qi5MZt`;myuRum4+HyEfs+%6g-!n>T8Nh}fi&3b^vrqw%*a377N31{HZ z&E`qS29DNq8!7uurSSOJMUO`E)+ZWaO&1V-8Au72!Zp$E;Aj){osDFXrtK8WE_yVQ zx9NVO(dkFR({P;`J}QA~?!qqp>lbthZ=|-+*cFeakBE<4E7?nr`Vw6<61~2%OIRai zyzv*}Vi#@iHPX7giD*qRu%C~-E9uclON>TZfi|Wf;QS_v(!|`0D9SYACRZapR30F^~kIk||ok^`8=F4~7_YN1ee{g%8}lb2kLL|aLZMk?2)T@*mBmF(rm9-=ix zf<})<+7(+#k0yT5#!%yD#6sRi+PGXvk0yFF(r)fbdNk4Fg6Esq4Bb)7d`x|df(DV7 zT@=umipPbk!kSTRV&PO>v(pLEY6BldSyQKV5t09;i$T5bFilrdD7hNxv65Z1tK4)C1(0hcd+E_gySxc|5%ACM_Jp^AzwZ{a zRv6*kx+Web#kxgliQjw)dbgfo7YQS*TXf>Lw7uP}yO#K!hF-NBy+Tt5_fsw z(i%(ftpCLbYZesTEGYUq@t>F#Zhy!i{2^CVm$nf)K4^v>ExtkSc~W)Vus6@Y=)5NEh1dR3>B>X`6RD# z4W2$3HA2M9vqZfOA(oVz#XXb?rP?AKBId_X>$M1{i1|-|3O1LBIa_9(mEmmoEz?y* zikNFP2w1_AX)6>Q!%?~b{77;CCJRx>;O^&BLOvCe1VZ?|&8$>rj^+($vy0ar} zgXkfu5FWlk^khf0c9~c&dWqg5=G6|xwTM0hw$*x%4wq* zDq2ONi20LW5L7q~X%R^xW|74};|GUS%F{t;q%5?G z5h7-fUlD1L7BN!9oG+>=7T2+QxU@|oS%if*iwh_&CEFrWM9dm-O(@$a5f;`eQbo+~ zLX=}MEh0_C+{b6S0pVtUkS<~#64Y2jgIYy~h`A(SV^)j2&>~aB43dqLDn1CKMT|`n z{G}vK)*{A;m@eF*$^OnNW~_*D$sksnqpB3fiI`|hPzjPT1&$XnX9X%KcU}|3L=m$! zh}bH!M9l30xg-I$pCn@52oNbn5GRY62LeRJdAf-CFhEqC*&^n@0U~iqttzI7(jGs{ zos2NGBVq84L}}t{at-*lTm!x**MJ|(#ouRg@%Oo0^?e~%eSeip!8Tg_^?yU`5{Vy@ z2-jy=Hu3}Ong`@!XSZDJJSZ1C56Q*O!?f5zp-dGq7l{`slC?@r6JgOUV!DWVO0YnG zn5+?-L=J|<7BNG_ydc7`zY&ORGRzcR!Ygvc<)Y7on`u;x8wj=n(YH_BFgRujeds2U zFYF?2EWC3?-@b9#aLf_B6+YoE9u#I2^770Bw?ru9d4BChloa&K9<0 zPIQUT$Hlz`id)!HIMH*2E|XX)Y-h%`A*fXJ?-Tb893?_e+$4%I62)nWc+Fh&?;95Z zhezncHid3mjueWqJMuL|0;x@%%WBZ+f1f-g)NUAmBNc375Zm{~di^1nx{nv}ZtE~PVV(_|vzeU_25(?$~k*HeSfnnHB7YV0t z7IA-v^JbAS6O-fUglSnf@vxDQqh;-st{LIlmEsnWuuyLiw?d3glNRm-&Hni|k)Zl# zz6#PJZWjq|**7zxRoo#G%DK9+^;^ZABB6rSQ~tX|f;StAx!B(#?iLAIHcp35@%Sh& zkfLk*aAe^saXZjGA|ch*cn@4x%NpS4M|XIyNEmpew!1{a5*zJ8az&A~WsA5^B-H5A zvsK(L5=Q7QN*_`l5D6FQTz`~uyG6noHrAFN;CN6ZoMWR6O_!DWkVuFPqz<-+MZztz z3`me^yhlXBeU^!he_D(PkBWrbWrW}vv$j<{CK6`YSi?}eY!Q!(ga(zBOT-f*VJ};z zMLa1IPLm4xOAwNt5(%#f7f(%F#nXt&<-}^}7=tji3I}Q=1GI`~M8bodM}N39S*v(f zBs|KKija^N@tjEL9mqDa^*n^g9CMqu&JXs8gu5j#8=ME`xQ$5v1=yS`=xkI_8pRUk zQUzsKVYs$MyeJYbRGDrOFNuUATVU{P5ig5`5?jcS+A3ZV3Fk4Fs)c(+!t3l0$wRAn zRV2L4L*o%4uOVHYo%usxrq@Nn8XkrN#5h_Hz9~-Y689Sn@^6czqPQ|R-V(zjH;R2? zt9VBwb+@_dLDL#(_qSE-7fBCsBVl5TcvmF#l*Unn_e4@J3!zoKFOt5r#`pk}RM`$y zj1NW9Pm=H%l5icQ)S3kaHjA(EY@`k67V(itDwI{8ONd2M1O)w7Bo$is^Af2=d@7P6 zLeh6J!+P-t@kdy!BQ^dcl3owkLAFv-2pS#`N&m3~{S!%RjsM2eCr$BZk@U9CRbH9! zFCyt~sb(tnuq~}msYxABi}(y?3W(J%5ub~sa3*XOUx=gxo7vNSDU!;h%6_B$RU{?K zxK^XJiKLMMDv~%TlGFrPRo}mfq-!~b^8Z~VZTI_SdRxUIk@UVUtJ!+-5Al^qdNe5c zt>SBulq=Y#E-HPbbXX)6Sj-xz6Kd-IjYuk&;T)tgvR-^Ez7t802gGxgLA|&_d@quo zA$zupe~P5HxUJIzY5gFQb_B916>Skeilj=523&0>$8&_AM3Nf(t<-;sr1}t*%w_Pu zMbi0}Apewu*A+jDUqsTMI!tQ4_>cInNcvm`S`%v*Nvk`OXs*gNTpdjMt0i6_a$E&w z?z%oAwXq5gAI+0_DreJ^6I(@?p6uolBC&KU0ljT%JCF0bR<42K#A^p@*RF6{5|yKyRH77`iM+omDRXa#OTS_Q+}i$Uu6%YvFaKu z1c1cCDf&v4lj)ASp<;6-+U{m0Ao#n|MRCl(QF=)~eW!G|xK^@z*p zPL1U@Ob@Lo0};Fpg4PXrgFu;~-zWU?&XI=(HCyRL1A6DcdDs+|1EPnvL-eBZ6bt(q(!RLT{yv>-1)W;$5WVSrc(Jf7z;);du5F@^O#gDEf408O z=GVFrTH~xm8apWcM!KK7P4q2h@vkD{Ir`y1sb7eQRO;8`k}s9I|DrGA^plGJj5uz+ z2z%hleiy04cF?6&x}|E~TSc7y<($qa+;(27=OWA!=Q@E&H`2O6cPtZhtrO!4ZKB^O zVr90LAy&lnVm&jUA8r0aY#i=Mz&h)}P@{Y+p@;k zE)t{}o=1#%flQF$&_ISt4L(KS1uC%Sa-9NiqFw=a&FNlw;I6sWNGB89#PC8^qt_U; z41ESjD)ZY(><%$Prr8f!QYV0S@ntRqjg%R96{@4&HQlN^FXRhoH;QDbix)B-?=|Uv z2voS6Aay6z96F&v>M0XB^)29BsdocSrG6Xus4}BsUawe3$ufQ(tmo^cPCx72U=6-s z3u~lGwf_d@6~R)GXquE1fp&U-PP0>RHx@D4Fn3cnMnDto&4hAo1|Nqde4QuaK^S1h-S@v?$G z0h!MP>OM7IWOV}iC?xnH0xu32`b7ki0ntL|a?9H;Cd#D$3&z$Eo9qyiWnewBtMs~y zZ`MMu)1_V>-{HZBk$ecsCy(-gvJ@%sX(k_Y za%aXTaeSE7$j4fI=tW0jTsQgHj!*XZ7?4i|`KYmxju83Sl8-VqoIVU4+QYC7t?eD% zyZ_K`V!@EQ-oDti&0^9Zi^UOh@FnX6mmKcSkO33? zrGr4cp5-qccoY57oa5j<%?wSa(gR_bGQQI+l;ObyvS^Z&Z$=pr%yF3j%SeA&shp=f znVx(kUDHwmL^IV<{(|+#$GacFe43WlLsiUlQX#8vCRdmYKbt{T$7@9hWg}@*49$+)u z7}Il<65PI|#2K=UP*a_$Byk16g>}CWe-+Lfr4eJkS#_wTVCP@jfz&PtPC(OUlU-$% zpQCCyj6Bz>Gv%w$pB5L~>3vzEaTn)C9MwBfoTUK~ zWVqr0JCzubN&*qcTN+Lslao58GCwmH)g{#IJRHpZR{40YuglP@vr*kr#jZqxf6wPSmR7U4RwsyfpF31aMcCpITN49d= zqpzmw$-Oq@tWkD`rO1kFaoipVya=c40pB{DvIh6>&0;L4OXsiV5Fv-Aiv==->k}vm zOFu$pyhNlR^T@NR$#nE@4ij|lFZ&AWCoU6c3RvYT*<8^`!bX95&F0%AV30&*y3sj+nMY8ig?l zRU>FwkE+C(D7tHfY-4hiz77{G7GOfV-NHv)*Zbv=Z-+mWd^ZSxt>X%Iqd&Ba>y1q3 zCQe7%l%~a$s!a1{zw9d3QF{?HI|Co)n1F>UF1P#~*_9AH^sy5-1|pm|(7ypz0Ga?d z1O5$|3MTqD6uKCwN2Naqmqwoz7he)>4<5GH$@>mHKEfmHPk89`@6-(XpJ#Rl7S(oK zcGN^1xl+hA0S|+|xMH765Bj!BSTQ|tZ%-5SS-G%$8ziqH*LpmkmT=x)10JZ0$N-Hk4GXLy?3_SNcg+f~A6#Mw7V zjxKiivkZL`NSpNt(jx+T>_knYpY0uiJ6?|P+N%uv4aQ)5zG2U|J8!bvc0M8i(Ji}BCDio#QjNXNH!=%?9t!5N-qcYZ|5J3cbHI=(gf*qxih zoJ80f<}-#N+z3Nj6P!2Nu06jTpOR=ZvMBVuRg0u(gi6sjuY&9phw~{ihHaO~H=K@t zA~GjsTe-{V5Aj{?>kLrBqDaOks}UpL@thHf6uN-Q>E7trWlVKEV~kKlW_`APml5xH z%Fl^bV7KM(GEyB+8}W9Z5%HS+N{Vj##G#)ijPU_hV%*zABA+rcAabPC=Yv(djC993 zq~gq%^qh2cWFX%dZ@y7EBHR6IHXFL_JuQpNe5h`x0y2#)i4Au`?@rW$H@`eZOE98DtzDSU@r7~PQ+N$8VrL@zQT{7G@n zOvbN~v!1Z8(~c@1cB3=jw)ui`!)ZGlmZjr!G|~=hP3~4G*30oC+Eu=xZ$^puO||>j zEC)=Mt%fjqIDX(t($8lMf6Yk9H&T7Z;BSq{EF%JQJk!RFZRRolk znxhAAHiIvQNg|Od=^o=V5?(U~<{Kl>u)j6*axa1 z7ad4=kIAter4&Kk3H1n0?tH#jg+9S<+l~(%aH$giVUzze@W(j9PdCC*eeL#Hh70lR z`L=6iq=R}sI@NVX_k1G^{hs{`@Sz{DJL}Q;m@G;zw+Y(4z|Zy)>Zr@{6gnhUMUtYs&EWkoMn8vhoiqgn(XD9~{Eb5xLlIMTuL122m>howho!F!aA{GsEC1E$7%602Sv@mW<$3Y_a(3nwc++wSa&qv$WZlb3%W}M4 zO}ezzCjSXYDW0D*Z`S+_H{PJ}AcgAMNfdb&{)6lB=r9`pe6`9)|0MSKSF>m3bkt zlTRo%rS0_>VbY1jEn65Achrf*%{<>zTMhdXI)O6Dz`s#OEIdyrQgIVSDsIxuMwNyC z7rks`JUWF56N1`r{A8`pQ(9eBR)-estElv#k-Lj(w9@KI`UlkXzn8N(mTXZICLD)E zmCgyofEM_#$3APFj=B*jzUgjjyJCQ!dnx4P2Ihi!guI6Oo%8uqFVybTDBx0JK%t=I5 z9nDF_p$5_>g6UN=H!Cw2YvrMKVip%9CT%o|pw3rZ>vPMD=iuMjr`A_hoLBF0*Z6Ab zztEr`_2K`^Q%Gi+r@Xk{>vNZRydIw?Z+@CPC#P^mLqiS~rBx2*bQB3jutreEI6_X~ zf8dyj<&=7J3L)M@1c`24elP z(iUK1&m(?t>2+xFk@3FTdXE~Q@gIze=-(8{|6oM<%_a!CXBuOdxmE})e% z+{r5{I~FR5UqMIJ`^a)74`yC|1D;qxP@l>g1y{AyoWdjRT~LAvlBHf@{dBT%dGQsf z+*zsq$`jl)wYFg-$|mnaHoEx2)p7q7hUzCyAEoeYC1_yxosk>&<_Ub|Q5s zBhBruszx=fZp45l$0-b^b?DGaeHC)>KL&^9qt+_R@nlp8Sb7>3V5l19)SD9Kwt_WS zRapi0TB?e@Srjrdo)wh6GpF5Ar%Y|__zf7J7$lW)T;-{&FI^s=sJfJs%-LM3^+)D_ zHT%_chw?1Ys`jbqZB7WG`mdxT%?B`{qUn^Hcq~9Sp93iL)pk^T$6t6O;~T0g$|h?y zn2h>7YWEFuy5Iyci8JT5=Jcwweiij`F3$$IV>$&vW6;|n!Pdzb7MRSwV=4H%nZop z3Ki>ij~6@j9h=y{lAxaAC}U96Fxb8($Z+5ALwcf@h?sXY@KH_>`3_+S6JF+Zh&y{LE@Xqc6idg|)rp1d6Tap;E08n$Gr%gx{{3h_^i zta8vEOu4*qhW&oXRi^=g{W*24C#@4$+hOA(HOKP*heLNKuv?=F=D6d`qAi{Pw_H6A z^A=Z@loiJp`{IjJN^5E+jGd5{Io#|~WfDiO^(=}7t!@2U9d{I|6zBdJabzVpkrdQQ zh@un)7W%vr>NwjWQ}Cc%%H+{i1}48j^KD*31WoScl(v(Uj(;H#s>NLDuaB;TW!B$m z^2alm4@%l>%P{9rbEb?ftEaied(3~XjV(xY{_0Pn1F=_3Ts8H?#i&jzZ>%W80@PhmS6w=Oyt}-yuC%zS zoVKf(U#l#s_2LFVUB&s75PZIBL`KqlqX3u*bs7sr0s&EVN>qK7RI8!9v8EP^LU;{& zuX4f>1T?DG;t%$*8WTif}odmukLBa+6trs#L@(6|+i3 zVZmheIu+N4D--bfs>?NGRHo*uULk3mn6J8?aZ`CMrQoZEVwIjs7EG9Q0j`9t*g+Qb z=hmO2I-Hu4(3+B37GA6T*yoVq5R3zss(N>6QJNw8g+bWV@y{@y6GB|X}a9BA_*`{Zy<{F2O>&8NB18ZQGG^I z^(s+6y>5ARE$Ki3N+LOxFmg&caJb8r5=3PWNAUYNj^C%WXTL@LfZvi@PzC}V77_A# z4F#b$V6&9sRAL2OjJ2|ga#`IfS8%N>uEQ=BYTvR-Ms$MVWfuxo)yoDqPWCllgS(0_ z<_{}Tt-W4%9Tgb_6MJ0^u_%WdO;cR~LgT{Z6g#$>7kfAjxFg`>Bs{1Z5JEQw8nlYK zVs8~L2@$SYUJP7X<*W4qmsKpog`_7YYA`!BX+~a&(0pNXAJpZB4k>VO2jlgYRWwi`=a9x;Z!vNv|aa1cX>@UriaL#a+IQ8)_Q7dwQ^Jecf^%C6IMw8*v%qi6vAO4+(N$00o&XqbG;>dzo_CqoVWntk9?nXBj>Z zG%jzXO>QzLH%t^SC5mx@y(mNKwy-EHMjfv8V8}wQ%wb1Lg}9P>4Nt+}VvF00ez2}~!`6?5;UR(I;sMw=)7avoBb z=gsNp-a$uv8R+HACG}Cnsk;av>F^b%$pQVS7xODvR6V^O-T$;1fO&j%F|TfJ6-W*F7d9&tV`IJ}WE=mP~dNc$d?V49dTAYvp0!u_(;Rv=wAc$9wrb*Ni z#J{uSzu|XMHt{Ed>s>kX)7^BPG(h8frLb43+}#V@UM%uIMW&YrG-511GYqpgqulGM%(% z@J#p6)XcoQArmwYw&Z40h!QLm1*x_YOr1Je-)O5XT4?SCfg2VJF%Fj&Lyc8;|ml8zXu7w5x$%1J-1yC{<-1+;XO6US9s9x|$-i zEHo{Xn>Hw5Xy)?J)sg_a3YuPDS-Db~LY~;Ex=*uf_cCvFNwF8*CXRyW26kn!|5~T) zf~TeDSt?b2PD4)4+#+0RnTs~Ac309YJ@$C4Iy^P*^6J_S?^x$FP#^1L)6==jpYQXW7bv83T!WgAOqrXtlgzuc(hvxq#?de& zrI}aE{f=~1-cl=+RR=fBz33ova!SYomVQ=cov0b;JF&q-Tz)OGCUt5GvrFThjG+GC zJcKp}w2Yw6{y6%1*`>oa<1rn=`0cJJ&s$W7EdiQ`_=gVk^f@`BdFh`se~ddPlO`gt zwj2dg-N+1b5|s7{_3m^p@2HvdE2qw)*%Bu(cJf({Ux+npN70in3tFmR+6h@VFgl=q zQjM^@fsk`T1|R$D8{Dtl|EHD@O68=AAb6yww?2-LX(^z}iq2ajES{ETxmcc;-%;b7 zgUOA*Y3BIr`TVh%^30zS=rWQ|y{&p_K6S*)DJR!;oVqqgpAMpX^#*sZ$Ly5-}wmCnre_G=V`z-FJ{9$qCI6ID<4a zSTm@`vy-ye5xSk6lvH!|)ZrYIwZ>||zi6;(j|2k~ERO=75{rTuO><5CA!7|rsh*YMIs4rFuDJEJn8D$q)wPro{!>bmL8gZVx> zq3s|ib1A_Lp2fLw$V&Hs^A=(B49whB%BpYSn@SLMwDTvP_F>D6Pi>S?$$uP28;C3? zm8Lpwl@^QZESuC>&!cJP40`v6vSb=Yu1Xq$tz3lZd0M8Qg-;G?-bgK!4xCK;^J+w% z5q3JIK1+$7Gb->5-1DCmd_bv;5GSk&hW5Qk@6BpbAF?kdFl`OwSV) ztBhk9o?33u)uEHT@G&hfJwy=Mg08W1UuNllthY;;=ZHR~qbuAHM6+N^M`pIt6S6vEIiV}s79q!b$GFodCv*c|NwGHf6S3#K6YLG<%^&Zk zZA)wG=KV4AHG>noUwMo!L-ro0AfKmnLETbt=OP#Fq;tq#fDFL=nk01GtUEAsM>qPp z-XE>ik3LGPKV<`0M*_TY(1Ffq^c}vsl5^5kBk+!doJ5^U)i~~r8kAE|Ia@0zA560; zLrS>=^Qt00VnpIh$r0(~E6wq??4u78nH^V{fZ2~LmpBnAm^*b!0o|M_ap$@DItjML zE9g~MIuI!H;GBcb9cuaH4191`(Su$b!b}CXblo^uajyh;motr@6bC2ML?9M+^IZQ-c2oNZ(Tm*KW?{*ZsAyc!?twHa{a|ehlb<`Y4e3ofz{&F6QT2 z%ulkIUtBRit73jXC0l)}#QgAy`E?WX6DH;-Nz8ALm>(H2KOJIzCB*y|i211x^CKSS z$AZjHc9@^ySgJm2VSX7XTYVvDsluBT-mUNfg`??1W%$J_oUL$;!kZP|t?+(@usn^P8-7(toki@5m}wP!4ufFhljghE$|W{y_ra4K6k7wCi}q#IDjv& zSO&UWE5Q+Tfe3!fl_2;^&ie>4E~2 zabS+i3BS^KS}~G?^qmlAunpB~L{) zuslvBR&}r{Q8MQZWXs+-ZaH9Y9JG301@1De)M%WosG6)jkETeyFnzrZpC;DmJ4u59 z_z!XSiA(gbE&40?P*wl1o{PeY%QayO(?*7+PDH5oiXOIKf6x}T$97}5&AwOIy2RST zlWh?bY>ojoXN=92Wa~QG7CFQgHN+NOZ0pv|*1empN0F^(xGg5u)+^Q4d#tTbtgTKHrAFr#FjGAHfoqHwZAQGm@U1(EhEO3>9CFNZyVF!HnzWQT%>J$s%=8LZDN00 zR)5>1{HSK^)aUb=)?YwA^7$x@La%$_{#G_ z;NJk20xrToh4BC{1JnR~fOUY&081eAa^RN$Zvx%`d;sWzFZXo=`~#mp&jGFntN}a# zcoNVC_zsW^{RU&Oqkk{E1keh24)8kQ7eE)-ejI?ls^r9l)1>1Q=r};06FbFQ-M>@r_o%8o=X#uK^?Q1@mQqn*sE_ z_Fk|D{u7RN8Q^gMeFghPz&^n50G|U61D*%G0R==0AB%8 zF}axvC;(gzxEAm(pa6ZZ_W=5YIZYJa1knF9 z{sQnfKs$i`M{yQlDqt~S6W~R_VZaa!?q>on20Q`yGaw2!Sq9hw_zR#5+R0GBNPru# z32+7A2EZqPzXQGl{0tEB_>?8U0q6;+9*S>c0}lp_0Hg!P1Ev6S0doOQ0Tuwx0mK5z z0o8y8z$(DSfK7m@fNg+2gJ6H)y8#aZo&>xEcnk1bz(K%|00VrHfZl*XfFwW$APbNK zmotN_#k&IdFD)&n*HwgPSd+zxmE@EBmpU`=}#_+`KwfOi2O0}cS%0AB%q z0Q?uw1-wr~pD5ryfPsKSKnh?CU=kn)Fc)wppcJqQuma!%TmrZnU>kyOs{-Eupnt1y z8}J=~`vJQFPXL|*ehcYyui5r8N_H$Y#&FhCL@ z127se8ITRg11toT0crtj0qX#p0apQT2HXmG2=EBtCBQ3ycL1LNeh>Hza1ih<;1|HZ z06zl+>Uub!JD@*cFdzw#1;_y`2FwGL0G0!40jmHP12zG+0&W1@4!95SDBxMZD}c8E z9|8Ud_!6)e_3{b8S8#s^cpUB@f&T+=!rdJZ4;Tta0*nHT1>^ws0r~?90A~PR0lW!V z1aJd9fGR*eAO_F^xB$=$*Z|lJ*ao->a0lQ4z+-^t051dH0PF{R1o$K1uYj)rKL8R@ zkDQ5+3y1^^0GtL$0b~Fs0Hy$D0_Fl10J@!q@e;TgunbTMs0XYDTnuOhTn)GmFcq*9 zK>uN~JK`+?egIGccpUH?;1$3=z=wd}1O5US4Bo#1e+T$CK!-m9K>t3nFJKU$0FVU8 z089i-2h0L20GtCT2h;#o0@eXq0b2k!0PX-h0C)oM0^kk6`+!dYUjV)a{0pEDL%jyR z3E+e~2GAdn4JZK21C#?U2RsV+6!0ye%W&8e&>heV5D!QKYy(6jq3;6R3z#+nIs>i& z+yM9z;24Rv1IP!g05k!b0ha+D0Xz-(6F^T!I)E}j9pFO1<$$e#I|0uEUIhFdkd;E; z2uJ#Wy8wFtZvy@V_yX`1;70&H=dP^>M5iKc!2N*F0T-o#50Hk*)Wv}B0O=W^1NH!Z z0F2H=T)=)nx6zt59Iy;fgaK{=v;oeX2sa=s3;6*&0I*F$`vyD<_zEz7GTIQ} z03hykv={VS>A+(F69Ll!1%N`p0>D{-rGPTPazGWp2WSGU0W4R9Br z^PfOanD$M3C`SYc_@@EdBAX0}R}34Cj`@2aT-PEsbE^cEgMX-OkkjA4wLc{Z`S&KMyh@zWYKxVpOVNylLCu=u}?UhUE8LMZYVAzDmh4w}hN3-K#*S`mhvzD~cx^_`S<=dTV!ur29M2$v*M@B(GTyzhQnX z8L|U1$kn9_Mr>*?sv^Xy;DMlrWR*j*Ko2o7)9JsIksNc&QB^x_tD>7*x1*K(R~6md zQZ(z~$Z#d6w`9biz`q>Oqp>GomX{)H1&V)XoTQj?c7WcI{!es5-wft>$iFn(lq~Vv z;O~Z%5y(F@AO8J8rMpMj+3c52!vN4xy3;{tJ1afYm7E5qD~|JdlpJ#l*37Tn zLFqU0KfDk0L6CoKwiE{Q^6z!UZ*FZ5Q}pgGJ|6 zpi{c$Q9_cUw{sAe%_(SdRJ?Qv5;z$1si1aa!&ahD6-*83Wsp_rSpDzc!BRQ9LLcdALF^|ep zmHci!{avA=NR$kENAjnGPI|&b{4@1AU(w@3?D>kKn@4}93)`x!U+;aO57i@xVP6b`EpE+%sdNCWqaTgnR&Mi?->e3$43%V>B znnb!5N=ZvasznO}myS*dCiWZgx=D^b+1q}`mbt+7DeE zMd0fY^m(O9H;$n3Nka@7W!2^@SYk5&SMU95sw+LXJu?vtW?uEH`c+j| z*H>4)_qv_WW%H7a7)47(B1ip?c?B!nXq!k5{Q!!dS32cekTREBqNqB=H$Q4si4^n1 z_Mug8(ih5`U=Xu6LHZ@R)u8`416`kK9?%gZZTZ@uXLjs;*IatKC>K3m0P>Mk*6T!y zhp}$ZF{#Eap3h+UFN35%bYEcrVjzZWA&Qgl%KEl?ol`N;RJ2uFZHHb!ZXp?NE$$S znR2T$fURGt3pF=GooN4Xx%!{g=Mj$f+O^H4(imy|A#PMPCnI!*NP#68X>00>M2kaj z2K?M`a{#*s(B1Sl!@LY)yMUq(c=Py#!bXwzR3@@rh@RD>xeR80?V$?`+}`%Zt%L(` zx&){+S3j-FLEKVnX%9pD4GE4ugKMGv#Y-fuHhgw(*W}cCNg{ zvMElLQ6g2-$u)#4=Im1S`C%RT^)8m!QD2*AwvsM~9#_NcO?z3^;C$Uf2a}h*Wi^BJ z;*Ti%WMAM*O(vjSeq%Y~p)$@ zcw$e|kKOdItgiMuyS##eOox6{AHA?V9{dsoZ6sXxx7WpND00!n5o9(;2l08uq<%T) zx(&bEErC;asE3s2Bo^%B(l1ob`Eb-9@C$|~Z6GXmJWnO>F?eG9%Z4XyFrrbO&y&3f z&mV_}J(mBH;Ym+hL@&+)f13Yu;Ndhglz-pwq62QkfB5ituuTm*U?6Poh}0jn0*EJ}i#suzIC=(J%BZ z{9=5=aHKcm$naZmpzkI4lXmb(%JMQ9UXy;1;2VY~y&LiYv19(^eF#rne#$$LCU9TZ z#0ve$@O*x*AzZkI|93Aa3IFn}V)rR7B!A+U8yZS_F9Vc)B>%6#kMTSAYedq|SW$*g zF5Qm_p7M>P@_b^Kmj4$7${DY9+aRa4WC!MHeKUX3-++avL%>rm?1JGL4^h11HR~70e-O|6x&};~|8?ggZJh$Fox!Bb9R&+r^4m<(~eJ_Wq+SbzI14cRw*8ehs9yZbDEiynfA zZv~Eeh$lT`_#~RUVF~`QP!1uE+(*XX4>mN$ugMLh_|_Qw+kM4=wsgaNdXriCnoL;eog%NIN7e{@g%nhj&rS_J#pVd XE+lqo`DMDO!rysQQST>!