diff --git a/base/base/EnumReflection.h b/base/base/EnumReflection.h index 4a9de4d17a3..e4e0ef672fd 100644 --- a/base/base/EnumReflection.h +++ b/base/base/EnumReflection.h @@ -32,7 +32,7 @@ constexpr void static_for(F && f) template struct fmt::formatter : fmt::formatter { - constexpr auto format(T value, auto& format_context) + constexpr auto format(T value, auto& format_context) const { return formatter::format(magic_enum::enum_name(value), format_context); } diff --git a/base/base/wide_integer_to_string.h b/base/base/wide_integer_to_string.h index c2cbe8d82e3..f703a722afa 100644 --- a/base/base/wide_integer_to_string.h +++ b/base/base/wide_integer_to_string.h @@ -62,7 +62,7 @@ struct fmt::formatter> } template - auto format(const wide::integer & value, FormatContext & ctx) + auto format(const wide::integer & value, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}", to_string(value)); } diff --git a/contrib/fmtlib b/contrib/fmtlib index b6f4ceaed0a..a33701196ad 160000 --- a/contrib/fmtlib +++ b/contrib/fmtlib @@ -1 +1 @@ -Subproject commit b6f4ceaed0a0a24ccf575fab6c56dd50ccf6f1a9 +Subproject commit a33701196adfad74917046096bf5a2aa0ab0bb50 diff --git a/contrib/fmtlib-cmake/CMakeLists.txt b/contrib/fmtlib-cmake/CMakeLists.txt index fe399ddc6e1..6625e411295 100644 --- a/contrib/fmtlib-cmake/CMakeLists.txt +++ b/contrib/fmtlib-cmake/CMakeLists.txt @@ -13,7 +13,6 @@ set (SRCS ${FMT_SOURCE_DIR}/include/fmt/core.h ${FMT_SOURCE_DIR}/include/fmt/format.h ${FMT_SOURCE_DIR}/include/fmt/format-inl.h - ${FMT_SOURCE_DIR}/include/fmt/locale.h ${FMT_SOURCE_DIR}/include/fmt/os.h ${FMT_SOURCE_DIR}/include/fmt/ostream.h ${FMT_SOURCE_DIR}/include/fmt/printf.h diff --git a/src/Analyzer/Identifier.h b/src/Analyzer/Identifier.h index cbd8f5e7694..91190dc7cdb 100644 --- a/src/Analyzer/Identifier.h +++ b/src/Analyzer/Identifier.h @@ -406,7 +406,7 @@ struct fmt::formatter } template - auto format(const DB::Identifier & identifier, FormatContext & ctx) + auto format(const DB::Identifier & identifier, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}", identifier.getFullName()); } @@ -428,7 +428,7 @@ struct fmt::formatter } template - auto format(const DB::IdentifierView & identifier_view, FormatContext & ctx) + auto format(const DB::IdentifierView & identifier_view, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}", identifier_view.getFullName()); } diff --git a/src/Client/TestHint.h b/src/Client/TestHint.h index eaf854be5df..b76c4245df4 100644 --- a/src/Client/TestHint.h +++ b/src/Client/TestHint.h @@ -112,7 +112,7 @@ struct fmt::formatter } template - auto format(const DB::TestHint::ErrorVector & ErrorVector, FormatContext & ctx) + auto format(const DB::TestHint::ErrorVector & ErrorVector, FormatContext & ctx) const { if (ErrorVector.empty()) return fmt::format_to(ctx.out(), "{}", 0); diff --git a/src/Common/TransactionID.h b/src/Common/TransactionID.h index 97d0072bc14..466f3f5343b 100644 --- a/src/Common/TransactionID.h +++ b/src/Common/TransactionID.h @@ -108,7 +108,7 @@ struct fmt::formatter } template - auto format(const DB::TransactionID & tid, FormatContext & context) + auto format(const DB::TransactionID & tid, FormatContext & context) const { return fmt::format_to(context.out(), "({}, {}, {})", tid.start_csn, tid.local_tid, tid.host_id); } diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index ddd30c4eef2..7d574247aa5 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -647,7 +647,7 @@ public: template <> struct fmt::formatter : fmt::formatter { - constexpr auto format(Coordination::Error code, auto & ctx) + constexpr auto format(Coordination::Error code, auto & ctx) const { return formatter::format(Coordination::errorMessage(code), ctx); } diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 2d053c615d9..f8d209bc11f 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -1,4 +1,6 @@ #include "filesystemHelpers.h" +#include +#include #if defined(OS_LINUX) # include @@ -11,7 +13,7 @@ #include #include #include -#include +#include #include #include #include @@ -369,10 +371,11 @@ Poco::Timestamp getModificationTimestamp(const std::string & path) void setModificationTime(const std::string & path, time_t time) { - struct utimbuf tb; - tb.actime = time; - tb.modtime = time; - if (utime(path.c_str(), &tb) != 0) + struct timeval times[2]; + times[0].tv_usec = times[1].tv_usec = 0; + times[0].tv_sec = ::time(nullptr); + times[1].tv_sec = time; + if (utimes(path.c_str(), times) != 0) DB::ErrnoException::throwFromPath(DB::ErrorCodes::PATH_ACCESS_DENIED, path, "Cannot set modification time to file: {}", path); } diff --git a/src/Common/formatReadable.h b/src/Common/formatReadable.h index a05a2a7f9e2..0d7a437219a 100644 --- a/src/Common/formatReadable.h +++ b/src/Common/formatReadable.h @@ -49,7 +49,7 @@ struct fmt::formatter } template - auto format(const ReadableSize & size, FormatContext & ctx) + auto format(const ReadableSize & size, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}", formatReadableSizeWithBinarySuffix(size.value)); } diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index f25ccab86b1..23ff714a929 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -466,7 +466,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial #ifdef NDEBUG /// TODO (alesapin) remove this, it should be always CORRUPTED_DATA. LOG_ERROR(getLogger("KeeperSnapshotManager"), "Children counter in stat.numChildren {}" - " is different from actual children size {} for node {}", itr.value.numChildren(), itr.value.getChildren().size(), itr.key); + " is different from actual children size {} for node {}", itr.value.numChildren(), itr.value.getChildren().size(), itr.key.toView()); #else throw Exception(ErrorCodes::LOGICAL_ERROR, "Children counter in stat.numChildren {}" " is different from actual children size {} for node {}", diff --git a/src/Coordination/RaftServerConfig.h b/src/Coordination/RaftServerConfig.h index 0ecbd6464c1..37b6a92ba70 100644 --- a/src/Coordination/RaftServerConfig.h +++ b/src/Coordination/RaftServerConfig.h @@ -57,7 +57,7 @@ using ClusterUpdateActions = std::vector; template <> struct fmt::formatter : fmt::formatter { - constexpr auto format(const DB::RaftServerConfig & server, format_context & ctx) + constexpr auto format(const DB::RaftServerConfig & server, format_context & ctx) const { return fmt::format_to( ctx.out(), "server.{}={};{};{}", server.id, server.endpoint, server.learner ? "learner" : "participant", server.priority); @@ -67,7 +67,7 @@ struct fmt::formatter : fmt::formatter template <> struct fmt::formatter : fmt::formatter { - constexpr auto format(const DB::ClusterUpdateAction & action, format_context & ctx) + constexpr auto format(const DB::ClusterUpdateAction & action, format_context & ctx) const { if (const auto * add = std::get_if(&action)) return fmt::format_to(ctx.out(), "(Add server {})", add->id); diff --git a/src/Core/Field.h b/src/Core/Field.h index a78b589c883..f1bb4a72b0d 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -1038,7 +1038,7 @@ struct fmt::formatter } template - auto format(const DB::Field & x, FormatContext & ctx) + auto format(const DB::Field & x, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}", toString(x)); } diff --git a/src/Core/QualifiedTableName.h b/src/Core/QualifiedTableName.h index bf05bd59caf..0fd72c32a54 100644 --- a/src/Core/QualifiedTableName.h +++ b/src/Core/QualifiedTableName.h @@ -125,7 +125,7 @@ namespace fmt } template - auto format(const DB::QualifiedTableName & name, FormatContext & ctx) + auto format(const DB::QualifiedTableName & name, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}.{}", DB::backQuoteIfNeed(name.database), DB::backQuoteIfNeed(name.table)); } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 46c30240ef8..bd6065ca270 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -623,7 +623,7 @@ struct fmt::formatter } template - auto format(const DB::DataTypePtr & type, FormatContext & ctx) + auto format(const DB::DataTypePtr & type, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}", type->getName()); } diff --git a/src/Functions/FunctionsStringDistance.cpp b/src/Functions/FunctionsStringDistance.cpp index 6cb23bbea9f..d0d8ebc946d 100644 --- a/src/Functions/FunctionsStringDistance.cpp +++ b/src/Functions/FunctionsStringDistance.cpp @@ -159,7 +159,7 @@ struct ByteJaccardIndexImpl } else { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal UTF-8 sequence, while processing '{}'", StringRef(haystack, haystack_end - haystack)); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal UTF-8 sequence, while processing '{}'", std::string_view(haystack, haystack_end - haystack)); } } } @@ -186,7 +186,7 @@ struct ByteJaccardIndexImpl } else { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal UTF-8 sequence, while processing '{}'", StringRef(needle, needle_end - needle)); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal UTF-8 sequence, while processing '{}'", std::string_view(needle, needle_end - needle)); } } } diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index d4b2d8ea0dc..6b0de441e94 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -1420,7 +1420,7 @@ struct fmt::formatter } template - auto format(const DB::UUID & uuid, FormatContext & context) + auto format(const DB::UUID & uuid, FormatContext & context) const { return fmt::format_to(context.out(), "{}", toString(uuid)); } diff --git a/src/Interpreters/StorageID.h b/src/Interpreters/StorageID.h index 96e3cefe00c..69dac8ea32d 100644 --- a/src/Interpreters/StorageID.h +++ b/src/Interpreters/StorageID.h @@ -136,7 +136,7 @@ namespace fmt } template - auto format(const DB::StorageID & storage_id, FormatContext & ctx) + auto format(const DB::StorageID & storage_id, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}", storage_id.getNameForLogs()); } diff --git a/src/Parsers/formatAST.h b/src/Parsers/formatAST.h index dd72a59b4a2..e34902663dd 100644 --- a/src/Parsers/formatAST.h +++ b/src/Parsers/formatAST.h @@ -40,7 +40,7 @@ struct fmt::formatter } template - auto format(const DB::ASTPtr & ast, FormatContext & context) + auto format(const DB::ASTPtr & ast, FormatContext & context) const { return fmt::format_to(context.out(), "{}", DB::serializeAST(*ast)); } diff --git a/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp index 6a3475a1830..56c774782c2 100644 --- a/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp @@ -421,7 +421,7 @@ void BSONEachRowRowInputFormat::readTuple(IColumn & column, const DataTypePtr & "Cannot parse tuple column with type {} from BSON array/embedded document field: " "tuple doesn't have element with name \"{}\"", data_type->getName(), - name); + name.toView()); index = *try_get_index; } @@ -806,7 +806,7 @@ bool BSONEachRowRowInputFormat::readRow(MutableColumns & columns, RowReadExtensi else { if (seen_columns[index]) - throw Exception(ErrorCodes::INCORRECT_DATA, "Duplicate field found while parsing BSONEachRow format: {}", name); + throw Exception(ErrorCodes::INCORRECT_DATA, "Duplicate field found while parsing BSONEachRow format: {}", name.toView()); seen_columns[index] = true; read_columns[index] = readField(*columns[index], types[index], BSONType(type)); diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index af340c4aab8..b9f61d30182 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -37,7 +37,7 @@ struct fmt::formatter } template - auto format(const DB::RowNumber & x, FormatContext & ctx) + auto format(const DB::RowNumber & x, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}:{}", x.block, x.row); } diff --git a/src/Storages/Kafka/KafkaConsumer.cpp b/src/Storages/Kafka/KafkaConsumer.cpp index 7075dcb71ca..9ba42b9875e 100644 --- a/src/Storages/Kafka/KafkaConsumer.cpp +++ b/src/Storages/Kafka/KafkaConsumer.cpp @@ -1,7 +1,4 @@ -// Needs to go first because its partial specialization of fmt::formatter -// should be defined before any instantiation -#include - +#include #include #include diff --git a/src/Storages/Kafka/KafkaConsumer.h b/src/Storages/Kafka/KafkaConsumer.h index a3bc97779b3..a2d047933be 100644 --- a/src/Storages/Kafka/KafkaConsumer.h +++ b/src/Storages/Kafka/KafkaConsumer.h @@ -1,5 +1,7 @@ #pragma once +#include +#include #include #include @@ -197,3 +199,6 @@ private: }; } + +template <> struct fmt::formatter : fmt::ostream_formatter {}; +template <> struct fmt::formatter : fmt::ostream_formatter {}; diff --git a/src/Storages/MergeTree/MarkRange.h b/src/Storages/MergeTree/MarkRange.h index 626d4e9e689..6b111f348bb 100644 --- a/src/Storages/MergeTree/MarkRange.h +++ b/src/Storages/MergeTree/MarkRange.h @@ -69,7 +69,7 @@ struct fmt::formatter } template - auto format(const DB::MarkRange & range, FormatContext & ctx) + auto format(const DB::MarkRange & range, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}", fmt::format("({}, {})", range.begin, range.end)); } diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index f3318a48883..79c0e6ad262 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -112,7 +112,7 @@ struct fmt::formatter static constexpr auto parse(format_parse_context & ctx) { return ctx.begin(); } template - auto format(const DB::Part & part, FormatContext & ctx) + auto format(const DB::Part & part, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{} in replicas [{}]", part.description.describe(), fmt::join(part.replicas, ", ")); } diff --git a/src/Storages/MergeTree/RangesInDataPart.cpp b/src/Storages/MergeTree/RangesInDataPart.cpp index c46385e84ef..50e0781b4e6 100644 --- a/src/Storages/MergeTree/RangesInDataPart.cpp +++ b/src/Storages/MergeTree/RangesInDataPart.cpp @@ -13,7 +13,7 @@ struct fmt::formatter static constexpr auto parse(format_parse_context & ctx) { return ctx.begin(); } template - auto format(const DB::RangesInDataPartDescription & range, FormatContext & ctx) + auto format(const DB::RangesInDataPartDescription & range, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}", range.describe()); } diff --git a/src/Storages/NamedCollectionsHelpers.h b/src/Storages/NamedCollectionsHelpers.h index a1909f514ea..b4aea096c59 100644 --- a/src/Storages/NamedCollectionsHelpers.h +++ b/src/Storages/NamedCollectionsHelpers.h @@ -158,7 +158,7 @@ struct fmt::formatter> } template - auto format(const DB::NamedCollectionValidateKey & elem, FormatContext & context) + auto format(const DB::NamedCollectionValidateKey & elem, FormatContext & context) const { return fmt::format_to(context.out(), "{}", elem.value); }