[CI] Add clang-tidy to lint (#18124)

* clang-tidy

* fix

* fix script

* test clang compiler

* fix clang-tidy rules

* Fix windows and other issues.

* Fix

* Improve information when running check-git-clang-tidy-output.sh on different OS
This commit is contained in:
mwtian 2021-09-09 00:41:53 -07:00 committed by GitHub
parent 8a066474d4
commit 26fd10c9e8
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
12 changed files with 657 additions and 13 deletions

View file

@ -32,8 +32,11 @@ RUN apt-get install -y -qq \
sudo unzip unrar apt-utils dialog tzdata wget rsync \
language-pack-en tmux cmake gdb vim htop \
libgtk2.0-dev zlib1g-dev libgl1-mesa-dev maven \
openjdk-8-jre openjdk-8-jdk clang-format-7
RUN ln -s /usr/bin/clang-format-7 /usr/bin/clang-format
openjdk-8-jre openjdk-8-jdk clang-format-7 jq \
clang-tidy-12 clang-12
RUN ln -s /usr/bin/clang-format-7 /usr/bin/clang-format && \
ln -s /usr/bin/clang-tidy-12 /usr/bin/clang-tidy && \
ln -s /usr/bin/clang-12 /usr/bin/clang
RUN curl -o- https://get.docker.com | sh
# System conf for tests

34
.clang-tidy Normal file
View file

@ -0,0 +1,34 @@
# Disable the following checks due to frequent false positives, noisiness,
# inconsistent style with existing codebase and other reasons:
# -misc-non-private-member-variables-in-classes (potentially too restrictive)
# -misc-unused-parameters (can be cleaned up in batch and enabled)
# -modernize-avoid-c-arrays (too restrictive)
# -modernize-pass-by-value (too restrictive)
# -modernize-return-braced-init-list (inconsistent style)
# -modernize-use-emplace (more subtle behavior)
# -modernize-use-trailing-return-type (inconsistent style)
#
# TODO: enable google-* and readability-* families of checks.
Checks: >
abseil-*,
bugprone-*,
misc-*,
-misc-non-private-member-variables-in-classes,
-misc-unused-parameters,
modernize-*,
-modernize-avoid-c-arrays,
-modernize-pass-by-value,
-modernize-return-braced-init-list,
-modernize-use-emplace,
-modernize-use-trailing-return-type,
performance-*,
CheckOptions:
# Reduce noisiness of the bugprone-narrowing-conversions check.
- key: bugprone-narrowing-conversions.IgnoreConversionFromTypes
value: 'size_t;ptrdiff_t;size_type;difference_type'
- key: bugprone-narrowing-conversions.WarnOnEquivalentBitWidth
value: 'false'
# Turn all the warnings from the checks above into errors.
WarningsAsErrors: "*"

View file

@ -1,13 +1,18 @@
# Bazel build
# C/C++ documentation: https://docs.bazel.build/versions/master/be/c-cpp.html
load("@rules_proto//proto:defs.bzl", "proto_library")
load("@rules_python//python:defs.bzl", "py_library")
load("@rules_cc//cc:defs.bzl", "cc_binary", "cc_library", "cc_test")
load("@rules_cc//cc:defs.bzl", "cc_binary", "cc_library", "cc_proto_library", "cc_test")
load("@com_github_grpc_grpc//bazel:cc_grpc_library.bzl", "cc_grpc_library")
load("@com_github_grpc_grpc//bazel:cython_library.bzl", "pyx_library")
load("@com_github_google_flatbuffers//:build_defs.bzl", "flatbuffer_cc_library")
load("//bazel:ray.bzl", "COPTS", "PYX_COPTS", "PYX_SRCS", "copy_to_workspace")
package(
default_visibility = ["//visibility:public"],
)
config_setting(
name = "msvc-cl",
flag_values = {"@bazel_tools//tools/cpp:compiler": "msvc-cl"},
@ -1809,18 +1814,25 @@ cc_library(
)
filegroup(
name = "iwyu_sh",
srcs = ["ci/travis/iwyu.sh"],
name = "extra_actions_base_proto",
srcs = [
"thirdparty/protobuf/extra_actions_base.proto",
],
)
proto_library(
name = "extra_actions_base_proto_lib",
srcs = ["thirdparty/protobuf/extra_actions_base.proto"],
)
cc_proto_library(
name = "extra_actions_cc_proto_lib",
deps = [":extra_actions_base_proto_lib"],
)
filegroup(
name = "extra_actions_base_proto",
srcs = [
# TODO: Replace our file with the built-in copy once this issue is resolved:
# https://github.com/bazelbuild/bazel/issues/8738
"thirdparty/protobuf/extra_actions_base.proto",
#"@bazel_tools//src/main/protobuf:extra_actions_base.proto",
],
name = "iwyu_sh",
srcs = ["ci/travis/iwyu.sh"],
)
action_listener(

22
bazel/BUILD.rapidjson Normal file
View file

@ -0,0 +1,22 @@
licenses(["notice"])
package(default_visibility = ["//visibility:public"])
filegroup(
name = "license",
srcs = ["license.txt"],
)
cc_library(
name = "rapidjson",
hdrs = glob([
"include/rapidjson/*.h",
"include/rapidjson/*/*.h",
]),
copts = [
"-Wno-non-virtual-dtor",
"-Wno-unused-variable",
"-Wno-implicit-fallthrough",
],
includes = ["include"],
)

View file

@ -128,6 +128,7 @@ def copy_to_workspace(name, srcs, dstdir = ""):
dstdir = "." + ("\\" + dstdir.replace("/", "\\")).rstrip("\\") + "\\",
),
local = 1,
tags = ["no-cache"],
)
def native_java_binary(module_name, name, native_binary_name):

View file

@ -278,3 +278,9 @@ def ray_deps_setup():
build_file = "@com_github_ray_project_ray//bazel:BUILD.nlohmann_json",
)
auto_http_archive(
name = "rapidjson",
url = "https://github.com/Tencent/rapidjson/archive/v1.1.0.zip",
build_file = True,
sha256 = "8e00c38829d6785a2dfb951bb87c6974fa07dfe488aa5b25deec4b8bc0f6a3ab",
)

View file

@ -0,0 +1,32 @@
# Tool for listening on Bazel actions and generate compile commands database.
#
# Using Bazel aspect to generate compile commands would be faster. Also Bazel
# action listeners are deprecated. We can switch to that if a stable solution
# exists, e.g. https://github.com/grailbio/bazel-compilation-database
cc_binary(
name = "extract_compile_command",
srcs = ["extract_compile_command.cc"],
# Build fails on Windows, and not part of Ray either.
tags = ["manual"],
deps = [
"//:extra_actions_cc_proto_lib",
"@com_google_protobuf//:protobuf",
"@rapidjson",
],
)
action_listener(
name = "compile_command_listener",
extra_actions = [":compile_command_action"],
mnemonics = ["CppCompile"],
)
extra_action(
name = "compile_command_action",
cmd = "$(location :extract_compile_command) \
$(EXTRA_ACTION_FILE) \
$(output $(ACTION_ID).compile_command.json)",
out_templates = ["$(ACTION_ID).compile_command.json"],
tools = [":extract_compile_command"],
)

View file

@ -0,0 +1,121 @@
/*
* Copyright 2016 The Kythe Authors. All rights reserved.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*
* Adapted from
* https://github.com/xulongwu4/bazel-compilation-database/blob/master/kythe/generate_compile_commands/extract_compile_command.cc
*/
#include <fcntl.h>
#include <sys/stat.h>
#include <sys/types.h>
#include <unistd.h>
#include <cstdio>
#include <string>
#include <vector>
#include "google/protobuf/io/coded_stream.h"
#include "google/protobuf/io/zero_copy_stream.h"
#include "google/protobuf/io/zero_copy_stream_impl.h"
#include "google/protobuf/stubs/common.h"
#include "rapidjson/stringbuffer.h"
#include "rapidjson/writer.h"
#include "thirdparty/protobuf/extra_actions_base.pb.h"
namespace {
using ::google::protobuf::io::CodedInputStream;
using ::google::protobuf::io::FileInputStream;
bool ReadExtraAction(const std::string &path, blaze::ExtraActionInfo *info,
blaze::CppCompileInfo *cpp_info) {
int fd = ::open(path.c_str(), O_RDONLY, S_IREAD | S_IWRITE);
if (fd < 0) {
perror("Failed to open input: ");
return false;
}
FileInputStream file_input(fd);
file_input.SetCloseOnDelete(true);
CodedInputStream input(&file_input);
if (!info->ParseFromCodedStream(&input)) return false;
if (!info->HasExtension(blaze::CppCompileInfo::cpp_compile_info)) return false;
*cpp_info = info->GetExtension(blaze::CppCompileInfo::cpp_compile_info);
return true;
}
std::string JoinCommand(const std::vector<std::string> &command) {
std::string output;
if (command.empty()) return output;
// TODO(shahms): Deal with embedded spaces and quotes.
auto iter = command.begin();
output = *iter++;
for (; iter != command.end(); ++iter) {
output += " " + *iter;
}
return output;
}
std::string FormatCompilationCommand(const std::string &source_file,
const std::vector<std::string> &command) {
rapidjson::StringBuffer buffer;
rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
writer.StartObject();
writer.Key("file");
writer.String(source_file.c_str());
writer.Key("directory");
writer.String("@BAZEL_ROOT@");
writer.Key("command");
writer.String(JoinCommand(command).c_str());
writer.EndObject();
return buffer.GetString();
}
} // namespace
int main(int argc, char **argv) {
GOOGLE_PROTOBUF_VERIFY_VERSION;
if (argc != 3) {
std::cerr << "usage: " << argv[0] << " extra-action-file output-file" << std::endl;
return 1;
}
std::string extra_action_file = argv[1];
std::string output_file = argv[2];
blaze::ExtraActionInfo info;
blaze::CppCompileInfo cpp_info;
if (!ReadExtraAction(extra_action_file, &info, &cpp_info)) return 1;
std::vector<std::string> args;
args.push_back(cpp_info.tool());
args.insert(args.end(), cpp_info.compiler_option().begin(),
cpp_info.compiler_option().end());
if (std::find(args.begin(), args.end(), "-c") == args.end()) {
args.push_back("-c");
args.push_back(cpp_info.source_file());
}
if (std::find(args.begin(), args.end(), "-o") == args.end()) {
args.push_back("-o");
args.push_back(cpp_info.output_file());
}
FILE *output = ::fopen(output_file.c_str(), "w");
if (output == nullptr) {
perror("Unable to open file for writing: ");
return 1;
}
::fputs(FormatCompilationCommand(cpp_info.source_file(), args).c_str(), output);
::fclose(output);
google::protobuf::ShutdownProtobufLibrary();
return 0;
}

View file

@ -0,0 +1,92 @@
#!/bin/bash
# TODO: integrate this script into pull request workflow.
printError() {
printf '\033[31mERROR:\033[0m %s\n' "$@"
}
printInfo() {
printf '\033[32mINFO:\033[0m %s\n' "$@"
}
log_err() {
printError "Setting up clang-tidy encountered an error"
}
set -eo pipefail
trap '[ $? -eq 0 ] || log_err' EXIT
printInfo "Fetching workspace info ..."
WORKSPACE=$(bazel info workspace)
BAZEL_ROOT=$(bazel info execution_root)
printInfo "Generating compilation database ..."
case "${OSTYPE}" in
linux*)
printInfo " Running on Linux, using clang to build C++ targets. Please make sure it is installed ..."
CC=clang bazel build //ci/generate_compile_commands:extract_compile_command //:ray_pkg \
--experimental_action_listener=//ci/generate_compile_commands:compile_command_listener;;
darwin*)
printInfo " Running on MacOS, assuming default C++ compiler is clang ..."
bazel build //ci/generate_compile_commands:extract_compile_command //:ray_pkg \
--experimental_action_listener=//ci/generate_compile_commands:compile_command_listener;;
msys*)
printInfo " Running on Windows, using clang-cl to build C++ targets. Please make sure it is installed ..."
CC=clang-cl bazel build //ci/generate_compile_commands:extract_compile_command //:ray_pkg \
--experimental_action_listener=//ci/generate_compile_commands:compile_command_listener;;
esac
printInfo "Assembling compilation database ..."
TMPFILE=$(mktemp)
printf '[\n' >"$TMPFILE"
find "$BAZEL_ROOT" -name '*.compile_command.json' -exec cat {} + >>"$TMPFILE"
printf '\n]\n' >>"$TMPFILE"
if [[ "${OSTYPE}" =~ darwin* ]]; then
sed -i '' "s|@BAZEL_ROOT@|$BAZEL_ROOT|g" "$TMPFILE"
sed -i '' "s/}{/},\n{/g" "$TMPFILE"
else
sed -i "s|@BAZEL_ROOT@|$BAZEL_ROOT|g" "$TMPFILE"
sed -i "s/}{/},\n{/g" "$TMPFILE"
fi
OUTFILE=$WORKSPACE/compile_commands.json
if hash jq 2>/dev/null; then
printInfo "Formatting compilation database ..."
jq . "$TMPFILE" >"$OUTFILE"
else
printInfo "Can not find jq. Skip formatting compilation database."
cp --no-preserve=mode "$TMPFILE" "$OUTFILE"
fi
# Compare against the master branch, because most development is done against it.
base_commit="$(git merge-base HEAD master)"
if [ "$base_commit" = "$(git rev-parse HEAD)" ]; then
# Prefix of master branch, so compare against parent commit
base_commit="$(git rev-parse HEAD^)"
printInfo "Running clang-tidy against parent commit $base_commit"
else
printInfo "Running clang-tidy against parent commit $base_commit from master branch"
fi
trap - EXIT
if git diff -U0 "$base_commit" | ci/travis/clang-tidy-diff.py -p1 -fix; then
printInfo "clang-tidy passed."
else
printError "clang-tidy failed. See above for details including suggested fixes."
printError
printError "If you think the warning is too aggressive, the proposed fix is incorrect or are unsure about how to"
printError "fix, feel free to raise the issue on the PR or Anyscale #learning-cplusplus Slack channel."
printError
printError "To run clang-tidy locally with fix suggestions, make sure clang and clang-tidy are installed and"
printError "available in PATH (version 12 is preferred). Then run"
printError "scripts/check-git-clang-tidy-output.sh"
printError "from repo root."
fi

View file

@ -460,6 +460,13 @@ _lint() {
bazel query 'kind("cc_test", //...)' --output=xml | python "${ROOT_DIR}"/check-bazel-team-owner.py
bazel query 'kind("py_test", //...)' --output=xml | python "${ROOT_DIR}"/check-bazel-team-owner.py
popd
# Run clang-tidy last since it needs to rebuild.
if command -v clang-tidy > /dev/null; then
"${ROOT_DIR}"/check-git-clang-tidy-output.sh
else
{ echo "WARNING: Skipping running clang-tidy which is not installed."; } 2> /dev/null
fi
fi
}

307
ci/travis/clang-tidy-diff.py Executable file
View file

@ -0,0 +1,307 @@
#!/usr/bin/env python
#
# This file is based on
# https://github.com/llvm-mirror/clang-tools-extra/blob/5c40544fa40bfb85ec888b6a03421b3905e4a4e7/clang-tidy/tool/clang-tidy-diff.py
#
# ===- clang-tidy-diff.py - ClangTidy Diff Checker ----------*- python -*--===#
#
# Part of the LLVM Project, under the Apache License v2.0 with LLVM Exceptions.
# See https://llvm.org/LICENSE.txt for license information.
# SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception
#
# ===----------------------------------------------------------------------===#
r"""
ClangTidy Diff Checker
======================
This script reads input from a unified diff, runs clang-tidy on all changed
files and outputs clang-tidy warnings in changed lines only. This is useful to
detect clang-tidy regressions in the lines touched by a specific patch.
Example usage for git/svn users:
git diff -U0 HEAD^ | clang-tidy-diff.py -p1
svn diff --diff-cmd=diff -x-U0 | \
clang-tidy-diff.py -fix -checks=-*,modernize-use-override
"""
import argparse
import glob
import json
import multiprocessing
import os
import re
import shutil
import subprocess
import sys
import tempfile
import threading
import traceback
try:
import yaml
except ImportError:
yaml = None
is_py2 = sys.version[0] == "2"
if is_py2:
import Queue as queue
else:
import queue as queue
def run_tidy(task_queue, lock, timeout):
watchdog = None
while True:
command = task_queue.get()
try:
proc = subprocess.Popen(
command, stdout=subprocess.PIPE, stderr=subprocess.PIPE)
if timeout is not None:
watchdog = threading.Timer(timeout, proc.kill)
watchdog.start()
stdout, stderr = proc.communicate()
with lock:
sys.stdout.write(stdout.decode("utf-8") + "\n")
sys.stdout.flush()
if stderr:
sys.stderr.write(stderr.decode("utf-8") + "\n")
sys.stderr.flush()
except Exception as e:
with lock:
sys.stderr.write("Failed: " + str(e) + ": ".join(command) +
"\n")
finally:
with lock:
if timeout is not None and watchdog is not None:
if not watchdog.is_alive():
sys.stderr.write("Terminated by timeout: " +
" ".join(command) + "\n")
watchdog.cancel()
task_queue.task_done()
def start_workers(max_tasks, tidy_caller, task_queue, lock, timeout):
for _ in range(max_tasks):
t = threading.Thread(
target=tidy_caller, args=(task_queue, lock, timeout))
t.daemon = True
t.start()
def merge_replacement_files(tmpdir, mergefile):
"""Merge all replacement files in a directory into a single file"""
# The fixes suggested by clang-tidy >= 4.0.0 are given under
# the top level key 'Diagnostics' in the output yaml files
mergekey = "Diagnostics"
merged = []
for replacefile in glob.iglob(os.path.join(tmpdir, "*.yaml")):
content = yaml.safe_load(open(replacefile, "r"))
if not content:
continue # Skip empty files.
merged.extend(content.get(mergekey, []))
if merged:
# MainSourceFile: The key is required by the definition inside
# include/clang/Tooling/ReplacementsYaml.h, but the value
# is actually never used inside clang-apply-replacements,
# so we set it to '' here.
output = {"MainSourceFile": "", mergekey: merged}
with open(mergefile, "w") as out:
yaml.safe_dump(output, out)
else:
# Empty the file:
open(mergefile, "w").close()
def main():
parser = argparse.ArgumentParser(
description="Run clang-tidy against changed files, and "
"output diagnostics only for modified "
"lines.")
parser.add_argument(
"-clang-tidy-binary",
metavar="PATH",
default="clang-tidy",
help="path to clang-tidy binary")
parser.add_argument(
"-p",
metavar="NUM",
default=0,
help="strip the smallest prefix containing P slashes")
parser.add_argument(
"-regex",
metavar="PATTERN",
default=None,
help="custom pattern selecting file paths to check "
"(case sensitive, overrides -iregex)")
parser.add_argument(
"-iregex",
metavar="PATTERN",
default=r".*\.(cpp|cc|c\+\+|cxx|c|cl|h|hpp|m|mm|inc)",
help="custom pattern selecting file paths to check "
"(case insensitive, overridden by -regex)")
parser.add_argument(
"-j",
type=int,
default=1,
help="number of tidy instances to be run in parallel.")
parser.add_argument(
"-timeout",
type=int,
default=None,
help="timeout per each file in seconds.")
parser.add_argument(
"-fix",
action="store_true",
default=False,
help="apply suggested fixes")
parser.add_argument(
"-checks",
help="checks filter, when not specified, use clang-tidy "
"default",
default="")
parser.add_argument(
"-path",
dest="build_path",
help="Path used to read a compile command database.")
if yaml:
parser.add_argument(
"-export-fixes",
metavar="FILE",
dest="export_fixes",
help="Create a yaml file to store suggested fixes in, "
"which can be applied with clang-apply-replacements.")
parser.add_argument(
"-extra-arg",
dest="extra_arg",
action="append",
default=[],
help="Additional argument to append to the compiler "
"command line.")
parser.add_argument(
"-extra-arg-before",
dest="extra_arg_before",
action="append",
default=[],
help="Additional argument to prepend to the compiler "
"command line.")
parser.add_argument(
"-quiet",
action="store_true",
default=False,
help="Run clang-tidy in quiet mode")
clang_tidy_args = []
argv = sys.argv[1:]
if "--" in argv:
clang_tidy_args.extend(argv[argv.index("--"):])
argv = argv[:argv.index("--")]
args = parser.parse_args(argv)
# Extract changed lines for each file.
filename = None
lines_by_file = {}
for line in sys.stdin:
match = re.search('^\+\+\+\ \"?(.*?/){%s}([^ \t\n\"]*)' % args.p, line)
if match:
filename = match.group(2)
if filename is None:
continue
if args.regex is not None:
if not re.match("^%s$" % args.regex, filename):
continue
else:
if not re.match("^%s$" % args.iregex, filename, re.IGNORECASE):
continue
match = re.search("^@@.*\+(\d+)(,(\d+))?", line)
if match:
start_line = int(match.group(1))
line_count = 1
if match.group(3):
line_count = int(match.group(3))
if line_count == 0:
continue
end_line = start_line + line_count - 1
lines_by_file.setdefault(filename,
[]).append([start_line, end_line])
if not any(lines_by_file):
print("No relevant changes found.")
sys.exit(0)
max_task_count = args.j
if max_task_count == 0:
max_task_count = multiprocessing.cpu_count()
max_task_count = min(len(lines_by_file), max_task_count)
tmpdir = None
if yaml and args.export_fixes:
tmpdir = tempfile.mkdtemp()
# Tasks for clang-tidy.
task_queue = queue.Queue(max_task_count)
# A lock for console output.
lock = threading.Lock()
# Run a pool of clang-tidy workers.
start_workers(max_task_count, run_tidy, task_queue, lock, args.timeout)
# Form the common args list.
common_clang_tidy_args = []
if args.fix:
common_clang_tidy_args.append("-fix")
if args.checks != "":
common_clang_tidy_args.append("-checks=" + args.checks)
if args.quiet:
common_clang_tidy_args.append("-quiet")
if args.build_path is not None:
common_clang_tidy_args.append("-p=%s" % args.build_path)
for arg in args.extra_arg:
common_clang_tidy_args.append("-extra-arg=%s" % arg)
for arg in args.extra_arg_before:
common_clang_tidy_args.append("-extra-arg-before=%s" % arg)
for name in lines_by_file:
line_filter_json = json.dumps(
[{
"name": name,
"lines": lines_by_file[name]
}],
separators=(",", ":"))
# Run clang-tidy on files containing changes.
command = [args.clang_tidy_binary]
command.append("-line-filter=" + line_filter_json)
if yaml and args.export_fixes:
# Get a temporary file. We immediately close the handle so
# clang-tidy can overwrite it.
(handle, tmp_name) = tempfile.mkstemp(suffix=".yaml", dir=tmpdir)
os.close(handle)
command.append("-export-fixes=" + tmp_name)
command.extend(common_clang_tidy_args)
command.append(name)
command.extend(clang_tidy_args)
task_queue.put(command)
# Wait for all threads to be done.
task_queue.join()
if yaml and args.export_fixes:
print("Writing fixes to " + args.export_fixes + " ...")
try:
merge_replacement_files(tmpdir, args.export_fixes)
except Exception:
sys.stderr.write("Error exporting fixes.\n")
traceback.print_exc()
if tmpdir:
shutil.rmtree(tmpdir)
if __name__ == "__main__":
main()

View file

@ -177,7 +177,8 @@ An output like the following indicates failure:
* branch master -> FETCH_HEAD
python/ray/util/sgd/tf/tf_runner.py:4:1: F401 'numpy as np' imported but unused # Below is the failure
In addition, there are other formatting checkers for components like the following:
In addition, there are other formatting and semantic checkers for components like the following (not included in
``./ci/travis/format.sh``):
* Python README format:
@ -192,6 +193,12 @@ In addition, there are other formatting checkers for components like the followi
./ci/travis/bazel-format.sh
* clang-tidy for C++ anti-patterns, requires ``clang`` and ``clang-tidy`` version 12 to be installed:
.. code-block:: shell
./ci/travis/check-git-clang-tidy-output.sh
Understanding CI test jobs
--------------------------