Skip to content

Commit

Permalink
Added 'ranges' column
Browse files Browse the repository at this point in the history
  • Loading branch information
al13n321 committed Oct 14, 2023
1 parent b66d58e commit 0ad07da
Show file tree
Hide file tree
Showing 8 changed files with 254 additions and 30 deletions.
174 changes: 168 additions & 6 deletions src/Processors/Formats/Impl/DWARFBlockInputFormat.cpp
Expand Up @@ -2,6 +2,8 @@
#if USE_DWARF_PARSER && defined(__ELF__) && !defined(OS_FREEBSD)

#include <llvm/DebugInfo/DWARF/DWARFFormValue.h>
#include <llvm/DebugInfo/DWARF/DWARFDebugRnglists.h>
#include <llvm/DebugInfo/DWARF/DWARFDebugRangeList.h>
#include <llvm/BinaryFormat/Dwarf.h>

#include <base/hex.h>
Expand All @@ -10,11 +12,12 @@
#include <Columns/ColumnArray.h>
#include <Columns/ColumnLowCardinality.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnUnique.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypesNumber.h>
#include <IO/ReadBufferFromFileBase.h>
#include <IO/WithFileName.h>
Expand Down Expand Up @@ -63,9 +66,7 @@ enum DwarfColumn
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_RANGES,

COL_ATTR_NAME,
COL_ATTR_FORM,
Expand All @@ -89,6 +90,7 @@ static NamesAndTypesList getHeaderForDWARF()
cols[COL_LINKAGE_NAME] = {"linkage_name", std::make_shared<DataTypeString>()};
cols[COL_DECL_FILE] = {"decl_file", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())};
cols[COL_DECL_LINE] = {"decl_line", std::make_shared<DataTypeUInt32>()};
cols[COL_RANGES] = {"ranges", std::make_shared<DataTypeArray>(std::make_shared<DataTypeTuple>(DataTypes{std::make_shared<DataTypeUInt64>(), std::make_shared<DataTypeUInt64>()}))};
cols[COL_ATTR_NAME] = {"attr_name", std::make_shared<DataTypeArray>(std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()))};
cols[COL_ATTR_FORM] = {"attr_form", std::make_shared<DataTypeArray>(std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()))};
cols[COL_ATTR_INT] = {"attr_int", std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>())};
Expand Down Expand Up @@ -211,9 +213,14 @@ void DWARFBlockInputFormat::initializeIfNeeded()
/// (The StringRef points into Elf's mmap of the whole file, or into file_contents.)
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())
if (auto line_section = elf->findSectionByName(".debug_line"))
debug_line_extractor.emplace(llvm::StringRef(line_section->begin(), line_section->size()), /*IsLittleEndian*/ true, /*AddressSize*/ 8);
if (auto addr_section = elf->findSectionByName(".debug_addr"))
debug_addr_section.emplace(addr_section->begin(), addr_section->size());
if (auto rnglists_section = elf->findSectionByName(".debug_rnglists"))
debug_rnglists_extractor.emplace(llvm::StringRef(rnglists_section->begin(), rnglists_section->size()), /*IsLittleEndian*/ true, /*AddressSize*/ 8);
if (auto ranges_section = elf->findSectionByName(".debug_ranges"))
debug_ranges_extractor.emplace(llvm::StringRef(ranges_section->begin(), ranges_section->size()), /*IsLittleEndian*/ true, /*AddressSize*/ 8);

llvm::StringMap<std::unique_ptr<llvm::MemoryBuffer>> sections;
elf->iterateSections([&](const Elf::Section & section, size_t /*idx*/)
Expand Down Expand Up @@ -323,6 +330,9 @@ Chunk DWARFBlockInputFormat::parseEntries(UnitState & unit)
auto col_linkage_name = ColumnString::create();
ColumnLowCardinality::Index col_decl_file;
auto col_decl_line = ColumnVector<UInt32>::create();
auto col_ranges_start = ColumnVector<UInt64>::create();
auto col_ranges_end = ColumnVector<UInt64>::create();
auto col_ranges_offsets = ColumnVector<UInt64>::create();
auto col_attr_name = ColumnVector<UInt16>::create();
auto col_attr_form = ColumnVector<UInt16>::create();
auto col_attr_int = ColumnVector<UInt64>::create();
Expand Down Expand Up @@ -362,6 +372,7 @@ Chunk DWARFBlockInputFormat::parseEntries(UnitState & unit)
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_RANGES]) col_ranges_offsets->insertValue(col_ranges_start->size());
if (need[COL_ATTR_NAME]) col_attr_offsets->insertValue(col_attr_name->size());

if (unit.stack.empty())
Expand All @@ -383,6 +394,19 @@ Chunk DWARFBlockInputFormat::parseEntries(UnitState & unit)
bool need_decl_file = need[COL_DECL_FILE];
bool need_decl_line = need[COL_DECL_LINE];

/// Address ranges can be encoded in the following ways:
/// * DW_AT_ranges - list of ranges; points to other sections, nontrivial to parse;
/// DW_AT_low_pc may also be present, used as base address during parsing
/// * [DW_AT_low_pc, DW_AT_high_pc) if DW_AT_high_pc has form "address"
/// * [DW_AT_low_pc, DW_AT_low_pc + DW_AT_high_pc) if DW_AT_high_pc has form "constant"
/// * [DW_AT_low_pc, DW_AT_low_pc + 1) if DW_AT_high_pc is absent
bool need_ranges = need[COL_RANGES];
std::optional<UInt64> ranges;
bool ranges_rnglistx = false; // `ranges` has an extra indirection through offsets table
std::optional<UInt64> low_pc;
std::optional<UInt64> high_pc;
bool relative_high_pc = false;

for (auto attr : abbrev->attributes())
{
auto val = llvm::DWARFFormValue::createFromSValue(attr.Form, attr.isImplicitConst() ? attr.getImplicitConstValue() : 0);
Expand Down Expand Up @@ -427,6 +451,25 @@ Chunk DWARFBlockInputFormat::parseEntries(UnitState & unit)
if (attr.Attr == llvm::dwarf::DW_AT_decl_line && std::exchange(need_decl_line, false))
col_decl_line->insertValue(static_cast<UInt32>(val.getRawUValue()));

/// Starting offset of this unit's data in .debug_addr section.
if (attr.Attr == llvm::dwarf::DW_AT_addr_base)
unit.addr_base = val.getRawUValue();
/// Same for .debug_rnglists section.
if (attr.Attr == llvm::dwarf::DW_AT_rnglists_base)
unit.rnglists_base = val.getRawUValue();

if (attr.Attr == llvm::dwarf::DW_AT_high_pc)
{
high_pc = val.getRawUValue();
relative_high_pc = true;
}

if (attr.Attr == llvm::dwarf::DW_AT_ranges)
{
ranges = val.getRawUValue();
ranges_rnglistx = val.getForm() == llvm::dwarf::DW_FORM_rnglistx;
}

/// 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
Expand Down Expand Up @@ -468,6 +511,25 @@ Chunk DWARFBlockInputFormat::parseEntries(UnitState & unit)
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();

if (need_ranges && (attr.Attr == llvm::dwarf::DW_AT_low_pc || attr.Attr == llvm::dwarf::DW_AT_high_pc))
{
UInt64 addr;
if (val.getForm() == llvm::dwarf::DW_FORM_addr)
addr = val.getRawUValue();
else if (val.getForm() == llvm::dwarf::DW_FORM_addrx ||
(val.getForm() >= llvm::dwarf::DW_FORM_addrx1 &&
val.getForm() <= llvm::dwarf::DW_FORM_addrx4))
addr = fetchFromDebugAddr(unit.addr_base, val.getRawUValue());
else
throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Form {} for {} is not supported", llvm::dwarf::FormEncodingString(val.getForm()), llvm::dwarf::AttributeString(attr.Attr));

if (attr.Attr == llvm::dwarf::DW_AT_low_pc)
low_pc = addr;
else
high_pc = addr;
}

break;

/// A byte string.
Expand Down Expand Up @@ -552,6 +614,25 @@ Chunk DWARFBlockInputFormat::parseEntries(UnitState & unit)
if (need_decl_file) col_decl_file.insertPosition(0);
if (need_decl_line) col_decl_line->insertDefault();

if (need_ranges)
{
if (ranges.has_value())
parseRanges(*ranges, ranges_rnglistx, low_pc, unit, col_ranges_start, col_ranges_end);
else if (low_pc.has_value())
{
UInt64 high;
if (!high_pc.has_value())
high = *low_pc + 1;
else if (relative_high_pc)
high = *low_pc + *high_pc ;
else
high = *high_pc;
col_ranges_start->insertValue(*low_pc);
col_ranges_end->insertValue(high);
}
}
if (need[COL_RANGES]) col_ranges_offsets->insertValue(col_ranges_start->size());

if (abbrev->hasChildren())
unit.stack.push_back(StackEntry{.offset = die_offset, .tag = tag});
}
Expand Down Expand Up @@ -624,6 +705,9 @@ Chunk DWARFBlockInputFormat::parseEntries(UnitState & unit)
case COL_DECL_LINE:
cols.push_back(std::exchange(col_decl_line, nullptr));
break;
case COL_RANGES:
cols.push_back(ColumnArray::create(ColumnTuple::create(Columns{std::exchange(col_ranges_start, nullptr), std::exchange(col_ranges_end, nullptr)}), std::exchange(col_ranges_offsets, 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));
Expand Down Expand Up @@ -683,6 +767,84 @@ void DWARFBlockInputFormat::parseFilenameTable(UnitState & unit, uint64_t offset
unit.filename_table = ColumnUnique<ColumnString>::create(std::move(col), /*is_nullable*/ false);
}

uint64_t DWARFBlockInputFormat::fetchFromDebugAddr(uint64_t addr_base, uint64_t idx) const
{
if (!debug_addr_section.has_value())
throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Missing .debug_addr section.");
if (addr_base == UINT64_MAX)
throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Missing DW_AT_addr_base");
uint64_t offset = addr_base + idx * 8;
if (offset + 8 > debug_addr_section->size())
throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, ".debug_addr offset out of bounds: {} vs {}.", offset, debug_addr_section->size());
uint64_t res;
memcpy(&res, debug_addr_section->data() + offset, 8);
return res;
}

void DWARFBlockInputFormat::parseRanges(
uint64_t offset, bool form_rnglistx, std::optional<uint64_t> low_pc, const UnitState & unit, const ColumnVector<UInt64>::MutablePtr & col_ranges_start,
const ColumnVector<UInt64>::MutablePtr & col_ranges_end) const
{
llvm::Optional<llvm::object::SectionedAddress> base_addr;
if (low_pc.has_value())
base_addr = llvm::object::SectionedAddress{.Address = *low_pc};

llvm::DWARFAddressRangesVector ranges;

if (unit.dwarf_unit->getFormParams().Version < 5)
{
if (!debug_ranges_extractor.has_value())
throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Missing .debug_ranges section");

llvm::DWARFDebugRangeList list;
auto err = list.extract(*debug_ranges_extractor, &offset);
if (err)
throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Error parsing .debug_ranges list: {}", llvm::toString(std::move(err)));

ranges = list.getAbsoluteRanges(base_addr);
}
else
{
if (!debug_rnglists_extractor.has_value())
throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Missing .debug_rnglists section");

if (form_rnglistx)
{
/// Fetch offset from a table.
if (unit.rnglists_base == UINT64_MAX)
throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Missing DW_AT_rnglists_base");
uint64_t entry_size = unit.dwarf_unit->getFormParams().getDwarfOffsetByteSize();
uint64_t lists_offset = unit.rnglists_base + offset * entry_size;
if (lists_offset + entry_size > debug_rnglists_extractor->size())
throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "DW_FORM_rnglistx offset out of bounds: {} vs {}", lists_offset, debug_rnglists_extractor->size());

offset = 0;
memcpy(&offset, debug_rnglists_extractor->getData().data() + lists_offset, entry_size);

offset += unit.rnglists_base;
}

llvm::DWARFDebugRnglist list;
auto err = list.extract(*debug_rnglists_extractor, /*HeaderOffset, essentially unused*/ 0, &offset,
".debug_rnglists", "rnglists");
if (err)
throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Error parsing .debug_rnglists list: {}", llvm::toString(std::move(err)));

auto lookup_addr = [&](uint32_t idx) -> llvm::Optional<llvm::object::SectionedAddress>
{
uint64_t addr = fetchFromDebugAddr(unit.addr_base, idx);
return llvm::object::SectionedAddress{.Address = addr};
};
ranges = list.getAbsoluteRanges(base_addr, /*AddressByteSize*/ 8, lookup_addr);
}

for (const llvm::DWARFAddressRange & r : ranges)
{
col_ranges_start->insertValue(r.LowPC);
col_ranges_end->insertValue(r.HighPC);
}
}

Chunk DWARFBlockInputFormat::generate()
{
initializeIfNeeded();
Expand Down
18 changes: 16 additions & 2 deletions src/Processors/Formats/Impl/DWARFBlockInputFormat.h
Expand Up @@ -12,6 +12,7 @@
#include <Formats/FormatSettings.h>
#include <Common/Elf.h>
#include <Common/ThreadPool.h>
#include <Columns/ColumnVector.h>

namespace DB
{
Expand Down Expand Up @@ -52,6 +53,8 @@ class DWARFBlockInputFormat : public IInputFormat
std::string unit_name;
ColumnPtr filename_table; // from .debug_line
size_t filename_table_size = 0;
uint64_t addr_base = UINT64_MAX;
uint64_t rnglists_base = UINT64_MAX;

uint64_t offset = 0;
std::vector<StackEntry> stack;
Expand Down Expand Up @@ -85,8 +88,11 @@ class DWARFBlockInputFormat : public IInputFormat
PODArray<char> file_contents; // if we couldn't mmap it

std::unique_ptr<llvm::DWARFContext> dwarf_context;
std::optional<llvm::DWARFDataExtractor> extractor;
std::optional<llvm::DWARFDataExtractor> debug_line_extractor;
std::optional<llvm::DWARFDataExtractor> extractor; // .debug_info
std::optional<llvm::DWARFDataExtractor> debug_line_extractor; // .debug_line
std::optional<std::string_view> debug_addr_section; // .debug_addr
std::optional<llvm::DWARFDataExtractor> debug_rnglists_extractor; // .debug_rnglists
std::optional<llvm::DWARFDataExtractor> debug_ranges_extractor; // .debug_ranges

std::atomic<size_t> seen_debug_line_warnings {0};

Expand All @@ -95,6 +101,14 @@ class DWARFBlockInputFormat : public IInputFormat
void stopThreads();
void parseFilenameTable(UnitState & unit, uint64_t offset);
Chunk parseEntries(UnitState & unit);

/// Parse .debug_addr entry.
uint64_t fetchFromDebugAddr(uint64_t addr_base, uint64_t idx) const;
/// Parse .debug_ranges (DWARF4) or .debug_rnglists (DWARF5) entry.
void parseRanges(
uint64_t offset, bool form_rnglistx, std::optional<uint64_t> low_pc, const UnitState & unit,
const ColumnVector<UInt64>::MutablePtr & col_ranges_start,
const ColumnVector<UInt64>::MutablePtr & col_ranges_end) const;
};

class DWARFSchemaReader : public ISchemaReader
Expand Down

0 comments on commit 0ad07da

Please sign in to comment.