mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Non significant changes according to clang's -Weverything, part 2 [#CLICKHOUSE-2]
This commit is contained in:
parent
440ece0ff2
commit
1c2b024c39
@ -43,7 +43,7 @@ public:
|
||||
/// Get the result type.
|
||||
virtual DataTypePtr getReturnType() const = 0;
|
||||
|
||||
virtual ~IAggregateFunction() {};
|
||||
virtual ~IAggregateFunction() {}
|
||||
|
||||
/** Data manipulating functions. */
|
||||
|
||||
|
@ -113,16 +113,16 @@ void processIdentifier(const String & column_name, const ASTPtr & ast, TypeAndCo
|
||||
else
|
||||
{
|
||||
/// Alias
|
||||
auto it = aliases.aliases.find(column_name);
|
||||
if (it != aliases.aliases.end())
|
||||
auto jt = aliases.aliases.find(column_name);
|
||||
if (jt != aliases.aliases.end())
|
||||
{
|
||||
/// TODO Cyclic aliases.
|
||||
|
||||
if (it->second.kind != CollectAliases::Kind::Expression)
|
||||
if (jt->second.kind != CollectAliases::Kind::Expression)
|
||||
throw Exception("Logical error: unexpected kind of alias", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
processImpl(it->second.node, context, aliases, columns, info, lambdas, table_functions);
|
||||
info[column_name] = info[it->second.node->getColumnName()];
|
||||
processImpl(jt->second.node, context, aliases, columns, info, lambdas, table_functions);
|
||||
info[column_name] = info[jt->second.node->getColumnName()];
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -79,7 +79,7 @@ public:
|
||||
setDescription();
|
||||
}
|
||||
|
||||
virtual ~Connection() {};
|
||||
virtual ~Connection() {}
|
||||
|
||||
/// Set throttler of network traffic. One throttler could be used for multiple connections to limit total traffic.
|
||||
void setThrottler(const ThrottlerPtr & throttler_)
|
||||
|
@ -4,13 +4,11 @@
|
||||
|
||||
#if USE_ICU
|
||||
#pragma GCC diagnostic push
|
||||
#ifdef __APPLE__
|
||||
#pragma GCC diagnostic ignored "-Wold-style-cast"
|
||||
#endif
|
||||
#include <unicode/ucol.h>
|
||||
#pragma GCC diagnostic pop
|
||||
#else
|
||||
#if __clang__
|
||||
#ifdef __clang__
|
||||
#pragma clang diagnostic push
|
||||
#pragma clang diagnostic ignored "-Wunused-private-field"
|
||||
#endif
|
||||
|
@ -88,7 +88,7 @@ private:
|
||||
}
|
||||
|
||||
public:
|
||||
~ColumnAggregateFunction();
|
||||
~ColumnAggregateFunction() override;
|
||||
|
||||
void set(const AggregateFunctionPtr & func_)
|
||||
{
|
||||
|
@ -52,7 +52,7 @@ public:
|
||||
void insertData(const char * pos, size_t length) override;
|
||||
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override;
|
||||
const char * deserializeAndInsertFromArena(const char * pos) override;
|
||||
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override;;
|
||||
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override;
|
||||
void insert(const Field & x) override;
|
||||
void insertFrom(const IColumn & src, size_t n) override;
|
||||
|
||||
|
@ -47,7 +47,7 @@ private:
|
||||
|
||||
ColumnString(const ColumnString & src)
|
||||
: offsets(src.offsets.begin(), src.offsets.end()),
|
||||
chars(src.chars.begin(), src.chars.end()) {};
|
||||
chars(src.chars.begin(), src.chars.end()) {}
|
||||
|
||||
public:
|
||||
const char * getFamilyName() const override { return "String"; }
|
||||
|
@ -1,7 +1,17 @@
|
||||
#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>
|
||||
|
||||
|
||||
|
@ -5,9 +5,23 @@
|
||||
#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"
|
||||
#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>
|
||||
|
||||
|
@ -24,7 +24,7 @@ public:
|
||||
children.insert(children.end(), inputs.begin(), inputs.end());
|
||||
}
|
||||
|
||||
~CollapsingFinalBlockInputStream();
|
||||
~CollapsingFinalBlockInputStream() override;
|
||||
|
||||
String getName() const override { return "CollapsingFinal"; }
|
||||
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <DataStreams/ColumnGathererStream.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <iomanip>
|
||||
|
||||
|
||||
@ -20,7 +21,7 @@ namespace ErrorCodes
|
||||
ColumnGathererStream::ColumnGathererStream(
|
||||
const String & column_name_, const BlockInputStreams & source_streams, ReadBuffer & row_sources_buf_,
|
||||
size_t block_preferred_size_)
|
||||
: name(column_name_), row_sources_buf(row_sources_buf_)
|
||||
: column_name(column_name_), row_sources_buf(row_sources_buf_)
|
||||
, block_preferred_size(block_preferred_size_), log(&Logger::get("ColumnGathererStream"))
|
||||
{
|
||||
if (source_streams.empty())
|
||||
@ -35,28 +36,28 @@ void ColumnGathererStream::init()
|
||||
sources.reserve(children.size());
|
||||
for (size_t i = 0; i < children.size(); ++i)
|
||||
{
|
||||
sources.emplace_back(children[i]->read(), name);
|
||||
sources.emplace_back(children[i]->read(), column_name);
|
||||
|
||||
Block & block = sources.back().block;
|
||||
|
||||
/// Sometimes MergeTreeReader injects additional column with partitioning key
|
||||
if (block.columns() > 2)
|
||||
throw Exception(
|
||||
"Block should have 1 or 2 columns, but contains " + toString(block.columns()),
|
||||
ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS);
|
||||
if (!block.has(name))
|
||||
"Block should have 1 or 2 columns, but contains " + toString(block.columns()),
|
||||
ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS);
|
||||
if (!block.has(column_name))
|
||||
throw Exception(
|
||||
"Not found column `" + name + "' in block.",
|
||||
ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK);
|
||||
"Not found column '" + column_name + "' in block.",
|
||||
ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK);
|
||||
|
||||
if (i == 0)
|
||||
{
|
||||
column.name = name;
|
||||
column.type = block.getByName(name).type;
|
||||
column.name = column_name;
|
||||
column.type = block.getByName(column_name).type;
|
||||
column.column = column.type->createColumn();
|
||||
}
|
||||
|
||||
if (block.getByName(name).column->getName() != column.column->getName())
|
||||
if (block.getByName(column_name).column->getName() != column.column->getName())
|
||||
throw Exception("Column types don't match", ErrorCodes::INCOMPATIBLE_COLUMNS);
|
||||
}
|
||||
}
|
||||
@ -90,7 +91,7 @@ void ColumnGathererStream::fetchNewBlock(Source & source, size_t source_num)
|
||||
try
|
||||
{
|
||||
source.block = children[source_num]->read();
|
||||
source.update(name);
|
||||
source.update(column_name);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
@ -120,7 +121,7 @@ void ColumnGathererStream::readSuffixImpl()
|
||||
speed << ", " << profile_info.rows / seconds << " rows/sec., "
|
||||
<< profile_info.bytes / 1048576.0 / seconds << " MiB/sec.";
|
||||
LOG_TRACE(log, std::fixed << std::setprecision(2)
|
||||
<< "Gathered column " << name
|
||||
<< "Gathered column " << column_name
|
||||
<< " (" << static_cast<double>(profile_info.bytes) / profile_info.rows << " bytes/elem.)"
|
||||
<< " in " << seconds << " sec."
|
||||
<< speed.str());
|
||||
|
@ -56,8 +56,8 @@ class ColumnGathererStream : public IProfilingBlockInputStream
|
||||
{
|
||||
public:
|
||||
ColumnGathererStream(
|
||||
const String & column_name_, const BlockInputStreams & source_streams, ReadBuffer & row_sources_buf_,
|
||||
size_t block_preferred_size_ = DEFAULT_BLOCK_SIZE);
|
||||
const String & column_name_, const BlockInputStreams & source_streams, ReadBuffer & row_sources_buf_,
|
||||
size_t block_preferred_size_ = DEFAULT_BLOCK_SIZE);
|
||||
|
||||
String getName() const override { return "ColumnGatherer"; }
|
||||
|
||||
@ -96,7 +96,7 @@ private:
|
||||
void init();
|
||||
void fetchNewBlock(Source & source, size_t source_num);
|
||||
|
||||
String name;
|
||||
String column_name;
|
||||
ColumnWithTypeAndName column;
|
||||
|
||||
std::vector<Source> sources;
|
||||
@ -115,7 +115,7 @@ void ColumnGathererStream::gather(Column & column_res)
|
||||
{
|
||||
if (source_to_fully_copy) /// Was set on a previous iteration
|
||||
{
|
||||
output_block.getByPosition(0).column = source_to_fully_copy->block.getByName(name).column;
|
||||
output_block.getByPosition(0).column = source_to_fully_copy->block.getByName(column_name).column;
|
||||
source_to_fully_copy->pos = source_to_fully_copy->size;
|
||||
source_to_fully_copy = nullptr;
|
||||
return;
|
||||
@ -167,7 +167,7 @@ void ColumnGathererStream::gather(Column & column_res)
|
||||
return;
|
||||
}
|
||||
|
||||
output_block.getByPosition(0).column = source.block.getByName(name).column;
|
||||
output_block.getByPosition(0).column = source.block.getByName(column_name).column;
|
||||
source.pos += len;
|
||||
return;
|
||||
}
|
||||
|
@ -36,7 +36,7 @@ public:
|
||||
void writePrefix() override { stream->writePrefix(); }
|
||||
void writeSuffix() override { stream->writeSuffix(); }
|
||||
void flush() override { stream->flush(); }
|
||||
void onProgress(const Progress & progress) override { stream->onProgress(progress); }
|
||||
void onProgress(const Progress & current_progress) override { stream->onProgress(current_progress); }
|
||||
String getContentType() const override { return stream->getContentType(); }
|
||||
|
||||
protected:
|
||||
|
@ -21,8 +21,8 @@ public:
|
||||
*/
|
||||
virtual bool read(MutableColumns & columns) = 0;
|
||||
|
||||
virtual void readPrefix() {}; /// delimiter before begin of result
|
||||
virtual void readSuffix() {}; /// delimiter after end of result
|
||||
virtual void readPrefix() {} /// delimiter before begin of result
|
||||
virtual void readSuffix() {} /// delimiter after end of result
|
||||
|
||||
/// Skip data until next row.
|
||||
/// This is intended for text streams, that allow skipping of errors.
|
||||
@ -33,7 +33,7 @@ public:
|
||||
/// In case of parse error, try to roll back and parse last one or two rows very carefully
|
||||
/// and collect as much as possible diagnostic information about error.
|
||||
/// If not implemented, returns empty string.
|
||||
virtual std::string getDiagnosticInfo() { return {}; };
|
||||
virtual std::string getDiagnosticInfo() { return {}; }
|
||||
|
||||
virtual ~IRowInputStream() {}
|
||||
};
|
||||
|
@ -31,12 +31,12 @@ public:
|
||||
virtual void writeField(const IColumn & column, const IDataType & type, size_t row_num) = 0;
|
||||
|
||||
/** Write delimiter. */
|
||||
virtual void writeFieldDelimiter() {}; /// delimiter between values
|
||||
virtual void writeRowStartDelimiter() {}; /// delimiter before each row
|
||||
virtual void writeRowEndDelimiter() {}; /// delimiter after each row
|
||||
virtual void writeRowBetweenDelimiter() {}; /// delimiter between rows
|
||||
virtual void writePrefix() {}; /// delimiter before resultset
|
||||
virtual void writeSuffix() {}; /// delimiter after resultset
|
||||
virtual void writeFieldDelimiter() {} /// delimiter between values
|
||||
virtual void writeRowStartDelimiter() {} /// delimiter before each row
|
||||
virtual void writeRowEndDelimiter() {} /// delimiter after each row
|
||||
virtual void writeRowBetweenDelimiter() {} /// delimiter between rows
|
||||
virtual void writePrefix() {} /// delimiter before resultset
|
||||
virtual void writeSuffix() {} /// delimiter after resultset
|
||||
|
||||
/** Flush output buffers if any. */
|
||||
virtual void flush() {}
|
||||
|
@ -1,7 +1,17 @@
|
||||
#pragma once
|
||||
|
||||
#include <queue>
|
||||
#include <boost/intrusive_ptr.hpp>
|
||||
|
||||
#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>
|
||||
|
||||
|
@ -40,7 +40,7 @@ public:
|
||||
/// static constexpr bool is_parametric = false;
|
||||
|
||||
/// Name of data type (examples: UInt64, Array(String)).
|
||||
virtual String getName() const { return getFamilyName(); };
|
||||
virtual String getName() const { return getFamilyName(); }
|
||||
|
||||
/// Name of data type family (example: FixedString, Array).
|
||||
virtual const char * getFamilyName() const = 0;
|
||||
@ -319,7 +319,7 @@ public:
|
||||
virtual bool isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion() const
|
||||
{
|
||||
return isValueRepresentedByNumber() || isFixedString();
|
||||
};
|
||||
}
|
||||
|
||||
virtual bool isString() const { return false; }
|
||||
virtual bool isFixedString() const { return false; }
|
||||
|
@ -154,8 +154,6 @@ void DatabaseOrdinary::loadTables(
|
||||
size_t total_tables = file_names.size();
|
||||
LOG_INFO(log, "Total " << total_tables << " tables.");
|
||||
|
||||
String data_path = context.getPath() + "data/" + escapeForFileName(name) + "/";
|
||||
|
||||
AtomicStopwatch watch;
|
||||
std::atomic<size_t> tables_processed {0};
|
||||
|
||||
@ -398,7 +396,7 @@ void DatabaseOrdinary::renameTable(
|
||||
to_database_concrete->name,
|
||||
to_table_name);
|
||||
}
|
||||
catch (const Exception & e)
|
||||
catch (const Exception &)
|
||||
{
|
||||
throw;
|
||||
}
|
||||
|
@ -16,7 +16,7 @@
|
||||
*/
|
||||
|
||||
#if USE_VECTORCLASS
|
||||
#if __clang__
|
||||
#ifdef __clang__
|
||||
#pragma clang diagnostic push
|
||||
#pragma clang diagnostic ignored "-Wshift-negative-value"
|
||||
#endif
|
||||
@ -25,7 +25,7 @@
|
||||
#include <vectormath_exp.h>
|
||||
#include <vectormath_trig.h>
|
||||
|
||||
#if __clang__
|
||||
#ifdef __clang__
|
||||
#pragma clang diagnostic pop
|
||||
#endif
|
||||
#endif
|
||||
|
@ -422,7 +422,7 @@ public:
|
||||
return function->checkNumberOfArguments(number_of_arguments);
|
||||
}
|
||||
|
||||
String getName() const override { return function->getName(); };
|
||||
String getName() const override { return function->getName(); }
|
||||
bool isVariadic() const override { return function->isVariadic(); }
|
||||
size_t getNumberOfArguments() const override { return function->getNumberOfArguments(); }
|
||||
|
||||
|
@ -161,8 +161,8 @@ public:
|
||||
/// Compute and set actual user settings, client_info.current_user should be set
|
||||
void calculateUserSettings();
|
||||
|
||||
ClientInfo & getClientInfo() { return client_info; };
|
||||
const ClientInfo & getClientInfo() const { return client_info; };
|
||||
ClientInfo & getClientInfo() { return client_info; }
|
||||
const ClientInfo & getClientInfo() const { return client_info; }
|
||||
|
||||
void setQuota(const String & name, const String & quota_key, const String & user_name, const Poco::Net::IPAddress & address);
|
||||
QuotaForIntervals & getQuota();
|
||||
@ -285,8 +285,8 @@ public:
|
||||
void setSessionContext(Context & context_) { session_context = &context_; }
|
||||
void setGlobalContext(Context & context_) { global_context = &context_; }
|
||||
|
||||
const Settings & getSettingsRef() const { return settings; };
|
||||
Settings & getSettingsRef() { return settings; };
|
||||
const Settings & getSettingsRef() const { return settings; }
|
||||
Settings & getSettingsRef() { return settings; }
|
||||
|
||||
|
||||
void setProgressCallback(ProgressCallback callback);
|
||||
|
@ -16,7 +16,7 @@ class ASTSelectQuery : public IAST
|
||||
{
|
||||
public:
|
||||
/** Get the text that identifies this element. */
|
||||
String getID() const override { return "SelectQuery"; };
|
||||
String getID() const override { return "SelectQuery"; }
|
||||
|
||||
ASTPtr clone() const override;
|
||||
|
||||
|
@ -16,7 +16,6 @@ struct StringRange
|
||||
const char * second = nullptr;
|
||||
|
||||
StringRange() = default;
|
||||
StringRange(const StringRange & other) = default;
|
||||
StringRange(const char * begin, const char * end) : first(begin), second(end) {}
|
||||
explicit StringRange(TokenIterator token) : first(token->begin), second(token->end) {}
|
||||
|
||||
|
@ -12,7 +12,16 @@ namespace DB
|
||||
*/
|
||||
void formatAST(const IAST & ast, std::ostream & s, bool hilite = true, bool one_line = false);
|
||||
|
||||
inline std::ostream & operator<<(std::ostream & os, const IAST & ast) { return formatAST(ast, os, false, true), os; }
|
||||
inline std::ostream & operator<<(std::ostream & os, const ASTPtr & ast) { return formatAST(*ast, os, false, true), os; }
|
||||
inline std::ostream & operator<<(std::ostream & os, const IAST & ast)
|
||||
{
|
||||
formatAST(ast, os, false, true);
|
||||
return os;
|
||||
}
|
||||
|
||||
inline std::ostream & operator<<(std::ostream & os, const ASTPtr & ast)
|
||||
{
|
||||
formatAST(*ast, os, false, true);
|
||||
return os;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -131,10 +131,10 @@ public:
|
||||
TableFullWriteLock lockForAlter(const std::string & who = "Alter")
|
||||
{
|
||||
/// The calculation order is important.
|
||||
auto data_lock = lockDataForAlter(who);
|
||||
auto structure_lock = lockStructureForAlter(who);
|
||||
auto res_data_lock = lockDataForAlter(who);
|
||||
auto res_structure_lock = lockStructureForAlter(who);
|
||||
|
||||
return {std::move(data_lock), std::move(structure_lock)};
|
||||
return {std::move(res_data_lock), std::move(res_structure_lock)};
|
||||
}
|
||||
|
||||
/** Does not allow changing the data in the table. (Moreover, does not give a look at the structure of the table with the intention to change the data).
|
||||
|
@ -34,7 +34,7 @@ public:
|
||||
/// Create storage according to the query.
|
||||
StoragePtr execute(const ASTPtr & ast_function, const Context & context) const;
|
||||
|
||||
virtual ~ITableFunction() {};
|
||||
virtual ~ITableFunction() {}
|
||||
|
||||
private:
|
||||
virtual StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context) const = 0;
|
||||
|
Loading…
Reference in New Issue
Block a user