Skip to content

Commit

Permalink
Require semicolon after velox macros, avoid triggering -Wextra-semi
Browse files Browse the repository at this point in the history
Reviewed By: zertosh

Differential Revision: D59007258

fbshipit-source-id: 33b9f6ba7f86a0540e62ccdb77c008bf2df57611
  • Loading branch information
David Tolnay authored and facebook-github-bot committed Sep 18, 2024
1 parent 8a6ab15 commit a8a21a4
Show file tree
Hide file tree
Showing 82 changed files with 138 additions and 131 deletions.
25 changes: 16 additions & 9 deletions velox/common/base/Exceptions.h
Original file line number Diff line number Diff line change
Expand Up @@ -156,7 +156,7 @@ std::string errorMessage(fmt::string_view fmt, const Args&... args) {

#define _VELOX_THROW_IMPL( \
exception, exprStr, errorSource, errorCode, isRetriable, ...) \
{ \
do { \
/* GCC 9.2.1 doesn't accept this code with constexpr. */ \
static const ::facebook::velox::detail::VeloxCheckFailArgs \
veloxCheckFailArgs = { \
Expand All @@ -172,14 +172,21 @@ std::string errorMessage(fmt::string_view fmt, const Args&... args) {
exception, \
typename ::facebook::velox::detail::VeloxCheckFailStringType< \
decltype(message)>::type>(veloxCheckFailArgs, message); \
}

#define _VELOX_CHECK_AND_THROW_IMPL( \
expr, exprStr, exception, errorSource, errorCode, isRetriable, ...) \
if (UNLIKELY(!(expr))) { \
_VELOX_THROW_IMPL( \
exception, exprStr, errorSource, errorCode, isRetriable, __VA_ARGS__); \
}
} while (0)

#define _VELOX_CHECK_AND_THROW_IMPL( \
expr, exprStr, exception, errorSource, errorCode, isRetriable, ...) \
do { \
if (UNLIKELY(!(expr))) { \
_VELOX_THROW_IMPL( \
exception, \
exprStr, \
errorSource, \
errorCode, \
isRetriable, \
__VA_ARGS__); \
} \
} while (0)

#define _VELOX_THROW(exception, ...) \
_VELOX_THROW_IMPL(exception, "", ##__VA_ARGS__)
Expand Down
2 changes: 1 addition & 1 deletion velox/common/caching/SsdFile.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -939,7 +939,7 @@ void SsdFile::maybeVerifyChecksum(
"IOERR: Corrupt SSD cache entry - File: {}, Offset: {}, Size: {}",
fileName_,
ssdRun.offset(),
ssdRun.size())
ssdRun.size());
}
}

Expand Down
4 changes: 2 additions & 2 deletions velox/common/memory/MemoryArbitrator.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@ class FactoryRegistry {
VELOX_USER_CHECK(
map_.find(kind) != map_.end(),
"Arbitrator factory for kind {} not registered",
kind)
kind);
return map_[kind];
}

Expand All @@ -54,7 +54,7 @@ class FactoryRegistry {
VELOX_USER_CHECK(
map_.find(kind) != map_.end(),
"Arbitrator factory for kind {} not registered",
kind)
kind);
return map_.erase(kind);
}

Expand Down
2 changes: 1 addition & 1 deletion velox/connectors/hive/HiveDataSink.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -503,7 +503,7 @@ void HiveDataSink::computePartitionAndBucketIds(const RowVectorPtr& input) {
VELOX_USER_CHECK(
!col->isNullAt(i),
"Partition key must not be null: {}",
input->type()->asRow().nameOf(partitionIdx))
input->type()->asRow().nameOf(partitionIdx));
}
}
}
Expand Down
2 changes: 1 addition & 1 deletion velox/connectors/hive/HiveDataSink.h
Original file line number Diff line number Diff line change
Expand Up @@ -218,7 +218,7 @@ class HiveInsertTableHandle : public ConnectorInsertTableHandle {
if (compressionKind.has_value()) {
VELOX_CHECK(
compressionKind.value() != common::CompressionKind_MAX,
"Unsupported compression type: CompressionKind_MAX")
"Unsupported compression type: CompressionKind_MAX");
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ void AzuriteServer::start() {
serverProcess_->exit_code(),
383,
"AzuriteServer process exited, code: ",
serverProcess_->exit_code())
serverProcess_->exit_code());
} catch (const std::exception& e) {
VELOX_FAIL("Failed to launch Azurite server: {}", e.what());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ class HdfsFileSystem::Impl {
hdfsClient_,
"Unable to connect to HDFS: {}, got error: {}.",
endpoint.identity(),
hdfsGetLastError())
hdfsGetLastError());
}

~Impl() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -91,6 +91,6 @@ void HdfsReadFile::checkFileReadParameters(uint64_t offset, uint64_t length)
"Cannot read HDFS file beyond its size: {}, offset: {}, end point: {}",
fileSize,
offset,
endPoint)
endPoint);
}
} // namespace facebook::velox
2 changes: 1 addition & 1 deletion velox/connectors/hive/storage_adapters/hdfs/HdfsReadFile.h
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ struct HdfsFile {

int32_t read(char* pos, uint64_t length) const {
auto bytesRead = hdfsRead(client_, handle_, pos, length);
VELOX_CHECK(bytesRead >= 0, "Read failure in HDFSReadFile::preadInternal.")
VELOX_CHECK(bytesRead >= 0, "Read failure in HDFSReadFile::preadInternal.");
return bytesRead;
}
};
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ void HdfsMiniCluster::start() {
serverProcess_->exit_code(),
383,
"Minicluster process exited, code: ",
serverProcess_->exit_code())
serverProcess_->exit_code());
} catch (const std::exception& e) {
VELOX_FAIL("Failed to launch Minicluster server: {}", e.what());
}
Expand Down Expand Up @@ -91,7 +91,7 @@ void HdfsMiniCluster::addFile(std::string source, std::string destination) {
if (!isExited) {
VELOX_FAIL(
"Failed to add file to hdfs, exit code: {}",
filePutProcess->exit_code())
filePutProcess->exit_code());
}
}

Expand Down
2 changes: 1 addition & 1 deletion velox/connectors/hive/storage_adapters/s3fs/S3Util.h
Original file line number Diff line number Diff line change
Expand Up @@ -180,7 +180,7 @@ inline std::string getRequestID(
if (error.GetResponseCode() == Aws::Http::HttpResponseCode::NOT_FOUND) { \
VELOX_FILE_NOT_FOUND_ERROR(errMsg); \
} \
VELOX_FAIL(errMsg) \
VELOX_FAIL(errMsg); \
} \
}

Expand Down
2 changes: 1 addition & 1 deletion velox/duckdb/conversion/DuckConversion.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ using ::duckdb::string_t;
using ::duckdb::timestamp_t;

variant decimalVariant(const Value& val) {
VELOX_DCHECK(val.type().id() == LogicalTypeId::DECIMAL)
VELOX_DCHECK(val.type().id() == LogicalTypeId::DECIMAL);
switch (val.type().InternalType()) {
case ::duckdb::PhysicalType::INT128: {
auto unscaledValue = val.GetValueUnsafe<::duckdb::hugeint_t>();
Expand Down
6 changes: 3 additions & 3 deletions velox/duckdb/conversion/DuckParser.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -594,7 +594,7 @@ std::shared_ptr<const core::IExpr> parseLambdaExpr(
} else {
VELOX_FAIL(
"Unexpected left-hand-side expression for the lambda expression: {}",
capture->toString())
capture->toString());
}

return std::make_shared<const core::LambdaExpr>(
Expand Down Expand Up @@ -695,7 +695,7 @@ bool isAscending(::duckdb::OrderType orderType, const std::string& exprString) {
return true;
case ::duckdb::OrderType::INVALID:
default:
VELOX_FAIL("Cannot parse ORDER BY clause: {}", exprString)
VELOX_FAIL("Cannot parse ORDER BY clause: {}", exprString);
}
}

Expand All @@ -712,7 +712,7 @@ bool isNullsFirst(
return false;
case ::duckdb::OrderByNullType::INVALID:
default:
VELOX_FAIL("Cannot parse ORDER BY clause: {}", exprString)
VELOX_FAIL("Cannot parse ORDER BY clause: {}", exprString);
}

VELOX_UNREACHABLE();
Expand Down
2 changes: 1 addition & 1 deletion velox/dwio/common/tests/utils/FilterGenerator.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@ VectorPtr getChildBySubfield(
const RowTypePtr& rootType) {
const Type* type = rootType ? rootType.get() : rowVector->type().get();
auto& path = subfield.path();
VELOX_CHECK(!path.empty())
VELOX_CHECK(!path.empty());
auto* rowType = &type->asRow();
auto* field = dynamic_cast<const Subfield::NestedField*>(path[0].get());
VELOX_CHECK(field);
Expand Down
2 changes: 1 addition & 1 deletion velox/dwio/parquet/reader/DeltaBpDecoder.h
Original file line number Diff line number Diff line change
Expand Up @@ -142,7 +142,7 @@ class DeltaBpDecoder {
VELOX_DCHECK_GT(totalValuesRemaining_, 0, "initBlock called at EOF");

if (!getZigZagVlqInt(minDelta_)) {
VELOX_FAIL("initBlock EOF")
VELOX_FAIL("initBlock EOF");
}

// read the bitwidth of each miniblock
Expand Down
2 changes: 1 addition & 1 deletion velox/dwio/parquet/reader/PageReader.h
Original file line number Diff line number Diff line change
Expand Up @@ -326,7 +326,7 @@ class PageReader {
int>::type = 0>
void
callDecoder(const uint64_t* nulls, bool& nullsFromFastPath, Visitor visitor) {
VELOX_CHECK(!isDictionary(), "BOOLEAN types are never dictionary-encoded")
VELOX_CHECK(!isDictionary(), "BOOLEAN types are never dictionary-encoded");
if (nulls) {
nullsFromFastPath = false;
booleanDecoder_->readWithVisitor<true>(nulls, visitor);
Expand Down
2 changes: 1 addition & 1 deletion velox/dwio/parquet/reader/ParquetReader.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -598,7 +598,7 @@ std::unique_ptr<ParquetTypeWithId> ReaderBase::getParquetColumnInfo(
return leafTypePtr;
}

VELOX_FAIL("Unable to extract Parquet column info.")
VELOX_FAIL("Unable to extract Parquet column info.");
return nullptr;
}

Expand Down
2 changes: 1 addition & 1 deletion velox/dwio/parquet/writer/Writer.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -157,7 +157,7 @@ void validateSchemaRecursive(const RowTypePtr& schema) {

folly::F14FastSet<std::string> uniqueNames;
for (const auto& name : fieldNames) {
VELOX_USER_CHECK(!name.empty(), "Field name must not be empty.")
VELOX_USER_CHECK(!name.empty(), "Field name must not be empty.");
auto result = uniqueNames.insert(name);
VELOX_USER_CHECK(
result.second,
Expand Down
4 changes: 2 additions & 2 deletions velox/exec/Driver.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -237,7 +237,7 @@ void BlockingState::setResume(std::shared_ptr<BlockingState> state) {
VELOX_FAIL(
"A ContinueFuture for task {} was realized with error: {}",
state->driver_->task()->taskId(),
e.what())
e.what());
} catch (const VeloxException&) {
state->driver_->task()->setError(std::current_exception());
}
Expand Down Expand Up @@ -841,7 +841,7 @@ void Driver::run(std::shared_ptr<Driver> self) {
VELOX_CHECK_NULL(
nullResult,
"The last operator (sink) must not produce any results. "
"Results need to be consumed by either a callback or another operator. ")
"Results need to be consumed by either a callback or another operator. ");

// There can be a race between Task terminating and the Driver being on the
// thread and exiting the runInternal() in a blocked state. If this happens
Expand Down
2 changes: 1 addition & 1 deletion velox/exec/Expand.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,7 @@ Expand::Expand(
} else {
VELOX_USER_FAIL(
"Expand operator doesn't support this expression. Only column references and constants are supported. {}",
columnProjection->toString())
columnProjection->toString());
}
}

Expand Down
2 changes: 1 addition & 1 deletion velox/exec/FilterProject.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -129,7 +129,7 @@ RowVectorPtr FilterProject::getOutput() {
vector_size_t size = input_->size();
LocalSelectivityVector localRows(*operatorCtx_->execCtx(), size);
auto* rows = localRows.get();
VELOX_DCHECK_NOT_NULL(rows)
VELOX_DCHECK_NOT_NULL(rows);
rows->setAll();
EvalCtx evalCtx(operatorCtx_->execCtx(), exprs_.get(), input_.get());

Expand Down
2 changes: 1 addition & 1 deletion velox/exec/HashProbe.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -903,7 +903,7 @@ void HashProbe::checkStateTransition(ProbeOperatorState state) {
} else {
VELOX_CHECK(
state_ == ProbeOperatorState::kWaitForBuild ||
state_ == ProbeOperatorState::kWaitForPeers)
state_ == ProbeOperatorState::kWaitForPeers);
}
break;
case ProbeOperatorState::kWaitForPeers:
Expand Down
4 changes: 2 additions & 2 deletions velox/exec/SortWindowBuild.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -296,13 +296,13 @@ void SortWindowBuild::loadNextPartitionFromSpill() {

std::shared_ptr<WindowPartition> SortWindowBuild::nextPartition() {
if (merge_ != nullptr) {
VELOX_CHECK(!sortedRows_.empty(), "No window partitions available")
VELOX_CHECK(!sortedRows_.empty(), "No window partitions available");
auto partition = folly::Range(sortedRows_.data(), sortedRows_.size());
return std::make_shared<WindowPartition>(
data_.get(), partition, inversedInputChannels_, sortKeyInfo_);
}

VELOX_CHECK(!partitionStartRows_.empty(), "No window partitions available")
VELOX_CHECK(!partitionStartRows_.empty(), "No window partitions available");

currentPartition_++;
VELOX_CHECK_LE(
Expand Down
2 changes: 1 addition & 1 deletion velox/exec/Unnest.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ Unnest::Unnest(
const auto& unnestVariables = unnestNode->unnestVariables();
for (const auto& variable : unnestVariables) {
if (!variable->type()->isArray() && !variable->type()->isMap()) {
VELOX_UNSUPPORTED("Unnest operator supports only ARRAY and MAP types")
VELOX_UNSUPPORTED("Unnest operator supports only ARRAY and MAP types");
}
unnestChannels_.push_back(inputType->getChildIdx(variable->name()));
}
Expand Down
2 changes: 1 addition & 1 deletion velox/exec/WindowFunction.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -94,7 +94,7 @@ std::unique_ptr<WindowFunction> WindowFunction::create(
"Unexpected return type for window function {}. Expected {}. Got {}.",
toString(name, argTypes),
type->toString(),
resultType->toString())
resultType->toString());
return func.value()->factory(
args, resultType, ignoreNulls, pool, stringAllocator, config);
}
Expand Down
2 changes: 1 addition & 1 deletion velox/exec/fuzzer/AggregationFuzzer.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -332,7 +332,7 @@ bool supportsDistinctInputs(
void AggregationFuzzer::go() {
VELOX_CHECK(
FLAGS_steps > 0 || FLAGS_duration_sec > 0,
"Either --steps or --duration_sec needs to be greater than zero.")
"Either --steps or --duration_sec needs to be greater than zero.");

auto startTime = std::chrono::system_clock::now();
size_t iteration = 0;
Expand Down
2 changes: 1 addition & 1 deletion velox/exec/fuzzer/CacheFuzzer.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -325,7 +325,7 @@ void CacheFuzzer::read(uint32_t fileIdx, int32_t fragmentIdx) {
void CacheFuzzer::go() {
VELOX_CHECK(
FLAGS_steps > 0 || FLAGS_duration_sec > 0,
"Either --steps or --duration_sec needs to be greater than zero.")
"Either --steps or --duration_sec needs to be greater than zero.");

auto startTime = std::chrono::system_clock::now();
size_t iteration = 0;
Expand Down
2 changes: 1 addition & 1 deletion velox/exec/fuzzer/JoinFuzzer.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -1467,7 +1467,7 @@ std::vector<std::vector<RowVectorPtr>> JoinFuzzer::splitInputByGroup(
void JoinFuzzer::go() {
VELOX_USER_CHECK(
FLAGS_steps > 0 || FLAGS_duration_sec > 0,
"Either --steps or --duration_sec needs to be greater than zero.")
"Either --steps or --duration_sec needs to be greater than zero.");
VELOX_USER_CHECK_GE(FLAGS_batch_size, 10, "Batch size must be at least 10.");

const auto startTime = std::chrono::system_clock::now();
Expand Down
2 changes: 1 addition & 1 deletion velox/exec/fuzzer/MemoryArbitrationFuzzer.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -734,7 +734,7 @@ void MemoryArbitrationFuzzer::verify() {
void MemoryArbitrationFuzzer::go() {
VELOX_USER_CHECK(
FLAGS_steps > 0 || FLAGS_duration_sec > 0,
"Either --steps or --duration_sec needs to be greater than zero.")
"Either --steps or --duration_sec needs to be greater than zero.");
VELOX_USER_CHECK_GE(FLAGS_batch_size, 10, "Batch size must be at least 10.");

const auto startTime = std::chrono::system_clock::now();
Expand Down
2 changes: 1 addition & 1 deletion velox/exec/fuzzer/RowNumberFuzzer.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -460,7 +460,7 @@ void RowNumberFuzzer::verify() {
void RowNumberFuzzer::go() {
VELOX_USER_CHECK(
FLAGS_steps > 0 || FLAGS_duration_sec > 0,
"Either --steps or --duration_sec needs to be greater than zero.")
"Either --steps or --duration_sec needs to be greater than zero.");
VELOX_USER_CHECK_GE(FLAGS_batch_size, 10, "Batch size must be at least 10.");

const auto startTime = std::chrono::system_clock::now();
Expand Down
2 changes: 1 addition & 1 deletion velox/exec/fuzzer/WindowFuzzer.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -208,7 +208,7 @@ std::vector<SortingKeyAndOrder> WindowFuzzer::generateSortingKeysAndOrders(
void WindowFuzzer::go() {
VELOX_CHECK(
FLAGS_steps > 0 || FLAGS_duration_sec > 0,
"Either --steps or --duration_sec needs to be greater than zero.")
"Either --steps or --duration_sec needs to be greater than zero.");

auto startTime = std::chrono::system_clock::now();
size_t iteration = 0;
Expand Down
2 changes: 1 addition & 1 deletion velox/exec/fuzzer/WriterFuzzer.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -287,7 +287,7 @@ WriterFuzzer::WriterFuzzer(
void WriterFuzzer::go() {
VELOX_CHECK(
FLAGS_steps > 0 || FLAGS_duration_sec > 0,
"Either --steps or --duration_sec needs to be greater than zero.")
"Either --steps or --duration_sec needs to be greater than zero.");

auto startTime = std::chrono::system_clock::now();
size_t iteration = 0;
Expand Down
2 changes: 1 addition & 1 deletion velox/exec/prefixsort/PrefixSortAlgorithm.h
Original file line number Diff line number Diff line change
Expand Up @@ -252,7 +252,7 @@ class PrefixSortRunner {
const detail::PrefixSortIterator& start,
const detail::PrefixSortIterator& end,
TCompare compare) const {
VELOX_CHECK(end >= start, "Invalid sort range.")
VELOX_CHECK(end >= start, "Invalid sort range.");
const uint64_t len = end - start;

// Insertion sort on smallest arrays
Expand Down
Loading

0 comments on commit a8a21a4

Please sign in to comment.