Merge pull request #70112 from ClickHouse/impl-libfuzzer-3

CI: enable libfuzzer
This commit is contained in:
Yakov Olkhovskiy 2024-11-05 05:44:17 +00:00 committed by GitHub
commit d3e2d9d820
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
22 changed files with 306 additions and 44 deletions

View File

@ -88,6 +88,7 @@ string (TOUPPER ${CMAKE_BUILD_TYPE} CMAKE_BUILD_TYPE_UC)
list(REVERSE CMAKE_FIND_LIBRARY_SUFFIXES)
option (ENABLE_FUZZING "Fuzzy testing using libfuzzer" OFF)
option (ENABLE_FUZZER_TEST "Build testing fuzzers in order to test libFuzzer functionality" OFF)
if (ENABLE_FUZZING)
# Also set WITH_COVERAGE=1 for better fuzzing process

View File

@ -33,8 +33,6 @@ RUN apt-get update \
COPY requirements.txt /
RUN pip3 install --no-cache-dir -r /requirements.txt
ENV FUZZER_ARGS="-max_total_time=60"
SHELL ["/bin/bash", "-c"]
# docker run --network=host --volume <workspace>:/workspace -e PR_TO_TEST=<> -e SHA_TO_TEST=<> clickhouse/libfuzzer

View File

@ -1,2 +1,2 @@
clickhouse_add_executable(aggregate_function_state_deserialization_fuzzer aggregate_function_state_deserialization_fuzzer.cpp ${SRCS})
target_link_libraries(aggregate_function_state_deserialization_fuzzer PRIVATE clickhouse_aggregate_functions)
target_link_libraries(aggregate_function_state_deserialization_fuzzer PRIVATE clickhouse_aggregate_functions dbms)

View File

@ -1,2 +1,2 @@
clickhouse_add_executable (names_and_types_fuzzer names_and_types_fuzzer.cpp)
target_link_libraries (names_and_types_fuzzer PRIVATE)
target_link_libraries (names_and_types_fuzzer PRIVATE dbms)

View File

@ -1,2 +1,3 @@
clickhouse_add_executable(data_type_deserialization_fuzzer data_type_deserialization_fuzzer.cpp ${SRCS})
target_link_libraries(data_type_deserialization_fuzzer PRIVATE clickhouse_aggregate_functions)
target_link_libraries(data_type_deserialization_fuzzer PRIVATE clickhouse_aggregate_functions dbms)

View File

@ -3,6 +3,7 @@
#include <IO/ReadBufferFromMemory.h>
#include <IO/ReadHelpers.h>
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypeFactory.h>
#include <Common/MemoryTracker.h>

View File

@ -1,2 +1,2 @@
clickhouse_add_executable(format_fuzzer format_fuzzer.cpp ${SRCS})
target_link_libraries(format_fuzzer PRIVATE clickhouse_aggregate_functions)
target_link_libraries(format_fuzzer PRIVATE clickhouse_aggregate_functions dbms)

View File

@ -3,5 +3,6 @@ target_link_libraries(execute_query_fuzzer PRIVATE
dbms
clickhouse_table_functions
clickhouse_aggregate_functions
clickhouse_functions
clickhouse_dictionaries
clickhouse_dictionaries_embedded)

View File

@ -2,10 +2,10 @@ clickhouse_add_executable(lexer_fuzzer lexer_fuzzer.cpp ${SRCS})
target_link_libraries(lexer_fuzzer PRIVATE clickhouse_parsers)
clickhouse_add_executable(select_parser_fuzzer select_parser_fuzzer.cpp ${SRCS})
target_link_libraries(select_parser_fuzzer PRIVATE clickhouse_parsers dbms)
target_link_libraries(select_parser_fuzzer PRIVATE clickhouse_parsers clickhouse_functions dbms)
clickhouse_add_executable(create_parser_fuzzer create_parser_fuzzer.cpp ${SRCS})
target_link_libraries(create_parser_fuzzer PRIVATE clickhouse_parsers dbms)
target_link_libraries(create_parser_fuzzer PRIVATE clickhouse_parsers clickhouse_functions dbms)
add_subdirectory(codegen_fuzzer)

View File

@ -47,4 +47,4 @@ target_compile_options (codegen_select_fuzzer PRIVATE -Wno-newline-eof)
target_link_libraries(protoc ch_contrib::fuzzer)
target_include_directories(codegen_select_fuzzer SYSTEM BEFORE PRIVATE "${CMAKE_CURRENT_BINARY_DIR}")
target_link_libraries(codegen_select_fuzzer PRIVATE ch_contrib::protobuf_mutator ch_contrib::protoc dbms)
target_link_libraries(codegen_select_fuzzer PRIVATE ch_contrib::protobuf_mutator ch_contrib::protoc clickhouse_functions dbms)

View File

@ -4,4 +4,4 @@ clickhouse_add_executable (mergetree_checksum_fuzzer mergetree_checksum_fuzzer.c
target_link_libraries (mergetree_checksum_fuzzer PRIVATE dbms)
clickhouse_add_executable (columns_description_fuzzer columns_description_fuzzer.cpp)
target_link_libraries (columns_description_fuzzer PRIVATE)
target_link_libraries (columns_description_fuzzer PRIVATE dbms)

View File

@ -539,9 +539,9 @@ class CI:
JobNames.LIBFUZZER_TEST: JobConfig(
required_builds=[BuildNames.FUZZERS],
run_by_labels=[Tags.libFuzzer],
timeout=10800,
timeout=5400,
run_command='libfuzzer_test_check.py "$CHECK_NAME"',
runner_type=Runners.STYLE_CHECKER,
runner_type=Runners.FUNC_TESTER,
),
JobNames.DOCKER_SERVER: CommonJobConfigs.DOCKER_SERVER.with_properties(
required_builds=[BuildNames.PACKAGE_RELEASE, BuildNames.PACKAGE_AARCH64]

View File

@ -3,20 +3,37 @@
import argparse
import logging
import os
import re
import sys
import zipfile
from pathlib import Path
from typing import List
from botocore.exceptions import ClientError
from build_download_helper import download_fuzzers
from clickhouse_helper import CiLogsCredentials
from docker_images_helper import DockerImage, get_docker_image, pull_image
from env_helper import REPO_COPY, REPORT_PATH, TEMP_PATH
from env_helper import REPO_COPY, REPORT_PATH, S3_BUILDS_BUCKET, TEMP_PATH
from pr_info import PRInfo
from report import FAILURE, SUCCESS, JobReport, TestResult
from s3_helper import S3Helper
from stopwatch import Stopwatch
from tee_popen import TeePopen
TIMEOUT = 60
NO_CHANGES_MSG = "Nothing to run"
s3 = S3Helper()
def zipdir(path, ziph):
# ziph is zipfile handle
for root, _, files in os.walk(path):
for file in files:
ziph.write(
os.path.join(root, file),
os.path.relpath(os.path.join(root, file), os.path.join(path, "..")),
)
def get_additional_envs(check_name, run_by_hash_num, run_by_hash_total):
@ -59,16 +76,19 @@ def get_run_command(
envs = [
# a static link, don't use S3_URL or S3_DOWNLOAD
'-e S3_URL="https://s3.amazonaws.com/clickhouse-datasets"',
'-e S3_URL="https://s3.amazonaws.com"',
]
envs += [f"-e {e}" for e in additional_envs]
env_str = " ".join(envs)
uid = os.getuid()
gid = os.getgid()
return (
f"docker run "
f"{ci_logs_args} "
f"--user {uid}:{gid} "
f"--workdir=/fuzzers "
f"--volume={fuzzers_path}:/fuzzers "
f"--volume={repo_path}/tests:/usr/share/clickhouse-test "
@ -85,6 +105,115 @@ def parse_args():
return parser.parse_args()
def download_corpus(path: str):
logging.info("Download corpus...")
try:
s3.download_file(
bucket=S3_BUILDS_BUCKET,
s3_path="fuzzer/corpus.zip",
local_file_path=path,
)
except ClientError as e:
if e.response["Error"]["Code"] == "NoSuchKey":
logging.debug("No active corpus exists")
else:
raise
with zipfile.ZipFile(f"{path}/corpus.zip", "r") as zipf:
zipf.extractall(path)
os.remove(f"{path}/corpus.zip")
units = 0
for _, _, files in os.walk(path):
units += len(files)
logging.info("...downloaded %d units", units)
def upload_corpus(path: str):
with zipfile.ZipFile(f"{path}/corpus.zip", "w", zipfile.ZIP_DEFLATED) as zipf:
zipdir(f"{path}/corpus/", zipf)
s3.upload_file(
bucket=S3_BUILDS_BUCKET,
file_path=f"{path}/corpus.zip",
s3_path="fuzzer/corpus.zip",
)
def process_error(path: Path) -> list:
ERROR = r"^==\d+==\s?ERROR: (\S+): (.*)"
# error_source = ""
# error_reason = ""
# test_unit = ""
# TEST_UNIT_LINE = r"artifact_prefix='.*\/'; Test unit written to (.*)"
error_info = []
is_error = False
with open(path, "r", encoding="utf-8") as file:
for line in file:
line = line.rstrip("\n")
if is_error:
error_info.append(line)
# match = re.search(TEST_UNIT_LINE, line)
# if match:
# test_unit = match.group(1)
continue
match = re.search(ERROR, line)
if match:
error_info.append(line)
# error_source = match.group(1)
# error_reason = match.group(2)
is_error = True
return error_info
def read_status(status_path: Path):
result = []
with open(status_path, "r", encoding="utf-8") as file:
for line in file:
result.append(line.rstrip("\n"))
return result
def process_results(result_path: Path):
test_results = []
oks = 0
errors = 0
fails = 0
for file in result_path.glob("*.status"):
fuzzer = file.stem
file_path = file.parent / fuzzer
file_path_unit = file_path.with_suffix(".unit")
file_path_out = file_path.with_suffix(".out")
file_path_stdout = file_path.with_suffix(".stdout")
status = read_status(file)
result = TestResult(fuzzer, status[0], float(status[2]))
if status[0] == "OK":
oks += 1
elif status[0] == "ERROR":
errors += 1
if file_path_out.exists():
result.set_log_files(f"['{file_path_out}']")
elif file_path_stdout.exists():
result.set_log_files(f"['{file_path_stdout}']")
else:
fails += 1
if file_path_out.exists():
result.set_raw_logs("\n".join(process_error(file_path_out)))
if file_path_unit.exists():
result.set_log_files(f"['{file_path_unit}']")
elif file_path_out.exists():
result.set_log_files(f"['{file_path_out}']")
elif file_path_stdout.exists():
result.set_log_files(f"['{file_path_stdout}']")
test_results.append(result)
return [oks, errors, fails, test_results]
def main():
logging.basicConfig(level=logging.INFO)
@ -114,15 +243,18 @@ def main():
fuzzers_path = temp_path / "fuzzers"
fuzzers_path.mkdir(parents=True, exist_ok=True)
download_corpus(fuzzers_path)
download_fuzzers(check_name, reports_path, fuzzers_path)
for file in os.listdir(fuzzers_path):
if file.endswith("_fuzzer"):
os.chmod(fuzzers_path / file, 0o777)
elif file.endswith("_seed_corpus.zip"):
corpus_path = fuzzers_path / (file.removesuffix("_seed_corpus.zip") + ".in")
seed_corpus_path = fuzzers_path / (
file.removesuffix("_seed_corpus.zip") + ".in"
)
with zipfile.ZipFile(fuzzers_path / file, "r") as zfd:
zfd.extractall(corpus_path)
zfd.extractall(seed_corpus_path)
result_path = temp_path / "result_path"
result_path.mkdir(parents=True, exist_ok=True)
@ -133,6 +265,8 @@ def main():
check_name, run_by_hash_num, run_by_hash_total
)
additional_envs.append(f"TIMEOUT={TIMEOUT}")
ci_logs_credentials = CiLogsCredentials(Path(temp_path) / "export-logs-config.sh")
ci_logs_args = ci_logs_credentials.get_docker_arguments(
pr_info, stopwatch.start_time_str, check_name
@ -152,10 +286,25 @@ def main():
retcode = process.wait()
if retcode == 0:
logging.info("Run successfully")
upload_corpus(fuzzers_path)
else:
logging.info("Run failed")
sys.exit(0)
results = process_results(result_path)
success = results[1] == 0 and results[2] == 0
JobReport(
description=f"OK: {results[0]}, ERROR: {results[1]}, FAIL: {results[2]}",
test_results=results[3],
status=SUCCESS if success else FAILURE,
start_time=stopwatch.start_time_str,
duration=stopwatch.duration_seconds,
additional_files=[],
).dump()
if not success:
sys.exit(1)
if __name__ == "__main__":

View File

@ -311,24 +311,32 @@ class S3Helper:
def list_prefix(
self, s3_prefix_path: str, bucket: str = S3_BUILDS_BUCKET
) -> List[str]:
objects = self.client.list_objects_v2(Bucket=bucket, Prefix=s3_prefix_path)
paginator = self.client.get_paginator("list_objects_v2")
pages = paginator.paginate(Bucket=bucket, Prefix=s3_prefix_path)
result = []
if "Contents" in objects:
for obj in objects["Contents"]:
result.append(obj["Key"])
for page in pages:
if "Contents" in page:
for obj in page["Contents"]:
result.append(obj["Key"])
return result
def list_prefix_non_recursive(
self, s3_prefix_path: str, bucket: str = S3_BUILDS_BUCKET
self,
s3_prefix_path: str,
bucket: str = S3_BUILDS_BUCKET,
only_dirs: bool = False,
) -> List[str]:
objects = self.client.list_objects_v2(Bucket=bucket, Prefix=s3_prefix_path)
paginator = self.client.get_paginator("list_objects_v2")
pages = paginator.paginate(Bucket=bucket, Prefix=s3_prefix_path, Delimiter="/")
result = []
if "Contents" in objects:
for obj in objects["Contents"]:
if "/" not in obj["Key"][len(s3_prefix_path) + 1 :]:
for page in pages:
if not only_dirs and "Contents" in page:
for obj in page["Contents"]:
result.append(obj["Key"])
if "CommonPrefixes" in page:
for obj in page["CommonPrefixes"]:
result.append(obj["Prefix"])
return result
def url_if_exists(self, key: str, bucket: str = S3_BUILDS_BUCKET) -> str:

View File

@ -1,5 +1,8 @@
#!/bin/bash -eu
# rename clickhouse
mv $OUT/clickhouse $OUT/clickhouse_fuzzer
# copy fuzzer options and dictionaries
cp $SRC/tests/fuzz/*.dict $OUT/
cp $SRC/tests/fuzz/*.options $OUT/

View File

@ -0,0 +1,2 @@
[CI]
FUZZER_ARGS = true

View File

@ -1,26 +1,49 @@
#!/usr/bin/env python3
import configparser
import datetime
import logging
import os
import subprocess
from pathlib import Path
DEBUGGER = os.getenv("DEBUGGER", "")
FUZZER_ARGS = os.getenv("FUZZER_ARGS", "")
TIMEOUT = int(os.getenv("TIMEOUT", "0"))
OUTPUT = "/test_output"
def run_fuzzer(fuzzer: str):
class Stopwatch:
def __init__(self):
self.reset()
@property
def duration_seconds(self) -> float:
return (datetime.datetime.utcnow() - self.start_time).total_seconds()
@property
def start_time_str(self) -> str:
return self.start_time_str_value
def reset(self) -> None:
self.start_time = datetime.datetime.utcnow()
self.start_time_str_value = self.start_time.strftime("%Y-%m-%d %H:%M:%S")
def run_fuzzer(fuzzer: str, timeout: int):
logging.info("Running fuzzer %s...", fuzzer)
corpus_dir = f"{fuzzer}.in"
with Path(corpus_dir) as path:
seed_corpus_dir = f"{fuzzer}.in"
with Path(seed_corpus_dir) as path:
if not path.exists() or not path.is_dir():
corpus_dir = ""
seed_corpus_dir = ""
active_corpus_dir = f"corpus/{fuzzer}"
if not os.path.exists(active_corpus_dir):
os.makedirs(active_corpus_dir)
options_file = f"{fuzzer}.options"
custom_libfuzzer_options = ""
fuzzer_arguments = ""
use_fuzzer_args = False
with Path(options_file) as path:
if path.exists() and path.is_file():
@ -44,7 +67,9 @@ def run_fuzzer(fuzzer: str):
if parser.has_section("libfuzzer"):
custom_libfuzzer_options = " ".join(
f"-{key}={value}" for key, value in parser["libfuzzer"].items()
f"-{key}={value}"
for key, value in parser["libfuzzer"].items()
if key not in ("jobs", "exact_artifact_path")
)
if parser.has_section("fuzzer_arguments"):
@ -53,19 +78,70 @@ def run_fuzzer(fuzzer: str):
for key, value in parser["fuzzer_arguments"].items()
)
cmd_line = f"{DEBUGGER} ./{fuzzer} {FUZZER_ARGS} {corpus_dir}"
if custom_libfuzzer_options:
cmd_line += f" {custom_libfuzzer_options}"
if fuzzer_arguments:
cmd_line += f" {fuzzer_arguments}"
use_fuzzer_args = parser.getboolean("CI", "FUZZER_ARGS", fallback=False)
if not "-dict=" in cmd_line and Path(f"{fuzzer}.dict").exists():
cmd_line += f" -dict={fuzzer}.dict"
exact_artifact_path = f"{OUTPUT}/{fuzzer}.unit"
status_path = f"{OUTPUT}/{fuzzer}.status"
out_path = f"{OUTPUT}/{fuzzer}.out"
stdout_path = f"{OUTPUT}/{fuzzer}.stdout"
cmd_line += " < /dev/null"
if not "-dict=" in custom_libfuzzer_options and Path(f"{fuzzer}.dict").exists():
custom_libfuzzer_options += f" -dict={fuzzer}.dict"
custom_libfuzzer_options += f" -exact_artifact_path={exact_artifact_path}"
logging.info("...will execute: %s", cmd_line)
subprocess.check_call(cmd_line, shell=True)
libfuzzer_corpora = f"{active_corpus_dir} {seed_corpus_dir}"
cmd_line = f"{DEBUGGER} ./{fuzzer} {fuzzer_arguments}"
env = None
with_fuzzer_args = ""
if use_fuzzer_args:
env = {"FUZZER_ARGS": f"{custom_libfuzzer_options} {libfuzzer_corpora}".strip()}
with_fuzzer_args = f" with FUZZER_ARGS '{env['FUZZER_ARGS']}'"
else:
cmd_line += f" {custom_libfuzzer_options} {libfuzzer_corpora}"
logging.info("...will execute: '%s'%s", cmd_line, with_fuzzer_args)
stopwatch = Stopwatch()
try:
with open(out_path, "wb") as out, open(stdout_path, "wb") as stdout:
subprocess.run(
cmd_line.split(),
stdin=subprocess.DEVNULL,
stdout=stdout,
stderr=out,
text=True,
check=True,
shell=False,
errors="replace",
timeout=timeout,
env=env,
)
except subprocess.CalledProcessError:
logging.info("Fail running %s", fuzzer)
with open(status_path, "w", encoding="utf-8") as status:
status.write(
f"FAIL\n{stopwatch.start_time_str}\n{stopwatch.duration_seconds}\n"
)
except subprocess.TimeoutExpired:
logging.info("Successful running %s", fuzzer)
with open(status_path, "w", encoding="utf-8") as status:
status.write(
f"OK\n{stopwatch.start_time_str}\n{stopwatch.duration_seconds}\n"
)
except Exception as e:
logging.info("Unexpected exception running %s: %s", fuzzer, e)
with open(status_path, "w", encoding="utf-8") as status:
status.write(
f"ERROR\n{stopwatch.start_time_str}\n{stopwatch.duration_seconds}\n"
)
else:
logging.info("Error running %s", fuzzer)
with open(status_path, "w", encoding="utf-8") as status:
status.write(
f"ERROR\n{stopwatch.start_time_str}\n{stopwatch.duration_seconds}\n"
)
def main():
@ -73,10 +149,14 @@ def main():
subprocess.check_call("ls -al", shell=True)
timeout = 30 if TIMEOUT == 0 else TIMEOUT
with Path() as current:
for fuzzer in current.iterdir():
if (current / fuzzer).is_file() and os.access(current / fuzzer, os.X_OK):
run_fuzzer(fuzzer)
run_fuzzer(fuzzer.name, timeout)
subprocess.check_call(f"ls -al {OUTPUT}", shell=True)
if __name__ == "__main__":

View File

@ -23,3 +23,7 @@ if (ENABLE_UTILS)
add_subdirectory (keeper-data-dumper)
add_subdirectory (memcpy-bench)
endif ()
if (ENABLE_FUZZING AND ENABLE_FUZZER_TEST)
add_subdirectory (libfuzzer-test)
endif ()

View File

@ -0,0 +1 @@
add_subdirectory (test_basic_fuzzer)

View File

@ -0,0 +1 @@
This folder contains various stuff intended to test libfuzzer functionality.

View File

@ -0,0 +1 @@
add_executable (test_basic_fuzzer main.cpp)

View File

@ -0,0 +1,11 @@
#include <stdint.h>
#include <stddef.h>
extern "C" int LLVMFuzzerTestOneInput(const uint8_t *data, size_t size)
{
if (size > 0 && data[0] == 'H')
if (size > 1 && data[1] == 'I')
if (size > 2 && data[2] == '!')
__builtin_trap();
return 0;
}