mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Preparation for extra warnings [#CLICKHOUSE-2]
This commit is contained in:
parent
419bc587c0
commit
36db216abf
@ -17,4 +17,4 @@ include/libcpuid/recog_amd.h
|
||||
include/libcpuid/recog_intel.h
|
||||
)
|
||||
|
||||
target_include_directories (cpuid PUBLIC include)
|
||||
target_include_directories (cpuid SYSTEM PUBLIC include)
|
||||
|
@ -56,5 +56,5 @@ ${RDKAFKA_SOURCE_DIR}/rdgz.c
|
||||
|
||||
add_library(rdkafka STATIC ${SRCS})
|
||||
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})
|
||||
|
@ -226,6 +226,6 @@ ConnectionPoolWithFailover::tryGetEntry(
|
||||
}
|
||||
}
|
||||
return result;
|
||||
};
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -3,10 +3,7 @@
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_ICU
|
||||
#pragma GCC diagnostic push
|
||||
#pragma GCC diagnostic ignored "-Wold-style-cast"
|
||||
#include <unicode/ucol.h>
|
||||
#pragma GCC diagnostic pop
|
||||
#else
|
||||
#ifdef __clang__
|
||||
#pragma clang diagnostic push
|
||||
|
@ -1,17 +1,7 @@
|
||||
#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_ref_counter.hpp>
|
||||
|
||||
#ifdef __clang__
|
||||
#pragma clang diagnostic pop
|
||||
#endif
|
||||
|
||||
#include <initializer_list>
|
||||
|
||||
|
||||
|
@ -37,7 +37,7 @@ namespace detail
|
||||
{
|
||||
MoveOrCopyIfThrow<T>()(std::forward<T>(src), dst);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/** 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.
|
||||
|
@ -130,11 +130,11 @@ int getCurrentExceptionCode()
|
||||
{
|
||||
return e.code();
|
||||
}
|
||||
catch (const Poco::Exception & e)
|
||||
catch (const Poco::Exception &)
|
||||
{
|
||||
return ErrorCodes::POCO_EXCEPTION;
|
||||
}
|
||||
catch (const std::exception & e)
|
||||
catch (const std::exception &)
|
||||
{
|
||||
return ErrorCodes::STD_EXCEPTION;
|
||||
}
|
||||
|
@ -74,7 +74,7 @@ bool check(const T x) { return x == 0; }
|
||||
template <typename T>
|
||||
void set(T & x) { x = 0; }
|
||||
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
/** Compile-time interface for cell of the hash table.
|
||||
|
@ -94,8 +94,8 @@ public:
|
||||
{
|
||||
if (auto it = aliases.find(name); it != aliases.end())
|
||||
return it->second;
|
||||
else if (auto it = case_insensitive_aliases.find(Poco::toLower(name)); it != case_insensitive_aliases.end())
|
||||
return it->second;
|
||||
else if (auto jt = case_insensitive_aliases.find(Poco::toLower(name)); jt != case_insensitive_aliases.end())
|
||||
return jt->second;
|
||||
|
||||
throw Exception(getFactoryName() + ": name '" + name + "' is not alias", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
@ -5,24 +5,9 @@
|
||||
#include <algorithm>
|
||||
#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/iterator_adaptors.hpp>
|
||||
|
||||
#ifdef __clang__
|
||||
#pragma clang diagnostic pop
|
||||
#endif
|
||||
|
||||
#pragma GCC diagnostic pop
|
||||
|
||||
#include <common/likely.h>
|
||||
#include <common/strong_typedef.h>
|
||||
|
||||
|
@ -340,4 +340,4 @@ private:
|
||||
size_t m_capacity;
|
||||
};
|
||||
|
||||
};
|
||||
}
|
||||
|
@ -50,4 +50,4 @@ private:
|
||||
static size_t getFailedOpIndex(int32_t code, const Responses & responses);
|
||||
};
|
||||
|
||||
};
|
||||
}
|
||||
|
@ -83,4 +83,4 @@ void ZooKeeperHolder::init(Args&&... args)
|
||||
|
||||
using ZooKeeperHolderPtr = std::shared_ptr<ZooKeeperHolder>;
|
||||
|
||||
};
|
||||
}
|
||||
|
@ -646,4 +646,4 @@ private:
|
||||
CurrentMetrics::Increment active_session_metric_increment{CurrentMetrics::ZooKeeperSession};
|
||||
};
|
||||
|
||||
};
|
||||
}
|
||||
|
@ -77,7 +77,10 @@ void formatIPv6(const unsigned char * src, char *& dst, UInt8 zeroed_tail_bytes_
|
||||
if (words[i] == 0)
|
||||
{
|
||||
if (cur.base == -1)
|
||||
cur.base = i, cur.len = 1;
|
||||
{
|
||||
cur.base = i;
|
||||
cur.len = 1;
|
||||
}
|
||||
else
|
||||
cur.len++;
|
||||
}
|
||||
|
@ -108,7 +108,7 @@ void localBackup(const Poco::Path & source_path, const Poco::Path & destination_
|
||||
|
||||
continue;
|
||||
}
|
||||
catch (const Poco::FileNotFoundException & e)
|
||||
catch (const Poco::FileNotFoundException &)
|
||||
{
|
||||
++try_no;
|
||||
if (try_no == max_tries)
|
||||
|
@ -60,7 +60,7 @@ add_executable (space_saving space_saving.cpp)
|
||||
target_link_libraries (space_saving clickhouse_common_io)
|
||||
|
||||
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)
|
||||
|
||||
add_executable (allocator allocator.cpp)
|
||||
|
@ -25,7 +25,7 @@ namespace ErrorCodes
|
||||
class Field;
|
||||
using Array = 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.
|
||||
|
@ -6,6 +6,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
STRONG_TYPEDEF(UInt128, UUID);
|
||||
STRONG_TYPEDEF(UInt128, UUID)
|
||||
|
||||
}
|
||||
|
@ -63,7 +63,7 @@ Block AggregatingSortedBlockInputStream::readImpl()
|
||||
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());
|
||||
|
||||
merge(merged_columns, queue);
|
||||
merge(merged_columns, queue_without_collation);
|
||||
return header.cloneWithColumns(std::move(merged_columns));
|
||||
}
|
||||
|
||||
|
@ -102,7 +102,7 @@ Block CollapsingSortedBlockInputStream::readImpl()
|
||||
if (merged_columns.empty())
|
||||
return {};
|
||||
|
||||
merge(merged_columns, queue);
|
||||
merge(merged_columns, queue_without_collation);
|
||||
return header.cloneWithColumns(std::move(merged_columns));
|
||||
}
|
||||
|
||||
|
@ -102,7 +102,7 @@ Block GraphiteRollupSortedBlockInputStream::readImpl()
|
||||
if (merged_columns.empty())
|
||||
return Block();
|
||||
|
||||
merge(merged_columns, queue);
|
||||
merge(merged_columns, queue_without_collation);
|
||||
return header.cloneWithColumns(std::move(merged_columns));
|
||||
}
|
||||
|
||||
|
@ -188,7 +188,7 @@ static bool handleOverflowMode(OverflowMode mode, const String & message, int co
|
||||
default:
|
||||
throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
bool IProfilingBlockInputStream::checkTimeLimit()
|
||||
|
@ -183,7 +183,7 @@ MergeSortingBlocksBlockInputStream::MergeSortingBlocksBlockInputStream(
|
||||
if (!has_collation)
|
||||
{
|
||||
for (size_t i = 0; i < cursors.size(); ++i)
|
||||
queue.push(SortCursor(&cursors[i]));
|
||||
queue_without_collation.push(SortCursor(&cursors[i]));
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -206,7 +206,7 @@ Block MergeSortingBlocksBlockInputStream::readImpl()
|
||||
}
|
||||
|
||||
return !has_collation
|
||||
? mergeImpl<SortCursor>(queue)
|
||||
? mergeImpl<SortCursor>(queue_without_collation)
|
||||
: mergeImpl<SortCursorWithCollation>(queue_with_collation);
|
||||
}
|
||||
|
||||
|
@ -55,7 +55,7 @@ private:
|
||||
|
||||
bool has_collation = false;
|
||||
|
||||
std::priority_queue<SortCursor> queue;
|
||||
std::priority_queue<SortCursor> queue_without_collation;
|
||||
std::priority_queue<SortCursorWithCollation> queue_with_collation;
|
||||
|
||||
/** Two different cursors are supported - with and without Collation.
|
||||
|
@ -320,7 +320,7 @@ void MergingAggregatedMemoryEfficientBlockInputStream::mergeThread(MemoryTracker
|
||||
* - 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)
|
||||
break;
|
||||
@ -330,7 +330,7 @@ void MergingAggregatedMemoryEfficientBlockInputStream::mergeThread(MemoryTracker
|
||||
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;
|
||||
}
|
||||
|
||||
@ -344,9 +344,9 @@ void MergingAggregatedMemoryEfficientBlockInputStream::mergeThread(MemoryTracker
|
||||
: 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
|
||||
|| parallel_merge_data->finish;
|
||||
|
@ -58,7 +58,7 @@ void MergingSortedBlockInputStream::init(MutableColumns & merged_columns)
|
||||
if (has_collation)
|
||||
initQueue(queue_with_collation);
|
||||
else
|
||||
initQueue(queue);
|
||||
initQueue(queue_without_collation);
|
||||
}
|
||||
|
||||
/// Let's check that all source blocks have the same structure.
|
||||
@ -105,7 +105,7 @@ Block MergingSortedBlockInputStream::readImpl()
|
||||
if (has_collation)
|
||||
merge(merged_columns, queue_with_collation);
|
||||
else
|
||||
merge(merged_columns, queue);
|
||||
merge(merged_columns, queue_without_collation);
|
||||
|
||||
return header.cloneWithColumns(std::move(merged_columns));
|
||||
}
|
||||
@ -200,7 +200,7 @@ void MergingSortedBlockInputStream::merge(MutableColumns & merged_columns, std::
|
||||
|
||||
// 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)
|
||||
{
|
||||
|
@ -2,17 +2,8 @@
|
||||
|
||||
#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>
|
||||
|
||||
#ifdef __clang__
|
||||
#pragma clang diagnostic pop
|
||||
#endif
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
#include <Core/Row.h>
|
||||
@ -161,7 +152,7 @@ protected:
|
||||
CursorImpls cursors;
|
||||
|
||||
using Queue = std::priority_queue<SortCursor>;
|
||||
Queue queue;
|
||||
Queue queue_without_collation;
|
||||
|
||||
using QueueWithCollation = std::priority_queue<SortCursorWithCollation>;
|
||||
QueueWithCollation queue_with_collation;
|
||||
|
@ -58,21 +58,21 @@ RemoteBlockInputStream::RemoteBlockInputStream(
|
||||
|
||||
create_multiplexed_connections = [this, pool, throttler]()
|
||||
{
|
||||
const Settings & settings = context.getSettingsRef();
|
||||
const Settings & current_settings = context.getSettingsRef();
|
||||
|
||||
std::vector<IConnectionPool::Entry> connections;
|
||||
if (main_table)
|
||||
{
|
||||
auto try_results = pool->getManyChecked(&settings, pool_mode, *main_table);
|
||||
auto try_results = pool->getManyChecked(¤t_settings, pool_mode, *main_table);
|
||||
connections.reserve(try_results.size());
|
||||
for (auto & try_result : try_results)
|
||||
connections.emplace_back(std::move(try_result.entry));
|
||||
}
|
||||
else
|
||||
connections = pool->getMany(&settings, pool_mode);
|
||||
connections = pool->getMany(¤t_settings, pool_mode);
|
||||
|
||||
return std::make_unique<MultiplexedConnections>(
|
||||
std::move(connections), settings, throttler, append_extra_info);
|
||||
std::move(connections), current_settings, throttler, append_extra_info);
|
||||
};
|
||||
}
|
||||
|
||||
|
@ -44,7 +44,7 @@ Block ReplacingSortedBlockInputStream::readImpl()
|
||||
if (merged_columns.empty())
|
||||
return Block();
|
||||
|
||||
merge(merged_columns, queue);
|
||||
merge(merged_columns, queue_without_collation);
|
||||
return header.cloneWithColumns(std::move(merged_columns));
|
||||
}
|
||||
|
||||
|
@ -286,7 +286,7 @@ Block SummingSortedBlockInputStream::readImpl()
|
||||
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));
|
||||
|
||||
/// Place aggregation results into block.
|
||||
|
@ -76,7 +76,7 @@ Block VersionedCollapsingSortedBlockInputStream::readImpl()
|
||||
if (merged_columns.empty())
|
||||
return {};
|
||||
|
||||
merge(merged_columns, queue);
|
||||
merge(merged_columns, queue_without_collation);
|
||||
return header.cloneWithColumns(std::move(merged_columns));
|
||||
}
|
||||
|
||||
|
@ -97,7 +97,7 @@ DataTypeEnum<Type>::DataTypeEnum(const Values & values_) : values{values_}
|
||||
});
|
||||
|
||||
fillMaps();
|
||||
name = generateName(values);
|
||||
type_name = generateName(values);
|
||||
}
|
||||
|
||||
template <typename Type>
|
||||
@ -145,9 +145,9 @@ template <typename Type>
|
||||
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.
|
||||
std::string name;
|
||||
readEscapedString(name, istr);
|
||||
static_cast<ColumnType &>(column).getData().push_back(getValue(StringRef(name)));
|
||||
std::string field_name;
|
||||
readEscapedString(field_name, istr);
|
||||
static_cast<ColumnType &>(column).getData().push_back(getValue(StringRef(field_name)));
|
||||
}
|
||||
|
||||
template <typename Type>
|
||||
@ -159,9 +159,9 @@ void DataTypeEnum<Type>::serializeTextQuoted(const IColumn & column, size_t row_
|
||||
template <typename Type>
|
||||
void DataTypeEnum<Type>::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
std::string name;
|
||||
readQuotedStringWithSQLStyle(name, istr);
|
||||
static_cast<ColumnType &>(column).getData().push_back(getValue(StringRef(name)));
|
||||
std::string field_name;
|
||||
readQuotedStringWithSQLStyle(field_name, istr);
|
||||
static_cast<ColumnType &>(column).getData().push_back(getValue(StringRef(field_name)));
|
||||
}
|
||||
|
||||
template <typename Type>
|
||||
@ -179,9 +179,9 @@ void DataTypeEnum<Type>::serializeTextXML(const IColumn & column, size_t row_num
|
||||
template <typename Type>
|
||||
void DataTypeEnum<Type>::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
std::string name;
|
||||
readJSONString(name, istr);
|
||||
static_cast<ColumnType &>(column).getData().push_back(getValue(StringRef(name)));
|
||||
std::string field_name;
|
||||
readJSONString(field_name, istr);
|
||||
static_cast<ColumnType &>(column).getData().push_back(getValue(StringRef(field_name)));
|
||||
}
|
||||
|
||||
template <typename Type>
|
||||
@ -193,9 +193,9 @@ void DataTypeEnum<Type>::serializeTextCSV(const IColumn & column, size_t row_num
|
||||
template <typename Type>
|
||||
void DataTypeEnum<Type>::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
std::string name;
|
||||
readCSVString(name, istr, settings.csv);
|
||||
static_cast<ColumnType &>(column).getData().push_back(getValue(StringRef(name)));
|
||||
std::string field_name;
|
||||
readCSVString(field_name, istr, settings.csv);
|
||||
static_cast<ColumnType &>(column).getData().push_back(getValue(StringRef(field_name)));
|
||||
}
|
||||
|
||||
template <typename Type>
|
||||
@ -237,7 +237,7 @@ void DataTypeEnum<Type>::insertDefaultInto(IColumn & column) const
|
||||
template <typename Type>
|
||||
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",
|
||||
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>();
|
||||
|
||||
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};
|
||||
|
||||
values.emplace_back(name, value);
|
||||
values.emplace_back(field_name, value);
|
||||
}
|
||||
|
||||
return std::make_shared<DataTypeEnum>(values);
|
||||
|
@ -53,7 +53,7 @@ private:
|
||||
Values values;
|
||||
NameToValueMap name_to_value_map;
|
||||
ValueToNameMap value_to_name_map;
|
||||
std::string name;
|
||||
std::string type_name;
|
||||
|
||||
static std::string generateName(const Values & values);
|
||||
void fillMaps();
|
||||
@ -62,7 +62,7 @@ public:
|
||||
explicit DataTypeEnum(const Values & 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 StringRef & getNameForValue(const FieldType & value) const
|
||||
@ -74,11 +74,11 @@ public:
|
||||
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))
|
||||
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;
|
||||
}
|
||||
|
@ -22,6 +22,6 @@ namespace Nested
|
||||
|
||||
/// Collect Array columns in a form of `column_name.element_name` to single Array(Tuple(...)) column.
|
||||
NamesAndTypesList collect(const NamesAndTypesList & names_and_types);
|
||||
};
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -39,15 +39,15 @@ Tables DatabaseDictionary::loadTables()
|
||||
Tables tables;
|
||||
for (const auto & pair : dictionaries)
|
||||
{
|
||||
const std::string & name = pair.first;
|
||||
if (deleted_tables.count(name))
|
||||
const std::string & dict_name = pair.first;
|
||||
if (deleted_tables.count(dict_name))
|
||||
continue;
|
||||
auto dict_ptr = std::static_pointer_cast<IDictionaryBase>(pair.second.loadable);
|
||||
if (dict_ptr)
|
||||
{
|
||||
const DictionaryStructure & dictionary_structure = dict_ptr->getStructure();
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -101,12 +101,12 @@ void CacheDictionary::isInImpl(
|
||||
{
|
||||
/// Transform all children to parents until ancestor id or null_value will be reached.
|
||||
|
||||
size_t size = out.size();
|
||||
memset(out.data(), 0xFF, size); /// 0xFF means "not calculated"
|
||||
size_t out_size = out.size();
|
||||
memset(out.data(), 0xFF, out_size); /// 0xFF means "not calculated"
|
||||
|
||||
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());
|
||||
|
||||
while (true)
|
||||
@ -115,7 +115,7 @@ void CacheDictionary::isInImpl(
|
||||
size_t parents_idx = 0;
|
||||
size_t new_children_idx = 0;
|
||||
|
||||
while (out_idx < size)
|
||||
while (out_idx < out_size)
|
||||
{
|
||||
/// Already calculated
|
||||
if (out[out_idx] != 0xFF)
|
||||
@ -203,7 +203,7 @@ void CacheDictionary::isInConstantVector(
|
||||
}
|
||||
|
||||
/// 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();
|
||||
}
|
||||
|
||||
@ -936,12 +936,12 @@ void CacheDictionary::setAttributeValue(Attribute & attribute, const Key idx, co
|
||||
if (string_ref.data && string_ref.data != null_value_ref.data())
|
||||
string_arena->free(const_cast<char *>(string_ref.data), string_ref.size);
|
||||
|
||||
const auto size = string.size();
|
||||
if (size != 0)
|
||||
const auto str_size = string.size();
|
||||
if (str_size != 0)
|
||||
{
|
||||
auto string_ptr = string_arena->alloc(size + 1);
|
||||
std::copy(string.data(), string.data() + size + 1, string_ptr);
|
||||
string_ref = StringRef{string_ptr, size};
|
||||
auto string_ptr = string_arena->alloc(str_size + 1);
|
||||
std::copy(string.data(), string.data() + str_size + 1, string_ptr);
|
||||
string_ref = StringRef{string_ptr, str_size};
|
||||
}
|
||||
else
|
||||
string_ref = {};
|
||||
|
@ -307,13 +307,13 @@ private:
|
||||
|
||||
/// buffer[column_size * cat_features_count] -> char * => cat_features[column_size][cat_features_count] -> char *
|
||||
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)
|
||||
{
|
||||
*cat_features = buffer;
|
||||
++cat_features;
|
||||
buffer += cat_features_count;
|
||||
buffer += cat_features_count_current;
|
||||
}
|
||||
}
|
||||
|
||||
@ -321,7 +321,7 @@ private:
|
||||
/// * CalcModelPredictionFlat if no cat features
|
||||
/// * CalcModelPrediction if all cat features are strings
|
||||
/// * 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
|
||||
{
|
||||
std::string error_msg = "Error occurred while applying CatBoost model: ";
|
||||
@ -334,12 +334,12 @@ private:
|
||||
PODArray<const float *> float_features(column_size);
|
||||
auto float_features_buf = float_features.data();
|
||||
/// 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,
|
||||
float_features_buf, float_features_count,
|
||||
float_features_buf, float_features_count_current,
|
||||
result_buf, column_size))
|
||||
{
|
||||
|
||||
@ -352,18 +352,18 @@ private:
|
||||
if (cat_features_are_strings)
|
||||
{
|
||||
/// 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);
|
||||
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.
|
||||
auto fixed_strings_data = placeStringColumns(columns, float_features_count,
|
||||
cat_features_count, cat_features_holder.data());
|
||||
auto fixed_strings_data = placeStringColumns(columns, float_features_count_current,
|
||||
cat_features_count_current, cat_features_holder.data());
|
||||
|
||||
if (!api->CalcModelPrediction(handle->get(), column_size,
|
||||
float_features_buf, float_features_count,
|
||||
cat_features_buf, cat_features_count,
|
||||
float_features_buf, float_features_count_current,
|
||||
cat_features_buf, cat_features_count_current,
|
||||
result_buf, column_size))
|
||||
{
|
||||
throw Exception(error_msg + api->GetErrorString(), ErrorCodes::CANNOT_APPLY_CATBOOST_MODEL);
|
||||
@ -373,13 +373,13 @@ private:
|
||||
{
|
||||
PODArray<const int *> cat_features(column_size);
|
||||
auto cat_features_buf = cat_features.data();
|
||||
auto cat_features_col = placeNumericColumns<int>(columns, float_features_count,
|
||||
cat_features_count, cat_features_buf);
|
||||
calcHashes(columns, float_features_count, cat_features_count, cat_features_buf);
|
||||
auto cat_features_col = placeNumericColumns<int>(columns, float_features_count_current,
|
||||
cat_features_count_current, cat_features_buf);
|
||||
calcHashes(columns, float_features_count_current, cat_features_count_current, cat_features_buf);
|
||||
if (!api->CalcModelPredictionWithHashedCatFeatures(
|
||||
handle->get(), column_size,
|
||||
float_features_buf, float_features_count,
|
||||
cat_features_buf, cat_features_count,
|
||||
float_features_buf, float_features_count_current,
|
||||
cat_features_buf, cat_features_count_current,
|
||||
result_buf, column_size))
|
||||
{
|
||||
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
|
||||
{
|
||||
init(lib_path);
|
||||
init();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
@ -463,7 +463,7 @@ CatBoostModel::CatBoostModel(std::string name_, std::string model_path_, std::st
|
||||
creation_time = std::chrono::system_clock::now();
|
||||
}
|
||||
|
||||
void CatBoostModel::init(const std::string & lib_path)
|
||||
void CatBoostModel::init()
|
||||
{
|
||||
api_provider = getCatBoostWrapperHolder(lib_path);
|
||||
api = &api_provider->getAPI();
|
||||
|
@ -80,7 +80,7 @@ private:
|
||||
std::chrono::time_point<std::chrono::system_clock> creation_time;
|
||||
std::exception_ptr creation_exception;
|
||||
|
||||
void init(const std::string & lib_path);
|
||||
void init();
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -28,12 +28,12 @@ void ComplexKeyCacheDictionary::setAttributeValue(Attribute & attribute, const s
|
||||
if (string_ref.data && string_ref.data != null_value_ref.data())
|
||||
string_arena->free(const_cast<char *>(string_ref.data), string_ref.size);
|
||||
|
||||
const auto size = string.size();
|
||||
if (size != 0)
|
||||
const auto str_size = string.size();
|
||||
if (str_size != 0)
|
||||
{
|
||||
auto string_ptr = string_arena->alloc(size + 1);
|
||||
std::copy(string.data(), string.data() + size + 1, string_ptr);
|
||||
string_ref = StringRef{string_ptr, size};
|
||||
auto string_ptr = string_arena->alloc(str_size + 1);
|
||||
std::copy(string.data(), string.data() + str_size + 1, string_ptr);
|
||||
string_ref = StringRef{string_ptr, str_size};
|
||||
}
|
||||
else
|
||||
string_ref = {};
|
||||
|
@ -223,47 +223,47 @@ Block DictionaryBlockInputStream<DictionaryType, Key>::getBlock(size_t start, si
|
||||
template <typename DictionaryType, typename Key>
|
||||
template <typename Type, typename Container>
|
||||
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*/,
|
||||
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 Container>
|
||||
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*/,
|
||||
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 Type, typename Container>
|
||||
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,
|
||||
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 Container>
|
||||
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,
|
||||
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 <template <typename> class Getter, typename StringGetter>
|
||||
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());
|
||||
|
||||
@ -283,7 +283,7 @@ Block DictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
|
||||
const DictionaryStructure & structure = dictionary->getStructure();
|
||||
|
||||
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()))
|
||||
{
|
||||
@ -293,7 +293,7 @@ Block DictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
|
||||
ColumnPtr column;
|
||||
#define GET_COLUMN_FORM_ATTRIBUTE(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)
|
||||
{
|
||||
case AttributeUnderlyingType::UInt8:
|
||||
@ -346,37 +346,37 @@ Block DictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
|
||||
template <typename DictionaryType, typename Key>
|
||||
template <typename AttributeType, typename Getter>
|
||||
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 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())
|
||||
size = keys.front()->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);
|
||||
}
|
||||
|
||||
template <typename DictionaryType, typename Key>
|
||||
template <typename Getter>
|
||||
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 DictionaryAttribute& attribute, const DictionaryType& dictionary) const
|
||||
const DictionaryAttribute& attribute, const DictionaryType & dict) const
|
||||
{
|
||||
auto column_string = ColumnString::create();
|
||||
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);
|
||||
}
|
||||
|
||||
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();
|
||||
column_vector->getData().reserve(ids.size());
|
||||
for (UInt64 id : ids)
|
||||
column_vector->getData().reserve(ids_to_fill.size());
|
||||
for (UInt64 id : ids_to_fill)
|
||||
column_vector->insert(id);
|
||||
return std::move(column_vector);
|
||||
}
|
||||
|
@ -7,7 +7,7 @@ namespace DB
|
||||
class DictionaryBlockInputStreamBase : public IProfilingBlockInputStream
|
||||
{
|
||||
protected:
|
||||
Block block;
|
||||
//Block block;
|
||||
|
||||
DictionaryBlockInputStreamBase(size_t rows_count, size_t max_block_size);
|
||||
|
||||
|
@ -20,10 +20,7 @@
|
||||
#include <Dictionaries/MongoDBDictionarySource.h>
|
||||
#endif
|
||||
#if USE_POCO_SQLODBC || USE_POCO_DATAODBC
|
||||
#pragma GCC diagnostic push
|
||||
#pragma GCC diagnostic ignored "-Wunused-parameter"
|
||||
#include <Poco/Data/ODBC/Connector.h>
|
||||
#pragma GCC diagnostic pop
|
||||
#include <Dictionaries/ODBCDictionarySource.h>
|
||||
#endif
|
||||
#if USE_MYSQL
|
||||
|
@ -240,22 +240,22 @@ std::vector<DictionaryAttribute> DictionaryStructure::getAttributes(
|
||||
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix,
|
||||
const bool hierarchy_allowed, const bool allow_null_values)
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
config.keys(config_prefix, keys);
|
||||
Poco::Util::AbstractConfiguration::Keys config_elems;
|
||||
config.keys(config_prefix, config_elems);
|
||||
auto has_hierarchy = false;
|
||||
|
||||
std::vector<DictionaryAttribute> attributes;
|
||||
std::vector<DictionaryAttribute> res_attributes;
|
||||
|
||||
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;
|
||||
|
||||
const auto prefix = config_prefix + '.' + key + '.';
|
||||
const auto prefix = config_prefix + '.' + config_elem + '.';
|
||||
Poco::Util::AbstractConfiguration::Keys attribute_keys;
|
||||
config.keys(config_prefix + '.' + key, attribute_keys);
|
||||
config.keys(config_prefix + '.' + config_elem, attribute_keys);
|
||||
|
||||
checkAttributeKeys(attribute_keys);
|
||||
|
||||
@ -300,12 +300,12 @@ std::vector<DictionaryAttribute> DictionaryStructure::getAttributes(
|
||||
|
||||
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
|
||||
});
|
||||
}
|
||||
|
||||
return attributes;
|
||||
return res_attributes;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -5,13 +5,10 @@
|
||||
#include <string>
|
||||
#include <sstream>
|
||||
|
||||
#pragma GCC diagnostic push
|
||||
#pragma GCC diagnostic ignored "-Wunused-parameter"
|
||||
#include <Poco/MongoDB/Connection.h>
|
||||
#include <Poco/MongoDB/Cursor.h>
|
||||
#include <Poco/MongoDB/Element.h>
|
||||
#include <Poco/MongoDB/ObjectId.h>
|
||||
#pragma GCC diagnostic pop
|
||||
#include <Poco/MongoDB/Connection.h>
|
||||
#include <Poco/MongoDB/Cursor.h>
|
||||
#include <Poco/MongoDB/Element.h>
|
||||
#include <Poco/MongoDB/ObjectId.h>
|
||||
|
||||
#include <Dictionaries/DictionaryStructure.h>
|
||||
#include <Dictionaries/MongoDBBlockInputStream.h>
|
||||
|
@ -2,14 +2,11 @@
|
||||
#if USE_POCO_MONGODB
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
||||
#pragma GCC diagnostic push
|
||||
#pragma GCC diagnostic ignored "-Wunused-parameter"
|
||||
#include <Poco/MongoDB/Connection.h>
|
||||
#include <Poco/MongoDB/Database.h>
|
||||
#include <Poco/MongoDB/Cursor.h>
|
||||
#include <Poco/MongoDB/Array.h>
|
||||
#include <Poco/MongoDB/ObjectId.h>
|
||||
#pragma GCC diagnostic pop
|
||||
#include <Poco/MongoDB/Connection.h>
|
||||
#include <Poco/MongoDB/Database.h>
|
||||
#include <Poco/MongoDB/Cursor.h>
|
||||
#include <Poco/MongoDB/Array.h>
|
||||
#include <Poco/MongoDB/ObjectId.h>
|
||||
|
||||
#include <Poco/Version.h>
|
||||
|
||||
|
@ -167,10 +167,10 @@ std::string MySQLDictionarySource::quoteForLike(const std::string s)
|
||||
|
||||
LocalDateTime MySQLDictionarySource::getLastModification() const
|
||||
{
|
||||
LocalDateTime update_time{std::time(nullptr)};
|
||||
LocalDateTime modification_time{std::time(nullptr)};
|
||||
|
||||
if (dont_check_update_time)
|
||||
return update_time;
|
||||
return modification_time;
|
||||
|
||||
try
|
||||
{
|
||||
@ -190,8 +190,8 @@ LocalDateTime MySQLDictionarySource::getLastModification() const
|
||||
|
||||
if (!update_time_value.isNull())
|
||||
{
|
||||
update_time = update_time_value.getDateTime();
|
||||
LOG_TRACE(log, "Got update time: " << update_time);
|
||||
modification_time = update_time_value.getDateTime();
|
||||
LOG_TRACE(log, "Got modification time: " << modification_time);
|
||||
}
|
||||
|
||||
/// 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
|
||||
return update_time;
|
||||
return modification_time;
|
||||
}
|
||||
|
||||
std::string MySQLDictionarySource::doInvalidateQuery(const std::string & request) const
|
||||
{
|
||||
Block sample_block;
|
||||
Block invalidate_sample_block;
|
||||
ColumnPtr column(ColumnString::create());
|
||||
sample_block.insert(ColumnWithTypeAndName(column, std::make_shared<DataTypeString>(), "Sample Block"));
|
||||
MySQLBlockInputStream block_input_stream(pool.Get(), request, sample_block, 1);
|
||||
invalidate_sample_block.insert(ColumnWithTypeAndName(column, std::make_shared<DataTypeString>(), "Sample Block"));
|
||||
MySQLBlockInputStream block_input_stream(pool.Get(), request, invalidate_sample_block, 1);
|
||||
return readInvalidateQuery(block_input_stream);
|
||||
}
|
||||
|
||||
|
@ -4,12 +4,9 @@
|
||||
#include <DataStreams/IProfilingBlockInputStream.h>
|
||||
#include <Dictionaries/ExternalResultDescription.h>
|
||||
|
||||
#pragma GCC diagnostic push
|
||||
#pragma GCC diagnostic ignored "-Wunused-parameter"
|
||||
#include <Poco/Data/Session.h>
|
||||
#include <Poco/Data/Statement.h>
|
||||
#include <Poco/Data/RecordSet.h>
|
||||
#pragma GCC diagnostic pop
|
||||
#include <Poco/Data/Session.h>
|
||||
#include <Poco/Data/Statement.h>
|
||||
#include <Poco/Data/RecordSet.h>
|
||||
|
||||
#include <string>
|
||||
|
||||
|
@ -145,10 +145,10 @@ bool ODBCDictionarySource::isModified() const
|
||||
|
||||
std::string ODBCDictionarySource::doInvalidateQuery(const std::string & request) const
|
||||
{
|
||||
Block sample_block;
|
||||
Block invalidate_sample_block;
|
||||
ColumnPtr column(ColumnString::create());
|
||||
sample_block.insert(ColumnWithTypeAndName(column, std::make_shared<DataTypeString>(), "Sample Block"));
|
||||
ODBCBlockInputStream block_input_stream(pool->get(), request, sample_block, 1);
|
||||
invalidate_sample_block.insert(ColumnWithTypeAndName(column, std::make_shared<DataTypeString>(), "Sample Block"));
|
||||
ODBCBlockInputStream block_input_stream(pool->get(), request, invalidate_sample_block, 1);
|
||||
return readInvalidateQuery(block_input_stream);
|
||||
}
|
||||
|
||||
|
@ -1,13 +1,12 @@
|
||||
#pragma once
|
||||
|
||||
#pragma GCC diagnostic push
|
||||
#pragma GCC diagnostic ignored "-Wunused-parameter"
|
||||
#include <Poco/Data/SessionPool.h>
|
||||
#pragma GCC diagnostic pop
|
||||
|
||||
#include <Dictionaries/IDictionarySource.h>
|
||||
#include <Dictionaries/ExternalQueryBuilder.h>
|
||||
#include <Dictionaries/DictionaryStructure.h>
|
||||
|
||||
|
||||
namespace Poco
|
||||
{
|
||||
namespace Util
|
||||
|
@ -189,22 +189,22 @@ std::string validateODBCConnectionString(const std::string & connection_string)
|
||||
{
|
||||
reconstructed_connection_string += '{';
|
||||
|
||||
const char * pos = value.data();
|
||||
const char * end = pos + value.size();
|
||||
const char * value_pos = value.data();
|
||||
const char * value_end = value_pos + value.size();
|
||||
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;
|
||||
}
|
||||
else
|
||||
{
|
||||
reconstructed_connection_string.append(pos, next_pos - pos);
|
||||
reconstructed_connection_string.append(value_pos, next_pos - value_pos);
|
||||
reconstructed_connection_string.append("}}");
|
||||
pos = next_pos + 1;
|
||||
value_pos = next_pos + 1;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -16,18 +16,18 @@
|
||||
*/
|
||||
|
||||
#if USE_VECTORCLASS
|
||||
#ifdef __clang__
|
||||
#pragma clang diagnostic push
|
||||
#pragma clang diagnostic ignored "-Wshift-negative-value"
|
||||
#endif
|
||||
#ifdef __clang__
|
||||
#pragma clang diagnostic push
|
||||
#pragma clang diagnostic ignored "-Wshift-negative-value"
|
||||
#endif
|
||||
|
||||
#include <vectorf128.h> // Y_IGNORE
|
||||
#include <vectormath_exp.h> // Y_IGNORE
|
||||
#include <vectormath_trig.h> // Y_IGNORE
|
||||
#include <vectorf128.h> // Y_IGNORE
|
||||
#include <vectormath_exp.h> // Y_IGNORE
|
||||
#include <vectormath_trig.h> // Y_IGNORE
|
||||
|
||||
#ifdef __clang__
|
||||
#pragma clang diagnostic pop
|
||||
#endif
|
||||
#ifdef __clang__
|
||||
#pragma clang diagnostic pop
|
||||
#endif
|
||||
#endif
|
||||
|
||||
|
||||
|
@ -44,7 +44,7 @@ public:
|
||||
return curr_buffer;
|
||||
}
|
||||
|
||||
~CascadeWriteBuffer();
|
||||
~CascadeWriteBuffer() override;
|
||||
|
||||
private:
|
||||
|
||||
|
@ -31,7 +31,7 @@ public:
|
||||
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));
|
||||
|
||||
~InterserverWriteBuffer();
|
||||
~InterserverWriteBuffer() override;
|
||||
void finalize();
|
||||
void cancel();
|
||||
|
||||
|
@ -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 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, true>(UInt8 * op, const UInt8 *& match, const size_t offset) { copyOverlap8Shuffle(op, match, offset); };
|
||||
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 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); }
|
||||
|
||||
|
||||
inline void copy16(UInt8 * dst, const UInt8 * src)
|
||||
@ -337,10 +337,10 @@ inline void copyOverlap16Shuffle(UInt8 * op, const UInt8 *& match, const size_t
|
||||
#endif
|
||||
|
||||
|
||||
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 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 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 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); }
|
||||
|
||||
|
||||
/// See also https://stackoverflow.com/a/30669632
|
||||
|
@ -75,7 +75,7 @@ struct PerformanceStatistics
|
||||
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.
|
||||
/// And in that case prefer variant with less number of invocations.
|
||||
@ -83,7 +83,7 @@ struct PerformanceStatistics
|
||||
if (adjustedCount() < 2)
|
||||
return adjustedCount() - 1;
|
||||
else
|
||||
return std::normal_distribution<>(mean(), sigma())(rng);
|
||||
return std::normal_distribution<>(mean(), sigma())(stat_rng);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -14,7 +14,7 @@ namespace DB
|
||||
class MMapReadBufferFromFileDescriptor : public ReadBuffer
|
||||
{
|
||||
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);
|
||||
|
@ -33,7 +33,7 @@ public:
|
||||
return setChunk();
|
||||
}
|
||||
|
||||
~ReadBufferFromMemoryWriteBuffer()
|
||||
~ReadBufferFromMemoryWriteBuffer() override
|
||||
{
|
||||
for (const auto & range : chunk_list)
|
||||
free(range.begin(), range.size());
|
||||
|
@ -133,7 +133,7 @@ bool ReadBufferFromFileDescriptor::poll(size_t timeout_microseconds)
|
||||
FD_SET(fd, &fds);
|
||||
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)
|
||||
throwFromErrno("Cannot select", ErrorCodes::CANNOT_SELECT);
|
||||
|
@ -30,7 +30,7 @@ bool ReadBufferFromPocoSocket::nextImpl()
|
||||
{
|
||||
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);
|
||||
}
|
||||
|
@ -128,7 +128,7 @@ public:
|
||||
send_progress_interval_ms = send_progress_interval_ms_;
|
||||
}
|
||||
|
||||
~WriteBufferFromHTTPServerResponse();
|
||||
~WriteBufferFromHTTPServerResponse() override;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -36,7 +36,7 @@ void WriteBufferFromPocoSocket::nextImpl()
|
||||
{
|
||||
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);
|
||||
}
|
||||
|
@ -419,12 +419,12 @@ inline void writeProbablyQuotedStringImpl(const String & s, WriteBuffer & buf, F
|
||||
|
||||
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)
|
||||
{
|
||||
writeProbablyQuotedStringImpl(s, buf, [](const String & s, WriteBuffer & buf) { return writeDoubleQuotedString(s, buf); });
|
||||
writeProbablyQuotedStringImpl(s, buf, [](const String & s_, WriteBuffer & buf_) { return writeDoubleQuotedString(s_, buf_); });
|
||||
}
|
||||
|
||||
|
||||
|
@ -17,9 +17,9 @@ ZlibDeflatingWriteBuffer::ZlibDeflatingWriteBuffer(
|
||||
zstr.zalloc = Z_NULL;
|
||||
zstr.zfree = Z_NULL;
|
||||
zstr.opaque = Z_NULL;
|
||||
zstr.next_in = 0;
|
||||
zstr.next_in = nullptr;
|
||||
zstr.avail_in = 0;
|
||||
zstr.next_out = 0;
|
||||
zstr.next_out = nullptr;
|
||||
zstr.avail_out = 0;
|
||||
|
||||
int window_bits = 15;
|
||||
|
@ -17,9 +17,9 @@ ZlibInflatingReadBuffer::ZlibInflatingReadBuffer(
|
||||
zstr.zalloc = Z_NULL;
|
||||
zstr.zfree = Z_NULL;
|
||||
zstr.opaque = Z_NULL;
|
||||
zstr.next_in = 0;
|
||||
zstr.next_in = nullptr;
|
||||
zstr.avail_in = 0;
|
||||
zstr.next_out = 0;
|
||||
zstr.next_out = nullptr;
|
||||
zstr.avail_out = 0;
|
||||
|
||||
int window_bits = 15;
|
||||
|
@ -1601,7 +1601,7 @@ public:
|
||||
|
||||
Block getHeader() const override { return aggregator.getHeader(final); }
|
||||
|
||||
~MergingAndConvertingBlockInputStream()
|
||||
~MergingAndConvertingBlockInputStream() override
|
||||
{
|
||||
LOG_TRACE(&Logger::get(__PRETTY_FUNCTION__), "Waiting for threads to finish");
|
||||
|
||||
|
@ -88,7 +88,7 @@ struct HostID
|
||||
{
|
||||
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
|
||||
return false;
|
||||
@ -578,7 +578,7 @@ void DDLWorker::processTask(DDLTask & task)
|
||||
tryExecuteQuery(rewritten_query, task, task.execution_status);
|
||||
}
|
||||
}
|
||||
catch (const zkutil::KeeperException & e)
|
||||
catch (const zkutil::KeeperException &)
|
||||
{
|
||||
throw;
|
||||
}
|
||||
|
@ -38,11 +38,11 @@ static bool isNetworkError()
|
||||
if (e.code() == ErrorCodes::TIMEOUT_EXCEEDED || e.code() == ErrorCodes::ALL_CONNECTION_TRIES_FAILED)
|
||||
return true;
|
||||
}
|
||||
catch (Poco::Net::DNSException & e)
|
||||
catch (Poco::Net::DNSException &)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
catch (Poco::TimeoutException & e)
|
||||
catch (Poco::TimeoutException &)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
@ -127,7 +127,7 @@ DictionaryPtr DictionaryFactory::create(const std::string & name, const Poco::Ut
|
||||
|
||||
throw Exception{name + ": unknown dictionary layout type: " + layout_type,
|
||||
ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG};
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -326,7 +326,7 @@ static DatabaseAndTableWithAlias getTableNameWithAliasFromTableExpression(const
|
||||
throw Exception("Logical error: no known elements in ASTTableExpression", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return database_and_table_with_alias;
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
void ExpressionAnalyzer::translateQualifiedNames()
|
||||
|
@ -24,7 +24,7 @@ public:
|
||||
size_t subquery_depth_ = 0,
|
||||
bool only_analyze = false);
|
||||
|
||||
~InterpreterSelectWithUnionQuery();
|
||||
~InterpreterSelectWithUnionQuery() override;
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
|
@ -18,7 +18,7 @@ add_executable (hash_map3 hash_map3.cpp)
|
||||
target_link_libraries (hash_map3 dbms ${FARMHASH_LIBRARIES} ${METROHASH_LIBRARIES})
|
||||
|
||||
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)
|
||||
|
||||
add_executable (hash_map_string_2 hash_map_string_2.cpp)
|
||||
|
@ -41,7 +41,7 @@ inline ASTPtr setAlias(ASTPtr ast, const String & alias)
|
||||
{
|
||||
ast->setAlias(alias);
|
||||
return ast;
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -67,5 +67,5 @@ struct StringRangePointersEqualTo
|
||||
}
|
||||
};
|
||||
|
||||
};
|
||||
}
|
||||
|
||||
|
@ -135,7 +135,7 @@ public:
|
||||
LOG_TRACE(log, "Row delimiter is: " << row_delimiter);
|
||||
}
|
||||
|
||||
~ReadBufferFromKafkaConsumer() { reset(); }
|
||||
~ReadBufferFromKafkaConsumer() override { reset(); }
|
||||
|
||||
/// Commit messages read with this consumer
|
||||
void commit()
|
||||
@ -144,7 +144,7 @@ public:
|
||||
if (read_messages == 0)
|
||||
return;
|
||||
|
||||
auto err = rd_kafka_commit(consumer, NULL, 1 /* async */);
|
||||
auto err = rd_kafka_commit(consumer, nullptr, 1 /* async */);
|
||||
if (err)
|
||||
throw Exception("Failed to commit offsets: " + String(rd_kafka_err2str(err)), ErrorCodes::UNKNOWN_EXCEPTION);
|
||||
|
||||
@ -199,7 +199,7 @@ public:
|
||||
return reader->read();
|
||||
}
|
||||
|
||||
Block getHeader() const override { return reader->getHeader(); };
|
||||
Block getHeader() const override { return reader->getHeader(); }
|
||||
|
||||
void readPrefixImpl() override
|
||||
{
|
||||
|
@ -125,7 +125,7 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo
|
||||
|
||||
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.
|
||||
throw;
|
||||
|
@ -6,7 +6,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
STRONG_TYPEDEF(UInt32, MergeTreeDataFormatVersion);
|
||||
STRONG_TYPEDEF(UInt32, MergeTreeDataFormatVersion)
|
||||
|
||||
const MergeTreeDataFormatVersion MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING {1};
|
||||
|
||||
|
@ -339,7 +339,7 @@ void MergeTreeDataPart::remove() const
|
||||
{
|
||||
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."
|
||||
" Most likely this is due to manual removing. This should be discouraged. Ignoring.");
|
||||
|
@ -323,7 +323,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts(
|
||||
if (relative_sample_size == RelativeSize(1))
|
||||
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);
|
||||
|
||||
if (relative_sample_offset > 1)
|
||||
@ -374,7 +374,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts(
|
||||
if (!data.sampling_expression)
|
||||
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);
|
||||
|
||||
RelativeSize size_of_universum = 0;
|
||||
|
@ -43,7 +43,7 @@ Block MergeTreeThreadBlockInputStream::getHeader() const
|
||||
auto res = pool->getHeader();
|
||||
injectVirtualColumns(res);
|
||||
return res;
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
/// Requests read task from MergeTreeReadPool and signals whether it got one
|
||||
|
@ -231,7 +231,7 @@ void ReplicatedMergeTreePartCheckThread::checkPart(const String & part_name)
|
||||
|
||||
LOG_INFO(log, "Part " << part_name << " looks good.");
|
||||
}
|
||||
catch (const Exception & e)
|
||||
catch (const Exception &)
|
||||
{
|
||||
/// TODO Better to check error code.
|
||||
|
||||
|
@ -114,7 +114,7 @@ void ReplicatedMergeTreeRestartingThread::run()
|
||||
{
|
||||
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.
|
||||
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
||||
|
@ -36,7 +36,7 @@ namespace ErrorCodes
|
||||
extern const int INCORRECT_FILE_NAME;
|
||||
extern const int FILE_DOESNT_EXIST;
|
||||
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)
|
||||
|
@ -50,7 +50,7 @@ void StorageJoin::truncate(const ASTPtr &)
|
||||
increment = 0;
|
||||
join = std::make_shared<Join>(key_names, key_names, NameSet(), false /* use_nulls */, SizeLimits(), kind, strictness);
|
||||
join->setSampleBlock(getSampleBlock().sortColumns());
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
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); }
|
||||
size_t StorageJoin::getSize() const { return join->getTotalRowCount(); };
|
||||
size_t StorageJoin::getSize() const { return join->getTotalRowCount(); }
|
||||
|
||||
|
||||
void registerStorageJoin(StorageFactory & factory)
|
||||
|
@ -70,7 +70,7 @@ public:
|
||||
res.insert({ name_type.type->createColumn(), name_type.type, name_type.name });
|
||||
|
||||
return Nested::flatten(res);
|
||||
};
|
||||
}
|
||||
|
||||
protected:
|
||||
Block readImpl() override;
|
||||
|
@ -645,7 +645,7 @@ Int64 StorageMergeTree::getCurrentMutationVersion(
|
||||
return 0;
|
||||
--it;
|
||||
return it->first;
|
||||
};
|
||||
}
|
||||
|
||||
void StorageMergeTree::clearOldMutations()
|
||||
{
|
||||
|
@ -2,12 +2,7 @@
|
||||
|
||||
#include <ext/shared_ptr_helper.h>
|
||||
#include <Storages/IStorage.h>
|
||||
|
||||
#pragma GCC diagnostic push
|
||||
#pragma GCC diagnostic ignored "-Wunused-parameter"
|
||||
#include <Poco/Data/SessionPool.h>
|
||||
#pragma GCC diagnostic pop
|
||||
|
||||
#include <Poco/Data/SessionPool.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1822,7 +1822,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry)
|
||||
{
|
||||
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."
|
||||
<< " Reason: " << getCurrentExceptionMessage(false));
|
||||
|
@ -131,7 +131,7 @@ void StorageSet::truncate(const ASTPtr &)
|
||||
increment = 0;
|
||||
set = std::make_shared<Set>(SizeLimits(), false);
|
||||
set->setHeader(header);
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
void StorageSetOrJoinBase::restore()
|
||||
|
@ -73,7 +73,7 @@ public:
|
||||
Block getHeader() const override
|
||||
{
|
||||
return header;
|
||||
};
|
||||
}
|
||||
|
||||
protected:
|
||||
Block readImpl() override
|
||||
@ -143,7 +143,7 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
~StripeLogBlockOutputStream()
|
||||
~StripeLogBlockOutputStream() override
|
||||
{
|
||||
try
|
||||
{
|
||||
|
@ -69,7 +69,7 @@ public:
|
||||
res.insert({ name_type.type->createColumn(), name_type.type, name_type.name });
|
||||
|
||||
return Nested::flatten(res);
|
||||
};
|
||||
}
|
||||
|
||||
protected:
|
||||
Block readImpl() override;
|
||||
|
@ -22,7 +22,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
};
|
||||
}
|
||||
|
||||
StorageURL::StorageURL(const Poco::URI & uri_,
|
||||
const std::string & table_name_,
|
||||
|
@ -94,10 +94,10 @@ void StorageSystemColumns::fillData(MutableColumns & res_columns, const Context
|
||||
|
||||
/// We compose the result.
|
||||
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 table_name = (*filtered_table_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)[row_no].get<std::string>();
|
||||
|
||||
NamesAndTypesList columns;
|
||||
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.
|
||||
* 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();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -16,13 +16,9 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Core/Defines.h>
|
||||
|
||||
#pragma GCC diagnostic push
|
||||
#pragma GCC diagnostic ignored "-Wunused-parameter"
|
||||
#pragma GCC diagnostic ignored "-Wsign-compare"
|
||||
#include <Poco/Data/ODBC/ODBCException.h>
|
||||
#include <Poco/Data/ODBC/SessionImpl.h>
|
||||
#include <Poco/Data/ODBC/Utility.h>
|
||||
#pragma GCC diagnostic pop
|
||||
#include <Poco/Data/ODBC/ODBCException.h>
|
||||
#include <Poco/Data/ODBC/SessionImpl.h>
|
||||
#include <Poco/Data/ODBC/Utility.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -15,7 +15,7 @@
|
||||
#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.
|
||||
|
@ -38,7 +38,7 @@
|
||||
*/
|
||||
|
||||
|
||||
POCO_DECLARE_EXCEPTION(Foundation_API, JSONException, Poco::Exception);
|
||||
POCO_DECLARE_EXCEPTION(Foundation_API, JSONException, Poco::Exception)
|
||||
|
||||
|
||||
class JSON
|
||||
@ -79,13 +79,13 @@ public:
|
||||
|
||||
ElementType getType() const;
|
||||
|
||||
bool isObject() const { return getType() == TYPE_OBJECT; };
|
||||
bool isArray() const { return getType() == TYPE_ARRAY; };
|
||||
bool isNumber() const { return getType() == TYPE_NUMBER; };
|
||||
bool isString() const { return getType() == TYPE_STRING; };
|
||||
bool isBool() const { return getType() == TYPE_BOOL; };
|
||||
bool isNull() const { return getType() == TYPE_NULL; };
|
||||
bool isNameValuePair() const { return getType() == TYPE_NAME_VALUE_PAIR; };
|
||||
bool isObject() const { return getType() == TYPE_OBJECT; }
|
||||
bool isArray() const { return getType() == TYPE_ARRAY; }
|
||||
bool isNumber() const { return getType() == TYPE_NUMBER; }
|
||||
bool isString() const { return getType() == TYPE_STRING; }
|
||||
bool isBool() const { return getType() == TYPE_BOOL; }
|
||||
bool isNull() const { return getType() == TYPE_NULL; }
|
||||
bool isNameValuePair() const { return getType() == TYPE_NAME_VALUE_PAIR; }
|
||||
|
||||
/// Количество элементов в массиве или объекте; если элемент - не массив или объект, то исключение.
|
||||
size_t size() const;
|
||||
|
@ -290,7 +290,7 @@ namespace ZeroTraits
|
||||
{
|
||||
inline bool check(StringRef x) { return 0 == x.size; }
|
||||
inline void set(StringRef & x) { x.size = 0; }
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
inline bool operator==(StringRef lhs, const char * rhs)
|
||||
|
@ -16,7 +16,7 @@ namespace ext
|
||||
To res {};
|
||||
memcpy(static_cast<void*>(&res), &from, std::min(sizeof(res), sizeof(from)));
|
||||
return res;
|
||||
};
|
||||
}
|
||||
|
||||
/** \brief Returns value `from` converted to type `To` while retaining bit representation.
|
||||
* `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");
|
||||
return bit_cast<To, From>(from);
|
||||
};
|
||||
}
|
||||
}
|
||||
|
@ -12,7 +12,7 @@ namespace ext
|
||||
using value_type = typename Collection::value_type;
|
||||
|
||||
return ResultCollection<value_type>(std::begin(collection), std::end(collection));
|
||||
};
|
||||
}
|
||||
|
||||
/** \brief Returns collection of specified type.
|
||||
* Performs implicit conversion of between source and result value_type, if available and required. */
|
||||
|
@ -21,7 +21,7 @@ namespace ext
|
||||
return Collection<value_type>(
|
||||
boost::make_transform_iterator(std::begin(collection), mapper),
|
||||
boost::make_transform_iterator(std::end(collection), mapper));
|
||||
};
|
||||
}
|
||||
|
||||
/** \brief Returns collection of specified container-type,
|
||||
* with each element transformed by the application of `mapper`.
|
||||
@ -35,7 +35,7 @@ namespace ext
|
||||
return ResultCollection<value_type>(
|
||||
boost::make_transform_iterator(std::begin(collection), mapper),
|
||||
boost::make_transform_iterator(std::end(collection), mapper));
|
||||
};
|
||||
}
|
||||
|
||||
/** \brief Returns collection of specified type,
|
||||
* 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
Loading…
Reference in New Issue
Block a user