mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge pull request #3993 from yandex/weverything-2
Added more warnings from clang's -Weverything, that are available in clang 8
This commit is contained in:
commit
2884b870b3
@ -36,7 +36,7 @@ option (WEVERYTHING "Enables -Weverything option with some exceptions. This is i
|
||||
if (CMAKE_CXX_COMPILER_ID STREQUAL "Clang")
|
||||
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wpedantic -Wno-vla-extension -Wno-zero-length-array -Wno-gnu-anonymous-struct -Wno-nested-anon-types")
|
||||
|
||||
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wextra-semi -Wcomma -Winconsistent-missing-destructor-override -Wunused-exception-parameter -Wshadow-uncaptured-local -Wcovered-switch-default -Wshadow -Wold-style-cast -Wrange-loop-analysis -Wunused-member-function -Wunreachable-code -Wunreachable-code-return -Wnewline-eof -Wembedded-directive -Wgnu-case-range -Wunused-macros -Wconditional-uninitialized -Wdeprecated -Wundef -Wreserved-id-macro -Wredundant-parens -Wzero-as-null-pointer-constant")
|
||||
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wshadow -Wshadow-uncaptured-local -Wextra-semi -Wcomma -Winconsistent-missing-destructor-override -Wunused-exception-parameter -Wcovered-switch-default -Wold-style-cast -Wrange-loop-analysis -Wunused-member-function -Wunreachable-code -Wunreachable-code-return -Wnewline-eof -Wembedded-directive -Wgnu-case-range -Wunused-macros -Wconditional-uninitialized -Wdeprecated -Wundef -Wreserved-id-macro -Wredundant-parens -Wzero-as-null-pointer-constant")
|
||||
|
||||
if (WEVERYTHING)
|
||||
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Weverything -Wno-c++98-compat -Wno-c++98-compat-pedantic -Wno-missing-noreturn -Wno-padded -Wno-switch-enum -Wno-shadow-field-in-constructor -Wno-deprecated-dynamic-exception-spec -Wno-float-equal -Wno-weak-vtables -Wno-shift-sign-overflow -Wno-sign-conversion -Wno-conversion -Wno-exit-time-destructors -Wno-undefined-func-template -Wno-documentation-unknown-command -Wno-missing-variable-declarations -Wno-unused-template -Wno-global-constructors -Wno-c99-extensions -Wno-missing-prototypes -Wno-weak-template-vtables -Wno-zero-length-array -Wno-gnu-anonymous-struct -Wno-nested-anon-types -Wno-double-promotion -Wno-disabled-macro-expansion -Wno-used-but-marked-unused -Wno-vla-extension -Wno-vla -Wno-packed")
|
||||
@ -53,10 +53,10 @@ if (CMAKE_CXX_COMPILER_ID STREQUAL "Clang")
|
||||
endif ()
|
||||
|
||||
if (NOT CMAKE_CXX_COMPILER_VERSION VERSION_LESS 8)
|
||||
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wextra-semi-stmt")
|
||||
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wextra-semi-stmt -Wshadow-field -Wstring-plus-int")
|
||||
|
||||
if (WEVERYTHING)
|
||||
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-shadow-field") # TODO Enable
|
||||
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS}")
|
||||
endif ()
|
||||
endif ()
|
||||
endif ()
|
||||
|
@ -187,8 +187,8 @@ template <bool result_is_nullable>
|
||||
class AggregateFunctionNullUnary final : public AggregateFunctionNullBase<result_is_nullable, AggregateFunctionNullUnary<result_is_nullable>>
|
||||
{
|
||||
public:
|
||||
AggregateFunctionNullUnary(AggregateFunctionPtr nested_function)
|
||||
: AggregateFunctionNullBase<result_is_nullable, AggregateFunctionNullUnary<result_is_nullable>>(nested_function)
|
||||
AggregateFunctionNullUnary(AggregateFunctionPtr nested_function_)
|
||||
: AggregateFunctionNullBase<result_is_nullable, AggregateFunctionNullUnary<result_is_nullable>>(std::move(nested_function_))
|
||||
{
|
||||
}
|
||||
|
||||
@ -209,8 +209,8 @@ template <bool result_is_nullable>
|
||||
class AggregateFunctionNullVariadic final : public AggregateFunctionNullBase<result_is_nullable, AggregateFunctionNullVariadic<result_is_nullable>>
|
||||
{
|
||||
public:
|
||||
AggregateFunctionNullVariadic(AggregateFunctionPtr nested_function, const DataTypes & arguments)
|
||||
: AggregateFunctionNullBase<result_is_nullable, AggregateFunctionNullVariadic<result_is_nullable>>(nested_function),
|
||||
AggregateFunctionNullVariadic(AggregateFunctionPtr nested_function_, const DataTypes & arguments)
|
||||
: AggregateFunctionNullBase<result_is_nullable, AggregateFunctionNullVariadic<result_is_nullable>>(std::move(nested_function_)),
|
||||
number_of_arguments(arguments.size())
|
||||
{
|
||||
if (number_of_arguments == 1)
|
||||
|
@ -20,7 +20,7 @@ private:
|
||||
|
||||
public:
|
||||
const char * getFamilyName() const override { return "Nothing"; }
|
||||
MutableColumnPtr cloneDummy(size_t s) const override { return ColumnNothing::create(s); }
|
||||
MutableColumnPtr cloneDummy(size_t s_) const override { return ColumnNothing::create(s_); }
|
||||
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
};
|
||||
|
@ -150,8 +150,8 @@ std::unique_ptr<ShellCommand> ShellCommand::execute(const std::string & command,
|
||||
{
|
||||
/// Arguments in non-constant chunks of memory (as required for `execv`).
|
||||
/// Moreover, their copying must be done before calling `vfork`, so after `vfork` do a minimum of things.
|
||||
std::vector<char> argv0("sh", "sh" + strlen("sh") + 1);
|
||||
std::vector<char> argv1("-c", "-c" + strlen("-c") + 1);
|
||||
std::vector<char> argv0("sh", &("sh"[3]));
|
||||
std::vector<char> argv1("-c", &("-c"[3]));
|
||||
std::vector<char> argv2(command.data(), command.data() + command.size() + 1);
|
||||
|
||||
char * const argv[] = { argv0.data(), argv1.data(), argv2.data(), nullptr };
|
||||
|
@ -194,9 +194,9 @@ template <bool CaseSensitive, bool ASCII> struct VolnitskyImpl;
|
||||
/// Case sensitive comparison
|
||||
template <bool ASCII> struct VolnitskyImpl<true, ASCII> : VolnitskyBase<VolnitskyImpl<true, ASCII>>
|
||||
{
|
||||
VolnitskyImpl(const char * const needle, const size_t needle_size, const size_t haystack_size_hint = 0)
|
||||
: VolnitskyBase<VolnitskyImpl<true, ASCII>>{needle, needle_size, haystack_size_hint},
|
||||
fallback_searcher{needle, needle_size}
|
||||
VolnitskyImpl(const char * const needle_, const size_t needle_size_, const size_t haystack_size_hint = 0)
|
||||
: VolnitskyBase<VolnitskyImpl<true, ASCII>>{needle_, needle_size_, haystack_size_hint},
|
||||
fallback_searcher{needle_, needle_size_}
|
||||
{
|
||||
}
|
||||
|
||||
@ -222,8 +222,8 @@ template <bool ASCII> struct VolnitskyImpl<true, ASCII> : VolnitskyBase<Volnitsk
|
||||
/// Case-insensitive ASCII
|
||||
template <> struct VolnitskyImpl<false, true> : VolnitskyBase<VolnitskyImpl<false, true>>
|
||||
{
|
||||
VolnitskyImpl(const char * const needle, const size_t needle_size, const size_t haystack_size_hint = 0)
|
||||
: VolnitskyBase{needle, needle_size, haystack_size_hint}, fallback_searcher{needle, needle_size}
|
||||
VolnitskyImpl(const char * const needle_, const size_t needle_size_, const size_t haystack_size_hint = 0)
|
||||
: VolnitskyBase{needle_, needle_size_, haystack_size_hint}, fallback_searcher{needle_, needle_size_}
|
||||
{
|
||||
}
|
||||
|
||||
@ -248,8 +248,8 @@ template <> struct VolnitskyImpl<false, true> : VolnitskyBase<VolnitskyImpl<fals
|
||||
/// Case-sensitive UTF-8
|
||||
template <> struct VolnitskyImpl<false, false> : VolnitskyBase<VolnitskyImpl<false, false>>
|
||||
{
|
||||
VolnitskyImpl(const char * const needle, const size_t needle_size, const size_t haystack_size_hint = 0)
|
||||
: VolnitskyBase{needle, needle_size, haystack_size_hint}, fallback_searcher{needle, needle_size}
|
||||
VolnitskyImpl(const char * const needle_, const size_t needle_size_, const size_t haystack_size_hint = 0)
|
||||
: VolnitskyBase{needle_, needle_size_, haystack_size_hint}, fallback_searcher{needle_, needle_size_}
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -839,7 +839,7 @@ int32_t ZooKeeper::tryMultiNoThrow(const Coordination::Requests & requests, Coor
|
||||
}
|
||||
|
||||
|
||||
size_t KeeperMultiException::getFailedOpIndex(int32_t code, const Coordination::Responses & responses)
|
||||
size_t KeeperMultiException::getFailedOpIndex(int32_t exception_code, const Coordination::Responses & responses)
|
||||
{
|
||||
if (responses.empty())
|
||||
throw DB::Exception("Responses for multi transaction is empty", DB::ErrorCodes::LOGICAL_ERROR);
|
||||
@ -848,17 +848,17 @@ size_t KeeperMultiException::getFailedOpIndex(int32_t code, const Coordination::
|
||||
if (responses[index]->error)
|
||||
return index;
|
||||
|
||||
if (!Coordination::isUserError(code))
|
||||
throw DB::Exception("There are no failed OPs because '" + ZooKeeper::error2string(code) + "' is not valid response code for that",
|
||||
if (!Coordination::isUserError(exception_code))
|
||||
throw DB::Exception("There are no failed OPs because '" + ZooKeeper::error2string(exception_code) + "' is not valid response code for that",
|
||||
DB::ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
throw DB::Exception("There is no failed OpResult", DB::ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
|
||||
KeeperMultiException::KeeperMultiException(int32_t code, const Coordination::Requests & requests, const Coordination::Responses & responses)
|
||||
: KeeperException("Transaction failed", code),
|
||||
requests(requests), responses(responses), failed_op_index(getFailedOpIndex(code, responses))
|
||||
KeeperMultiException::KeeperMultiException(int32_t exception_code, const Coordination::Requests & requests, const Coordination::Responses & responses)
|
||||
: KeeperException("Transaction failed", exception_code),
|
||||
requests(requests), responses(responses), failed_op_index(getFailedOpIndex(exception_code, responses))
|
||||
{
|
||||
addMessage("Op #" + std::to_string(failed_op_index) + ", path: " + getPathForFirstFailedOp());
|
||||
}
|
||||
@ -869,15 +869,15 @@ std::string KeeperMultiException::getPathForFirstFailedOp() const
|
||||
return requests[failed_op_index]->getPath();
|
||||
}
|
||||
|
||||
void KeeperMultiException::check(int32_t code, const Coordination::Requests & requests, const Coordination::Responses & responses)
|
||||
void KeeperMultiException::check(int32_t exception_code, const Coordination::Requests & requests, const Coordination::Responses & responses)
|
||||
{
|
||||
if (!code)
|
||||
if (!exception_code)
|
||||
return;
|
||||
|
||||
if (Coordination::isUserError(code))
|
||||
throw KeeperMultiException(code, requests, responses);
|
||||
if (Coordination::isUserError(exception_code))
|
||||
throw KeeperMultiException(exception_code, requests, responses);
|
||||
else
|
||||
throw KeeperException(code);
|
||||
throw KeeperException(exception_code);
|
||||
}
|
||||
|
||||
|
||||
|
@ -263,7 +263,8 @@ inline bool_if_not_safe_conversion<A, B> equalsOp(A a, B b)
|
||||
template <typename A, typename B>
|
||||
inline bool_if_safe_conversion<A, B> equalsOp(A a, B b)
|
||||
{
|
||||
return a == b;
|
||||
using LargestType = std::conditional_t<sizeof(A) >= sizeof(B), A, B>;
|
||||
return static_cast<LargestType>(a) == static_cast<LargestType>(b);
|
||||
}
|
||||
|
||||
template <>
|
||||
|
@ -507,13 +507,13 @@ void DatabaseOrdinary::shutdown()
|
||||
|
||||
void DatabaseOrdinary::alterTable(
|
||||
const Context & context,
|
||||
const String & name,
|
||||
const String & table_name,
|
||||
const ColumnsDescription & columns,
|
||||
const ASTModifier & storage_modifier)
|
||||
{
|
||||
/// Read the definition of the table and replace the necessary parts with new ones.
|
||||
|
||||
String table_name_escaped = escapeForFileName(name);
|
||||
String table_name_escaped = escapeForFileName(table_name);
|
||||
String table_metadata_tmp_path = metadata_path + "/" + table_name_escaped + ".sql.tmp";
|
||||
String table_metadata_path = metadata_path + "/" + table_name_escaped + ".sql";
|
||||
String statement;
|
||||
|
@ -23,8 +23,8 @@ namespace
|
||||
class ShellCommandOwningBlockInputStream : public OwningBlockInputStream<ShellCommand>
|
||||
{
|
||||
public:
|
||||
ShellCommandOwningBlockInputStream(const BlockInputStreamPtr & stream, std::unique_ptr<ShellCommand> own)
|
||||
: OwningBlockInputStream(std::move(stream), std::move(own))
|
||||
ShellCommandOwningBlockInputStream(const BlockInputStreamPtr & impl, std::unique_ptr<ShellCommand> own_)
|
||||
: OwningBlockInputStream(std::move(impl), std::move(own_))
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -88,7 +88,7 @@ Block BlockInputStreamFromRowInputStream::readImpl()
|
||||
throw;
|
||||
|
||||
++num_errors;
|
||||
Float64 current_error_ratio = static_cast<Float64>(num_errors) / total_rows;
|
||||
Float32 current_error_ratio = static_cast<Float32>(num_errors) / total_rows;
|
||||
|
||||
if (num_errors > allow_errors_num
|
||||
&& current_error_ratio > allow_errors_ratio)
|
||||
|
@ -45,7 +45,7 @@ private:
|
||||
BlockMissingValues block_missing_values;
|
||||
|
||||
UInt64 allow_errors_num;
|
||||
Float64 allow_errors_ratio;
|
||||
Float32 allow_errors_ratio;
|
||||
|
||||
size_t total_rows = 0;
|
||||
size_t num_errors = 0;
|
||||
|
@ -60,7 +60,7 @@ struct FormatSettings
|
||||
DateTimeInputFormat date_time_input_format = DateTimeInputFormat::Basic;
|
||||
|
||||
UInt64 input_allow_errors_num = 0;
|
||||
Float64 input_allow_errors_ratio = 0;
|
||||
Float32 input_allow_errors_ratio = 0;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -8,8 +8,8 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
JSONCompactRowOutputStream::JSONCompactRowOutputStream(WriteBuffer & ostr_, const Block & sample_, const FormatSettings & settings)
|
||||
: JSONRowOutputStream(ostr_, sample_, settings)
|
||||
JSONCompactRowOutputStream::JSONCompactRowOutputStream(WriteBuffer & ostr_, const Block & sample_, const FormatSettings & settings_)
|
||||
: JSONRowOutputStream(ostr_, sample_, settings_)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -11,8 +11,8 @@ namespace DB
|
||||
class PrettyCompactBlockOutputStream : public PrettyBlockOutputStream
|
||||
{
|
||||
public:
|
||||
PrettyCompactBlockOutputStream(WriteBuffer & ostr_, const Block & header_, const FormatSettings & format_settings)
|
||||
: PrettyBlockOutputStream(ostr_, header_, format_settings) {}
|
||||
PrettyCompactBlockOutputStream(WriteBuffer & ostr_, const Block & header_, const FormatSettings & format_settings_)
|
||||
: PrettyBlockOutputStream(ostr_, header_, format_settings_) {}
|
||||
|
||||
void write(const Block & block) override;
|
||||
|
||||
|
@ -11,8 +11,8 @@ namespace DB
|
||||
class PrettySpaceBlockOutputStream : public PrettyBlockOutputStream
|
||||
{
|
||||
public:
|
||||
PrettySpaceBlockOutputStream(WriteBuffer & ostr_, const Block & header_, const FormatSettings & format_settings)
|
||||
: PrettyBlockOutputStream(ostr_, header_, format_settings) {}
|
||||
PrettySpaceBlockOutputStream(WriteBuffer & ostr_, const Block & header_, const FormatSettings & format_settings_)
|
||||
: PrettyBlockOutputStream(ostr_, header_, format_settings_) {}
|
||||
|
||||
void write(const Block & block) override;
|
||||
void writeSuffix() override;
|
||||
|
@ -9,8 +9,8 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
TSKVRowOutputStream::TSKVRowOutputStream(WriteBuffer & ostr_, const Block & sample_, const FormatSettings & format_settings)
|
||||
: TabSeparatedRowOutputStream(ostr_, sample_, false, false, format_settings)
|
||||
TSKVRowOutputStream::TSKVRowOutputStream(WriteBuffer & ostr_, const Block & sample_, const FormatSettings & format_settings_)
|
||||
: TabSeparatedRowOutputStream(ostr_, sample_, false, false, format_settings_)
|
||||
{
|
||||
NamesAndTypesList columns(sample_.getNamesAndTypesList());
|
||||
fields.assign(columns.begin(), columns.end());
|
||||
|
@ -13,8 +13,8 @@ namespace DB
|
||||
class TabSeparatedRawRowOutputStream : public TabSeparatedRowOutputStream
|
||||
{
|
||||
public:
|
||||
TabSeparatedRawRowOutputStream(WriteBuffer & ostr_, const Block & sample_, bool with_names_, bool with_types_, const FormatSettings & format_settings)
|
||||
: TabSeparatedRowOutputStream(ostr_, sample_, with_names_, with_types_, format_settings) {}
|
||||
TabSeparatedRawRowOutputStream(WriteBuffer & ostr_, const Block & sample_, bool with_names_, bool with_types_, const FormatSettings & format_settings_)
|
||||
: TabSeparatedRowOutputStream(ostr_, sample_, with_names_, with_types_, format_settings_) {}
|
||||
|
||||
void writeField(const IColumn & column, const IDataType & type, size_t row_num) override
|
||||
{
|
||||
|
@ -183,7 +183,7 @@ private:
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
formatReadableSizeWithBinarySuffix(vec_from[i], buf_to);
|
||||
formatReadableSizeWithBinarySuffix(static_cast<double>(vec_from[i]), buf_to);
|
||||
writeChar(0, buf_to);
|
||||
offsets_to[i] = buf_to.count();
|
||||
}
|
||||
|
@ -4,9 +4,11 @@
|
||||
|
||||
#include <Functions/GatherUtils/Sources.h>
|
||||
#include <Functions/GatherUtils/Sinks.h>
|
||||
#include <Core/AccurateComparison.h>
|
||||
|
||||
#include <ext/range.h>
|
||||
|
||||
|
||||
namespace DB::ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
@ -31,7 +33,7 @@ void writeSlice(const NumericArraySlice<T> & slice, NumericArraySink<U> & sink)
|
||||
sink.elements.resize(sink.current_offset + slice.size);
|
||||
for (size_t i = 0; i < slice.size; ++i)
|
||||
{
|
||||
sink.elements[sink.current_offset] = slice.data[i];
|
||||
sink.elements[sink.current_offset] = static_cast<U>(slice.data[i]);
|
||||
++sink.current_offset;
|
||||
}
|
||||
}
|
||||
@ -421,17 +423,12 @@ bool sliceHasImpl(const FirstSliceType & first, const SecondSliceType & second,
|
||||
return all;
|
||||
}
|
||||
|
||||
#pragma GCC diagnostic push
|
||||
#pragma GCC diagnostic ignored "-Wsign-compare"
|
||||
|
||||
template <typename T, typename U>
|
||||
bool sliceEqualElements(const NumericArraySlice<T> & first, const NumericArraySlice<U> & second, size_t first_ind, size_t second_ind)
|
||||
{
|
||||
return first.data[first_ind] == second.data[second_ind];
|
||||
return accurate::equalsOp(first.data[first_ind], second.data[second_ind]);
|
||||
}
|
||||
|
||||
#pragma GCC diagnostic pop
|
||||
|
||||
template <typename T>
|
||||
bool sliceEqualElements(const NumericArraySlice<T> &, const GenericArraySlice &, size_t, size_t)
|
||||
{
|
||||
|
@ -118,18 +118,18 @@ struct ConstSource : public Base
|
||||
size_t total_rows;
|
||||
size_t row_num = 0;
|
||||
|
||||
explicit ConstSource(const ColumnConst & col)
|
||||
: Base(static_cast<const typename Base::Column &>(col.getDataColumn())), total_rows(col.size())
|
||||
explicit ConstSource(const ColumnConst & col_)
|
||||
: Base(static_cast<const typename Base::Column &>(col_.getDataColumn())), total_rows(col_.size())
|
||||
{
|
||||
}
|
||||
|
||||
template <typename ColumnType>
|
||||
ConstSource(const ColumnType & col, size_t total_rows) : Base(col), total_rows(total_rows)
|
||||
ConstSource(const ColumnType & col_, size_t total_rows_) : Base(col_), total_rows(total_rows_)
|
||||
{
|
||||
}
|
||||
|
||||
template <typename ColumnType>
|
||||
ConstSource(const ColumnType & col, const NullMap & null_map, size_t total_rows) : Base(col, null_map), total_rows(total_rows)
|
||||
ConstSource(const ColumnType & col_, const NullMap & null_map_, size_t total_rows_) : Base(col_, null_map_), total_rows(total_rows_)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -12,8 +12,8 @@ namespace DB
|
||||
class HashingReadBuffer : public IHashingBuffer<ReadBuffer>
|
||||
{
|
||||
public:
|
||||
HashingReadBuffer(ReadBuffer & in_, size_t block_size = DBMS_DEFAULT_HASHING_BLOCK_SIZE) :
|
||||
IHashingBuffer<ReadBuffer>(block_size), in(in_)
|
||||
HashingReadBuffer(ReadBuffer & in_, size_t block_size_ = DBMS_DEFAULT_HASHING_BLOCK_SIZE) :
|
||||
IHashingBuffer<ReadBuffer>(block_size_), in(in_)
|
||||
{
|
||||
working_buffer = in.buffer();
|
||||
pos = in.position();
|
||||
|
@ -53,14 +53,14 @@ ReadBufferFromFile::ReadBufferFromFile(
|
||||
|
||||
|
||||
ReadBufferFromFile::ReadBufferFromFile(
|
||||
int fd,
|
||||
int fd_,
|
||||
const std::string & original_file_name,
|
||||
size_t buf_size,
|
||||
char * existing_memory,
|
||||
size_t alignment)
|
||||
:
|
||||
ReadBufferFromFileDescriptor(fd, buf_size, existing_memory, alignment),
|
||||
file_name(original_file_name.empty() ? "(fd = " + toString(fd) + ")" : original_file_name)
|
||||
ReadBufferFromFileDescriptor(fd_, buf_size, existing_memory, alignment),
|
||||
file_name(original_file_name.empty() ? "(fd = " + toString(fd_) + ")" : original_file_name)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -96,13 +96,13 @@ class ReadWriteBufferFromHTTP : public detail::ReadWriteBufferFromHTTPBase<HTTPS
|
||||
using Parent = detail::ReadWriteBufferFromHTTPBase<HTTPSessionPtr>;
|
||||
|
||||
public:
|
||||
explicit ReadWriteBufferFromHTTP(Poco::URI uri,
|
||||
const std::string & method = {},
|
||||
explicit ReadWriteBufferFromHTTP(Poco::URI uri_,
|
||||
const std::string & method_ = {},
|
||||
OutStreamCallback out_stream_callback = {},
|
||||
const ConnectionTimeouts & timeouts = {},
|
||||
const Poco::Net::HTTPBasicCredentials & credentials = {},
|
||||
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE)
|
||||
: Parent(makeHTTPSession(uri, timeouts), uri, method, out_stream_callback, credentials, buffer_size_)
|
||||
: Parent(makeHTTPSession(uri_, timeouts), uri_, method_, out_stream_callback, credentials, buffer_size_)
|
||||
{
|
||||
}
|
||||
};
|
||||
@ -111,16 +111,16 @@ class PooledReadWriteBufferFromHTTP : public detail::ReadWriteBufferFromHTTPBase
|
||||
using Parent = detail::ReadWriteBufferFromHTTPBase<PooledHTTPSessionPtr>;
|
||||
|
||||
public:
|
||||
explicit PooledReadWriteBufferFromHTTP(Poco::URI uri,
|
||||
const std::string & method = {},
|
||||
explicit PooledReadWriteBufferFromHTTP(Poco::URI uri_,
|
||||
const std::string & method_ = {},
|
||||
OutStreamCallback out_stream_callback = {},
|
||||
const ConnectionTimeouts & timeouts = {},
|
||||
const Poco::Net::HTTPBasicCredentials & credentials = {},
|
||||
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
size_t max_connections_per_endpoint = DEFAULT_COUNT_OF_HTTP_CONNECTIONS_PER_ENDPOINT)
|
||||
: Parent(makePooledHTTPSession(uri, timeouts, max_connections_per_endpoint),
|
||||
uri,
|
||||
method,
|
||||
: Parent(makePooledHTTPSession(uri_, timeouts, max_connections_per_endpoint),
|
||||
uri_,
|
||||
method_,
|
||||
out_stream_callback,
|
||||
credentials,
|
||||
buffer_size_)
|
||||
|
@ -58,14 +58,14 @@ WriteBufferFromFile::WriteBufferFromFile(
|
||||
|
||||
/// Use pre-opened file descriptor.
|
||||
WriteBufferFromFile::WriteBufferFromFile(
|
||||
int fd,
|
||||
int fd_,
|
||||
const std::string & original_file_name,
|
||||
size_t buf_size,
|
||||
char * existing_memory,
|
||||
size_t alignment)
|
||||
:
|
||||
WriteBufferFromFileDescriptor(fd, buf_size, existing_memory, alignment),
|
||||
file_name(original_file_name.empty() ? "(fd = " + toString(fd) + ")" : original_file_name)
|
||||
WriteBufferFromFileDescriptor(fd_, buf_size, existing_memory, alignment),
|
||||
file_name(original_file_name.empty() ? "(fd = " + toString(fd_) + ")" : original_file_name)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -16,9 +16,7 @@ namespace ErrorCodes
|
||||
|
||||
|
||||
WriteBufferFromTemporaryFile::WriteBufferFromTemporaryFile(std::unique_ptr<Poco::TemporaryFile> && tmp_file_)
|
||||
:
|
||||
WriteBufferFromFile(tmp_file_->path(), DBMS_DEFAULT_BUFFER_SIZE, O_RDWR | O_TRUNC | O_CREAT, 0600),
|
||||
tmp_file(std::move(tmp_file_))
|
||||
: WriteBufferFromFile(tmp_file_->path(), DBMS_DEFAULT_BUFFER_SIZE, O_RDWR | O_TRUNC | O_CREAT, 0600), tmp_file(std::move(tmp_file_))
|
||||
{}
|
||||
|
||||
|
||||
@ -34,7 +32,6 @@ WriteBufferFromTemporaryFile::Ptr WriteBufferFromTemporaryFile::create(const std
|
||||
class ReadBufferFromTemporaryWriteBuffer : public ReadBufferFromFile
|
||||
{
|
||||
public:
|
||||
|
||||
static ReadBufferPtr createFrom(WriteBufferFromTemporaryFile * origin)
|
||||
{
|
||||
int fd = origin->getFD();
|
||||
@ -47,8 +44,8 @@ public:
|
||||
return std::make_shared<ReadBufferFromTemporaryWriteBuffer>(fd, file_name, std::move(origin->tmp_file));
|
||||
}
|
||||
|
||||
ReadBufferFromTemporaryWriteBuffer(int fd, const std::string & file_name, std::unique_ptr<Poco::TemporaryFile> && tmp_file_)
|
||||
: ReadBufferFromFile(fd, file_name), tmp_file(std::move(tmp_file_))
|
||||
ReadBufferFromTemporaryWriteBuffer(int fd_, const std::string & file_name_, std::unique_ptr<Poco::TemporaryFile> && tmp_file_)
|
||||
: ReadBufferFromFile(fd_, file_name_), tmp_file(std::move(tmp_file_))
|
||||
{}
|
||||
|
||||
std::unique_ptr<Poco::TemporaryFile> tmp_file;
|
||||
|
@ -93,12 +93,12 @@ void PartLogElement::appendToBlock(Block & block) const
|
||||
}
|
||||
|
||||
|
||||
bool PartLog::addNewPart(Context & context, const MutableDataPartPtr & part, UInt64 elapsed_ns, const ExecutionStatus & execution_status)
|
||||
bool PartLog::addNewPart(Context & current_context, const MutableDataPartPtr & part, UInt64 elapsed_ns, const ExecutionStatus & execution_status)
|
||||
{
|
||||
return addNewParts(context, {part}, elapsed_ns, execution_status);
|
||||
return addNewParts(current_context, {part}, elapsed_ns, execution_status);
|
||||
}
|
||||
|
||||
bool PartLog::addNewParts(Context & context, const PartLog::MutableDataPartsVector & parts, UInt64 elapsed_ns,
|
||||
bool PartLog::addNewParts(Context & current_context, const PartLog::MutableDataPartsVector & parts, UInt64 elapsed_ns,
|
||||
const ExecutionStatus & execution_status)
|
||||
{
|
||||
if (parts.empty())
|
||||
@ -108,7 +108,7 @@ bool PartLog::addNewParts(Context & context, const PartLog::MutableDataPartsVect
|
||||
|
||||
try
|
||||
{
|
||||
part_log = context.getPartLog(parts.front()->storage.getDatabaseName()); // assume parts belong to the same table
|
||||
part_log = current_context.getPartLog(parts.front()->storage.getDatabaseName()); // assume parts belong to the same table
|
||||
if (!part_log)
|
||||
return false;
|
||||
|
||||
|
@ -22,18 +22,18 @@ MergeTreeSelectBlockInputStream::MergeTreeSelectBlockInputStream(
|
||||
Names column_names,
|
||||
const MarkRanges & mark_ranges_,
|
||||
bool use_uncompressed_cache_,
|
||||
const PrewhereInfoPtr & prewhere_info,
|
||||
const PrewhereInfoPtr & prewhere_info_,
|
||||
bool check_columns,
|
||||
size_t min_bytes_to_use_direct_io_,
|
||||
size_t max_read_buffer_size_,
|
||||
bool save_marks_in_cache_,
|
||||
const Names & virt_column_names,
|
||||
const Names & virt_column_names_,
|
||||
size_t part_index_in_query_,
|
||||
bool quiet)
|
||||
:
|
||||
MergeTreeBaseSelectBlockInputStream{storage_, prewhere_info, max_block_size_rows_,
|
||||
MergeTreeBaseSelectBlockInputStream{storage_, prewhere_info_, max_block_size_rows_,
|
||||
preferred_block_size_bytes_, preferred_max_column_in_block_size_bytes_, min_bytes_to_use_direct_io_,
|
||||
max_read_buffer_size_, use_uncompressed_cache_, save_marks_in_cache_, virt_column_names},
|
||||
max_read_buffer_size_, use_uncompressed_cache_, save_marks_in_cache_, virt_column_names_},
|
||||
required_columns{column_names},
|
||||
data_part{owned_data_part_},
|
||||
part_columns_lock(data_part->columns_lock),
|
||||
|
@ -11,18 +11,18 @@ MergeTreeThreadSelectBlockInputStream::MergeTreeThreadSelectBlockInputStream(
|
||||
const size_t thread,
|
||||
const MergeTreeReadPoolPtr & pool,
|
||||
const size_t min_marks_to_read_,
|
||||
const size_t max_block_size_rows,
|
||||
size_t preferred_block_size_bytes,
|
||||
size_t preferred_max_column_in_block_size_bytes,
|
||||
const MergeTreeData & storage,
|
||||
const bool use_uncompressed_cache,
|
||||
const PrewhereInfoPtr & prewhere_info,
|
||||
const size_t max_block_size_rows_,
|
||||
size_t preferred_block_size_bytes_,
|
||||
size_t preferred_max_column_in_block_size_bytes_,
|
||||
const MergeTreeData & storage_,
|
||||
const bool use_uncompressed_cache_,
|
||||
const PrewhereInfoPtr & prewhere_info_,
|
||||
const Settings & settings,
|
||||
const Names & virt_column_names)
|
||||
const Names & virt_column_names_)
|
||||
:
|
||||
MergeTreeBaseSelectBlockInputStream{storage, prewhere_info, max_block_size_rows,
|
||||
preferred_block_size_bytes, preferred_max_column_in_block_size_bytes, settings.min_bytes_to_use_direct_io,
|
||||
settings.max_read_buffer_size, use_uncompressed_cache, true, virt_column_names},
|
||||
MergeTreeBaseSelectBlockInputStream{storage_, prewhere_info_, max_block_size_rows_,
|
||||
preferred_block_size_bytes_, preferred_max_column_in_block_size_bytes_, settings.min_bytes_to_use_direct_io,
|
||||
settings.max_read_buffer_size, use_uncompressed_cache_, true, virt_column_names_},
|
||||
thread{thread},
|
||||
pool{pool}
|
||||
{
|
||||
|
Loading…
Reference in New Issue
Block a user