mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Useless changes
This commit is contained in:
parent
1d69518c4d
commit
0d14a2c67e
@ -284,7 +284,6 @@ void ExternalDictionaryLibraryBridgeRequestHandler::handleRequest(HTTPServerRequ
|
||||
else if (method == "extDict_loadIds")
|
||||
{
|
||||
LOG_DEBUG(log, "Getting diciontary ids for dictionary with id: {}", dictionary_id);
|
||||
String ids_string;
|
||||
std::vector<uint64_t> ids = parseIdsFromBinary(request.getStream());
|
||||
|
||||
auto library_handler = ExternalDictionaryLibraryHandlerFactory::instance().get(dictionary_id);
|
||||
|
@ -14,7 +14,7 @@ namespace ErrorCodes
|
||||
|
||||
SharedLibrary::SharedLibrary(std::string_view path, int flags)
|
||||
{
|
||||
handle = dlopen(path.data(), flags);
|
||||
handle = dlopen(path.data(), flags); // NOLINT
|
||||
if (!handle)
|
||||
throw Exception(ErrorCodes::CANNOT_DLOPEN, "Cannot dlopen: ({})", dlerror()); // NOLINT(concurrency-mt-unsafe) // MT-Safe on Linux, see man dlerror
|
||||
|
||||
@ -34,7 +34,7 @@ void * SharedLibrary::getImpl(std::string_view name, bool no_throw)
|
||||
{
|
||||
dlerror(); // NOLINT(concurrency-mt-unsafe) // MT-Safe on Linux, see man dlerror
|
||||
|
||||
auto * res = dlsym(handle, name.data());
|
||||
auto * res = dlsym(handle, name.data()); // NOLINT
|
||||
|
||||
if (char * error = dlerror()) // NOLINT(concurrency-mt-unsafe) // MT-Safe on Linux, see man dlerror
|
||||
{
|
||||
|
@ -674,8 +674,7 @@ private:
|
||||
|
||||
if (pos + length > end)
|
||||
length = end - pos;
|
||||
if (length > sizeof(CodePoint))
|
||||
length = sizeof(CodePoint);
|
||||
length = std::min(length, sizeof(CodePoint));
|
||||
|
||||
CodePoint res = 0;
|
||||
memcpy(&res, pos, length);
|
||||
@ -883,9 +882,7 @@ public:
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error in markov model");
|
||||
|
||||
size_t offset_from_begin_of_string = pos - data;
|
||||
size_t determinator_sliding_window_size = params.determinator_sliding_window_size;
|
||||
if (determinator_sliding_window_size > determinator_size)
|
||||
determinator_sliding_window_size = determinator_size;
|
||||
size_t determinator_sliding_window_size = std::min(params.determinator_sliding_window_size, determinator_size);
|
||||
|
||||
size_t determinator_sliding_window_overflow = offset_from_begin_of_string + determinator_sliding_window_size > determinator_size
|
||||
? offset_from_begin_of_string + determinator_sliding_window_size - determinator_size : 0;
|
||||
|
@ -119,8 +119,7 @@ void ODBCSource::insertValue(
|
||||
time_t time = 0;
|
||||
const DataTypeDateTime & datetime_type = assert_cast<const DataTypeDateTime &>(*data_type);
|
||||
readDateTimeText(time, in, datetime_type.getTimeZone());
|
||||
if (time < 0)
|
||||
time = 0;
|
||||
time = std::max<time_t>(time, 0);
|
||||
column.insert(static_cast<UInt32>(time));
|
||||
break;
|
||||
}
|
||||
|
@ -245,7 +245,7 @@ bool AccessRightsElements::sameOptions() const
|
||||
|
||||
void AccessRightsElements::eraseNonGrantable()
|
||||
{
|
||||
boost::range::remove_erase_if(*this, [](AccessRightsElement & element)
|
||||
boost::range::remove_erase_if(*this, [](AccessRightsElement & element) // NOLINT
|
||||
{
|
||||
element.eraseNonGrantable();
|
||||
return element.empty();
|
||||
|
@ -308,7 +308,7 @@ void AllowedClientHosts::removeAddress(const IPAddress & address)
|
||||
if (address.isLoopback())
|
||||
local_host = false;
|
||||
else
|
||||
boost::range::remove_erase(addresses, address);
|
||||
boost::range::remove_erase(addresses, address); // NOLINT
|
||||
}
|
||||
|
||||
void AllowedClientHosts::addSubnet(const IPSubnet & subnet)
|
||||
@ -328,7 +328,7 @@ void AllowedClientHosts::removeSubnet(const IPSubnet & subnet)
|
||||
else if (subnet.isMaskAllBitsOne())
|
||||
removeAddress(subnet.getPrefix());
|
||||
else
|
||||
boost::range::remove_erase(subnets, subnet);
|
||||
boost::range::remove_erase(subnets, subnet); // NOLINT
|
||||
}
|
||||
|
||||
void AllowedClientHosts::addName(const String & name)
|
||||
@ -344,7 +344,7 @@ void AllowedClientHosts::removeName(const String & name)
|
||||
if (boost::iequals(name, "localhost"))
|
||||
local_host = false;
|
||||
else
|
||||
boost::range::remove_erase(names, name);
|
||||
boost::range::remove_erase(names, name); // NOLINT
|
||||
}
|
||||
|
||||
void AllowedClientHosts::addNameRegexp(const String & name_regexp)
|
||||
@ -364,7 +364,7 @@ void AllowedClientHosts::removeNameRegexp(const String & name_regexp)
|
||||
else if (name_regexp == ".*")
|
||||
any_host = false;
|
||||
else
|
||||
boost::range::remove_erase(name_regexps, name_regexp);
|
||||
boost::range::remove_erase(name_regexps, name_regexp); // NOLINT
|
||||
}
|
||||
|
||||
void AllowedClientHosts::addLikePattern(const String & pattern)
|
||||
@ -384,7 +384,7 @@ void AllowedClientHosts::removeLikePattern(const String & pattern)
|
||||
else if ((pattern == "%") || (pattern == "0.0.0.0/0") || (pattern == "::/0"))
|
||||
any_host = false;
|
||||
else
|
||||
boost::range::remove_erase(like_patterns, pattern);
|
||||
boost::range::remove_erase(like_patterns, pattern); // NOLINT
|
||||
}
|
||||
|
||||
void AllowedClientHosts::addLocalHost()
|
||||
|
@ -234,8 +234,7 @@ public:
|
||||
BetterFloat qr = (sum + l_count + r->count * 0.5) / count;
|
||||
BetterFloat err2 = qr * (1 - qr);
|
||||
|
||||
if (err > err2)
|
||||
err = err2;
|
||||
err = std::min(err, err2);
|
||||
|
||||
BetterFloat k = count_epsilon_4 * err;
|
||||
|
||||
|
@ -132,7 +132,7 @@ public:
|
||||
void free(char * ptr, const size_t size)
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
return ArenaWithFreeLists::free(ptr, size);
|
||||
ArenaWithFreeLists::free(ptr, size);
|
||||
}
|
||||
|
||||
/// Size of the allocated pool in bytes
|
||||
|
@ -253,8 +253,7 @@ namespace cctz_extension
|
||||
|
||||
size_t Read(void * buf, size_t bytes) override
|
||||
{
|
||||
if (bytes > size)
|
||||
bytes = size;
|
||||
bytes = std::min(bytes, size);
|
||||
memcpy(buf, data, bytes);
|
||||
data += bytes;
|
||||
size -= bytes;
|
||||
|
@ -11,6 +11,7 @@
|
||||
#include <base/types.h>
|
||||
#include <absl/container/flat_hash_map.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
/// Finite State Transducer is an efficient way to represent term dictionary.
|
||||
|
@ -291,7 +291,7 @@ public:
|
||||
* the compiler can not guess about this, and generates the `load`, `increment`, `store` code.
|
||||
*/
|
||||
if (inserted)
|
||||
new (&it->getMapped()) typename Cell::Mapped();
|
||||
new (reinterpret_cast<void*>(&it->getMapped())) typename Cell::Mapped();
|
||||
|
||||
return it->getMapped();
|
||||
}
|
||||
|
@ -67,7 +67,7 @@ std::string flushJemallocProfile(const std::string & file_prefix)
|
||||
checkJemallocProfilingEnabled();
|
||||
char * prefix_buffer;
|
||||
size_t prefix_size = sizeof(prefix_buffer);
|
||||
int n = mallctl("opt.prof_prefix", &prefix_buffer, &prefix_size, nullptr, 0);
|
||||
int n = mallctl("opt.prof_prefix", &prefix_buffer, &prefix_size, nullptr, 0); // NOLINT
|
||||
if (!n && std::string_view(prefix_buffer) != "jeprof")
|
||||
{
|
||||
LOG_TRACE(getLogger("SystemJemalloc"), "Flushing memory profile with prefix {}", prefix_buffer);
|
||||
@ -80,7 +80,7 @@ std::string flushJemallocProfile(const std::string & file_prefix)
|
||||
const auto * profile_dump_path_str = profile_dump_path.c_str();
|
||||
|
||||
LOG_TRACE(getLogger("SystemJemalloc"), "Flushing memory profile to {}", profile_dump_path_str);
|
||||
mallctl("prof.dump", nullptr, nullptr, &profile_dump_path_str, sizeof(profile_dump_path_str));
|
||||
mallctl("prof.dump", nullptr, nullptr, &profile_dump_path_str, sizeof(profile_dump_path_str)); // NOLINT
|
||||
return profile_dump_path;
|
||||
}
|
||||
|
||||
|
@ -27,7 +27,7 @@ void LogFrequencyLimiterIml::log(Poco::Message & message)
|
||||
SipHash hash;
|
||||
hash.update(logger->name());
|
||||
/// Format strings are compile-time constants, so they are uniquely identified by pointer and size
|
||||
hash.update(pattern.data());
|
||||
hash.update(reinterpret_cast<uintptr_t>(pattern.data()));
|
||||
hash.update(pattern.size());
|
||||
|
||||
time_t now = time(nullptr);
|
||||
|
@ -112,7 +112,7 @@ struct NetlinkMessage
|
||||
|
||||
if (bytes_sent <= 0)
|
||||
{
|
||||
if (errno == EAGAIN)
|
||||
if (bytes_sent < 0 && errno == EAGAIN)
|
||||
continue;
|
||||
else
|
||||
throw ErrnoException(ErrorCodes::NETLINK_ERROR, "Can't send a Netlink command");
|
||||
|
@ -40,8 +40,7 @@ size_t shortest_literal_length(const Literals & literals)
|
||||
if (literals.empty()) return 0;
|
||||
size_t shortest = std::numeric_limits<size_t>::max();
|
||||
for (const auto & lit : literals)
|
||||
if (shortest > lit.literal.size())
|
||||
shortest = lit.literal.size();
|
||||
shortest = std::min(shortest, lit.literal.size());
|
||||
return shortest;
|
||||
}
|
||||
|
||||
@ -451,7 +450,7 @@ try
|
||||
{
|
||||
Literals alternative_literals;
|
||||
Literal required_literal;
|
||||
analyzeImpl(regexp_, regexp_.data(), required_literal, is_trivial, alternative_literals);
|
||||
analyzeImpl(regexp_, regexp_.data(), required_literal, is_trivial, alternative_literals); // NOLINT
|
||||
required_substring = std::move(required_literal.literal);
|
||||
required_substring_is_prefix = required_literal.prefix;
|
||||
for (auto & lit : alternative_literals)
|
||||
@ -649,8 +648,7 @@ unsigned OptimizedRegularExpression::match(const char * subject, size_t subject_
|
||||
if (limit == 0)
|
||||
return 0;
|
||||
|
||||
if (limit > number_of_subpatterns + 1)
|
||||
limit = number_of_subpatterns + 1;
|
||||
limit = std::min(limit, number_of_subpatterns + 1);
|
||||
|
||||
if (is_trivial)
|
||||
{
|
||||
|
@ -228,8 +228,7 @@ PoolWithFailoverBase<TNestedPool>::getMany(
|
||||
std::vector<ShuffledPool> shuffled_pools = getShuffledPools(max_ignored_errors, get_priority);
|
||||
|
||||
/// Limit `max_tries` value by `max_error_cap` to avoid unlimited number of retries
|
||||
if (max_tries > max_error_cap)
|
||||
max_tries = max_error_cap;
|
||||
max_tries = std::min(max_tries, max_error_cap);
|
||||
|
||||
/// We will try to get a connection from each pool until a connection is produced or max_tries is reached.
|
||||
std::vector<TryResult> try_results(shuffled_pools.size());
|
||||
|
@ -163,8 +163,7 @@ void ProgressIndication::writeProgress(WriteBufferFromFileDescriptor & message)
|
||||
WriteBufferFromOwnString profiling_msg_builder;
|
||||
|
||||
/// We don't want -0. that can appear due to rounding errors.
|
||||
if (cpu_usage <= 0)
|
||||
cpu_usage = 0;
|
||||
cpu_usage = std::max(cpu_usage, 0.);
|
||||
|
||||
profiling_msg_builder << "(" << fmt::format("{:.1f}", cpu_usage) << " CPU";
|
||||
|
||||
|
@ -152,8 +152,7 @@ void Timer::createIfNecessary(UInt64 thread_id, int clock_type, int pause_signal
|
||||
void Timer::set(UInt32 period)
|
||||
{
|
||||
/// Too high frequency can introduce infinite busy loop of signal handlers. We will limit maximum frequency (with 1000 signals per second).
|
||||
if (period < 1000000)
|
||||
period = 1000000;
|
||||
period = std::max<UInt32>(period, 1000000);
|
||||
/// Randomize offset as uniform random value from 0 to period - 1.
|
||||
/// It will allow to sample short queries even if timer period is large.
|
||||
/// (For example, with period of 1 second, query with 50 ms duration will be sampled with 1 / 20 probability).
|
||||
|
@ -206,14 +206,25 @@ public:
|
||||
while (true)
|
||||
{
|
||||
if (!queue.empty())
|
||||
return processQueue(std::move(lock));
|
||||
if (postponed.empty())
|
||||
{
|
||||
processQueue(std::move(lock));
|
||||
return;
|
||||
}
|
||||
else if (postponed.empty())
|
||||
{
|
||||
wait(lock);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (postponed.front().key <= now())
|
||||
return processPostponed(std::move(lock));
|
||||
waitUntil(lock, postponed.front().key);
|
||||
{
|
||||
processPostponed(std::move(lock));
|
||||
return;
|
||||
}
|
||||
else
|
||||
{
|
||||
waitUntil(lock, postponed.front().key);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -21,11 +21,11 @@ namespace
|
||||
{ 1.533, 2.132, 2.776, 3.747, 4.604, 7.173 },
|
||||
{ 1.476, 2.015, 2.571, 3.365, 4.032, 5.893 },
|
||||
{ 1.440, 1.943, 2.447, 3.143, 3.707, 5.208 },
|
||||
{ 1.415, 1.895, 2.365, 2.998, 3.499, 4.782 },
|
||||
{ 1.415, 1.895, 2.365, 2.998, 3.499, 4.782 }, // NOLINT
|
||||
{ 1.397, 1.860, 2.306, 2.896, 3.355, 4.499 },
|
||||
{ 1.383, 1.833, 2.262, 2.821, 3.250, 4.296 },
|
||||
{ 1.372, 1.812, 2.228, 2.764, 3.169, 4.143 },
|
||||
{ 1.363, 1.796, 2.201, 2.718, 3.106, 4.024 },
|
||||
{ 1.363, 1.796, 2.201, 2.718, 3.106, 4.024 }, // NOLINT
|
||||
{ 1.356, 1.782, 2.179, 2.681, 3.055, 3.929 },
|
||||
{ 1.350, 1.771, 2.160, 2.650, 3.012, 3.852 },
|
||||
{ 1.345, 1.761, 2.145, 2.624, 2.977, 3.787 },
|
||||
@ -50,7 +50,7 @@ namespace
|
||||
{ 1.308, 1.692, 2.035, 2.445, 2.733, 3.356 },
|
||||
{ 1.307, 1.691, 2.032, 2.441, 2.728, 3.348 },
|
||||
{ 1.306, 1.690, 2.030, 2.438, 2.724, 3.340 },
|
||||
{ 1.306, 1.688, 2.028, 2.434, 2.719, 3.333 },
|
||||
{ 1.306, 1.688, 2.028, 2.434, 2.719, 3.333 }, // NOLINT
|
||||
{ 1.305, 1.687, 2.026, 2.431, 2.715, 3.326 },
|
||||
{ 1.304, 1.686, 2.024, 2.429, 2.712, 3.319 },
|
||||
{ 1.304, 1.685, 2.023, 2.426, 2.708, 3.313 },
|
||||
@ -137,8 +137,7 @@ void StudentTTest::add(size_t distribution, double value)
|
||||
/// Confidence_level_index can be set in range [0, 5]. Corresponding values can be found above.
|
||||
std::pair<bool, std::string> StudentTTest::compareAndReport(size_t confidence_level_index) const
|
||||
{
|
||||
if (confidence_level_index > 5)
|
||||
confidence_level_index = 5;
|
||||
confidence_level_index = std::min<size_t>(confidence_level_index, 5);
|
||||
|
||||
if (data[0].size == 0 || data[1].size == 0)
|
||||
return {true, ""};
|
||||
|
@ -141,8 +141,7 @@ void collectSymbolsFromProgramHeaders(
|
||||
__msan_unpoison(buckets, hash[0] * sizeof(buckets[0]));
|
||||
|
||||
for (ElfW(Word) i = 0; i < hash[0]; ++i)
|
||||
if (buckets[i] > sym_cnt)
|
||||
sym_cnt = buckets[i];
|
||||
sym_cnt = std::max<size_t>(sym_cnt, buckets[i]);
|
||||
|
||||
if (sym_cnt)
|
||||
{
|
||||
|
@ -86,8 +86,7 @@ void SystemLogQueue<LogElement>::push(LogElement&& element)
|
||||
// It is enough to only wake the flushing thread once, after the message
|
||||
// count increases past half available size.
|
||||
const uint64_t queue_end = queue_front_index + queue.size();
|
||||
if (requested_flush_up_to < queue_end)
|
||||
requested_flush_up_to = queue_end;
|
||||
requested_flush_up_to = std::max(requested_flush_up_to, queue_end);
|
||||
|
||||
flush_event.notify_all();
|
||||
}
|
||||
|
@ -110,7 +110,8 @@ private:
|
||||
static inline UInt64 getClockMonotonic()
|
||||
{
|
||||
struct timespec ts;
|
||||
clock_gettime(CLOCK_MONOTONIC, &ts);
|
||||
if (0 != clock_gettime(CLOCK_MONOTONIC, &ts))
|
||||
throw std::system_error(std::error_code(errno, std::system_category()));
|
||||
return ts.tv_sec * 1000000000ULL + ts.tv_nsec;
|
||||
}
|
||||
};
|
||||
|
@ -244,12 +244,12 @@ public:
|
||||
/// Yield leadership and become follower.
|
||||
void yieldLeadership()
|
||||
{
|
||||
return server->yieldLeadership();
|
||||
server->yieldLeadership();
|
||||
}
|
||||
|
||||
void recalculateStorageStats()
|
||||
{
|
||||
return server->recalculateStorageStats();
|
||||
server->recalculateStorageStats();
|
||||
}
|
||||
|
||||
static void cleanResources();
|
||||
|
@ -22,8 +22,8 @@ class WriteBuffer;
|
||||
|
||||
enum class SettingsWriteFormat : uint8_t
|
||||
{
|
||||
BINARY, /// Part of the settings are serialized as strings, and other part as variants. This is the old behaviour.
|
||||
STRINGS_WITH_FLAGS, /// All settings are serialized as strings. Before each value the flag `is_important` is serialized.
|
||||
BINARY = 0, /// Part of the settings are serialized as strings, and other part as variants. This is the old behaviour.
|
||||
STRINGS_WITH_FLAGS = 1, /// All settings are serialized as strings. Before each value the flag `is_important` is serialized.
|
||||
DEFAULT = STRINGS_WITH_FLAGS,
|
||||
};
|
||||
|
||||
|
@ -34,11 +34,11 @@ void addProgramOptionsAsMultitokens(T &cmd_settings, boost::program_options::opt
|
||||
/// Adds program options to set the settings from a command line.
|
||||
/// (Don't forget to call notify() on the `variables_map` after parsing it!)
|
||||
template <typename T>
|
||||
void addProgramOption(T &cmd_settings, boost::program_options::options_description & options, std::string_view name, const typename T::SettingFieldRef & field)
|
||||
void addProgramOption(T & cmd_settings, boost::program_options::options_description & options, std::string_view name, const typename T::SettingFieldRef & field)
|
||||
{
|
||||
auto on_program_option = boost::function1<void, const std::string &>([&cmd_settings, name](const std::string & value) { cmd_settings.set(name, value); });
|
||||
options.add(boost::shared_ptr<boost::program_options::option_description>(new boost::program_options::option_description(
|
||||
name.data(), boost::program_options::value<std::string>()->composing()->notifier(on_program_option), field.getDescription())));
|
||||
name.data(), boost::program_options::value<std::string>()->composing()->notifier(on_program_option), field.getDescription()))); // NOLINT
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
|
@ -831,7 +831,7 @@ public:
|
||||
[[maybe_unused]] Messaging::MessageTransport & mt,
|
||||
const Poco::Net::SocketAddress & address) override
|
||||
{
|
||||
return setPassword(user_name, "", session, address);
|
||||
setPassword(user_name, "", session, address);
|
||||
}
|
||||
|
||||
AuthenticationType getType() const override
|
||||
@ -855,7 +855,7 @@ public:
|
||||
if (type == Messaging::FrontMessageType::PASSWORD_MESSAGE)
|
||||
{
|
||||
std::unique_ptr<Messaging::PasswordMessage> password = mt.receive<Messaging::PasswordMessage>();
|
||||
return setPassword(user_name, password->password, session, address);
|
||||
setPassword(user_name, password->password, session, address);
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT,
|
||||
|
@ -195,7 +195,7 @@ inline DataTypePtr createDecimal(UInt64 precision_value, UInt64 scale_value)
|
||||
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Wrong precision: it must be between {} and {}, got {}",
|
||||
DecimalUtils::min_precision, DecimalUtils::max_precision<Decimal256>, precision_value);
|
||||
|
||||
if (static_cast<UInt64>(scale_value) > precision_value)
|
||||
if (scale_value > precision_value)
|
||||
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Negative scales and scales larger than precision are not supported");
|
||||
|
||||
if (precision_value <= DecimalUtils::max_precision<Decimal32>)
|
||||
|
@ -12,7 +12,6 @@
|
||||
#include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
#include <Disks/WriteMode.h>
|
||||
#include <Disks/DirectoryIterator.h>
|
||||
#include <Disks/IDiskTransaction.h>
|
||||
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
@ -55,7 +54,6 @@ using DisksMap = std::map<String, DiskPtr>;
|
||||
|
||||
class IReservation;
|
||||
using ReservationPtr = std::unique_ptr<IReservation>;
|
||||
using Reservations = std::vector<ReservationPtr>;
|
||||
|
||||
class ReadBufferFromFileBase;
|
||||
class WriteBufferFromFileBase;
|
||||
|
@ -3,12 +3,17 @@
|
||||
#include <string>
|
||||
#include <vector>
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <Disks/IDisk.h>
|
||||
#include <sys/types.h>
|
||||
#include <Disks/IDisk.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct ReadSettings;
|
||||
struct WriteSettings;
|
||||
class WriteBufferFromFileBase;
|
||||
|
||||
struct RemoveRequest
|
||||
{
|
||||
std::string path; /// Relative path.
|
||||
@ -47,7 +52,7 @@ public:
|
||||
/// Move directory from `from_path` to `to_path`.
|
||||
virtual void moveDirectory(const std::string & from_path, const std::string & to_path) = 0;
|
||||
|
||||
virtual void moveFile(const String & from_path, const String & to_path) = 0;
|
||||
virtual void moveFile(const std::string & from_path, const std::string & to_path) = 0;
|
||||
|
||||
virtual void createFile(const String & path) = 0;
|
||||
|
||||
@ -73,10 +78,10 @@ public:
|
||||
const WriteSettings & settings = {},
|
||||
bool autocommit = true) = 0;
|
||||
|
||||
using WriteBlobFunction = std::function<size_t(const Strings & blob_path, WriteMode mode, const std::optional<ObjectAttributes> & object_attributes)>;
|
||||
using WriteBlobFunction = std::function<size_t(const std::vector<std::string> & blob_path, WriteMode mode, const std::optional<ObjectAttributes> & object_attributes)>;
|
||||
|
||||
/// Write a file using a custom function to write an object to the disk's object storage.
|
||||
virtual void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) = 0;
|
||||
virtual void writeFileUsingBlobWritingFunction(const std::string & path, WriteMode mode, WriteBlobFunction && write_blob_function) = 0;
|
||||
|
||||
/// Remove file. Throws exception if file doesn't exists or it's a directory.
|
||||
virtual void removeFile(const std::string & path) = 0;
|
||||
|
@ -39,7 +39,7 @@ public:
|
||||
|
||||
void setReadUntilPosition(size_t position) override;
|
||||
|
||||
void setReadUntilEnd() override { return setReadUntilPosition(getFileSize()); }
|
||||
void setReadUntilEnd() override { setReadUntilPosition(getFileSize()); }
|
||||
|
||||
size_t getFileSize() override { return getTotalSize(blobs_to_read); }
|
||||
|
||||
|
@ -2,7 +2,6 @@
|
||||
|
||||
#include <mutex>
|
||||
#include <IO/Archives/IArchiveReader.h>
|
||||
#include <IO/Archives/LibArchiveReader.h>
|
||||
#include "config.h"
|
||||
|
||||
|
||||
|
@ -75,7 +75,7 @@ void MMappedFileDescriptor::set(int fd_, size_t offset_)
|
||||
{
|
||||
size_t file_size = getFileSize(fd_);
|
||||
|
||||
if (offset > static_cast<size_t>(file_size))
|
||||
if (offset > file_size)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "MMappedFileDescriptor: requested offset is greater than file size");
|
||||
|
||||
set(fd_, offset_, file_size - offset);
|
||||
@ -101,5 +101,3 @@ MMappedFileDescriptor::~MMappedFileDescriptor()
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
@ -283,9 +283,7 @@ void PeekableReadBuffer::resizeOwnMemoryIfNecessary(size_t bytes_to_append)
|
||||
{
|
||||
size_t pos_offset = pos - memory.data();
|
||||
|
||||
size_t new_size_amortized = memory.size() * 2;
|
||||
if (new_size_amortized < new_size)
|
||||
new_size_amortized = new_size;
|
||||
size_t new_size_amortized = std::max(memory.size() * 2, new_size);
|
||||
memory.resize(new_size_amortized);
|
||||
|
||||
if (need_update_checkpoint)
|
||||
|
@ -202,7 +202,7 @@ static Aws::String getAWSMetadataEndpoint()
|
||||
if (ec2_metadata_service_endpoint.empty())
|
||||
{
|
||||
Aws::String ec2_metadata_service_endpoint_mode = Aws::Environment::GetEnv("AWS_EC2_METADATA_SERVICE_ENDPOINT_MODE");
|
||||
if (ec2_metadata_service_endpoint_mode.length() == 0)
|
||||
if (ec2_metadata_service_endpoint_mode.empty())
|
||||
{
|
||||
ec2_metadata_service_endpoint = "http://169.254.169.254"; //default to IPv4 default endpoint
|
||||
}
|
||||
|
@ -339,7 +339,10 @@ void WriteBufferFromS3::allocateBuffer()
|
||||
chassert(0 == hidden_size);
|
||||
|
||||
if (buffer_allocation_policy->getBufferNumber() == 1)
|
||||
return allocateFirstBuffer();
|
||||
{
|
||||
allocateFirstBuffer();
|
||||
return;
|
||||
}
|
||||
|
||||
memory = Memory(buffer_allocation_policy->getBufferSize());
|
||||
WriteBuffer::set(memory.data(), memory.size());
|
||||
|
@ -12,13 +12,13 @@
|
||||
#include <Storages/StorageS3Settings.h>
|
||||
#include <Common/threadPoolCallbackRunner.h>
|
||||
#include <IO/S3/BlobStorageLogWriter.h>
|
||||
#include <Common/ThreadPoolTaskTracker.h>
|
||||
#include <Common/BufferAllocationPolicy.h>
|
||||
|
||||
#include <memory>
|
||||
#include <vector>
|
||||
#include <list>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
/**
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Common/logger_useful.h>
|
||||
#include "Interpreters/Cache/Guards.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -144,59 +144,59 @@ public:
|
||||
/// Loads a specified object.
|
||||
/// The function does nothing if it's already loaded.
|
||||
/// The function doesn't throw an exception if it's failed to load.
|
||||
template <typename ReturnType = LoadablePtr, typename = std::enable_if_t<is_scalar_load_result_type<ReturnType>, void>>
|
||||
template <typename ReturnType = LoadablePtr, typename = std::enable_if_t<is_scalar_load_result_type<ReturnType>, void>> // NOLINT
|
||||
ReturnType tryLoad(const String & name, Duration timeout = WAIT) const;
|
||||
|
||||
/// Loads objects by filter.
|
||||
/// The function does nothing for already loaded objects, it just returns them.
|
||||
/// The function doesn't throw an exception if it's failed to load something.
|
||||
template <typename ReturnType = Loadables, typename = std::enable_if_t<is_vector_load_result_type<ReturnType>, void>>
|
||||
template <typename ReturnType = Loadables, typename = std::enable_if_t<is_vector_load_result_type<ReturnType>, void>> // NOLINT
|
||||
ReturnType tryLoad(const FilterByNameFunction & filter, Duration timeout = WAIT) const;
|
||||
|
||||
/// Loads all objects.
|
||||
/// The function does nothing for already loaded objects, it just returns them.
|
||||
/// The function doesn't throw an exception if it's failed to load something.
|
||||
template <typename ReturnType = Loadables, typename = std::enable_if_t<is_vector_load_result_type<ReturnType>, void>>
|
||||
template <typename ReturnType = Loadables, typename = std::enable_if_t<is_vector_load_result_type<ReturnType>, void>> // NOLINT
|
||||
ReturnType tryLoadAll(Duration timeout = WAIT) const { return tryLoad<ReturnType>(FilterByNameFunction{}, timeout); }
|
||||
|
||||
/// Loads a specified object.
|
||||
/// The function does nothing if it's already loaded.
|
||||
/// The function throws an exception if it's failed to load.
|
||||
template <typename ReturnType = LoadablePtr, typename = std::enable_if_t<is_scalar_load_result_type<ReturnType>, void>>
|
||||
template <typename ReturnType = LoadablePtr, typename = std::enable_if_t<is_scalar_load_result_type<ReturnType>, void>> // NOLINT
|
||||
ReturnType load(const String & name) const;
|
||||
|
||||
/// Loads objects by filter.
|
||||
/// The function does nothing for already loaded objects, it just returns them.
|
||||
/// The function throws an exception if it's failed to load something.
|
||||
template <typename ReturnType = Loadables, typename = std::enable_if_t<is_vector_load_result_type<ReturnType>, void>>
|
||||
template <typename ReturnType = Loadables, typename = std::enable_if_t<is_vector_load_result_type<ReturnType>, void>> // NOLINT
|
||||
ReturnType load(const FilterByNameFunction & filter) const;
|
||||
|
||||
/// Loads all objects. Not recommended to use.
|
||||
/// The function does nothing for already loaded objects, it just returns them.
|
||||
/// The function throws an exception if it's failed to load something.
|
||||
template <typename ReturnType = Loadables, typename = std::enable_if_t<is_vector_load_result_type<ReturnType>, void>>
|
||||
template <typename ReturnType = Loadables, typename = std::enable_if_t<is_vector_load_result_type<ReturnType>, void>> // NOLINT
|
||||
ReturnType loadAll() const { return load<ReturnType>(FilterByNameFunction{}); }
|
||||
|
||||
/// Loads or reloads a specified object.
|
||||
/// The function reloads the object if it's already loaded.
|
||||
/// The function throws an exception if it's failed to load or reload.
|
||||
template <typename ReturnType = LoadablePtr, typename = std::enable_if_t<is_scalar_load_result_type<ReturnType>, void>>
|
||||
template <typename ReturnType = LoadablePtr, typename = std::enable_if_t<is_scalar_load_result_type<ReturnType>, void>> // NOLINT
|
||||
ReturnType loadOrReload(const String & name) const;
|
||||
|
||||
/// Loads or reloads objects by filter.
|
||||
/// The function reloads the objects which are already loaded.
|
||||
/// The function throws an exception if it's failed to load or reload something.
|
||||
template <typename ReturnType = Loadables, typename = std::enable_if_t<is_vector_load_result_type<ReturnType>, void>>
|
||||
template <typename ReturnType = Loadables, typename = std::enable_if_t<is_vector_load_result_type<ReturnType>, void>> // NOLINT
|
||||
ReturnType loadOrReload(const FilterByNameFunction & filter) const;
|
||||
|
||||
/// Load or reloads all objects. Not recommended to use.
|
||||
/// The function throws an exception if it's failed to load or reload something.
|
||||
template <typename ReturnType = Loadables, typename = std::enable_if_t<is_vector_load_result_type<ReturnType>, void>>
|
||||
template <typename ReturnType = Loadables, typename = std::enable_if_t<is_vector_load_result_type<ReturnType>, void>> // NOLINT
|
||||
ReturnType loadOrReloadAll() const { return loadOrReload<ReturnType>(FilterByNameFunction{}); }
|
||||
|
||||
/// Reloads objects by filter which were tried to load before (successfully or not).
|
||||
/// The function throws an exception if it's failed to load or reload something.
|
||||
template <typename ReturnType = Loadables, typename = std::enable_if_t<is_vector_load_result_type<ReturnType>, void>>
|
||||
template <typename ReturnType = Loadables, typename = std::enable_if_t<is_vector_load_result_type<ReturnType>, void>> // NOLINT
|
||||
ReturnType reloadAllTriedToLoad() const;
|
||||
|
||||
/// Check if object with name exists in configuration
|
||||
|
@ -180,7 +180,7 @@ namespace
|
||||
if (settings)
|
||||
changes = assert_cast<ASTSetQuery *>(settings.get())->changes;
|
||||
|
||||
boost::remove_erase_if(
|
||||
boost::remove_erase_if( // NOLINT
|
||||
changes,
|
||||
[](const SettingChange & change)
|
||||
{
|
||||
|
@ -38,11 +38,7 @@ struct ASTCheckTableQuery : public ASTQueryWithTableAndOutput
|
||||
protected:
|
||||
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
|
||||
{
|
||||
std::string nl_or_nothing = settings.one_line ? "" : "\n";
|
||||
|
||||
std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' ');
|
||||
std::string nl_or_ws = settings.one_line ? " " : "\n";
|
||||
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "CHECK TABLE " << (settings.hilite ? hilite_none : "");
|
||||
|
||||
if (table)
|
||||
@ -83,11 +79,7 @@ struct ASTCheckAllTablesQuery : public ASTQueryWithOutput
|
||||
protected:
|
||||
void formatQueryImpl(const FormatSettings & settings, FormatState & /* state */, FormatStateStacked frame) const override
|
||||
{
|
||||
std::string nl_or_nothing = settings.one_line ? "" : "\n";
|
||||
|
||||
std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' ');
|
||||
std::string nl_or_ws = settings.one_line ? " " : "\n";
|
||||
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "CHECK ALL TABLES" << (settings.hilite ? hilite_none : "");
|
||||
}
|
||||
};
|
||||
|
@ -14,10 +14,7 @@ IParser::Pos & IParser::Pos::operator=(const IParser::Pos & rhs)
|
||||
{
|
||||
depth = rhs.depth;
|
||||
max_depth = rhs.max_depth;
|
||||
|
||||
if (rhs.backtracks > backtracks)
|
||||
backtracks = rhs.backtracks;
|
||||
|
||||
backtracks = std::max(backtracks, rhs.backtracks);
|
||||
max_backtracks = rhs.max_backtracks;
|
||||
|
||||
if (rhs < *this)
|
||||
|
@ -484,7 +484,7 @@ Token Lexer::nextTokenImpl()
|
||||
if (heredoc_name_end_position != std::string::npos)
|
||||
{
|
||||
size_t heredoc_size = heredoc_name_end_position + 1;
|
||||
std::string_view heredoc = {token_stream.data(), heredoc_size};
|
||||
std::string_view heredoc = {token_stream.data(), heredoc_size}; // NOLINT
|
||||
|
||||
size_t heredoc_end_position = token_stream.find(heredoc, heredoc_size);
|
||||
if (heredoc_end_position != std::string::npos)
|
||||
|
@ -304,7 +304,7 @@ namespace
|
||||
changes = assert_cast<ASTSetQuery *>(settings.get())->changes;
|
||||
}
|
||||
|
||||
boost::remove_erase_if(changes, [](const SettingChange & change) { return change.name == "async"; });
|
||||
boost::remove_erase_if(changes, [](const SettingChange & change) { return change.name == "async"; }); // NOLINT
|
||||
changes.emplace_back("async", async);
|
||||
|
||||
auto new_settings = std::make_shared<ASTSetQuery>();
|
||||
|
@ -12,7 +12,7 @@ namespace DB
|
||||
ASTPtr makeASTForLogicalAnd(ASTs && arguments)
|
||||
{
|
||||
bool partial_result = true;
|
||||
boost::range::remove_erase_if(arguments, [&](const ASTPtr & argument) -> bool
|
||||
boost::range::remove_erase_if(arguments, [&](const ASTPtr & argument) -> bool // NOLINT
|
||||
{
|
||||
bool b;
|
||||
if (!tryGetLiteralBool(argument.get(), b))
|
||||
@ -41,7 +41,7 @@ ASTPtr makeASTForLogicalAnd(ASTs && arguments)
|
||||
ASTPtr makeASTForLogicalOr(ASTs && arguments)
|
||||
{
|
||||
bool partial_result = false;
|
||||
boost::range::remove_erase_if(arguments, [&](const ASTPtr & argument) -> bool
|
||||
boost::range::remove_erase_if(arguments, [&](const ASTPtr & argument) -> bool // NOLINT
|
||||
{
|
||||
bool b;
|
||||
if (!tryGetLiteralBool(argument.get(), b))
|
||||
|
Loading…
Reference in New Issue
Block a user