Preparation for extra warnings [#CLICKHOUSE-2]

This commit is contained in:
Alexey Milovidov 2018-08-10 07:02:56 +03:00
parent 419bc587c0
commit 36db216abf
102 changed files with 267 additions and 323 deletions

View File

@ -17,4 +17,4 @@ include/libcpuid/recog_amd.h
include/libcpuid/recog_intel.h include/libcpuid/recog_intel.h
) )
target_include_directories (cpuid PUBLIC include) target_include_directories (cpuid SYSTEM PUBLIC include)

View File

@ -56,5 +56,5 @@ ${RDKAFKA_SOURCE_DIR}/rdgz.c
add_library(rdkafka STATIC ${SRCS}) add_library(rdkafka STATIC ${SRCS})
target_include_directories(rdkafka PRIVATE include) target_include_directories(rdkafka PRIVATE include)
target_include_directories(rdkafka PUBLIC ${RDKAFKA_SOURCE_DIR}) target_include_directories(rdkafka SYSTEM PUBLIC ${RDKAFKA_SOURCE_DIR})
target_link_libraries(rdkafka PUBLIC ${ZLIB_LIBRARIES} ${OPENSSL_SSL_LIBRARY} ${OPENSSL_CRYPTO_LIBRARY}) target_link_libraries(rdkafka PUBLIC ${ZLIB_LIBRARIES} ${OPENSSL_SSL_LIBRARY} ${OPENSSL_CRYPTO_LIBRARY})

View File

@ -226,6 +226,6 @@ ConnectionPoolWithFailover::tryGetEntry(
} }
} }
return result; return result;
}; }
} }

View File

@ -3,10 +3,7 @@
#include <Common/config.h> #include <Common/config.h>
#if USE_ICU #if USE_ICU
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wold-style-cast"
#include <unicode/ucol.h> #include <unicode/ucol.h>
#pragma GCC diagnostic pop
#else #else
#ifdef __clang__ #ifdef __clang__
#pragma clang diagnostic push #pragma clang diagnostic push

View File

@ -1,17 +1,7 @@
#pragma once #pragma once
#ifdef __clang__
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <boost/smart_ptr/intrusive_ptr.hpp> #include <boost/smart_ptr/intrusive_ptr.hpp>
#include <boost/smart_ptr/intrusive_ref_counter.hpp> #include <boost/smart_ptr/intrusive_ref_counter.hpp>
#ifdef __clang__
#pragma clang diagnostic pop
#endif
#include <initializer_list> #include <initializer_list>

View File

@ -37,7 +37,7 @@ namespace detail
{ {
MoveOrCopyIfThrow<T>()(std::forward<T>(src), dst); MoveOrCopyIfThrow<T>()(std::forward<T>(src), dst);
} }
}; }
/** A very simple thread-safe queue of limited size. /** A very simple thread-safe queue of limited size.
* If you try to pop an item from an empty queue, the thread is blocked until the queue becomes nonempty. * If you try to pop an item from an empty queue, the thread is blocked until the queue becomes nonempty.

View File

@ -130,11 +130,11 @@ int getCurrentExceptionCode()
{ {
return e.code(); return e.code();
} }
catch (const Poco::Exception & e) catch (const Poco::Exception &)
{ {
return ErrorCodes::POCO_EXCEPTION; return ErrorCodes::POCO_EXCEPTION;
} }
catch (const std::exception & e) catch (const std::exception &)
{ {
return ErrorCodes::STD_EXCEPTION; return ErrorCodes::STD_EXCEPTION;
} }

View File

@ -74,7 +74,7 @@ bool check(const T x) { return x == 0; }
template <typename T> template <typename T>
void set(T & x) { x = 0; } void set(T & x) { x = 0; }
}; }
/** Compile-time interface for cell of the hash table. /** Compile-time interface for cell of the hash table.

View File

@ -94,8 +94,8 @@ public:
{ {
if (auto it = aliases.find(name); it != aliases.end()) if (auto it = aliases.find(name); it != aliases.end())
return it->second; return it->second;
else if (auto it = case_insensitive_aliases.find(Poco::toLower(name)); it != case_insensitive_aliases.end()) else if (auto jt = case_insensitive_aliases.find(Poco::toLower(name)); jt != case_insensitive_aliases.end())
return it->second; return jt->second;
throw Exception(getFactoryName() + ": name '" + name + "' is not alias", ErrorCodes::LOGICAL_ERROR); throw Exception(getFactoryName() + ": name '" + name + "' is not alias", ErrorCodes::LOGICAL_ERROR);
} }

View File

@ -5,24 +5,9 @@
#include <algorithm> #include <algorithm>
#include <memory> #include <memory>
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wold-style-cast"
#ifdef __clang__
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#pragma clang diagnostic ignored "-Wreserved-id-macro"
#endif
#include <boost/noncopyable.hpp> #include <boost/noncopyable.hpp>
#include <boost/iterator_adaptors.hpp> #include <boost/iterator_adaptors.hpp>
#ifdef __clang__
#pragma clang diagnostic pop
#endif
#pragma GCC diagnostic pop
#include <common/likely.h> #include <common/likely.h>
#include <common/strong_typedef.h> #include <common/strong_typedef.h>

View File

@ -340,4 +340,4 @@ private:
size_t m_capacity; size_t m_capacity;
}; };
}; }

View File

@ -50,4 +50,4 @@ private:
static size_t getFailedOpIndex(int32_t code, const Responses & responses); static size_t getFailedOpIndex(int32_t code, const Responses & responses);
}; };
}; }

View File

@ -83,4 +83,4 @@ void ZooKeeperHolder::init(Args&&... args)
using ZooKeeperHolderPtr = std::shared_ptr<ZooKeeperHolder>; using ZooKeeperHolderPtr = std::shared_ptr<ZooKeeperHolder>;
}; }

View File

@ -646,4 +646,4 @@ private:
CurrentMetrics::Increment active_session_metric_increment{CurrentMetrics::ZooKeeperSession}; CurrentMetrics::Increment active_session_metric_increment{CurrentMetrics::ZooKeeperSession};
}; };
}; }

View File

@ -77,7 +77,10 @@ void formatIPv6(const unsigned char * src, char *& dst, UInt8 zeroed_tail_bytes_
if (words[i] == 0) if (words[i] == 0)
{ {
if (cur.base == -1) if (cur.base == -1)
cur.base = i, cur.len = 1; {
cur.base = i;
cur.len = 1;
}
else else
cur.len++; cur.len++;
} }

View File

@ -108,7 +108,7 @@ void localBackup(const Poco::Path & source_path, const Poco::Path & destination_
continue; continue;
} }
catch (const Poco::FileNotFoundException & e) catch (const Poco::FileNotFoundException &)
{ {
++try_no; ++try_no;
if (try_no == max_tries) if (try_no == max_tries)

View File

@ -60,7 +60,7 @@ add_executable (space_saving space_saving.cpp)
target_link_libraries (space_saving clickhouse_common_io) target_link_libraries (space_saving clickhouse_common_io)
add_executable (integer_hash_tables_and_hashes integer_hash_tables_and_hashes.cpp) add_executable (integer_hash_tables_and_hashes integer_hash_tables_and_hashes.cpp)
target_include_directories (integer_hash_tables_and_hashes BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR}) target_include_directories (integer_hash_tables_and_hashes SYSTEM BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR})
target_link_libraries (integer_hash_tables_and_hashes clickhouse_common_io) target_link_libraries (integer_hash_tables_and_hashes clickhouse_common_io)
add_executable (allocator allocator.cpp) add_executable (allocator allocator.cpp)

View File

@ -25,7 +25,7 @@ namespace ErrorCodes
class Field; class Field;
using Array = std::vector<Field>; using Array = std::vector<Field>;
using TupleBackend = std::vector<Field>; using TupleBackend = std::vector<Field>;
STRONG_TYPEDEF(TupleBackend, Tuple); /// Array and Tuple are different types with equal representation inside Field. STRONG_TYPEDEF(TupleBackend, Tuple) /// Array and Tuple are different types with equal representation inside Field.
/** 32 is enough. Round number is used for alignment and for better arithmetic inside std::vector. /** 32 is enough. Round number is used for alignment and for better arithmetic inside std::vector.

View File

@ -6,6 +6,6 @@
namespace DB namespace DB
{ {
STRONG_TYPEDEF(UInt128, UUID); STRONG_TYPEDEF(UInt128, UUID)
} }

View File

@ -63,7 +63,7 @@ Block AggregatingSortedBlockInputStream::readImpl()
for (size_t i = 0, size = columns_to_aggregate.size(); i < size; ++i) for (size_t i = 0, size = columns_to_aggregate.size(); i < size; ++i)
columns_to_aggregate[i] = typeid_cast<ColumnAggregateFunction *>(merged_columns[column_numbers_to_aggregate[i]].get()); columns_to_aggregate[i] = typeid_cast<ColumnAggregateFunction *>(merged_columns[column_numbers_to_aggregate[i]].get());
merge(merged_columns, queue); merge(merged_columns, queue_without_collation);
return header.cloneWithColumns(std::move(merged_columns)); return header.cloneWithColumns(std::move(merged_columns));
} }

View File

@ -102,7 +102,7 @@ Block CollapsingSortedBlockInputStream::readImpl()
if (merged_columns.empty()) if (merged_columns.empty())
return {}; return {};
merge(merged_columns, queue); merge(merged_columns, queue_without_collation);
return header.cloneWithColumns(std::move(merged_columns)); return header.cloneWithColumns(std::move(merged_columns));
} }

View File

@ -102,7 +102,7 @@ Block GraphiteRollupSortedBlockInputStream::readImpl()
if (merged_columns.empty()) if (merged_columns.empty())
return Block(); return Block();
merge(merged_columns, queue); merge(merged_columns, queue_without_collation);
return header.cloneWithColumns(std::move(merged_columns)); return header.cloneWithColumns(std::move(merged_columns));
} }

View File

@ -188,7 +188,7 @@ static bool handleOverflowMode(OverflowMode mode, const String & message, int co
default: default:
throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR);
} }
}; }
bool IProfilingBlockInputStream::checkTimeLimit() bool IProfilingBlockInputStream::checkTimeLimit()

View File

@ -183,7 +183,7 @@ MergeSortingBlocksBlockInputStream::MergeSortingBlocksBlockInputStream(
if (!has_collation) if (!has_collation)
{ {
for (size_t i = 0; i < cursors.size(); ++i) for (size_t i = 0; i < cursors.size(); ++i)
queue.push(SortCursor(&cursors[i])); queue_without_collation.push(SortCursor(&cursors[i]));
} }
else else
{ {
@ -206,7 +206,7 @@ Block MergeSortingBlocksBlockInputStream::readImpl()
} }
return !has_collation return !has_collation
? mergeImpl<SortCursor>(queue) ? mergeImpl<SortCursor>(queue_without_collation)
: mergeImpl<SortCursorWithCollation>(queue_with_collation); : mergeImpl<SortCursorWithCollation>(queue_with_collation);
} }

View File

@ -55,7 +55,7 @@ private:
bool has_collation = false; bool has_collation = false;
std::priority_queue<SortCursor> queue; std::priority_queue<SortCursor> queue_without_collation;
std::priority_queue<SortCursorWithCollation> queue_with_collation; std::priority_queue<SortCursorWithCollation> queue_with_collation;
/** Two different cursors are supported - with and without Collation. /** Two different cursors are supported - with and without Collation.

View File

@ -320,7 +320,7 @@ void MergingAggregatedMemoryEfficientBlockInputStream::mergeThread(MemoryTracker
* - or, if no next blocks, set 'exhausted' flag. * - or, if no next blocks, set 'exhausted' flag.
*/ */
{ {
std::lock_guard<std::mutex> lock(parallel_merge_data->get_next_blocks_mutex); std::lock_guard<std::mutex> lock_next_blocks(parallel_merge_data->get_next_blocks_mutex);
if (parallel_merge_data->exhausted || parallel_merge_data->finish) if (parallel_merge_data->exhausted || parallel_merge_data->finish)
break; break;
@ -330,7 +330,7 @@ void MergingAggregatedMemoryEfficientBlockInputStream::mergeThread(MemoryTracker
if (!blocks_to_merge || blocks_to_merge->empty()) if (!blocks_to_merge || blocks_to_merge->empty())
{ {
{ {
std::unique_lock<std::mutex> lock(parallel_merge_data->merged_blocks_mutex); std::unique_lock<std::mutex> lock_merged_blocks(parallel_merge_data->merged_blocks_mutex);
parallel_merge_data->exhausted = true; parallel_merge_data->exhausted = true;
} }
@ -344,9 +344,9 @@ void MergingAggregatedMemoryEfficientBlockInputStream::mergeThread(MemoryTracker
: blocks_to_merge->front().info.bucket_num; : blocks_to_merge->front().info.bucket_num;
{ {
std::unique_lock<std::mutex> lock(parallel_merge_data->merged_blocks_mutex); std::unique_lock<std::mutex> lock_merged_blocks(parallel_merge_data->merged_blocks_mutex);
parallel_merge_data->have_space.wait(lock, [this] parallel_merge_data->have_space.wait(lock_merged_blocks, [this]
{ {
return parallel_merge_data->merged_blocks.size() < merging_threads return parallel_merge_data->merged_blocks.size() < merging_threads
|| parallel_merge_data->finish; || parallel_merge_data->finish;

View File

@ -58,7 +58,7 @@ void MergingSortedBlockInputStream::init(MutableColumns & merged_columns)
if (has_collation) if (has_collation)
initQueue(queue_with_collation); initQueue(queue_with_collation);
else else
initQueue(queue); initQueue(queue_without_collation);
} }
/// Let's check that all source blocks have the same structure. /// Let's check that all source blocks have the same structure.
@ -105,7 +105,7 @@ Block MergingSortedBlockInputStream::readImpl()
if (has_collation) if (has_collation)
merge(merged_columns, queue_with_collation); merge(merged_columns, queue_with_collation);
else else
merge(merged_columns, queue); merge(merged_columns, queue_without_collation);
return header.cloneWithColumns(std::move(merged_columns)); return header.cloneWithColumns(std::move(merged_columns));
} }
@ -200,7 +200,7 @@ void MergingSortedBlockInputStream::merge(MutableColumns & merged_columns, std::
// std::cerr << "copied columns\n"; // std::cerr << "copied columns\n";
size_t merged_rows = merged_columns.at(0)->size(); merged_rows = merged_columns.at(0)->size();
if (limit && total_merged_rows + merged_rows > limit) if (limit && total_merged_rows + merged_rows > limit)
{ {

View File

@ -2,17 +2,8 @@
#include <queue> #include <queue>
#ifdef __clang__
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <boost/smart_ptr/intrusive_ptr.hpp> #include <boost/smart_ptr/intrusive_ptr.hpp>
#ifdef __clang__
#pragma clang diagnostic pop
#endif
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <Core/Row.h> #include <Core/Row.h>
@ -161,7 +152,7 @@ protected:
CursorImpls cursors; CursorImpls cursors;
using Queue = std::priority_queue<SortCursor>; using Queue = std::priority_queue<SortCursor>;
Queue queue; Queue queue_without_collation;
using QueueWithCollation = std::priority_queue<SortCursorWithCollation>; using QueueWithCollation = std::priority_queue<SortCursorWithCollation>;
QueueWithCollation queue_with_collation; QueueWithCollation queue_with_collation;

View File

@ -58,21 +58,21 @@ RemoteBlockInputStream::RemoteBlockInputStream(
create_multiplexed_connections = [this, pool, throttler]() create_multiplexed_connections = [this, pool, throttler]()
{ {
const Settings & settings = context.getSettingsRef(); const Settings & current_settings = context.getSettingsRef();
std::vector<IConnectionPool::Entry> connections; std::vector<IConnectionPool::Entry> connections;
if (main_table) if (main_table)
{ {
auto try_results = pool->getManyChecked(&settings, pool_mode, *main_table); auto try_results = pool->getManyChecked(&current_settings, pool_mode, *main_table);
connections.reserve(try_results.size()); connections.reserve(try_results.size());
for (auto & try_result : try_results) for (auto & try_result : try_results)
connections.emplace_back(std::move(try_result.entry)); connections.emplace_back(std::move(try_result.entry));
} }
else else
connections = pool->getMany(&settings, pool_mode); connections = pool->getMany(&current_settings, pool_mode);
return std::make_unique<MultiplexedConnections>( return std::make_unique<MultiplexedConnections>(
std::move(connections), settings, throttler, append_extra_info); std::move(connections), current_settings, throttler, append_extra_info);
}; };
} }

View File

@ -44,7 +44,7 @@ Block ReplacingSortedBlockInputStream::readImpl()
if (merged_columns.empty()) if (merged_columns.empty())
return Block(); return Block();
merge(merged_columns, queue); merge(merged_columns, queue_without_collation);
return header.cloneWithColumns(std::move(merged_columns)); return header.cloneWithColumns(std::move(merged_columns));
} }

View File

@ -286,7 +286,7 @@ Block SummingSortedBlockInputStream::readImpl()
desc.merged_column = header.safeGetByPosition(desc.column_numbers[0]).column->cloneEmpty(); desc.merged_column = header.safeGetByPosition(desc.column_numbers[0]).column->cloneEmpty();
} }
merge(merged_columns, queue); merge(merged_columns, queue_without_collation);
Block res = header.cloneWithColumns(std::move(merged_columns)); Block res = header.cloneWithColumns(std::move(merged_columns));
/// Place aggregation results into block. /// Place aggregation results into block.

View File

@ -76,7 +76,7 @@ Block VersionedCollapsingSortedBlockInputStream::readImpl()
if (merged_columns.empty()) if (merged_columns.empty())
return {}; return {};
merge(merged_columns, queue); merge(merged_columns, queue_without_collation);
return header.cloneWithColumns(std::move(merged_columns)); return header.cloneWithColumns(std::move(merged_columns));
} }

View File

@ -97,7 +97,7 @@ DataTypeEnum<Type>::DataTypeEnum(const Values & values_) : values{values_}
}); });
fillMaps(); fillMaps();
name = generateName(values); type_name = generateName(values);
} }
template <typename Type> template <typename Type>
@ -145,9 +145,9 @@ template <typename Type>
void DataTypeEnum<Type>::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const void DataTypeEnum<Type>::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{ {
/// NOTE It would be nice to do without creating a temporary object - at least extract std::string out. /// NOTE It would be nice to do without creating a temporary object - at least extract std::string out.
std::string name; std::string field_name;
readEscapedString(name, istr); readEscapedString(field_name, istr);
static_cast<ColumnType &>(column).getData().push_back(getValue(StringRef(name))); static_cast<ColumnType &>(column).getData().push_back(getValue(StringRef(field_name)));
} }
template <typename Type> template <typename Type>
@ -159,9 +159,9 @@ void DataTypeEnum<Type>::serializeTextQuoted(const IColumn & column, size_t row_
template <typename Type> template <typename Type>
void DataTypeEnum<Type>::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const void DataTypeEnum<Type>::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{ {
std::string name; std::string field_name;
readQuotedStringWithSQLStyle(name, istr); readQuotedStringWithSQLStyle(field_name, istr);
static_cast<ColumnType &>(column).getData().push_back(getValue(StringRef(name))); static_cast<ColumnType &>(column).getData().push_back(getValue(StringRef(field_name)));
} }
template <typename Type> template <typename Type>
@ -179,9 +179,9 @@ void DataTypeEnum<Type>::serializeTextXML(const IColumn & column, size_t row_num
template <typename Type> template <typename Type>
void DataTypeEnum<Type>::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const void DataTypeEnum<Type>::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{ {
std::string name; std::string field_name;
readJSONString(name, istr); readJSONString(field_name, istr);
static_cast<ColumnType &>(column).getData().push_back(getValue(StringRef(name))); static_cast<ColumnType &>(column).getData().push_back(getValue(StringRef(field_name)));
} }
template <typename Type> template <typename Type>
@ -193,9 +193,9 @@ void DataTypeEnum<Type>::serializeTextCSV(const IColumn & column, size_t row_num
template <typename Type> template <typename Type>
void DataTypeEnum<Type>::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const void DataTypeEnum<Type>::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{ {
std::string name; std::string field_name;
readCSVString(name, istr, settings.csv); readCSVString(field_name, istr, settings.csv);
static_cast<ColumnType &>(column).getData().push_back(getValue(StringRef(name))); static_cast<ColumnType &>(column).getData().push_back(getValue(StringRef(field_name)));
} }
template <typename Type> template <typename Type>
@ -237,7 +237,7 @@ void DataTypeEnum<Type>::insertDefaultInto(IColumn & column) const
template <typename Type> template <typename Type>
bool DataTypeEnum<Type>::equals(const IDataType & rhs) const bool DataTypeEnum<Type>::equals(const IDataType & rhs) const
{ {
return typeid(rhs) == typeid(*this) && name == static_cast<const DataTypeEnum<Type> &>(rhs).name; return typeid(rhs) == typeid(*this) && type_name == static_cast<const DataTypeEnum<Type> &>(rhs).type_name;
} }
@ -346,14 +346,14 @@ static DataTypePtr create(const ASTPtr & arguments)
throw Exception("Elements of Enum data type must be of form: 'name' = number, where name is string literal and number is an integer", throw Exception("Elements of Enum data type must be of form: 'name' = number, where name is string literal and number is an integer",
ErrorCodes::UNEXPECTED_AST_STRUCTURE); ErrorCodes::UNEXPECTED_AST_STRUCTURE);
const String & name = name_literal->value.get<String>(); const String & field_name = name_literal->value.get<String>();
const auto value = value_literal->value.get<typename NearestFieldType<FieldType>::Type>(); const auto value = value_literal->value.get<typename NearestFieldType<FieldType>::Type>();
if (value > std::numeric_limits<FieldType>::max() || value < std::numeric_limits<FieldType>::min()) if (value > std::numeric_limits<FieldType>::max() || value < std::numeric_limits<FieldType>::min())
throw Exception{"Value " + toString(value) + " for element '" + name + "' exceeds range of " + EnumName<FieldType>::value, throw Exception{"Value " + toString(value) + " for element '" + field_name + "' exceeds range of " + EnumName<FieldType>::value,
ErrorCodes::ARGUMENT_OUT_OF_BOUND}; ErrorCodes::ARGUMENT_OUT_OF_BOUND};
values.emplace_back(name, value); values.emplace_back(field_name, value);
} }
return std::make_shared<DataTypeEnum>(values); return std::make_shared<DataTypeEnum>(values);

View File

@ -53,7 +53,7 @@ private:
Values values; Values values;
NameToValueMap name_to_value_map; NameToValueMap name_to_value_map;
ValueToNameMap value_to_name_map; ValueToNameMap value_to_name_map;
std::string name; std::string type_name;
static std::string generateName(const Values & values); static std::string generateName(const Values & values);
void fillMaps(); void fillMaps();
@ -62,7 +62,7 @@ public:
explicit DataTypeEnum(const Values & values_); explicit DataTypeEnum(const Values & values_);
const Values & getValues() const { return values; } const Values & getValues() const { return values; }
std::string getName() const override { return name; } std::string getName() const override { return type_name; }
const char * getFamilyName() const override; const char * getFamilyName() const override;
const StringRef & getNameForValue(const FieldType & value) const const StringRef & getNameForValue(const FieldType & value) const
@ -74,11 +74,11 @@ public:
return it->second; return it->second;
} }
FieldType getValue(StringRef name) const FieldType getValue(StringRef field_name) const
{ {
const auto it = name_to_value_map.find(name); const auto it = name_to_value_map.find(field_name);
if (it == std::end(name_to_value_map)) if (it == std::end(name_to_value_map))
throw Exception{"Unknown element '" + name.toString() + "' for type " + getName(), ErrorCodes::LOGICAL_ERROR}; throw Exception{"Unknown element '" + field_name.toString() + "' for type " + getName(), ErrorCodes::LOGICAL_ERROR};
return it->second; return it->second;
} }

View File

@ -22,6 +22,6 @@ namespace Nested
/// Collect Array columns in a form of `column_name.element_name` to single Array(Tuple(...)) column. /// Collect Array columns in a form of `column_name.element_name` to single Array(Tuple(...)) column.
NamesAndTypesList collect(const NamesAndTypesList & names_and_types); NamesAndTypesList collect(const NamesAndTypesList & names_and_types);
}; }
} }

View File

@ -39,15 +39,15 @@ Tables DatabaseDictionary::loadTables()
Tables tables; Tables tables;
for (const auto & pair : dictionaries) for (const auto & pair : dictionaries)
{ {
const std::string & name = pair.first; const std::string & dict_name = pair.first;
if (deleted_tables.count(name)) if (deleted_tables.count(dict_name))
continue; continue;
auto dict_ptr = std::static_pointer_cast<IDictionaryBase>(pair.second.loadable); auto dict_ptr = std::static_pointer_cast<IDictionaryBase>(pair.second.loadable);
if (dict_ptr) if (dict_ptr)
{ {
const DictionaryStructure & dictionary_structure = dict_ptr->getStructure(); const DictionaryStructure & dictionary_structure = dict_ptr->getStructure();
auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure); auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure);
tables[name] = StorageDictionary::create(name, ColumnsDescription{columns}, dictionary_structure, name); tables[dict_name] = StorageDictionary::create(dict_name, ColumnsDescription{columns}, dictionary_structure, dict_name);
} }
} }

View File

@ -101,12 +101,12 @@ void CacheDictionary::isInImpl(
{ {
/// Transform all children to parents until ancestor id or null_value will be reached. /// Transform all children to parents until ancestor id or null_value will be reached.
size_t size = out.size(); size_t out_size = out.size();
memset(out.data(), 0xFF, size); /// 0xFF means "not calculated" memset(out.data(), 0xFF, out_size); /// 0xFF means "not calculated"
const auto null_value = std::get<UInt64>(hierarchical_attribute->null_values); const auto null_value = std::get<UInt64>(hierarchical_attribute->null_values);
PaddedPODArray<Key> children(size); PaddedPODArray<Key> children(out_size);
PaddedPODArray<Key> parents(child_ids.begin(), child_ids.end()); PaddedPODArray<Key> parents(child_ids.begin(), child_ids.end());
while (true) while (true)
@ -115,7 +115,7 @@ void CacheDictionary::isInImpl(
size_t parents_idx = 0; size_t parents_idx = 0;
size_t new_children_idx = 0; size_t new_children_idx = 0;
while (out_idx < size) while (out_idx < out_size)
{ {
/// Already calculated /// Already calculated
if (out[out_idx] != 0xFF) if (out[out_idx] != 0xFF)
@ -203,7 +203,7 @@ void CacheDictionary::isInConstantVector(
} }
/// Assuming short hierarchy, so linear search is Ok. /// Assuming short hierarchy, so linear search is Ok.
for (size_t i = 0, size = out.size(); i < size; ++i) for (size_t i = 0, out_size = out.size(); i < out_size; ++i)
out[i] = std::find(ancestors.begin(), ancestors.end(), ancestor_ids[i]) != ancestors.end(); out[i] = std::find(ancestors.begin(), ancestors.end(), ancestor_ids[i]) != ancestors.end();
} }
@ -936,12 +936,12 @@ void CacheDictionary::setAttributeValue(Attribute & attribute, const Key idx, co
if (string_ref.data && string_ref.data != null_value_ref.data()) if (string_ref.data && string_ref.data != null_value_ref.data())
string_arena->free(const_cast<char *>(string_ref.data), string_ref.size); string_arena->free(const_cast<char *>(string_ref.data), string_ref.size);
const auto size = string.size(); const auto str_size = string.size();
if (size != 0) if (str_size != 0)
{ {
auto string_ptr = string_arena->alloc(size + 1); auto string_ptr = string_arena->alloc(str_size + 1);
std::copy(string.data(), string.data() + size + 1, string_ptr); std::copy(string.data(), string.data() + str_size + 1, string_ptr);
string_ref = StringRef{string_ptr, size}; string_ref = StringRef{string_ptr, str_size};
} }
else else
string_ref = {}; string_ref = {};

View File

@ -307,13 +307,13 @@ private:
/// buffer[column_size * cat_features_count] -> char * => cat_features[column_size][cat_features_count] -> char * /// buffer[column_size * cat_features_count] -> char * => cat_features[column_size][cat_features_count] -> char *
void fillCatFeaturesBuffer(const char *** cat_features, const char ** buffer, void fillCatFeaturesBuffer(const char *** cat_features, const char ** buffer,
size_t column_size, size_t cat_features_count) const size_t column_size, size_t cat_features_count_current) const
{ {
for (size_t i = 0; i < column_size; ++i) for (size_t i = 0; i < column_size; ++i)
{ {
*cat_features = buffer; *cat_features = buffer;
++cat_features; ++cat_features;
buffer += cat_features_count; buffer += cat_features_count_current;
} }
} }
@ -321,7 +321,7 @@ private:
/// * CalcModelPredictionFlat if no cat features /// * CalcModelPredictionFlat if no cat features
/// * CalcModelPrediction if all cat features are strings /// * CalcModelPrediction if all cat features are strings
/// * CalcModelPredictionWithHashedCatFeatures if has int cat features. /// * CalcModelPredictionWithHashedCatFeatures if has int cat features.
ColumnPtr evalImpl(const ColumnRawPtrs & columns, size_t float_features_count, size_t cat_features_count, ColumnPtr evalImpl(const ColumnRawPtrs & columns, size_t float_features_count_current, size_t cat_features_count_current,
bool cat_features_are_strings) const bool cat_features_are_strings) const
{ {
std::string error_msg = "Error occurred while applying CatBoost model: "; std::string error_msg = "Error occurred while applying CatBoost model: ";
@ -334,12 +334,12 @@ private:
PODArray<const float *> float_features(column_size); PODArray<const float *> float_features(column_size);
auto float_features_buf = float_features.data(); auto float_features_buf = float_features.data();
/// Store all float data into single column. float_features is a list of pointers to it. /// Store all float data into single column. float_features is a list of pointers to it.
auto float_features_col = placeNumericColumns<float>(columns, 0, float_features_count, float_features_buf); auto float_features_col = placeNumericColumns<float>(columns, 0, float_features_count_current, float_features_buf);
if (cat_features_count == 0) if (cat_features_count_current == 0)
{ {
if (!api->CalcModelPredictionFlat(handle->get(), column_size, if (!api->CalcModelPredictionFlat(handle->get(), column_size,
float_features_buf, float_features_count, float_features_buf, float_features_count_current,
result_buf, column_size)) result_buf, column_size))
{ {
@ -352,18 +352,18 @@ private:
if (cat_features_are_strings) if (cat_features_are_strings)
{ {
/// cat_features_holder stores pointers to ColumnString data or fixed_strings_data. /// cat_features_holder stores pointers to ColumnString data or fixed_strings_data.
PODArray<const char *> cat_features_holder(cat_features_count * column_size); PODArray<const char *> cat_features_holder(cat_features_count_current * column_size);
PODArray<const char **> cat_features(column_size); PODArray<const char **> cat_features(column_size);
auto cat_features_buf = cat_features.data(); auto cat_features_buf = cat_features.data();
fillCatFeaturesBuffer(cat_features_buf, cat_features_holder.data(), column_size, cat_features_count); fillCatFeaturesBuffer(cat_features_buf, cat_features_holder.data(), column_size, cat_features_count_current);
/// Fixed strings are stored without termination zero, so have to copy data into fixed_strings_data. /// Fixed strings are stored without termination zero, so have to copy data into fixed_strings_data.
auto fixed_strings_data = placeStringColumns(columns, float_features_count, auto fixed_strings_data = placeStringColumns(columns, float_features_count_current,
cat_features_count, cat_features_holder.data()); cat_features_count_current, cat_features_holder.data());
if (!api->CalcModelPrediction(handle->get(), column_size, if (!api->CalcModelPrediction(handle->get(), column_size,
float_features_buf, float_features_count, float_features_buf, float_features_count_current,
cat_features_buf, cat_features_count, cat_features_buf, cat_features_count_current,
result_buf, column_size)) result_buf, column_size))
{ {
throw Exception(error_msg + api->GetErrorString(), ErrorCodes::CANNOT_APPLY_CATBOOST_MODEL); throw Exception(error_msg + api->GetErrorString(), ErrorCodes::CANNOT_APPLY_CATBOOST_MODEL);
@ -373,13 +373,13 @@ private:
{ {
PODArray<const int *> cat_features(column_size); PODArray<const int *> cat_features(column_size);
auto cat_features_buf = cat_features.data(); auto cat_features_buf = cat_features.data();
auto cat_features_col = placeNumericColumns<int>(columns, float_features_count, auto cat_features_col = placeNumericColumns<int>(columns, float_features_count_current,
cat_features_count, cat_features_buf); cat_features_count_current, cat_features_buf);
calcHashes(columns, float_features_count, cat_features_count, cat_features_buf); calcHashes(columns, float_features_count_current, cat_features_count_current, cat_features_buf);
if (!api->CalcModelPredictionWithHashedCatFeatures( if (!api->CalcModelPredictionWithHashedCatFeatures(
handle->get(), column_size, handle->get(), column_size,
float_features_buf, float_features_count, float_features_buf, float_features_count_current,
cat_features_buf, cat_features_count, cat_features_buf, cat_features_count_current,
result_buf, column_size)) result_buf, column_size))
{ {
throw Exception(error_msg + api->GetErrorString(), ErrorCodes::CANNOT_APPLY_CATBOOST_MODEL); throw Exception(error_msg + api->GetErrorString(), ErrorCodes::CANNOT_APPLY_CATBOOST_MODEL);
@ -453,7 +453,7 @@ CatBoostModel::CatBoostModel(std::string name_, std::string model_path_, std::st
{ {
try try
{ {
init(lib_path); init();
} }
catch (...) catch (...)
{ {
@ -463,7 +463,7 @@ CatBoostModel::CatBoostModel(std::string name_, std::string model_path_, std::st
creation_time = std::chrono::system_clock::now(); creation_time = std::chrono::system_clock::now();
} }
void CatBoostModel::init(const std::string & lib_path) void CatBoostModel::init()
{ {
api_provider = getCatBoostWrapperHolder(lib_path); api_provider = getCatBoostWrapperHolder(lib_path);
api = &api_provider->getAPI(); api = &api_provider->getAPI();

View File

@ -80,7 +80,7 @@ private:
std::chrono::time_point<std::chrono::system_clock> creation_time; std::chrono::time_point<std::chrono::system_clock> creation_time;
std::exception_ptr creation_exception; std::exception_ptr creation_exception;
void init(const std::string & lib_path); void init();
}; };
} }

View File

@ -28,12 +28,12 @@ void ComplexKeyCacheDictionary::setAttributeValue(Attribute & attribute, const s
if (string_ref.data && string_ref.data != null_value_ref.data()) if (string_ref.data && string_ref.data != null_value_ref.data())
string_arena->free(const_cast<char *>(string_ref.data), string_ref.size); string_arena->free(const_cast<char *>(string_ref.data), string_ref.size);
const auto size = string.size(); const auto str_size = string.size();
if (size != 0) if (str_size != 0)
{ {
auto string_ptr = string_arena->alloc(size + 1); auto string_ptr = string_arena->alloc(str_size + 1);
std::copy(string.data(), string.data() + size + 1, string_ptr); std::copy(string.data(), string.data() + str_size + 1, string_ptr);
string_ref = StringRef{string_ptr, size}; string_ref = StringRef{string_ptr, str_size};
} }
else else
string_ref = {}; string_ref = {};

View File

@ -223,47 +223,47 @@ Block DictionaryBlockInputStream<DictionaryType, Key>::getBlock(size_t start, si
template <typename DictionaryType, typename Key> template <typename DictionaryType, typename Key>
template <typename Type, typename Container> template <typename Type, typename Container>
void DictionaryBlockInputStream<DictionaryType, Key>::callGetter( void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
DictionaryGetter<Type> getter, const PaddedPODArray<Key> & ids, DictionaryGetter<Type> getter, const PaddedPODArray<Key> & ids_to_fill,
const Columns & /*keys*/, const DataTypes & /*data_types*/, const Columns & /*keys*/, const DataTypes & /*data_types*/,
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const Container & container, const DictionaryAttribute & attribute, const DictionaryType & dict) const
{ {
(dictionary.*getter)(attribute.name, ids, container); (dict.*getter)(attribute.name, ids_to_fill, container);
} }
template <typename DictionaryType, typename Key> template <typename DictionaryType, typename Key>
template <typename Container> template <typename Container>
void DictionaryBlockInputStream<DictionaryType, Key>::callGetter( void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
DictionaryStringGetter getter, const PaddedPODArray<Key> & ids, DictionaryStringGetter getter, const PaddedPODArray<Key> & ids_to_fill,
const Columns & /*keys*/, const DataTypes & /*data_types*/, const Columns & /*keys*/, const DataTypes & /*data_types*/,
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const Container & container, const DictionaryAttribute & attribute, const DictionaryType & dict) const
{ {
(dictionary.*getter)(attribute.name, ids, container); (dict.*getter)(attribute.name, ids_to_fill, container);
} }
template <typename DictionaryType, typename Key> template <typename DictionaryType, typename Key>
template <typename Type, typename Container> template <typename Type, typename Container>
void DictionaryBlockInputStream<DictionaryType, Key>::callGetter( void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
GetterByKey<Type> getter, const PaddedPODArray<Key> & /*ids*/, GetterByKey<Type> getter, const PaddedPODArray<Key> & /*ids_to_fill*/,
const Columns & keys, const DataTypes & data_types, const Columns & keys, const DataTypes & data_types,
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const Container & container, const DictionaryAttribute & attribute, const DictionaryType & dict) const
{ {
(dictionary.*getter)(attribute.name, keys, data_types, container); (dict.*getter)(attribute.name, keys, data_types, container);
} }
template <typename DictionaryType, typename Key> template <typename DictionaryType, typename Key>
template <typename Container> template <typename Container>
void DictionaryBlockInputStream<DictionaryType, Key>::callGetter( void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
StringGetterByKey getter, const PaddedPODArray<Key> & /*ids*/, StringGetterByKey getter, const PaddedPODArray<Key> & /*ids_to_fill*/,
const Columns & keys, const DataTypes & data_types, const Columns & keys, const DataTypes & data_types,
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const Container & container, const DictionaryAttribute & attribute, const DictionaryType & dict) const
{ {
(dictionary.*getter)(attribute.name, keys, data_types, container); (dict.*getter)(attribute.name, keys, data_types, container);
} }
template <typename DictionaryType, typename Key> template <typename DictionaryType, typename Key>
template <template <typename> class Getter, typename StringGetter> template <template <typename> class Getter, typename StringGetter>
Block DictionaryBlockInputStream<DictionaryType, Key>::fillBlock( Block DictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
const PaddedPODArray<Key> & ids, const Columns & keys, const DataTypes & types, ColumnsWithTypeAndName && view) const const PaddedPODArray<Key> & ids_to_fill, const Columns & keys, const DataTypes & types, ColumnsWithTypeAndName && view) const
{ {
std::unordered_set<std::string> names(column_names.begin(), column_names.end()); std::unordered_set<std::string> names(column_names.begin(), column_names.end());
@ -283,7 +283,7 @@ Block DictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
const DictionaryStructure & structure = dictionary->getStructure(); const DictionaryStructure & structure = dictionary->getStructure();
if (structure.id && names.find(structure.id->name) != names.end()) if (structure.id && names.find(structure.id->name) != names.end())
block_columns.emplace_back(getColumnFromIds(ids), std::make_shared<DataTypeUInt64>(), structure.id->name); block_columns.emplace_back(getColumnFromIds(ids_to_fill), std::make_shared<DataTypeUInt64>(), structure.id->name);
for (const auto idx : ext::range(0, structure.attributes.size())) for (const auto idx : ext::range(0, structure.attributes.size()))
{ {
@ -293,7 +293,7 @@ Block DictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
ColumnPtr column; ColumnPtr column;
#define GET_COLUMN_FORM_ATTRIBUTE(TYPE) \ #define GET_COLUMN_FORM_ATTRIBUTE(TYPE) \
column = getColumnFromAttribute<TYPE, Getter<TYPE>>( \ column = getColumnFromAttribute<TYPE, Getter<TYPE>>( \
&DictionaryType::get##TYPE, ids, keys, data_types, attribute, *dictionary) &DictionaryType::get##TYPE, ids_to_fill, keys, data_types, attribute, *dictionary)
switch (attribute.underlying_type) switch (attribute.underlying_type)
{ {
case AttributeUnderlyingType::UInt8: case AttributeUnderlyingType::UInt8:
@ -346,37 +346,37 @@ Block DictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
template <typename DictionaryType, typename Key> template <typename DictionaryType, typename Key>
template <typename AttributeType, typename Getter> template <typename AttributeType, typename Getter>
ColumnPtr DictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAttribute( ColumnPtr DictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAttribute(
Getter getter, const PaddedPODArray<Key> & ids, Getter getter, const PaddedPODArray<Key> & ids_to_fill,
const Columns & keys, const DataTypes & data_types, const Columns & keys, const DataTypes & data_types,
const DictionaryAttribute & attribute, const DictionaryType & dictionary) const const DictionaryAttribute & attribute, const DictionaryType & dict) const
{ {
auto size = ids.size(); auto size = ids_to_fill.size();
if (!keys.empty()) if (!keys.empty())
size = keys.front()->size(); size = keys.front()->size();
auto column_vector = ColumnVector<AttributeType>::create(size); auto column_vector = ColumnVector<AttributeType>::create(size);
callGetter(getter, ids, keys, data_types, column_vector->getData(), attribute, dictionary); callGetter(getter, ids_to_fill, keys, data_types, column_vector->getData(), attribute, dict);
return std::move(column_vector); return std::move(column_vector);
} }
template <typename DictionaryType, typename Key> template <typename DictionaryType, typename Key>
template <typename Getter> template <typename Getter>
ColumnPtr DictionaryBlockInputStream<DictionaryType, Key>::getColumnFromStringAttribute( ColumnPtr DictionaryBlockInputStream<DictionaryType, Key>::getColumnFromStringAttribute(
Getter getter, const PaddedPODArray<Key> & ids, Getter getter, const PaddedPODArray<Key> & ids_to_fill,
const Columns & keys, const DataTypes & data_types, const Columns & keys, const DataTypes & data_types,
const DictionaryAttribute& attribute, const DictionaryType& dictionary) const const DictionaryAttribute& attribute, const DictionaryType & dict) const
{ {
auto column_string = ColumnString::create(); auto column_string = ColumnString::create();
auto ptr = column_string.get(); auto ptr = column_string.get();
callGetter(getter, ids, keys, data_types, ptr, attribute, dictionary); callGetter(getter, ids_to_fill, keys, data_types, ptr, attribute, dict);
return std::move(column_string); return std::move(column_string);
} }
template <typename DictionaryType, typename Key> template <typename DictionaryType, typename Key>
ColumnPtr DictionaryBlockInputStream<DictionaryType, Key>::getColumnFromIds(const PaddedPODArray<Key> & ids) const ColumnPtr DictionaryBlockInputStream<DictionaryType, Key>::getColumnFromIds(const PaddedPODArray<Key> & ids_to_fill) const
{ {
auto column_vector = ColumnVector<UInt64>::create(); auto column_vector = ColumnVector<UInt64>::create();
column_vector->getData().reserve(ids.size()); column_vector->getData().reserve(ids_to_fill.size());
for (UInt64 id : ids) for (UInt64 id : ids_to_fill)
column_vector->insert(id); column_vector->insert(id);
return std::move(column_vector); return std::move(column_vector);
} }

View File

@ -7,7 +7,7 @@ namespace DB
class DictionaryBlockInputStreamBase : public IProfilingBlockInputStream class DictionaryBlockInputStreamBase : public IProfilingBlockInputStream
{ {
protected: protected:
Block block; //Block block;
DictionaryBlockInputStreamBase(size_t rows_count, size_t max_block_size); DictionaryBlockInputStreamBase(size_t rows_count, size_t max_block_size);

View File

@ -20,10 +20,7 @@
#include <Dictionaries/MongoDBDictionarySource.h> #include <Dictionaries/MongoDBDictionarySource.h>
#endif #endif
#if USE_POCO_SQLODBC || USE_POCO_DATAODBC #if USE_POCO_SQLODBC || USE_POCO_DATAODBC
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wunused-parameter"
#include <Poco/Data/ODBC/Connector.h> #include <Poco/Data/ODBC/Connector.h>
#pragma GCC diagnostic pop
#include <Dictionaries/ODBCDictionarySource.h> #include <Dictionaries/ODBCDictionarySource.h>
#endif #endif
#if USE_MYSQL #if USE_MYSQL

View File

@ -240,22 +240,22 @@ std::vector<DictionaryAttribute> DictionaryStructure::getAttributes(
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix,
const bool hierarchy_allowed, const bool allow_null_values) const bool hierarchy_allowed, const bool allow_null_values)
{ {
Poco::Util::AbstractConfiguration::Keys keys; Poco::Util::AbstractConfiguration::Keys config_elems;
config.keys(config_prefix, keys); config.keys(config_prefix, config_elems);
auto has_hierarchy = false; auto has_hierarchy = false;
std::vector<DictionaryAttribute> attributes; std::vector<DictionaryAttribute> res_attributes;
const FormatSettings format_settings; const FormatSettings format_settings;
for (const auto & key : keys) for (const auto & config_elem : config_elems)
{ {
if (!startsWith(key.data(), "attribute")) if (!startsWith(config_elem.data(), "attribute"))
continue; continue;
const auto prefix = config_prefix + '.' + key + '.'; const auto prefix = config_prefix + '.' + config_elem + '.';
Poco::Util::AbstractConfiguration::Keys attribute_keys; Poco::Util::AbstractConfiguration::Keys attribute_keys;
config.keys(config_prefix + '.' + key, attribute_keys); config.keys(config_prefix + '.' + config_elem, attribute_keys);
checkAttributeKeys(attribute_keys); checkAttributeKeys(attribute_keys);
@ -300,12 +300,12 @@ std::vector<DictionaryAttribute> DictionaryStructure::getAttributes(
has_hierarchy = has_hierarchy || hierarchical; has_hierarchy = has_hierarchy || hierarchical;
attributes.emplace_back(DictionaryAttribute{ res_attributes.emplace_back(DictionaryAttribute{
name, underlying_type, type, expression, null_value, hierarchical, injective, is_object_id name, underlying_type, type, expression, null_value, hierarchical, injective, is_object_id
}); });
} }
return attributes; return res_attributes;
} }
} }

View File

@ -5,13 +5,10 @@
#include <string> #include <string>
#include <sstream> #include <sstream>
#pragma GCC diagnostic push #include <Poco/MongoDB/Connection.h>
#pragma GCC diagnostic ignored "-Wunused-parameter" #include <Poco/MongoDB/Cursor.h>
#include <Poco/MongoDB/Connection.h> #include <Poco/MongoDB/Element.h>
#include <Poco/MongoDB/Cursor.h> #include <Poco/MongoDB/ObjectId.h>
#include <Poco/MongoDB/Element.h>
#include <Poco/MongoDB/ObjectId.h>
#pragma GCC diagnostic pop
#include <Dictionaries/DictionaryStructure.h> #include <Dictionaries/DictionaryStructure.h>
#include <Dictionaries/MongoDBBlockInputStream.h> #include <Dictionaries/MongoDBBlockInputStream.h>

View File

@ -2,14 +2,11 @@
#if USE_POCO_MONGODB #if USE_POCO_MONGODB
#include <Poco/Util/AbstractConfiguration.h> #include <Poco/Util/AbstractConfiguration.h>
#pragma GCC diagnostic push #include <Poco/MongoDB/Connection.h>
#pragma GCC diagnostic ignored "-Wunused-parameter" #include <Poco/MongoDB/Database.h>
#include <Poco/MongoDB/Connection.h> #include <Poco/MongoDB/Cursor.h>
#include <Poco/MongoDB/Database.h> #include <Poco/MongoDB/Array.h>
#include <Poco/MongoDB/Cursor.h> #include <Poco/MongoDB/ObjectId.h>
#include <Poco/MongoDB/Array.h>
#include <Poco/MongoDB/ObjectId.h>
#pragma GCC diagnostic pop
#include <Poco/Version.h> #include <Poco/Version.h>

View File

@ -167,10 +167,10 @@ std::string MySQLDictionarySource::quoteForLike(const std::string s)
LocalDateTime MySQLDictionarySource::getLastModification() const LocalDateTime MySQLDictionarySource::getLastModification() const
{ {
LocalDateTime update_time{std::time(nullptr)}; LocalDateTime modification_time{std::time(nullptr)};
if (dont_check_update_time) if (dont_check_update_time)
return update_time; return modification_time;
try try
{ {
@ -190,8 +190,8 @@ LocalDateTime MySQLDictionarySource::getLastModification() const
if (!update_time_value.isNull()) if (!update_time_value.isNull())
{ {
update_time = update_time_value.getDateTime(); modification_time = update_time_value.getDateTime();
LOG_TRACE(log, "Got update time: " << update_time); LOG_TRACE(log, "Got modification time: " << modification_time);
} }
/// fetch remaining rows to avoid "commands out of sync" error /// fetch remaining rows to avoid "commands out of sync" error
@ -211,15 +211,15 @@ LocalDateTime MySQLDictionarySource::getLastModification() const
} }
/// we suppose failure to get modification time is not an error, therefore return current time /// we suppose failure to get modification time is not an error, therefore return current time
return update_time; return modification_time;
} }
std::string MySQLDictionarySource::doInvalidateQuery(const std::string & request) const std::string MySQLDictionarySource::doInvalidateQuery(const std::string & request) const
{ {
Block sample_block; Block invalidate_sample_block;
ColumnPtr column(ColumnString::create()); ColumnPtr column(ColumnString::create());
sample_block.insert(ColumnWithTypeAndName(column, std::make_shared<DataTypeString>(), "Sample Block")); invalidate_sample_block.insert(ColumnWithTypeAndName(column, std::make_shared<DataTypeString>(), "Sample Block"));
MySQLBlockInputStream block_input_stream(pool.Get(), request, sample_block, 1); MySQLBlockInputStream block_input_stream(pool.Get(), request, invalidate_sample_block, 1);
return readInvalidateQuery(block_input_stream); return readInvalidateQuery(block_input_stream);
} }

View File

@ -4,12 +4,9 @@
#include <DataStreams/IProfilingBlockInputStream.h> #include <DataStreams/IProfilingBlockInputStream.h>
#include <Dictionaries/ExternalResultDescription.h> #include <Dictionaries/ExternalResultDescription.h>
#pragma GCC diagnostic push #include <Poco/Data/Session.h>
#pragma GCC diagnostic ignored "-Wunused-parameter" #include <Poco/Data/Statement.h>
#include <Poco/Data/Session.h> #include <Poco/Data/RecordSet.h>
#include <Poco/Data/Statement.h>
#include <Poco/Data/RecordSet.h>
#pragma GCC diagnostic pop
#include <string> #include <string>

View File

@ -145,10 +145,10 @@ bool ODBCDictionarySource::isModified() const
std::string ODBCDictionarySource::doInvalidateQuery(const std::string & request) const std::string ODBCDictionarySource::doInvalidateQuery(const std::string & request) const
{ {
Block sample_block; Block invalidate_sample_block;
ColumnPtr column(ColumnString::create()); ColumnPtr column(ColumnString::create());
sample_block.insert(ColumnWithTypeAndName(column, std::make_shared<DataTypeString>(), "Sample Block")); invalidate_sample_block.insert(ColumnWithTypeAndName(column, std::make_shared<DataTypeString>(), "Sample Block"));
ODBCBlockInputStream block_input_stream(pool->get(), request, sample_block, 1); ODBCBlockInputStream block_input_stream(pool->get(), request, invalidate_sample_block, 1);
return readInvalidateQuery(block_input_stream); return readInvalidateQuery(block_input_stream);
} }

View File

@ -1,13 +1,12 @@
#pragma once #pragma once
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wunused-parameter"
#include <Poco/Data/SessionPool.h> #include <Poco/Data/SessionPool.h>
#pragma GCC diagnostic pop
#include <Dictionaries/IDictionarySource.h> #include <Dictionaries/IDictionarySource.h>
#include <Dictionaries/ExternalQueryBuilder.h> #include <Dictionaries/ExternalQueryBuilder.h>
#include <Dictionaries/DictionaryStructure.h> #include <Dictionaries/DictionaryStructure.h>
namespace Poco namespace Poco
{ {
namespace Util namespace Util

View File

@ -189,22 +189,22 @@ std::string validateODBCConnectionString(const std::string & connection_string)
{ {
reconstructed_connection_string += '{'; reconstructed_connection_string += '{';
const char * pos = value.data(); const char * value_pos = value.data();
const char * end = pos + value.size(); const char * value_end = value_pos + value.size();
while (true) while (true)
{ {
const char * next_pos = find_first_symbols<'}'>(pos, end); const char * next_pos = find_first_symbols<'}'>(value_pos, value_end);
if (next_pos == end) if (next_pos == value_end)
{ {
reconstructed_connection_string.append(pos, next_pos - pos); reconstructed_connection_string.append(value_pos, next_pos - value_pos);
break; break;
} }
else else
{ {
reconstructed_connection_string.append(pos, next_pos - pos); reconstructed_connection_string.append(value_pos, next_pos - value_pos);
reconstructed_connection_string.append("}}"); reconstructed_connection_string.append("}}");
pos = next_pos + 1; value_pos = next_pos + 1;
} }
} }

View File

@ -16,18 +16,18 @@
*/ */
#if USE_VECTORCLASS #if USE_VECTORCLASS
#ifdef __clang__ #ifdef __clang__
#pragma clang diagnostic push #pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wshift-negative-value" #pragma clang diagnostic ignored "-Wshift-negative-value"
#endif #endif
#include <vectorf128.h> // Y_IGNORE #include <vectorf128.h> // Y_IGNORE
#include <vectormath_exp.h> // Y_IGNORE #include <vectormath_exp.h> // Y_IGNORE
#include <vectormath_trig.h> // Y_IGNORE #include <vectormath_trig.h> // Y_IGNORE
#ifdef __clang__ #ifdef __clang__
#pragma clang diagnostic pop #pragma clang diagnostic pop
#endif #endif
#endif #endif

View File

@ -44,7 +44,7 @@ public:
return curr_buffer; return curr_buffer;
} }
~CascadeWriteBuffer(); ~CascadeWriteBuffer() override;
private: private:

View File

@ -31,7 +31,7 @@ public:
const Poco::Timespan & send_timeout = Poco::Timespan(DEFAULT_REMOTE_WRITE_BUFFER_SEND_TIMEOUT, 0), const Poco::Timespan & send_timeout = Poco::Timespan(DEFAULT_REMOTE_WRITE_BUFFER_SEND_TIMEOUT, 0),
const Poco::Timespan & receive_timeout = Poco::Timespan(DEFAULT_REMOTE_WRITE_BUFFER_RECEIVE_TIMEOUT, 0)); const Poco::Timespan & receive_timeout = Poco::Timespan(DEFAULT_REMOTE_WRITE_BUFFER_RECEIVE_TIMEOUT, 0));
~InterserverWriteBuffer(); ~InterserverWriteBuffer() override;
void finalize(); void finalize();
void cancel(); void cancel();

View File

@ -206,10 +206,10 @@ inline void copyOverlap8Shuffle(UInt8 * op, const UInt8 *& match, const size_t o
template <> void inline copy<8>(UInt8 * dst, const UInt8 * src) { copy8(dst, src); }; template <> void inline copy<8>(UInt8 * dst, const UInt8 * src) { copy8(dst, src); }
template <> void inline wildCopy<8>(UInt8 * dst, const UInt8 * src, UInt8 * dst_end) { wildCopy8(dst, src, dst_end); }; template <> void inline wildCopy<8>(UInt8 * dst, const UInt8 * src, UInt8 * dst_end) { wildCopy8(dst, src, dst_end); }
template <> void inline copyOverlap<8, false>(UInt8 * op, const UInt8 *& match, const size_t offset) { copyOverlap8(op, match, offset); }; template <> void inline copyOverlap<8, false>(UInt8 * op, const UInt8 *& match, const size_t offset) { copyOverlap8(op, match, offset); }
template <> void inline copyOverlap<8, true>(UInt8 * op, const UInt8 *& match, const size_t offset) { copyOverlap8Shuffle(op, match, offset); }; template <> void inline copyOverlap<8, true>(UInt8 * op, const UInt8 *& match, const size_t offset) { copyOverlap8Shuffle(op, match, offset); }
inline void copy16(UInt8 * dst, const UInt8 * src) inline void copy16(UInt8 * dst, const UInt8 * src)
@ -337,10 +337,10 @@ inline void copyOverlap16Shuffle(UInt8 * op, const UInt8 *& match, const size_t
#endif #endif
template <> void inline copy<16>(UInt8 * dst, const UInt8 * src) { copy16(dst, src); }; template <> void inline copy<16>(UInt8 * dst, const UInt8 * src) { copy16(dst, src); }
template <> void inline wildCopy<16>(UInt8 * dst, const UInt8 * src, UInt8 * dst_end) { wildCopy16(dst, src, dst_end); }; template <> void inline wildCopy<16>(UInt8 * dst, const UInt8 * src, UInt8 * dst_end) { wildCopy16(dst, src, dst_end); }
template <> void inline copyOverlap<16, false>(UInt8 * op, const UInt8 *& match, const size_t offset) { copyOverlap16(op, match, offset); }; template <> void inline copyOverlap<16, false>(UInt8 * op, const UInt8 *& match, const size_t offset) { copyOverlap16(op, match, offset); }
template <> void inline copyOverlap<16, true>(UInt8 * op, const UInt8 *& match, const size_t offset) { copyOverlap16Shuffle(op, match, offset); }; template <> void inline copyOverlap<16, true>(UInt8 * op, const UInt8 *& match, const size_t offset) { copyOverlap16Shuffle(op, match, offset); }
/// See also https://stackoverflow.com/a/30669632 /// See also https://stackoverflow.com/a/30669632

View File

@ -75,7 +75,7 @@ struct PerformanceStatistics
sum += seconds / bytes; sum += seconds / bytes;
} }
double sample(pcg64 & rng) const double sample(pcg64 & stat_rng) const
{ {
/// If there is a variant with not enough statistics, always choose it. /// If there is a variant with not enough statistics, always choose it.
/// And in that case prefer variant with less number of invocations. /// And in that case prefer variant with less number of invocations.
@ -83,7 +83,7 @@ struct PerformanceStatistics
if (adjustedCount() < 2) if (adjustedCount() < 2)
return adjustedCount() - 1; return adjustedCount() - 1;
else else
return std::normal_distribution<>(mean(), sigma())(rng); return std::normal_distribution<>(mean(), sigma())(stat_rng);
} }
}; };

View File

@ -14,7 +14,7 @@ namespace DB
class MMapReadBufferFromFileDescriptor : public ReadBuffer class MMapReadBufferFromFileDescriptor : public ReadBuffer
{ {
protected: protected:
MMapReadBufferFromFileDescriptor() : ReadBuffer(nullptr, 0) {}; MMapReadBufferFromFileDescriptor() : ReadBuffer(nullptr, 0) {}
void init(int fd_, size_t offset, size_t length_); void init(int fd_, size_t offset, size_t length_);
void init(int fd_, size_t offset); void init(int fd_, size_t offset);

View File

@ -33,7 +33,7 @@ public:
return setChunk(); return setChunk();
} }
~ReadBufferFromMemoryWriteBuffer() ~ReadBufferFromMemoryWriteBuffer() override
{ {
for (const auto & range : chunk_list) for (const auto & range : chunk_list)
free(range.begin(), range.size()); free(range.begin(), range.size());

View File

@ -133,7 +133,7 @@ bool ReadBufferFromFileDescriptor::poll(size_t timeout_microseconds)
FD_SET(fd, &fds); FD_SET(fd, &fds);
timeval timeout = { time_t(timeout_microseconds / 1000000), suseconds_t(timeout_microseconds % 1000000) }; timeval timeout = { time_t(timeout_microseconds / 1000000), suseconds_t(timeout_microseconds % 1000000) };
int res = select(1, &fds, 0, 0, &timeout); int res = select(1, &fds, nullptr, nullptr, &timeout);
if (-1 == res) if (-1 == res)
throwFromErrno("Cannot select", ErrorCodes::CANNOT_SELECT); throwFromErrno("Cannot select", ErrorCodes::CANNOT_SELECT);

View File

@ -30,7 +30,7 @@ bool ReadBufferFromPocoSocket::nextImpl()
{ {
throw NetException(e.displayText(), "while reading from socket (" + peer_address.toString() + ")", ErrorCodes::NETWORK_ERROR); throw NetException(e.displayText(), "while reading from socket (" + peer_address.toString() + ")", ErrorCodes::NETWORK_ERROR);
} }
catch (const Poco::TimeoutException & e) catch (const Poco::TimeoutException &)
{ {
throw NetException("Timeout exceeded while reading from socket (" + peer_address.toString() + ")", ErrorCodes::SOCKET_TIMEOUT); throw NetException("Timeout exceeded while reading from socket (" + peer_address.toString() + ")", ErrorCodes::SOCKET_TIMEOUT);
} }

View File

@ -128,7 +128,7 @@ public:
send_progress_interval_ms = send_progress_interval_ms_; send_progress_interval_ms = send_progress_interval_ms_;
} }
~WriteBufferFromHTTPServerResponse(); ~WriteBufferFromHTTPServerResponse() override;
}; };
} }

View File

@ -36,7 +36,7 @@ void WriteBufferFromPocoSocket::nextImpl()
{ {
throw NetException(e.displayText() + " while writing to socket (" + peer_address.toString() + ")", ErrorCodes::NETWORK_ERROR); throw NetException(e.displayText() + " while writing to socket (" + peer_address.toString() + ")", ErrorCodes::NETWORK_ERROR);
} }
catch (const Poco::TimeoutException & e) catch (const Poco::TimeoutException &)
{ {
throw NetException("Timeout exceeded while writing to socket (" + peer_address.toString() + ")", ErrorCodes::SOCKET_TIMEOUT); throw NetException("Timeout exceeded while writing to socket (" + peer_address.toString() + ")", ErrorCodes::SOCKET_TIMEOUT);
} }

View File

@ -419,12 +419,12 @@ inline void writeProbablyQuotedStringImpl(const String & s, WriteBuffer & buf, F
inline void writeProbablyBackQuotedString(const String & s, WriteBuffer & buf) inline void writeProbablyBackQuotedString(const String & s, WriteBuffer & buf)
{ {
writeProbablyQuotedStringImpl(s, buf, [](const String & s, WriteBuffer & buf) { return writeBackQuotedString(s, buf); }); writeProbablyQuotedStringImpl(s, buf, [](const String & s_, WriteBuffer & buf_) { return writeBackQuotedString(s_, buf_); });
} }
inline void writeProbablyDoubleQuotedString(const String & s, WriteBuffer & buf) inline void writeProbablyDoubleQuotedString(const String & s, WriteBuffer & buf)
{ {
writeProbablyQuotedStringImpl(s, buf, [](const String & s, WriteBuffer & buf) { return writeDoubleQuotedString(s, buf); }); writeProbablyQuotedStringImpl(s, buf, [](const String & s_, WriteBuffer & buf_) { return writeDoubleQuotedString(s_, buf_); });
} }

View File

@ -17,9 +17,9 @@ ZlibDeflatingWriteBuffer::ZlibDeflatingWriteBuffer(
zstr.zalloc = Z_NULL; zstr.zalloc = Z_NULL;
zstr.zfree = Z_NULL; zstr.zfree = Z_NULL;
zstr.opaque = Z_NULL; zstr.opaque = Z_NULL;
zstr.next_in = 0; zstr.next_in = nullptr;
zstr.avail_in = 0; zstr.avail_in = 0;
zstr.next_out = 0; zstr.next_out = nullptr;
zstr.avail_out = 0; zstr.avail_out = 0;
int window_bits = 15; int window_bits = 15;

View File

@ -17,9 +17,9 @@ ZlibInflatingReadBuffer::ZlibInflatingReadBuffer(
zstr.zalloc = Z_NULL; zstr.zalloc = Z_NULL;
zstr.zfree = Z_NULL; zstr.zfree = Z_NULL;
zstr.opaque = Z_NULL; zstr.opaque = Z_NULL;
zstr.next_in = 0; zstr.next_in = nullptr;
zstr.avail_in = 0; zstr.avail_in = 0;
zstr.next_out = 0; zstr.next_out = nullptr;
zstr.avail_out = 0; zstr.avail_out = 0;
int window_bits = 15; int window_bits = 15;

View File

@ -1601,7 +1601,7 @@ public:
Block getHeader() const override { return aggregator.getHeader(final); } Block getHeader() const override { return aggregator.getHeader(final); }
~MergingAndConvertingBlockInputStream() ~MergingAndConvertingBlockInputStream() override
{ {
LOG_TRACE(&Logger::get(__PRETTY_FUNCTION__), "Waiting for threads to finish"); LOG_TRACE(&Logger::get(__PRETTY_FUNCTION__), "Waiting for threads to finish");

View File

@ -88,7 +88,7 @@ struct HostID
{ {
return DB::isLocalAddress(DNSResolver::instance().resolveAddress(host_name, port), clickhouse_port); return DB::isLocalAddress(DNSResolver::instance().resolveAddress(host_name, port), clickhouse_port);
} }
catch (const Poco::Net::NetException & e) catch (const Poco::Net::NetException &)
{ {
/// Avoid "Host not found" exceptions /// Avoid "Host not found" exceptions
return false; return false;
@ -578,7 +578,7 @@ void DDLWorker::processTask(DDLTask & task)
tryExecuteQuery(rewritten_query, task, task.execution_status); tryExecuteQuery(rewritten_query, task, task.execution_status);
} }
} }
catch (const zkutil::KeeperException & e) catch (const zkutil::KeeperException &)
{ {
throw; throw;
} }

View File

@ -38,11 +38,11 @@ static bool isNetworkError()
if (e.code() == ErrorCodes::TIMEOUT_EXCEEDED || e.code() == ErrorCodes::ALL_CONNECTION_TRIES_FAILED) if (e.code() == ErrorCodes::TIMEOUT_EXCEEDED || e.code() == ErrorCodes::ALL_CONNECTION_TRIES_FAILED)
return true; return true;
} }
catch (Poco::Net::DNSException & e) catch (Poco::Net::DNSException &)
{ {
return true; return true;
} }
catch (Poco::TimeoutException & e) catch (Poco::TimeoutException &)
{ {
return true; return true;
} }

View File

@ -127,7 +127,7 @@ DictionaryPtr DictionaryFactory::create(const std::string & name, const Poco::Ut
throw Exception{name + ": unknown dictionary layout type: " + layout_type, throw Exception{name + ": unknown dictionary layout type: " + layout_type,
ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG}; ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG};
}; }
} }

View File

@ -326,7 +326,7 @@ static DatabaseAndTableWithAlias getTableNameWithAliasFromTableExpression(const
throw Exception("Logical error: no known elements in ASTTableExpression", ErrorCodes::LOGICAL_ERROR); throw Exception("Logical error: no known elements in ASTTableExpression", ErrorCodes::LOGICAL_ERROR);
return database_and_table_with_alias; return database_and_table_with_alias;
}; }
void ExpressionAnalyzer::translateQualifiedNames() void ExpressionAnalyzer::translateQualifiedNames()

View File

@ -24,7 +24,7 @@ public:
size_t subquery_depth_ = 0, size_t subquery_depth_ = 0,
bool only_analyze = false); bool only_analyze = false);
~InterpreterSelectWithUnionQuery(); ~InterpreterSelectWithUnionQuery() override;
BlockIO execute() override; BlockIO execute() override;

View File

@ -18,7 +18,7 @@ add_executable (hash_map3 hash_map3.cpp)
target_link_libraries (hash_map3 dbms ${FARMHASH_LIBRARIES} ${METROHASH_LIBRARIES}) target_link_libraries (hash_map3 dbms ${FARMHASH_LIBRARIES} ${METROHASH_LIBRARIES})
add_executable (hash_map_string hash_map_string.cpp) add_executable (hash_map_string hash_map_string.cpp)
target_include_directories (hash_map_string BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR}) target_include_directories (hash_map_string SYSTEM BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR})
target_link_libraries (hash_map_string dbms) target_link_libraries (hash_map_string dbms)
add_executable (hash_map_string_2 hash_map_string_2.cpp) add_executable (hash_map_string_2 hash_map_string_2.cpp)

View File

@ -41,7 +41,7 @@ inline ASTPtr setAlias(ASTPtr ast, const String & alias)
{ {
ast->setAlias(alias); ast->setAlias(alias);
return ast; return ast;
}; }
} }

View File

@ -67,5 +67,5 @@ struct StringRangePointersEqualTo
} }
}; };
}; }

View File

@ -135,7 +135,7 @@ public:
LOG_TRACE(log, "Row delimiter is: " << row_delimiter); LOG_TRACE(log, "Row delimiter is: " << row_delimiter);
} }
~ReadBufferFromKafkaConsumer() { reset(); } ~ReadBufferFromKafkaConsumer() override { reset(); }
/// Commit messages read with this consumer /// Commit messages read with this consumer
void commit() void commit()
@ -144,7 +144,7 @@ public:
if (read_messages == 0) if (read_messages == 0)
return; return;
auto err = rd_kafka_commit(consumer, NULL, 1 /* async */); auto err = rd_kafka_commit(consumer, nullptr, 1 /* async */);
if (err) if (err)
throw Exception("Failed to commit offsets: " + String(rd_kafka_err2str(err)), ErrorCodes::UNKNOWN_EXCEPTION); throw Exception("Failed to commit offsets: " + String(rd_kafka_err2str(err)), ErrorCodes::UNKNOWN_EXCEPTION);
@ -199,7 +199,7 @@ public:
return reader->read(); return reader->read();
} }
Block getHeader() const override { return reader->getHeader(); }; Block getHeader() const override { return reader->getHeader(); }
void readPrefixImpl() override void readPrefixImpl() override
{ {

View File

@ -125,7 +125,7 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo
part->checksums.checkEqual(data_checksums, false); part->checksums.checkEqual(data_checksums, false);
} }
catch (const NetException & e) catch (const NetException &)
{ {
/// Network error or error on remote side. No need to enqueue part for check. /// Network error or error on remote side. No need to enqueue part for check.
throw; throw;

View File

@ -6,7 +6,7 @@
namespace DB namespace DB
{ {
STRONG_TYPEDEF(UInt32, MergeTreeDataFormatVersion); STRONG_TYPEDEF(UInt32, MergeTreeDataFormatVersion)
const MergeTreeDataFormatVersion MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING {1}; const MergeTreeDataFormatVersion MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING {1};

View File

@ -339,7 +339,7 @@ void MergeTreeDataPart::remove() const
{ {
from_dir.renameTo(to); from_dir.renameTo(to);
} }
catch (const Poco::FileNotFoundException & e) catch (const Poco::FileNotFoundException &)
{ {
LOG_ERROR(storage.log, "Directory " << from << " (part to remove) doesn't exist or one of nested files has gone." LOG_ERROR(storage.log, "Directory " << from << " (part to remove) doesn't exist or one of nested files has gone."
" Most likely this is due to manual removing. This should be discouraged. Ignoring."); " Most likely this is due to manual removing. This should be discouraged. Ignoring.");

View File

@ -323,7 +323,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts(
if (relative_sample_size == RelativeSize(1)) if (relative_sample_size == RelativeSize(1))
relative_sample_size = 0; relative_sample_size = 0;
if (relative_sample_offset > 0 && 0 == relative_sample_size) if (relative_sample_offset > 0 && RelativeSize(0) == relative_sample_size)
throw Exception("Sampling offset is incorrect because no sampling", ErrorCodes::ARGUMENT_OUT_OF_BOUND); throw Exception("Sampling offset is incorrect because no sampling", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
if (relative_sample_offset > 1) if (relative_sample_offset > 1)
@ -374,7 +374,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts(
if (!data.sampling_expression) if (!data.sampling_expression)
throw Exception("Illegal SAMPLE: table doesn't support sampling", ErrorCodes::SAMPLING_NOT_SUPPORTED); throw Exception("Illegal SAMPLE: table doesn't support sampling", ErrorCodes::SAMPLING_NOT_SUPPORTED);
if (sample_factor_column_queried && relative_sample_size != 0) if (sample_factor_column_queried && relative_sample_size != RelativeSize(0))
used_sample_factor = 1.0 / boost::rational_cast<Float64>(relative_sample_size); used_sample_factor = 1.0 / boost::rational_cast<Float64>(relative_sample_size);
RelativeSize size_of_universum = 0; RelativeSize size_of_universum = 0;

View File

@ -43,7 +43,7 @@ Block MergeTreeThreadBlockInputStream::getHeader() const
auto res = pool->getHeader(); auto res = pool->getHeader();
injectVirtualColumns(res); injectVirtualColumns(res);
return res; return res;
}; }
/// Requests read task from MergeTreeReadPool and signals whether it got one /// Requests read task from MergeTreeReadPool and signals whether it got one

View File

@ -231,7 +231,7 @@ void ReplicatedMergeTreePartCheckThread::checkPart(const String & part_name)
LOG_INFO(log, "Part " << part_name << " looks good."); LOG_INFO(log, "Part " << part_name << " looks good.");
} }
catch (const Exception & e) catch (const Exception &)
{ {
/// TODO Better to check error code. /// TODO Better to check error code.

View File

@ -114,7 +114,7 @@ void ReplicatedMergeTreeRestartingThread::run()
{ {
storage.setZooKeeper(storage.context.getZooKeeper()); storage.setZooKeeper(storage.context.getZooKeeper());
} }
catch (const zkutil::KeeperException & e) catch (const zkutil::KeeperException &)
{ {
/// The exception when you try to zookeeper_init usually happens if DNS does not work. We will try to do it again. /// The exception when you try to zookeeper_init usually happens if DNS does not work. We will try to do it again.
tryLogCurrentException(log, __PRETTY_FUNCTION__); tryLogCurrentException(log, __PRETTY_FUNCTION__);

View File

@ -36,7 +36,7 @@ namespace ErrorCodes
extern const int INCORRECT_FILE_NAME; extern const int INCORRECT_FILE_NAME;
extern const int FILE_DOESNT_EXIST; extern const int FILE_DOESNT_EXIST;
extern const int EMPTY_LIST_OF_COLUMNS_PASSED; extern const int EMPTY_LIST_OF_COLUMNS_PASSED;
}; }
static std::string getTablePath(const std::string & db_dir_path, const std::string & table_name, const std::string & format_name) static std::string getTablePath(const std::string & db_dir_path, const std::string & table_name, const std::string & format_name)

View File

@ -50,7 +50,7 @@ void StorageJoin::truncate(const ASTPtr &)
increment = 0; increment = 0;
join = std::make_shared<Join>(key_names, key_names, NameSet(), false /* use_nulls */, SizeLimits(), kind, strictness); join = std::make_shared<Join>(key_names, key_names, NameSet(), false /* use_nulls */, SizeLimits(), kind, strictness);
join->setSampleBlock(getSampleBlock().sortColumns()); join->setSampleBlock(getSampleBlock().sortColumns());
}; }
void StorageJoin::assertCompatible(ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_) const void StorageJoin::assertCompatible(ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_) const
@ -62,7 +62,7 @@ void StorageJoin::assertCompatible(ASTTableJoin::Kind kind_, ASTTableJoin::Stric
void StorageJoin::insertBlock(const Block & block) { join->insertFromBlock(block); } void StorageJoin::insertBlock(const Block & block) { join->insertFromBlock(block); }
size_t StorageJoin::getSize() const { return join->getTotalRowCount(); }; size_t StorageJoin::getSize() const { return join->getTotalRowCount(); }
void registerStorageJoin(StorageFactory & factory) void registerStorageJoin(StorageFactory & factory)

View File

@ -70,7 +70,7 @@ public:
res.insert({ name_type.type->createColumn(), name_type.type, name_type.name }); res.insert({ name_type.type->createColumn(), name_type.type, name_type.name });
return Nested::flatten(res); return Nested::flatten(res);
}; }
protected: protected:
Block readImpl() override; Block readImpl() override;

View File

@ -645,7 +645,7 @@ Int64 StorageMergeTree::getCurrentMutationVersion(
return 0; return 0;
--it; --it;
return it->first; return it->first;
}; }
void StorageMergeTree::clearOldMutations() void StorageMergeTree::clearOldMutations()
{ {

View File

@ -2,12 +2,7 @@
#include <ext/shared_ptr_helper.h> #include <ext/shared_ptr_helper.h>
#include <Storages/IStorage.h> #include <Storages/IStorage.h>
#include <Poco/Data/SessionPool.h>
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wunused-parameter"
#include <Poco/Data/SessionPool.h>
#pragma GCC diagnostic pop
namespace DB namespace DB

View File

@ -1822,7 +1822,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry)
{ {
src_data = data.checkStructureAndGetMergeTreeData(source_table); src_data = data.checkStructureAndGetMergeTreeData(source_table);
} }
catch (Exception & e) catch (Exception &)
{ {
LOG_INFO(log, "Can't use " << source_table_name << " as source table for REPLACE PARTITION command. Will fetch all parts." LOG_INFO(log, "Can't use " << source_table_name << " as source table for REPLACE PARTITION command. Will fetch all parts."
<< " Reason: " << getCurrentExceptionMessage(false)); << " Reason: " << getCurrentExceptionMessage(false));

View File

@ -131,7 +131,7 @@ void StorageSet::truncate(const ASTPtr &)
increment = 0; increment = 0;
set = std::make_shared<Set>(SizeLimits(), false); set = std::make_shared<Set>(SizeLimits(), false);
set->setHeader(header); set->setHeader(header);
}; }
void StorageSetOrJoinBase::restore() void StorageSetOrJoinBase::restore()

View File

@ -73,7 +73,7 @@ public:
Block getHeader() const override Block getHeader() const override
{ {
return header; return header;
}; }
protected: protected:
Block readImpl() override Block readImpl() override
@ -143,7 +143,7 @@ public:
{ {
} }
~StripeLogBlockOutputStream() ~StripeLogBlockOutputStream() override
{ {
try try
{ {

View File

@ -69,7 +69,7 @@ public:
res.insert({ name_type.type->createColumn(), name_type.type, name_type.name }); res.insert({ name_type.type->createColumn(), name_type.type, name_type.name });
return Nested::flatten(res); return Nested::flatten(res);
}; }
protected: protected:
Block readImpl() override; Block readImpl() override;

View File

@ -22,7 +22,7 @@ namespace DB
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}; }
StorageURL::StorageURL(const Poco::URI & uri_, StorageURL::StorageURL(const Poco::URI & uri_,
const std::string & table_name_, const std::string & table_name_,

View File

@ -94,10 +94,10 @@ void StorageSystemColumns::fillData(MutableColumns & res_columns, const Context
/// We compose the result. /// We compose the result.
size_t rows = filtered_database_column->size(); size_t rows = filtered_database_column->size();
for (size_t i = 0; i < rows; ++i) for (size_t row_no = 0; row_no < rows; ++row_no)
{ {
const std::string database_name = (*filtered_database_column)[i].get<std::string>(); const std::string database_name = (*filtered_database_column)[row_no].get<std::string>();
const std::string table_name = (*filtered_table_column)[i].get<std::string>(); const std::string table_name = (*filtered_table_column)[row_no].get<std::string>();
NamesAndTypesList columns; NamesAndTypesList columns;
ColumnDefaults column_defaults; ColumnDefaults column_defaults;
@ -130,13 +130,13 @@ void StorageSystemColumns::fillData(MutableColumns & res_columns, const Context
/** Info about sizes of columns for tables of MergeTree family. /** Info about sizes of columns for tables of MergeTree family.
* NOTE: It is possible to add getter for this info to IStorage interface. * NOTE: It is possible to add getter for this info to IStorage interface.
*/ */
if (auto storage_concrete = dynamic_cast<StorageMergeTree *>(storage.get())) if (auto storage_concrete_plain = dynamic_cast<StorageMergeTree *>(storage.get()))
{ {
column_sizes = storage_concrete->getData().getColumnSizes(); column_sizes = storage_concrete_plain->getData().getColumnSizes();
} }
else if (auto storage_concrete = dynamic_cast<StorageReplicatedMergeTree *>(storage.get())) else if (auto storage_concrete_replicated = dynamic_cast<StorageReplicatedMergeTree *>(storage.get()))
{ {
column_sizes = storage_concrete->getData().getColumnSizes(); column_sizes = storage_concrete_replicated->getData().getColumnSizes();
} }
} }

View File

@ -16,13 +16,9 @@
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <Core/Defines.h> #include <Core/Defines.h>
#pragma GCC diagnostic push #include <Poco/Data/ODBC/ODBCException.h>
#pragma GCC diagnostic ignored "-Wunused-parameter" #include <Poco/Data/ODBC/SessionImpl.h>
#pragma GCC diagnostic ignored "-Wsign-compare" #include <Poco/Data/ODBC/Utility.h>
#include <Poco/Data/ODBC/ODBCException.h>
#include <Poco/Data/ODBC/SessionImpl.h>
#include <Poco/Data/ODBC/Utility.h>
#pragma GCC diagnostic pop
namespace DB namespace DB

View File

@ -15,7 +15,7 @@
#define DATE_LUT_YEARS (1 + DATE_LUT_MAX_YEAR - DATE_LUT_MIN_YEAR) /// Number of years in lookup table #define DATE_LUT_YEARS (1 + DATE_LUT_MAX_YEAR - DATE_LUT_MIN_YEAR) /// Number of years in lookup table
STRONG_TYPEDEF(UInt16, DayNum); STRONG_TYPEDEF(UInt16, DayNum)
/** Lookup table to conversion of time to date, and to month / year / day of week / day of month and so on. /** Lookup table to conversion of time to date, and to month / year / day of week / day of month and so on.

View File

@ -38,7 +38,7 @@
*/ */
POCO_DECLARE_EXCEPTION(Foundation_API, JSONException, Poco::Exception); POCO_DECLARE_EXCEPTION(Foundation_API, JSONException, Poco::Exception)
class JSON class JSON
@ -79,13 +79,13 @@ public:
ElementType getType() const; ElementType getType() const;
bool isObject() const { return getType() == TYPE_OBJECT; }; bool isObject() const { return getType() == TYPE_OBJECT; }
bool isArray() const { return getType() == TYPE_ARRAY; }; bool isArray() const { return getType() == TYPE_ARRAY; }
bool isNumber() const { return getType() == TYPE_NUMBER; }; bool isNumber() const { return getType() == TYPE_NUMBER; }
bool isString() const { return getType() == TYPE_STRING; }; bool isString() const { return getType() == TYPE_STRING; }
bool isBool() const { return getType() == TYPE_BOOL; }; bool isBool() const { return getType() == TYPE_BOOL; }
bool isNull() const { return getType() == TYPE_NULL; }; bool isNull() const { return getType() == TYPE_NULL; }
bool isNameValuePair() const { return getType() == TYPE_NAME_VALUE_PAIR; }; bool isNameValuePair() const { return getType() == TYPE_NAME_VALUE_PAIR; }
/// Количество элементов в массиве или объекте; если элемент - не массив или объект, то исключение. /// Количество элементов в массиве или объекте; если элемент - не массив или объект, то исключение.
size_t size() const; size_t size() const;

View File

@ -290,7 +290,7 @@ namespace ZeroTraits
{ {
inline bool check(StringRef x) { return 0 == x.size; } inline bool check(StringRef x) { return 0 == x.size; }
inline void set(StringRef & x) { x.size = 0; } inline void set(StringRef & x) { x.size = 0; }
}; }
inline bool operator==(StringRef lhs, const char * rhs) inline bool operator==(StringRef lhs, const char * rhs)

View File

@ -16,7 +16,7 @@ namespace ext
To res {}; To res {};
memcpy(static_cast<void*>(&res), &from, std::min(sizeof(res), sizeof(from))); memcpy(static_cast<void*>(&res), &from, std::min(sizeof(res), sizeof(from)));
return res; return res;
}; }
/** \brief Returns value `from` converted to type `To` while retaining bit representation. /** \brief Returns value `from` converted to type `To` while retaining bit representation.
* `To` and `From` must satisfy `CopyConstructible`. * `To` and `From` must satisfy `CopyConstructible`.
@ -26,5 +26,5 @@ namespace ext
{ {
static_assert(sizeof(To) == sizeof(From), "bit cast on types of different width"); static_assert(sizeof(To) == sizeof(From), "bit cast on types of different width");
return bit_cast<To, From>(from); return bit_cast<To, From>(from);
}; }
} }

View File

@ -12,7 +12,7 @@ namespace ext
using value_type = typename Collection::value_type; using value_type = typename Collection::value_type;
return ResultCollection<value_type>(std::begin(collection), std::end(collection)); return ResultCollection<value_type>(std::begin(collection), std::end(collection));
}; }
/** \brief Returns collection of specified type. /** \brief Returns collection of specified type.
* Performs implicit conversion of between source and result value_type, if available and required. */ * Performs implicit conversion of between source and result value_type, if available and required. */

View File

@ -21,7 +21,7 @@ namespace ext
return Collection<value_type>( return Collection<value_type>(
boost::make_transform_iterator(std::begin(collection), mapper), boost::make_transform_iterator(std::begin(collection), mapper),
boost::make_transform_iterator(std::end(collection), mapper)); boost::make_transform_iterator(std::end(collection), mapper));
}; }
/** \brief Returns collection of specified container-type, /** \brief Returns collection of specified container-type,
* with each element transformed by the application of `mapper`. * with each element transformed by the application of `mapper`.
@ -35,7 +35,7 @@ namespace ext
return ResultCollection<value_type>( return ResultCollection<value_type>(
boost::make_transform_iterator(std::begin(collection), mapper), boost::make_transform_iterator(std::begin(collection), mapper),
boost::make_transform_iterator(std::end(collection), mapper)); boost::make_transform_iterator(std::end(collection), mapper));
}; }
/** \brief Returns collection of specified type, /** \brief Returns collection of specified type,
* with each element transformed by the application of `mapper`. * with each element transformed by the application of `mapper`.

Some files were not shown because too many files have changed in this diff Show More