Merge pull request #56102 from ClickHouse/coverage

Granular code coverage with introspection
This commit is contained in:
Alexey Milovidov 2023-11-16 23:23:40 +01:00 committed by GitHub
commit 482d8ca9d8
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
18 changed files with 369 additions and 68 deletions

View File

@ -187,9 +187,10 @@ if (NOT CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE")
endif ()
endif()
if (CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE"
OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO"
OR CMAKE_BUILD_TYPE_UC STREQUAL "MINSIZEREL")
if (NOT (SANITIZE_COVERAGE OR WITH_COVERAGE)
AND (CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE"
OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO"
OR CMAKE_BUILD_TYPE_UC STREQUAL "MINSIZEREL"))
set (OMIT_HEAVY_DEBUG_SYMBOLS_DEFAULT ON)
else()
set (OMIT_HEAVY_DEBUG_SYMBOLS_DEFAULT OFF)
@ -291,9 +292,6 @@ set (CMAKE_C_STANDARD 11)
set (CMAKE_C_EXTENSIONS ON) # required by most contribs written in C
set (CMAKE_C_STANDARD_REQUIRED ON)
# Compiler-specific coverage flags e.g. -fcoverage-mapping
option(WITH_COVERAGE "Profile the resulting binary/binaries" OFF)
if (COMPILER_CLANG)
# Enable C++14 sized global deallocation functions. It should be enabled by setting -std=c++14 but I'm not sure.
# See https://reviews.llvm.org/D112921
@ -309,12 +307,6 @@ if (COMPILER_CLANG)
set(BRANCHES_WITHIN_32B_BOUNDARIES "-mbranches-within-32B-boundaries")
set(COMPILER_FLAGS "${COMPILER_FLAGS} ${BRANCHES_WITHIN_32B_BOUNDARIES}")
endif()
if (WITH_COVERAGE)
set(COMPILER_FLAGS "${COMPILER_FLAGS} -fprofile-instr-generate -fcoverage-mapping")
# If we want to disable coverage for specific translation units
set(WITHOUT_COVERAGE "-fno-profile-instr-generate -fno-coverage-mapping")
endif()
endif ()
set (COMPILER_FLAGS "${COMPILER_FLAGS}")

View File

@ -1,3 +1,5 @@
add_compile_options($<$<OR:$<COMPILE_LANGUAGE:C>,$<COMPILE_LANGUAGE:CXX>>:${COVERAGE_FLAGS}>)
if (USE_CLANG_TIDY)
set (CMAKE_CXX_CLANG_TIDY "${CLANG_TIDY_PATH}")
endif ()

View File

@ -1,11 +1,15 @@
#include "coverage.h"
#if WITH_COVERAGE
#pragma GCC diagnostic ignored "-Wreserved-identifier"
# include <mutex>
# include <unistd.h>
/// WITH_COVERAGE enables the default implementation of code coverage,
/// that dumps a map to the filesystem.
#if WITH_COVERAGE
#include <mutex>
#include <unistd.h>
# if defined(__clang__)
@ -31,3 +35,131 @@ void dumpCoverageReportIfPossible()
#endif
}
/// SANITIZE_COVERAGE enables code instrumentation,
/// but leaves the callbacks implementation to us,
/// which we use to calculate coverage on a per-test basis
/// and to write it to system tables.
#if defined(SANITIZE_COVERAGE)
namespace
{
bool pc_guards_initialized = false;
bool pc_table_initialized = false;
uint32_t * guards_start = nullptr;
uint32_t * guards_end = nullptr;
uintptr_t * coverage_array = nullptr;
size_t coverage_array_size = 0;
uintptr_t * all_addresses_array = nullptr;
size_t all_addresses_array_size = 0;
}
extern "C"
{
/// This is called at least once for every DSO for initialization.
/// But we will use it only for the main DSO.
void __sanitizer_cov_trace_pc_guard_init(uint32_t * start, uint32_t * stop)
{
if (pc_guards_initialized)
return;
pc_guards_initialized = true;
/// The function can be called multiple times, but we need to initialize only once.
if (start == stop || *start)
return;
guards_start = start;
guards_end = stop;
coverage_array_size = stop - start;
/// Note: we will leak this.
coverage_array = static_cast<uintptr_t*>(malloc(sizeof(uintptr_t) * coverage_array_size));
resetCoverage();
}
/// This is called at least once for every DSO for initialization
/// and provides information about all instrumented addresses.
void __sanitizer_cov_pcs_init(const uintptr_t * pcs_begin, const uintptr_t * pcs_end)
{
if (pc_table_initialized)
return;
pc_table_initialized = true;
all_addresses_array = static_cast<uintptr_t*>(malloc(sizeof(uintptr_t) * coverage_array_size));
all_addresses_array_size = pcs_end - pcs_begin;
/// They are not a real pointers, but also contain a flag in the most significant bit,
/// in which we are not interested for now. Reset it.
for (size_t i = 0; i < all_addresses_array_size; ++i)
all_addresses_array[i] = pcs_begin[i] & 0x7FFFFFFFFFFFFFFFULL;
}
/// This is called at every basic block / edge, etc.
void __sanitizer_cov_trace_pc_guard(uint32_t * guard)
{
/// Duplicate the guard check.
if (!*guard)
return;
*guard = 0;
/// If you set *guard to 0 this code will not be called again for this edge.
/// Now we can get the PC and do whatever you want:
/// - store it somewhere or symbolize it and print right away.
/// The values of `*guard` are as you set them in
/// __sanitizer_cov_trace_pc_guard_init and so you can make them consecutive
/// and use them to dereference an array or a bit vector.
void * pc = __builtin_return_address(0);
coverage_array[guard - guards_start] = reinterpret_cast<uintptr_t>(pc);
}
}
__attribute__((no_sanitize("coverage")))
std::span<const uintptr_t> getCoverage()
{
return {coverage_array, coverage_array_size};
}
__attribute__((no_sanitize("coverage")))
std::span<const uintptr_t> getAllInstrumentedAddresses()
{
return {all_addresses_array, all_addresses_array_size};
}
__attribute__((no_sanitize("coverage")))
void resetCoverage()
{
memset(coverage_array, 0, coverage_array_size * sizeof(*coverage_array));
/// The guard defines whether the __sanitizer_cov_trace_pc_guard should be called.
/// For example, you can unset it after first invocation to prevent excessive work.
/// Initially set all the guards to 1 to enable callbacks.
for (uint32_t * x = guards_start; x < guards_end; ++x)
*x = 1;
}
#else
std::span<const uintptr_t> getCoverage()
{
return {};
}
std::span<const uintptr_t> getAllInstrumentedAddresses()
{
return {};
}
void resetCoverage()
{
}
#endif

View File

@ -1,5 +1,8 @@
#pragma once
#include <span>
#include <cstdint>
/// Flush coverage report to file, depending on coverage system
/// proposed by compiler (llvm for clang and gcov for gcc).
///
@ -7,3 +10,16 @@
/// Thread safe (use exclusive lock).
/// Idempotent, may be called multiple times.
void dumpCoverageReportIfPossible();
/// This is effective if SANITIZE_COVERAGE is enabled at build time.
/// Get accumulated unique program addresses of the instrumented parts of the code,
/// seen so far after program startup or after previous reset.
/// The returned span will be represented as a sparse map, containing mostly zeros, which you should filter away.
std::span<const uintptr_t> getCoverage();
/// Get all instrumented addresses that could be in the coverage.
std::span<const uintptr_t> getAllInstrumentedAddresses();
/// Reset the accumulated coverage.
/// This is useful to compare coverage of different tests, including differential coverage.
void resetCoverage();

View File

@ -1,5 +1,6 @@
#include "memcpy.h"
__attribute__((no_sanitize("coverage")))
extern "C" void * memcpy(void * __restrict dst, const void * __restrict src, size_t size)
{
return inline_memcpy(dst, src, size);

View File

@ -93,7 +93,7 @@
* See https://habr.com/en/company/yandex/blog/457612/
*/
__attribute__((no_sanitize("coverage")))
static inline void * inline_memcpy(void * __restrict dst_, const void * __restrict src_, size_t size)
{
/// We will use pointer arithmetic, so char pointer will be used.

View File

@ -58,3 +58,27 @@ if (SANITIZE)
message (FATAL_ERROR "Unknown sanitizer type: ${SANITIZE}")
endif ()
endif()
# Default coverage instrumentation (dumping the coverage map on exit)
option(WITH_COVERAGE "Instrumentation for code coverage with default implementation" OFF)
if (WITH_COVERAGE)
message (INFORMATION "Enabled instrumentation for code coverage")
set(COVERAGE_FLAGS "-fprofile-instr-generate -fcoverage-mapping")
endif()
option (SANITIZE_COVERAGE "Instrumentation for code coverage with custom callbacks" OFF)
if (SANITIZE_COVERAGE)
message (INFORMATION "Enabled instrumentation for code coverage")
# We set this define for whole build to indicate that at least some parts are compiled with coverage.
# And to expose it in system.build_options.
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -DSANITIZE_COVERAGE=1")
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -DSANITIZE_COVERAGE=1")
# But the actual coverage will be enabled on per-library basis: for ClickHouse code, but not for 3rd-party.
set (COVERAGE_FLAGS "-fsanitize-coverage=trace-pc-guard,pc-table")
endif()
set (WITHOUT_COVERAGE_FLAGS "-fno-profile-instr-generate -fno-coverage-mapping -fno-sanitize-coverage=trace-pc-guard,pc-table")

View File

@ -3,15 +3,6 @@
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w -ffunction-sections -fdata-sections")
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w -ffunction-sections -fdata-sections")
if (WITH_COVERAGE)
set (WITHOUT_COVERAGE_LIST ${WITHOUT_COVERAGE})
separate_arguments(WITHOUT_COVERAGE_LIST)
# disable coverage for contib files and build with optimisations
if (COMPILER_CLANG)
add_compile_options(-O3 -DNDEBUG -finline-functions -finline-hint-functions ${WITHOUT_COVERAGE_LIST})
endif()
endif()
if (SANITIZE STREQUAL "undefined")
# 3rd-party libraries usually not intended to work with UBSan.
add_compile_options(-fno-sanitize=undefined)

View File

@ -366,38 +366,6 @@ else ()
COMMAND_ECHO STDOUT)
endif ()
# add_custom_command (
# OUTPUT ${PROTOC_BUILD_DIR}
# COMMAND mkdir -p ${PROTOC_BUILD_DIR})
#
# add_custom_command (
# OUTPUT "${PROTOC_BUILD_DIR}/CMakeCache.txt"
#
# COMMAND ${CMAKE_COMMAND}
# -G"${CMAKE_GENERATOR}"
# -DCMAKE_MAKE_PROGRAM="${CMAKE_MAKE_PROGRAM}"
# -DCMAKE_C_COMPILER="${CMAKE_C_COMPILER}"
# -DCMAKE_CXX_COMPILER="${CMAKE_CXX_COMPILER}"
# -Dprotobuf_BUILD_TESTS=0
# -Dprotobuf_BUILD_CONFORMANCE=0
# -Dprotobuf_BUILD_EXAMPLES=0
# -Dprotobuf_BUILD_PROTOC_BINARIES=1
# "${protobuf_source_dir}/cmake"
#
# DEPENDS "${PROTOC_BUILD_DIR}"
# WORKING_DIRECTORY "${PROTOC_BUILD_DIR}"
# COMMENT "Configuring 'protoc' for host architecture."
# USES_TERMINAL)
#
# add_custom_command (
# OUTPUT "${PROTOC_BUILD_DIR}/protoc"
# COMMAND ${CMAKE_COMMAND} --build "${PROTOC_BUILD_DIR}"
# DEPENDS "${PROTOC_BUILD_DIR}/CMakeCache.txt"
# COMMENT "Building 'protoc' for host architecture."
# USES_TERMINAL)
#
# add_custom_target (protoc-host DEPENDS "${PROTOC_BUILD_DIR}/protoc")
add_executable(protoc IMPORTED GLOBAL)
set_target_properties (protoc PROPERTIES IMPORTED_LOCATION "${PROTOC_BUILD_DIR}/protoc")
add_dependencies(protoc "${PROTOC_BUILD_DIR}/protoc")

View File

@ -1,3 +1,5 @@
add_compile_options($<$<OR:$<COMPILE_LANGUAGE:C>,$<COMPILE_LANGUAGE:CXX>>:${COVERAGE_FLAGS}>)
if (USE_CLANG_TIDY)
set (CMAKE_CXX_CLANG_TIDY "${CLANG_TIDY_PATH}")
endif ()

View File

@ -676,6 +676,10 @@ try
global_context->addWarningMessage("Server was built with sanitizer. It will work slowly.");
#endif
#if defined(SANITIZE_COVERAGE) || WITH_COVERAGE
global_context->addWarningMessage("Server was built with code coverage. It will work slowly.");
#endif
const size_t physical_server_memory = getMemoryAmount();
LOG_INFO(log, "Available RAM: {}; physical cores: {}; logical cores: {}.",

View File

@ -1,3 +1,5 @@
add_compile_options($<$<OR:$<COMPILE_LANGUAGE:C>,$<COMPILE_LANGUAGE:CXX>>:${COVERAGE_FLAGS}>)
if (USE_INCLUDE_WHAT_YOU_USE)
set (CMAKE_CXX_INCLUDE_WHAT_YOU_USE ${IWYU_PATH})
endif ()
@ -293,7 +295,8 @@ set_source_files_properties(
Common/Elf.cpp
Common/Dwarf.cpp
Common/SymbolIndex.cpp
PROPERTIES COMPILE_FLAGS "-O2 ${WITHOUT_COVERAGE}")
Common/ThreadFuzzer.cpp
PROPERTIES COMPILE_FLAGS "-O2 ${WITHOUT_COVERAGE_FLAGS}")
target_link_libraries (clickhouse_common_io
PRIVATE

View File

@ -0,0 +1,94 @@
#if defined(SANITIZE_COVERAGE)
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnConst.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
#include <Interpreters/Context.h>
#include <base/coverage.h>
namespace DB
{
namespace
{
enum class Kind
{
Current,
All
};
/** If ClickHouse is build with coverage instrumentation, returns an array
* of currently accumulated (`coverage`) / all possible (`coverageAll`) unique code addresses.
*/
class FunctionCoverage : public IFunction
{
private:
Kind kind;
public:
String getName() const override
{
return kind == Kind::Current ? "coverage" : "coverageAll";
}
explicit FunctionCoverage(Kind kind_) : kind(kind_)
{
}
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override
{
return false;
}
size_t getNumberOfArguments() const override
{
return 0;
}
bool isDeterministic() const override
{
return false;
}
DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override
{
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override
{
auto coverage_table = kind == Kind::Current ? getCoverage() : getAllInstrumentedAddresses();
auto column_addresses = ColumnUInt64::create();
auto & data = column_addresses->getData();
for (auto ptr : coverage_table)
if (ptr)
data.push_back(ptr);
auto column_array = ColumnArray::create(
std::move(column_addresses),
ColumnArray::ColumnOffsets::create(1, data.size()));
return ColumnConst::create(std::move(column_array), input_rows_count);
}
};
}
REGISTER_FUNCTION(Coverage)
{
factory.registerFunction("coverage", [](ContextPtr){ return std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionCoverage>(Kind::Current)); });
factory.registerFunction("coverageAll", [](ContextPtr){ return std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionCoverage>(Kind::All)); });
}
}
#endif

View File

@ -57,6 +57,7 @@
#include <Parsers/ASTSystemQuery.h>
#include <Parsers/ASTCreateQuery.h>
#include <Common/ThreadFuzzer.h>
#include <base/coverage.h>
#include <csignal>
#include <algorithm>
#include <unistd.h>
@ -687,6 +688,12 @@ BlockIO InterpreterSystemQuery::execute()
FailPointInjection::disableFailPoint(query.fail_point_name);
break;
}
case Type::RESET_COVERAGE:
{
getContext()->checkAccess(AccessType::SYSTEM);
resetCoverage();
break;
}
default:
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown type of SYSTEM query");
}
@ -1301,6 +1308,7 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster()
case Type::START_THREAD_FUZZER:
case Type::ENABLE_FAILPOINT:
case Type::DISABLE_FAILPOINT:
case Type::RESET_COVERAGE:
case Type::UNKNOWN:
case Type::END: break;
}

View File

@ -86,6 +86,7 @@ public:
START_PULLING_REPLICATION_LOG,
STOP_CLEANUP,
START_CLEANUP,
RESET_COVERAGE,
END
};

View File

@ -448,14 +448,14 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
}
case Type::DROP_FORMAT_SCHEMA_CACHE:
{
if (ParserKeyword{"FOR"}.ignore(pos, expected))
{
if (ParserKeyword{"Protobuf"}.ignore(pos, expected))
res->schema_cache_format = "Protobuf";
else
return false;
}
break;
if (ParserKeyword{"FOR"}.ignore(pos, expected))
{
if (ParserKeyword{"Protobuf"}.ignore(pos, expected))
res->schema_cache_format = "Protobuf";
else
return false;
}
break;
}
case Type::UNFREEZE:
{

View File

@ -8972,7 +8972,7 @@ void StorageReplicatedMergeTree::createTableSharedID() const
else if (code == Coordination::Error::ZNONODE) /// table completely dropped, we can choose any id we want
{
id = toString(UUIDHelpers::Nil);
LOG_DEBUG(log, "Table was completely drop, we can use anything as ID (will use {})", id);
LOG_DEBUG(log, "Table was completely dropped, and we can use anything as ID (will use {})", id);
}
else if (code != Coordination::Error::ZOK)
{

View File

@ -1173,6 +1173,25 @@ class TestCase:
description_full += result.reason.value
description_full += result.description
if (
args.collect_per_test_coverage
and BuildFlags.SANITIZE_COVERAGE in args.build_flags
):
clickhouse_execute(
args,
f"INSERT INTO system.coverage SELECT now(), '{self.case}', coverage()",
retry_error_codes=True,
)
coverage = clickhouse_execute(
args,
"SELECT length(coverage())",
retry_error_codes=True,
).decode()
description_full += f" Coverage: {coverage}"
description_full += "\n"
if result.status == TestStatus.FAIL and self.testcase_args:
@ -1231,6 +1250,17 @@ class TestCase:
+ pattern
)
# We want to calculate per-test code coverage. That's why we reset it before each test.
if (
args.collect_per_test_coverage
and BuildFlags.SANITIZE_COVERAGE in args.build_flags
):
clickhouse_execute(
args,
"SYSTEM RESET COVERAGE",
retry_error_codes=True,
)
command = pattern.format(**params)
proc = Popen(command, shell=True, env=os.environ)
@ -1872,6 +1902,7 @@ class BuildFlags:
UNDEFINED = "ubsan"
MEMORY = "msan"
DEBUG = "debug"
SANITIZE_COVERAGE = "sanitize-coverage"
RELEASE = "release"
ORDINARY_DATABASE = "ordinary-database"
POLYMORPHIC_PARTS = "polymorphic-parts"
@ -1891,6 +1922,8 @@ def collect_build_flags(args):
result.append(BuildFlags.UNDEFINED)
elif b"-fsanitize=memory" in value:
result.append(BuildFlags.MEMORY)
elif b"-DSANITIZE_COVERAGE=1" in value:
result.append(BuildFlags.SANITIZE_COVERAGE)
value = clickhouse_execute(
args, "SELECT value FROM system.build_options WHERE name = 'BUILD_TYPE'"
@ -2072,6 +2105,8 @@ def reportCoverageFor(args, what, query, permissive=False):
return True
# This is high-level coverage on per-component basis (functions, data types, etc.)
# Don't be confused with the code coverage.
def reportCoverage(args):
clickhouse_execute(args, "SYSTEM FLUSH LOGS")
@ -2334,6 +2369,28 @@ def main(args):
print(f"Failed to create databases for tests: {e}")
server_died.set()
if (
args.collect_per_test_coverage
and BuildFlags.SANITIZE_COVERAGE in args.build_flags
):
clickhouse_execute(
args,
"""
CREATE TABLE IF NOT EXISTS system.coverage
(
time DateTime,
test_name String,
coverage Array(UInt64)
) ENGINE = MergeTree ORDER BY test_name;
""",
)
# Coverage collected at the system startup before running any tests:
clickhouse_execute(
args,
"INSERT INTO system.coverage SELECT now(), '', coverage()",
)
total_tests_run = 0
for suite in sorted(os.listdir(base_dir), key=suite_key_func):
@ -2678,6 +2735,12 @@ def parse_args():
default=False,
help="Check what high-level server components were covered by tests",
)
parser.add_argument(
"--collect-per-test-coverage",
action="store_true",
default=False,
help="Create `system.coverage` table on the server and collect information about low-level code coverage on a per test basis there",
)
parser.add_argument(
"--report-logs-stats",
action="store_true",