Skip to content

Commit

Permalink
[Issue 11632][C++] Turning on more compiler warnings, and enforcing w…
Browse files Browse the repository at this point in the history
…arnings as errors (apache#11668)

* [C++] Turning on more compiler warning flags, and enforcing warnings as errors.

This change enables several key warning flags that prevent common mistakes in C++: -Wall -Wformat-security and -Wvla, as well as ensuring the code won't build if it contains warnings.  This will help to keep the code base clean and stable long term.  I was also planning to enable "-Wextra" as it contains several helpful warnings, but I thought the changes required to get -Wall enabled were getting a bit huge as-is, so I'm going to split that effort into two PRs.

Most of the changes fall into four categories:
* The vast majority are fixing class member initialization order warnings.  These can lead to real bugs and are important to fix.
* Next was unused variables or functions - these were mostly found in the tests
* Functions with switches on enum values and no default/fallback case resulting in a code path with no return - just needed exception throw statements.
* Finally, I also fixed several misuses of the "static" keyword: when applied to global variables or functions, this actually means that the identifier has "internal linkage", meaning it's not accessible outside the current translation unit.  Putting this in a header is almost never what you want to do, but it's a common mistake since the meaning is different when applied to class members.  The "inline" keyword is a better choice in these circumstances.

Tests are still all passing.

[C++] Removing some unnecessary variable type changes I meant to remove from the previous commit.  I decided to disable the "signed comparison warning" since there are typically tons of these and they're tricky to fix because you have to go through and change all of your integer types to get them to line up right.  Most C++ code bases I've worked on in the past also disable this warning.  It's too much pain for too little gain.

* Fixed more warnings and issues arising from the Release mode build.

I had forgotten to build in release the first time, and it revealed more
warnings that needed to be addressed.  Finally, I formatted everything with clang-format.

* [C++] Compiling under clang with -Wall generates a slew of additional warnings, a few of which were quite serious.  This commit fixes these.

I also reformatted the CMakeLists file to use the newer options syntax and better support clang and GCC side by side.  This seems to work with recent GCC and clang versions.  Will need to further test with older compilers as well.

* Improved support for running tests outside the Docker environment, by allowing a locally installed gtest-parallel to be detected.

* Fixed two more unit tests

* Tweaking the compiler settings to work under the older compiler versions used in CentOS7

* Applying clang-format

* Forgot to actually fix the real unused variable warnings in this file when CRC32 is disabled

* Realized that I was running "check-format" with a version of clang-format that was too new, and it was formatting files differently.  Probably should add a better check for that at some point, but this change is already getting out of control.

* Apparently my attempts to allow gtest-parallel on a non-root path cause problems with the regular build.  Reverting them.  Also fixing potential mis-use of tar command in the test service start script.

Co-authored-by: Matteo Merli <[email protected]>
  • Loading branch information
oversearch and merlimat authored Aug 24, 2021
1 parent bd942e1 commit 4e60de6
Show file tree
Hide file tree
Showing 55 changed files with 177 additions and 232 deletions.
29 changes: 23 additions & 6 deletions pulsar-client-cpp/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -77,16 +77,28 @@ set(Boost_NO_BOOST_CMAKE ON)
set(CMAKE_CXX_STANDARD 11)
set(CMAKE_C_STANDARD 11)

if (MSVC)
# Visual Studio compiler flags
# Compiler specific configuration:
# https://stackoverflow.com/questions/10046114/in-cmake-how-can-i-test-if-the-compiler-is-clang
if (CMAKE_CXX_COMPILER_ID STREQUAL "MSVC")
add_definitions(-DWIN32_LEAN_AND_MEAN -DNOGDI -D_WIN32_WINNT=0x0501 -D_CRT_SECURE_NO_WARNINGS)
add_compile_options(/wd4244 /wd4267 /wd4018 /wd4715 /wd4251 /wd4275)
else()
add_compile_options(-Werror=switch -Wno-deprecated-declarations)
elseif (CMAKE_CXX_COMPILER_ID STREQUAL "Intel")
# ?? Don't have this to test with
else() # GCC or Clang are mostly compatible:
# Turn on warnings and enable warnings-as-errors:
add_compile_options(-Wall -Wformat-security -Wvla -Werror)
# Turn off certain warnings that are too much pain for too little gain:
add_compile_options(-Wno-sign-compare -Wno-deprecated-declarations -Wno-error=cpp)
if (CMAKE_SYSTEM_PROCESSOR STREQUAL "x86_64")
add_compile_options(-msse4.2 -mpclmul)
endif()
endif(MSVC)
# Options unique to Clang or GCC:
if (CMAKE_CXX_COMPILER_ID MATCHES "Clang")
add_compile_options(-Qunused-arguments)
elseif (CMAKE_CXX_COMPILER_ID STREQUAL "GNU" AND NOT (CMAKE_CXX_COMPILER_VERSION VERSION_LESS 4.9))
add_compile_options(-Wno-stringop-truncation)
endif()
endif()

set(CMAKE_POSITION_INDEPENDENT_CODE ON)

Expand Down Expand Up @@ -231,6 +243,8 @@ if (CMAKE_COMPILER_IS_GNUCC AND CMAKE_CXX_COMPILER_VERSION VERSION_LESS 4.9)
MESSAGE(STATUS "Using Boost::Regex")
else()
MESSAGE(STATUS "Using std::regex")
# Turn on color error messages and show additional help with errors (only available in GCC v4.9+):
add_compile_options(-fdiagnostics-show-option -fdiagnostics-color)
endif()

if(BUILD_PERF_TOOLS)
Expand Down Expand Up @@ -287,7 +301,10 @@ if (NOT APPLE AND NOT MSVC)
# we don't set options below to build _pulsar.so
set(CMAKE_CXX_FLAGS_PYTHON "${CMAKE_CXX_FLAGS}")
# Hide all non-exported symbols to avoid conflicts
set(CMAKE_CXX_FLAGS " -fvisibility=hidden -Wl,--exclude-libs,ALL ${CMAKE_CXX_FLAGS}")
add_compile_options(-fvisibility=hidden)
if (CMAKE_COMPILER_IS_GNUCC)
add_compile_options(-Wl,--exclude-libs,ALL)
endif ()
endif ()

if (LIB_ZSTD)
Expand Down
2 changes: 1 addition & 1 deletion pulsar-client-cpp/examples/SampleConsumerCApi.c
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@ int main() {
}

printf("Received message with payload: '%.*s'\n", pulsar_message_get_length(message),
pulsar_message_get_data(message));
(const char*)pulsar_message_get_data(message));

pulsar_consumer_acknowledge(consumer, message);
pulsar_message_free(message);
Expand Down
2 changes: 1 addition & 1 deletion pulsar-client-cpp/examples/SampleConsumerListenerCApi.c
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@

static void listener_callback(pulsar_consumer_t* consumer, pulsar_message_t* message, void* ctx) {
printf("Received message with payload: '%.*s'\n", pulsar_message_get_length(message),
pulsar_message_get_data(message));
(const char*)pulsar_message_get_data(message));

pulsar_consumer_acknowledge(consumer, message);
pulsar_message_free(message);
Expand Down
2 changes: 1 addition & 1 deletion pulsar-client-cpp/examples/SampleReaderCApi.c
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@ int main() {
}

printf("Received message with payload: '%.*s'\n", pulsar_message_get_length(message),
pulsar_message_get_data(message));
(const char*)pulsar_message_get_data(message));

pulsar_message_free(message);
}
Expand Down
3 changes: 2 additions & 1 deletion pulsar-client-cpp/include/pulsar/BrokerConsumerStats.h
Original file line number Diff line number Diff line change
Expand Up @@ -37,9 +37,10 @@ class PULSAR_PUBLIC BrokerConsumerStats {
std::shared_ptr<BrokerConsumerStatsImplBase> impl_;

public:
BrokerConsumerStats() = default;
explicit BrokerConsumerStats(std::shared_ptr<BrokerConsumerStatsImplBase> impl);

BrokerConsumerStats();
virtual ~BrokerConsumerStats() = default;

/** Returns true if the Stats are still valid **/
virtual bool isValid() const;
Expand Down
2 changes: 1 addition & 1 deletion pulsar-client-cpp/include/pulsar/ClientConfiguration.h
Original file line number Diff line number Diff line change
Expand Up @@ -166,7 +166,7 @@ class PULSAR_PUBLIC ClientConfiguration {
/**
* @return the path to the trusted TLS certificate file
*/
std::string getTlsTrustCertsFilePath() const;
const std::string& getTlsTrustCertsFilePath() const;

/**
* Configure whether the Pulsar client accepts untrusted TLS certificates from brokers.
Expand Down
9 changes: 3 additions & 6 deletions pulsar-client-cpp/lib/Backoff.cc
Original file line number Diff line number Diff line change
Expand Up @@ -18,16 +18,13 @@
*/
#include "Backoff.h"
#include <boost/random/uniform_int_distribution.hpp>
#include <algorithm>
#include <time.h> /* time */

namespace pulsar {

Backoff::Backoff(const TimeDuration& initial, const TimeDuration& max, const TimeDuration& mandatoryStop)
: initial_(initial),
max_(max),
next_(initial),
mandatoryStopMade_(false),
mandatoryStop_(mandatoryStop),
rng_(time(NULL)) {}
: initial_(initial), max_(max), next_(initial), mandatoryStop_(mandatoryStop), rng_(time(NULL)) {}

TimeDuration Backoff::next() {
TimeDuration current = next_;
Expand Down
6 changes: 2 additions & 4 deletions pulsar-client-cpp/lib/Backoff.h
Original file line number Diff line number Diff line change
Expand Up @@ -20,9 +20,6 @@
#define _PULSAR_BACKOFF_HEADER_
#include <boost/date_time/posix_time/posix_time.hpp>
#include <boost/random/mersenne_twister.hpp>
#include <stdlib.h> /* srand, rand */
#include <algorithm>
#include <time.h> /* time */
#include <pulsar/defines.h>

namespace pulsar {
Expand All @@ -42,7 +39,8 @@ class PULSAR_PUBLIC Backoff {
TimeDuration mandatoryStop_;
boost::posix_time::ptime firstBackoffTime_;
boost::random::mt19937 rng_;
bool mandatoryStopMade_;
bool mandatoryStopMade_ = false;

friend class PulsarFriend;
};
} // namespace pulsar
Expand Down
8 changes: 2 additions & 6 deletions pulsar-client-cpp/lib/BinaryProtoLookupService.cc
Original file line number Diff line number Diff line change
Expand Up @@ -34,15 +34,11 @@ namespace pulsar {
* Constructor
*/
BinaryProtoLookupService::BinaryProtoLookupService(ConnectionPool& cnxPool, const std::string& lookupUrl)
: cnxPool_(cnxPool), serviceUrl_(lookupUrl), mutex_(), requestIdGenerator_(0) {}
: serviceUrl_(lookupUrl), cnxPool_(cnxPool) {}

BinaryProtoLookupService::BinaryProtoLookupService(ConnectionPool& cnxPool, const std::string& lookupUrl,
const std::string& listenerName)
: cnxPool_(cnxPool),
serviceUrl_(lookupUrl),
listenerName_(listenerName),
mutex_(),
requestIdGenerator_(0) {}
: serviceUrl_(lookupUrl), listenerName_(listenerName), cnxPool_(cnxPool) {}

/*
* @param topicName topic name to get broker for
Expand Down
4 changes: 2 additions & 2 deletions pulsar-client-cpp/lib/BinaryProtoLookupService.h
Original file line number Diff line number Diff line change
Expand Up @@ -48,11 +48,11 @@ class PULSAR_PUBLIC BinaryProtoLookupService : public LookupService {

private:
std::mutex mutex_;
uint64_t requestIdGenerator_;
uint64_t requestIdGenerator_ = 0;

std::string serviceUrl_;
ConnectionPool& cnxPool_;
std::string listenerName_;
ConnectionPool& cnxPool_;

void sendTopicLookupRequest(const std::string& topicName, bool authoritative,
const std::string& listenerName, Result result,
Expand Down
2 changes: 0 additions & 2 deletions pulsar-client-cpp/lib/BrokerConsumerStats.cc
Original file line number Diff line number Diff line change
Expand Up @@ -23,8 +23,6 @@
namespace pulsar {
BrokerConsumerStats::BrokerConsumerStats(std::shared_ptr<BrokerConsumerStatsImplBase> impl) : impl_(impl) {}

BrokerConsumerStats::BrokerConsumerStats() {}

std::shared_ptr<BrokerConsumerStatsImplBase> BrokerConsumerStats::getImpl() const { return impl_; }

bool BrokerConsumerStats::isValid() const { return impl_->isValid(); }
Expand Down
1 change: 1 addition & 0 deletions pulsar-client-cpp/lib/BrokerConsumerStatsImplBase.h
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
namespace pulsar {
class BrokerConsumerStatsImplBase {
public:
virtual ~BrokerConsumerStatsImplBase() = default;
/** Returns true if the Stats are still valid **/
virtual bool isValid() const = 0;

Expand Down
4 changes: 3 additions & 1 deletion pulsar-client-cpp/lib/ClientConfiguration.cc
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,9 @@ ClientConfiguration& ClientConfiguration::setTlsTrustCertsFilePath(const std::st
return *this;
}

std::string ClientConfiguration::getTlsTrustCertsFilePath() const { return impl_->tlsTrustCertsFilePath; }
const std::string& ClientConfiguration::getTlsTrustCertsFilePath() const {
return impl_->tlsTrustCertsFilePath;
}

ClientConfiguration& ClientConfiguration::setTlsAllowInsecureConnection(bool allowInsecure) {
impl_->tlsAllowInsecureConnection = allowInsecure;
Expand Down
15 changes: 3 additions & 12 deletions pulsar-client-cpp/lib/ClientConnection.cc
Original file line number Diff line number Diff line change
Expand Up @@ -155,8 +155,7 @@ ClientConnection::ClientConnection(const std::string& logicalAddress, const std:
ExecutorServicePtr executor,
const ClientConfiguration& clientConfiguration,
const AuthenticationPtr& authentication)
: state_(Pending),
operationsTimeout_(seconds(clientConfiguration.getOperationTimeoutSeconds())),
: operationsTimeout_(seconds(clientConfiguration.getOperationTimeoutSeconds())),
authentication_(authentication),
serverProtocolVersion_(ProtocolVersion_MIN),
executor_(executor),
Expand All @@ -172,21 +171,13 @@ ClientConnection::ClientConnection(const std::string& logicalAddress, const std:
logicalAddress_(logicalAddress),
physicalAddress_(physicalAddress),
cnxString_("[<none> -> " + physicalAddress + "] "),
error_(boost::system::error_code()),
incomingBuffer_(SharedBuffer::allocate(DefaultBufferSize)),
incomingCmd_(),
connectTimeoutTask_(std::make_shared<PeriodicTask>(executor_->getIOService(),
clientConfiguration.getConnectionTimeout())),
pendingWriteBuffers_(),
pendingWriteOperations_(0),
outgoingBuffer_(SharedBuffer::allocate(DefaultBufferSize)),
outgoingCmd_(),
havePendingPingRequest_(false),
keepAliveTimer_(),
maxPendingLookupRequest_(clientConfiguration.getConcurrentLookupRequest()),
consumerStatsRequestTimer_(executor_->createDeadlineTimer()),
numOfPendingLookupRequest_(0),
isTlsAllowInsecureConnection_(false) {
maxPendingLookupRequest_(clientConfiguration.getConcurrentLookupRequest()) {

LOG_INFO(cnxString_ << "Create ClientConnection, timeout=" << clientConfiguration.getConnectionTimeout());
if (clientConfiguration.isUseTls()) {
#if BOOST_VERSION >= 105400
Expand Down
22 changes: 10 additions & 12 deletions pulsar-client-cpp/lib/ClientConnection.h
Original file line number Diff line number Diff line change
Expand Up @@ -249,7 +249,7 @@ class PULSAR_PUBLIC ClientConnection : public std::enable_shared_from_this<Clien
}
}

State state_;
State state_ = Pending;
TimeDuration operationsTimeout_;
AuthenticationPtr authentication_;
int serverProtocolVersion_;
Expand All @@ -264,9 +264,13 @@ class PULSAR_PUBLIC ClientConnection : public std::enable_shared_from_this<Clien
*/
SocketPtr socket_;
TlsSocketPtr tlsSocket_;
#if BOOST_VERSION >= 106600
boost::asio::strand<boost::asio::io_service::executor_type> strand_;
#else
boost::asio::io_service::strand strand_;
#endif

const std::string logicalAddress_;

/*
* stores address of the service, for ex. pulsar://localhost:6650
*/
Expand Down Expand Up @@ -312,7 +316,7 @@ class PULSAR_PUBLIC ClientConnection : public std::enable_shared_from_this<Clien

// Pending buffers to write on the socket
std::deque<boost::any> pendingWriteBuffers_;
int pendingWriteOperations_;
int pendingWriteOperations_ = 0;

SharedBuffer outgoingBuffer_;
proto::BaseCommand outgoingCmd_;
Expand All @@ -321,7 +325,7 @@ class PULSAR_PUBLIC ClientConnection : public std::enable_shared_from_this<Clien
HandlerAllocator writeHandlerAllocator_;

// Signals whether we're waiting for a response from broker
bool havePendingPingRequest_;
bool havePendingPingRequest_ = false;
DeadlineTimerPtr keepAliveTimer_;
DeadlineTimerPtr consumerStatsRequestTimer_;

Expand All @@ -330,16 +334,10 @@ class PULSAR_PUBLIC ClientConnection : public std::enable_shared_from_this<Clien

void startConsumerStatsTimer(std::vector<uint64_t> consumerStatsRequests);
uint32_t maxPendingLookupRequest_;
uint32_t numOfPendingLookupRequest_;
uint32_t numOfPendingLookupRequest_ = 0;
friend class PulsarFriend;

bool isTlsAllowInsecureConnection_;

#if BOOST_VERSION >= 106600
boost::asio::strand<boost::asio::io_service::executor_type> strand_;
#else
boost::asio::io_service::strand strand_;
#endif
bool isTlsAllowInsecureConnection_ = false;
};
} // namespace pulsar

Expand Down
6 changes: 3 additions & 3 deletions pulsar-client-cpp/lib/Commands.cc
Original file line number Diff line number Diff line change
Expand Up @@ -166,12 +166,11 @@ PairSharedBuffer Commands::newSend(SharedBuffer& headers, BaseCommand& cmd, uint
4 + cmdSize + magicAndChecksumLength + 4 + msgMetadataSize; // cmdLength + cmdSize + magicLength +
// checksumSize + msgMetadataLength + msgMetadataSize
int totalSize = headerContentSize + payloadSize;
int headersSize = 4 + headerContentSize; // totalSize + headerLength
int checksumReaderIndex = -1;

headers.reset();
assert(headers.writableBytes() >= headersSize);
headers.writeUnsignedInt(totalSize); // External frame
assert(headers.writableBytes() >= (4 + headerContentSize)); // totalSize + headerLength
headers.writeUnsignedInt(totalSize); // External frame

// Write cmd
headers.writeUnsignedInt(cmdSize);
Expand Down Expand Up @@ -644,6 +643,7 @@ std::string Commands::messageType(BaseCommand_Type type) {
return "END_TXN_ON_SUBSCRIPTION_RESPONSE";
break;
};
BOOST_THROW_EXCEPTION(std::logic_error("Invalid BaseCommand enumeration value"));
}

void Commands::initBatchMessageMetadata(const Message& msg, pulsar::proto::MessageMetadata& batchMetadata) {
Expand Down
3 changes: 3 additions & 0 deletions pulsar-client-cpp/lib/CompressionCodec.cc
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@ CompressionCodec& CompressionCodecProvider::getCodec(CompressionType compression
default:
return compressionCodecNone_;
}
BOOST_THROW_EXCEPTION(std::logic_error("Invalid CompressionType enumeration value"));
}

CompressionType CompressionCodecProvider::convertType(proto::CompressionType type) {
Expand All @@ -61,6 +62,7 @@ CompressionType CompressionCodecProvider::convertType(proto::CompressionType typ
case proto::SNAPPY:
return CompressionSNAPPY;
}
BOOST_THROW_EXCEPTION(std::logic_error("Invalid proto::CompressionType enumeration value"));
}

proto::CompressionType CompressionCodecProvider::convertType(CompressionType type) {
Expand All @@ -76,6 +78,7 @@ proto::CompressionType CompressionCodecProvider::convertType(CompressionType typ
case CompressionSNAPPY:
return proto::SNAPPY;
}
BOOST_THROW_EXCEPTION(std::logic_error("Invalid CompressionType enumeration value"));
}

SharedBuffer CompressionCodecNone::encode(const SharedBuffer& raw) { return raw; }
Expand Down
10 changes: 3 additions & 7 deletions pulsar-client-cpp/lib/ConsumerImpl.cc
Original file line number Diff line number Diff line change
Expand Up @@ -53,20 +53,14 @@ ConsumerImpl::ConsumerImpl(const ClientImplPtr client, const std::string& topic,
startMessageId_(startMessageId),
// This is the initial capacity of the queue
incomingMessages_(std::max(config_.getReceiverQueueSize(), 1)),
pendingReceives_(),
availablePermits_(0),
receiverQueueRefillThreshold_(config_.getReceiverQueueSize() / 2),
consumerId_(client->newConsumerId()),
consumerName_(config_.getConsumerName()),
partitionIndex_(-1),
consumerCreatedPromise_(),
messageListenerRunning_(true),
batchAcknowledgementTracker_(topic_, subscriptionName, (long)consumerId_),
brokerConsumerStats_(),
consumerStatsBasePtr_(),
negativeAcksTracker_(client, *this, conf),
ackGroupingTrackerPtr_(std::make_shared<AckGroupingTracker>()),
msgCrypto_(),
readCompacted_(conf.isReadCompacted()),
lastMessageInBroker_(Optional<MessageId>::of(MessageId())) {
std::stringstream consumerStrStream;
Expand Down Expand Up @@ -774,6 +768,7 @@ inline proto::CommandSubscribe_SubType ConsumerImpl::getSubType() {
case ConsumerKeyShared:
return proto::CommandSubscribe_SubType_Key_Shared;
}
BOOST_THROW_EXCEPTION(std::logic_error("Invalid ConsumerType enumeration value"));
}

inline proto::CommandSubscribe_InitialPosition ConsumerImpl::getInitialPosition() {
Expand All @@ -785,6 +780,7 @@ inline proto::CommandSubscribe_InitialPosition ConsumerImpl::getInitialPosition(
case InitialPositionEarliest:
return proto::CommandSubscribe_InitialPosition::CommandSubscribe_InitialPosition_Earliest;
}
BOOST_THROW_EXCEPTION(std::logic_error("Invalid InitialPosition enumeration value"));
}

void ConsumerImpl::statsCallback(Result res, ResultCallback callback, proto::CommandAck_AckType ackType) {
Expand Down Expand Up @@ -1156,7 +1152,7 @@ void ConsumerImpl::hasMessageAvailableAsync(HasMessageAvailableCallback callback
return;
}

getLastMessageIdAsync([this, lastDequed, callback](Result result, MessageId messageId) {
getLastMessageIdAsync([lastDequed, callback](Result result, MessageId messageId) {
if (result == ResultOk) {
if (messageId > lastDequed && messageId.entryId() != -1) {
callback(ResultOk, true);
Expand Down
2 changes: 1 addition & 1 deletion pulsar-client-cpp/lib/ConsumerImpl.h
Original file line number Diff line number Diff line change
Expand Up @@ -197,7 +197,7 @@ class ConsumerImpl : public ConsumerImplBase,
uint64_t consumerId_;
std::string consumerName_;
std::string consumerStr_;
int32_t partitionIndex_;
int32_t partitionIndex_ = -1;
Promise<Result, ConsumerImplBaseWeakPtr> consumerCreatedPromise_;
std::atomic_bool messageListenerRunning_;
CompressionCodecProvider compressionCodecProvider_;
Expand Down
Loading

0 comments on commit 4e60de6

Please sign in to comment.