mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Better [#CLICKHOUSE-2].
This commit is contained in:
parent
a85773bdbc
commit
54e601019c
@ -1220,14 +1220,14 @@ namespace libdivide_internal {
|
||||
#endif
|
||||
|
||||
/* Some bogus unswitch functions for unsigned types so the same (presumably templated) code can work for both signed and unsigned. */
|
||||
uint32_t crash_u32(uint32_t, const libdivide_u32_t *) { abort(); return *(uint32_t *)NULL; }
|
||||
uint64_t crash_u64(uint64_t, const libdivide_u64_t *) { abort(); return *(uint64_t *)NULL; }
|
||||
uint32_t crash_u32(uint32_t, const libdivide_u32_t *) { abort(); }
|
||||
uint64_t crash_u64(uint64_t, const libdivide_u64_t *) { abort(); }
|
||||
#ifdef __APPLE__
|
||||
UInt64 crash_u64(UInt64, const libdivide_u64_t *) { abort(); return *(UInt64 *)NULL; }
|
||||
UInt64 crash_u64(UInt64, const libdivide_u64_t *) { abort(); }
|
||||
#endif
|
||||
#if LIBDIVIDE_USE_SSE2
|
||||
__m128i crash_u32_vector(__m128i, const libdivide_u32_t *) { abort(); return *(__m128i *)NULL; }
|
||||
__m128i crash_u64_vector(__m128i, const libdivide_u64_t *) { abort(); return *(__m128i *)NULL; }
|
||||
__m128i crash_u32_vector(__m128i, const libdivide_u32_t *) { abort(); }
|
||||
__m128i crash_u64_vector(__m128i, const libdivide_u64_t *) { abort(); }
|
||||
#endif
|
||||
|
||||
template<typename IntType, typename DenomType, DenomType gen_func(IntType), int get_algo(const DenomType *), IntType do_func(IntType, const DenomType *), MAYBE_VECTOR_PARAM>
|
||||
|
@ -7,7 +7,7 @@ namespace DB
|
||||
namespace
|
||||
{
|
||||
|
||||
AggregateFunctionPtr createAggregateFunctionCount(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
||||
AggregateFunctionPtr createAggregateFunctionCount(const std::string & /*name*/, const DataTypes & /*argument_types*/, const Array & /*parameters*/)
|
||||
{
|
||||
return std::make_shared<AggregateFunctionCount>();
|
||||
}
|
||||
|
@ -50,7 +50,7 @@ ConnectionPoolWithFailover::ConnectionPoolWithFailover(
|
||||
}
|
||||
}
|
||||
|
||||
IConnectionPool::Entry ConnectionPoolWithFailover::get(const Settings * settings, bool force_connected)
|
||||
IConnectionPool::Entry ConnectionPoolWithFailover::get(const Settings * settings, bool /*force_connected*/)
|
||||
{
|
||||
TryGetEntryFunc try_get_entry = [&](NestedPool & pool, std::string & fail_message)
|
||||
{
|
||||
|
@ -11,7 +11,7 @@
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
int main(int, char **)
|
||||
{
|
||||
using Strings = std::vector<std::string>;
|
||||
using Hashes = std::vector<char>;
|
||||
|
@ -21,7 +21,7 @@
|
||||
using namespace DB;
|
||||
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
int main(int, char **)
|
||||
try
|
||||
{
|
||||
NamesAndTypesList names_and_types_list
|
||||
|
@ -15,7 +15,7 @@
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
int main(int, char **)
|
||||
try
|
||||
{
|
||||
using namespace DB;
|
||||
|
@ -32,7 +32,7 @@ void forkThread(ForkPtr fork)
|
||||
}
|
||||
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
int main(int, char **)
|
||||
try
|
||||
{
|
||||
Context context = Context::createGlobal();
|
||||
|
@ -16,7 +16,7 @@
|
||||
|
||||
using namespace DB;
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
int main(int, char **)
|
||||
try
|
||||
{
|
||||
Block sample;
|
||||
|
@ -410,24 +410,14 @@ struct AbsImpl
|
||||
{
|
||||
using ResultType = typename NumberTraits::ResultOfAbs<A>::Type;
|
||||
|
||||
template <typename T = A>
|
||||
static inline ResultType apply(T a,
|
||||
typename std::enable_if<std::is_integral<T>::value && std::is_signed<T>::value, void>::type * = nullptr)
|
||||
static inline ResultType apply(A a)
|
||||
{
|
||||
return a < 0 ? static_cast<ResultType>(~a) + 1 : a;
|
||||
}
|
||||
|
||||
template <typename T = A>
|
||||
static inline ResultType apply(T a,
|
||||
typename std::enable_if<std::is_integral<T>::value && std::is_unsigned<T>::value, void>::type * = nullptr)
|
||||
{
|
||||
return static_cast<ResultType>(a);
|
||||
}
|
||||
|
||||
template <typename T = A>
|
||||
static inline ResultType apply(T a, typename std::enable_if<std::is_floating_point<T>::value, void>::type * = nullptr)
|
||||
{
|
||||
return static_cast<ResultType>(std::abs(a));
|
||||
if constexpr (std::is_integral<A>::value && std::is_signed<A>::value)
|
||||
return a < 0 ? static_cast<ResultType>(~a) + 1 : a;
|
||||
else if constexpr (std::is_integral<A>::value && std::is_unsigned<A>::value)
|
||||
return static_cast<ResultType>(a);
|
||||
else if constexpr (std::is_floating_point<A>::value)
|
||||
return static_cast<ResultType>(std::abs(a));
|
||||
}
|
||||
};
|
||||
|
||||
@ -582,21 +572,17 @@ public:
|
||||
private:
|
||||
const Context & context;
|
||||
|
||||
/// Overload for InvalidType
|
||||
template <typename ResultDataType,
|
||||
typename std::enable_if<std::is_same<ResultDataType, InvalidType>::value>::type * = nullptr>
|
||||
template <typename ResultDataType>
|
||||
bool checkRightTypeImpl(DataTypePtr & type_res) const
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
/// Overload for well-defined operations
|
||||
template <typename ResultDataType,
|
||||
typename std::enable_if<!std::is_same<ResultDataType, InvalidType>::value>::type * = nullptr>
|
||||
bool checkRightTypeImpl(DataTypePtr & type_res) const
|
||||
{
|
||||
type_res = std::make_shared<ResultDataType>();
|
||||
return true;
|
||||
/// Overload for InvalidType
|
||||
if constexpr (std::is_same<ResultDataType, InvalidType>::value)
|
||||
return false;
|
||||
else
|
||||
{
|
||||
type_res = std::make_shared<ResultDataType>();
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
template <typename LeftDataType, typename RightDataType>
|
||||
@ -646,27 +632,21 @@ private:
|
||||
}
|
||||
|
||||
/// Overload for InvalidType
|
||||
template <typename LeftDataType, typename RightDataType, typename ResultDataType, typename ColumnType,
|
||||
typename std::enable_if<std::is_same<ResultDataType, InvalidType>::value>::type * = nullptr>
|
||||
bool executeRightTypeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result,
|
||||
const ColumnType * col_left)
|
||||
template <typename LeftDataType, typename RightDataType, typename ResultDataType, typename ColumnType>
|
||||
bool executeRightTypeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result, const ColumnType * col_left)
|
||||
{
|
||||
throw Exception("Types " + String(TypeName<typename LeftDataType::FieldType>::get())
|
||||
+ " and " + String(TypeName<typename LeftDataType::FieldType>::get())
|
||||
+ " are incompatible for function " + getName() + " or not upscaleable to common type", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
if constexpr (std::is_same<ResultDataType, InvalidType>::value)
|
||||
throw Exception("Types " + String(TypeName<typename LeftDataType::FieldType>::get())
|
||||
+ " and " + String(TypeName<typename LeftDataType::FieldType>::get())
|
||||
+ " are incompatible for function " + getName() + " or not upscaleable to common type", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
else
|
||||
{
|
||||
using T0 = typename LeftDataType::FieldType;
|
||||
using T1 = typename RightDataType::FieldType;
|
||||
using ResultType = typename ResultDataType::FieldType;
|
||||
|
||||
/// Overload for well-defined operations
|
||||
template <typename LeftDataType, typename RightDataType, typename ResultDataType, typename ColumnType,
|
||||
typename std::enable_if<!std::is_same<ResultDataType, InvalidType>::value>::type * = nullptr>
|
||||
bool executeRightTypeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result,
|
||||
const ColumnType * col_left)
|
||||
{
|
||||
using T0 = typename LeftDataType::FieldType;
|
||||
using T1 = typename RightDataType::FieldType;
|
||||
using ResultType = typename ResultDataType::FieldType;
|
||||
|
||||
return executeRightTypeImpl<T0, T1, ResultType>(block, arguments, result, col_left);
|
||||
return executeRightTypeImpl<T0, T1, ResultType>(block, arguments, result, col_left);
|
||||
}
|
||||
}
|
||||
|
||||
/// ColumnVector overload
|
||||
@ -923,7 +903,7 @@ class FunctionUnaryArithmetic : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionUnaryArithmetic>(); }
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionUnaryArithmetic>(); }
|
||||
|
||||
private:
|
||||
template <typename T0>
|
||||
@ -1010,7 +990,7 @@ public:
|
||||
return FunctionUnaryArithmeticMonotonicity<Name>::has();
|
||||
}
|
||||
|
||||
Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override
|
||||
Monotonicity getMonotonicityForRange(const IDataType &, const Field & left, const Field & right) const override
|
||||
{
|
||||
return FunctionUnaryArithmeticMonotonicity<Name>::get(left, right);
|
||||
}
|
||||
@ -1070,7 +1050,7 @@ using FunctionLCM = FunctionBinaryArithmetic<LCMImpl, NameLCM>;
|
||||
template <> struct FunctionUnaryArithmeticMonotonicity<NameNegate>
|
||||
{
|
||||
static bool has() { return true; }
|
||||
static IFunction::Monotonicity get(const Field & left, const Field & right)
|
||||
static IFunction::Monotonicity get(const Field &, const Field &)
|
||||
{
|
||||
return { true, false };
|
||||
}
|
||||
@ -1094,7 +1074,7 @@ template <> struct FunctionUnaryArithmeticMonotonicity<NameAbs>
|
||||
template <> struct FunctionUnaryArithmeticMonotonicity<NameBitNot>
|
||||
{
|
||||
static bool has() { return false; }
|
||||
static IFunction::Monotonicity get(const Field & left, const Field & right)
|
||||
static IFunction::Monotonicity get(const Field &, const Field &)
|
||||
{
|
||||
return {};
|
||||
}
|
||||
@ -1325,7 +1305,7 @@ private:
|
||||
{
|
||||
const auto size = value_col->size();
|
||||
bool is_const;
|
||||
const auto mask = createConstMask<T>(size, block, arguments, is_const);
|
||||
const auto mask = createConstMask<T>(block, arguments, is_const);
|
||||
const auto & val = value_col->getData();
|
||||
|
||||
const auto out_col = std::make_shared<ColumnVector<UInt8>>(size);
|
||||
@ -1351,7 +1331,7 @@ private:
|
||||
{
|
||||
const auto size = value_col->size();
|
||||
bool is_const;
|
||||
const auto mask = createConstMask<T>(size, block, arguments, is_const);
|
||||
const auto mask = createConstMask<T>(block, arguments, is_const);
|
||||
const auto val = value_col->template getValue<T>();
|
||||
|
||||
if (is_const)
|
||||
@ -1378,7 +1358,7 @@ private:
|
||||
}
|
||||
|
||||
template <typename ValueType>
|
||||
ValueType createConstMask(const size_t size, const Block & block, const ColumnNumbers & arguments, bool & is_const)
|
||||
ValueType createConstMask(const Block & block, const ColumnNumbers & arguments, bool & is_const)
|
||||
{
|
||||
is_const = true;
|
||||
ValueType mask = 0;
|
||||
|
@ -45,7 +45,7 @@ std::string Service::getId(const std::string & node_id) const
|
||||
return getEndpointId(node_id);
|
||||
}
|
||||
|
||||
void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & body, WriteBuffer & out, Poco::Net::HTTPServerResponse & response)
|
||||
void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*body*/, WriteBuffer & out, Poco::Net::HTTPServerResponse & response)
|
||||
{
|
||||
if (blocker.isCancelled())
|
||||
throw Exception("Transferring part to replica was cancelled", ErrorCodes::ABORTED);
|
||||
|
@ -39,7 +39,7 @@ protected:
|
||||
virtual bool getNewTask() = 0;
|
||||
|
||||
/// We will call progressImpl manually.
|
||||
void progress(const Progress & value) override {}
|
||||
void progress(const Progress &) override {}
|
||||
|
||||
Block readFromPart();
|
||||
|
||||
|
@ -94,8 +94,7 @@ MergeTreeData::MergeTreeData(
|
||||
const String & log_name_,
|
||||
bool require_part_metadata_,
|
||||
bool attach,
|
||||
BrokenPartCallback broken_part_callback_,
|
||||
PartsCleanCallback parts_clean_callback_)
|
||||
BrokenPartCallback broken_part_callback_)
|
||||
: ITableDeclaration{materialized_columns_, alias_columns_, column_defaults_}, context(context_),
|
||||
sampling_expression(sampling_expression_),
|
||||
index_granularity(settings_.index_granularity),
|
||||
@ -2120,7 +2119,7 @@ MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVector() const
|
||||
}
|
||||
|
||||
MergeTreeData::DataPartPtr MergeTreeData::getAnyPartInPartition(
|
||||
const String & partition_id, std::unique_lock<std::mutex> & data_parts_lock)
|
||||
const String & partition_id, std::unique_lock<std::mutex> & /*data_parts_lock*/)
|
||||
{
|
||||
auto min_block = std::numeric_limits<Int64>::min();
|
||||
MergeTreePartInfo dummy_part_info(partition_id, min_block, min_block, 0);
|
||||
|
@ -93,8 +93,6 @@ class MergeTreeData : public ITableDeclaration
|
||||
public:
|
||||
/// Function to call if the part is suspected to contain corrupt data.
|
||||
using BrokenPartCallback = std::function<void (const String &)>;
|
||||
/// Callback to delete outdated parts immediately
|
||||
using PartsCleanCallback = std::function<void ()>;
|
||||
using DataPart = MergeTreeDataPart;
|
||||
|
||||
using MutableDataPartPtr = std::shared_ptr<DataPart>;
|
||||
@ -290,9 +288,7 @@ public:
|
||||
const String & log_name_,
|
||||
bool require_part_metadata_,
|
||||
bool attach,
|
||||
BrokenPartCallback broken_part_callback_ = [](const String &){},
|
||||
PartsCleanCallback parts_clean_callback_ = nullptr
|
||||
);
|
||||
BrokenPartCallback broken_part_callback_ = [](const String &){});
|
||||
|
||||
/// Load the set of data parts from disk. Call once - immediately after the object is created.
|
||||
void loadDataParts(bool skip_sanity_checks);
|
||||
|
@ -358,7 +358,6 @@ void PKCondition::traverseAST(const ASTPtr & node, const Context & context, Bloc
|
||||
|
||||
bool PKCondition::canConstantBeWrappedByMonotonicFunctions(
|
||||
const ASTPtr & node,
|
||||
const Context & context,
|
||||
size_t & out_primary_key_column_num,
|
||||
DataTypePtr & out_primary_key_column_type,
|
||||
Field & out_value,
|
||||
@ -537,7 +536,7 @@ bool PKCondition::atomFromAST(const ASTPtr & node, const Context & context, Bloc
|
||||
key_arg_pos = 0;
|
||||
}
|
||||
else if (getConstant(args[1], block_with_constants, const_value, const_type)
|
||||
&& canConstantBeWrappedByMonotonicFunctions(args[0], context, key_column_num, key_expr_type, const_value, const_type))
|
||||
&& canConstantBeWrappedByMonotonicFunctions(args[0], key_column_num, key_expr_type, const_value, const_type))
|
||||
{
|
||||
key_arg_pos = 0;
|
||||
is_constant_transformed = true;
|
||||
@ -548,7 +547,7 @@ bool PKCondition::atomFromAST(const ASTPtr & node, const Context & context, Bloc
|
||||
key_arg_pos = 1;
|
||||
}
|
||||
else if (getConstant(args[0], block_with_constants, const_value, const_type)
|
||||
&& canConstantBeWrappedByMonotonicFunctions(args[1], context, key_column_num, key_expr_type, const_value, const_type))
|
||||
&& canConstantBeWrappedByMonotonicFunctions(args[1], key_column_num, key_expr_type, const_value, const_type))
|
||||
{
|
||||
key_arg_pos = 1;
|
||||
is_constant_transformed = true;
|
||||
|
@ -319,7 +319,6 @@ private:
|
||||
|
||||
bool canConstantBeWrappedByMonotonicFunctions(
|
||||
const ASTPtr & node,
|
||||
const Context & context,
|
||||
size_t & out_primary_key_column_num,
|
||||
DataTypePtr & out_primary_key_column_type,
|
||||
Field & out_value,
|
||||
|
@ -187,8 +187,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
|
||||
context_, primary_expr_ast_, date_column_name, partition_expr_ast_,
|
||||
sampling_expression_, merging_params_,
|
||||
settings_, database_name_ + "." + table_name, true, attach,
|
||||
[this] (const std::string & name) { enqueuePartForCheck(name); },
|
||||
[this] () { clearOldPartsAndRemoveFromZK(); }),
|
||||
[this] (const std::string & name) { enqueuePartForCheck(name); }),
|
||||
reader(data), writer(data), merger(data, context.getBackgroundPool()), queue(data.format_version),
|
||||
fetcher(data),
|
||||
shutdown_event(false), part_check_thread(*this),
|
||||
|
Loading…
Reference in New Issue
Block a user