Better [#CLICKHOUSE-2].

This commit is contained in:
Alexey Milovidov 2017-12-01 20:49:12 +03:00
parent 0ac2f5b479
commit f5ecb2db5d
27 changed files with 163 additions and 162 deletions

View File

@ -18,6 +18,8 @@ get_property (BUILD_INCLUDE_DIRECTORIES DIRECTORY ${ClickHouse_SOURCE_DIR} PROPE
string (TIMESTAMP BUILD_DATE "%Y-%m-%d" UTC)
configure_file (${CMAKE_CURRENT_SOURCE_DIR}/src/Common/config_build.cpp.in ${CONFIG_BUILD})
#set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wextra")
if (NOT NO_WERROR)
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Werror")
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Werror")

View File

@ -99,7 +99,7 @@ public:
void updateHashWithValue(size_t index, SipHash & hash) const override;
int compareAt(size_t p1, size_t p2, const IColumn & rhs_, int nan_direction_hint) const override
int compareAt(size_t p1, size_t p2, const IColumn & rhs_, int /*nan_direction_hint*/) const override
{
const ColumnFixedString & rhs = static_cast<const ColumnFixedString &>(rhs_);
return memcmp(&chars[p1 * n], &rhs.chars[p2 * n], n);

View File

@ -16,8 +16,8 @@ namespace DB
template <typename T>
struct CompareHelper
{
static bool less(T a, T b, int nan_direction_hint) { return a < b; }
static bool greater(T a, T b, int nan_direction_hint) { return a > b; }
static bool less(T a, T b, int /*nan_direction_hint*/) { return a < b; }
static bool greater(T a, T b, int /*nan_direction_hint*/) { return a > b; }
/** Compares two numbers. Returns a number less than zero, equal to zero, or greater than zero if a < b, a == b, a > b, respectively.
* If one of the values is NaN, then
@ -25,7 +25,7 @@ struct CompareHelper
* - if nan_direction_hint == 1 - NaN are considered to be larger than all numbers;
* Essentially: nan_direction_hint == -1 says that the comparison is for sorting in descending order.
*/
static int compare(T a, T b, int nan_direction_hint)
static int compare(T a, T b, int /*nan_direction_hint*/)
{
return a > b ? 1 : (a < b ? -1 : 0);
}
@ -156,7 +156,7 @@ public:
data.push_back(static_cast<const Self &>(src).getData()[n]);
}
void insertData(const char * pos, size_t length) override
void insertData(const char * pos, size_t /*length*/) override
{
data.push_back(*reinterpret_cast<const T *>(pos));
}

View File

@ -77,7 +77,7 @@ public:
/// Creates column with the same type and specified size.
/// If size is less current size, then data is cut.
/// If size is greater, than default values are appended.
virtual ColumnPtr cloneResized(size_t size) const { throw Exception("Cannot cloneResized() column " + getName(), ErrorCodes::NOT_IMPLEMENTED); }
virtual ColumnPtr cloneResized(size_t /*size*/) const { throw Exception("Cannot cloneResized() column " + getName(), ErrorCodes::NOT_IMPLEMENTED); }
/// Returns number of values in column.
virtual size_t size() const = 0;
@ -106,7 +106,7 @@ public:
/// If column stores integers, it returns n-th element transformed to UInt64 using static_cast.
/// If column stores floting point numbers, bits of n-th elements are copied to lower bits of UInt64, the remaining bits are zeros.
/// Is used to optimize some computations (in aggregation, for example).
virtual UInt64 get64(size_t n) const
virtual UInt64 get64(size_t /*n*/) const
{
throw Exception("Method get64 is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
@ -114,12 +114,12 @@ public:
/** If column is numeric, return value of n-th element, casted to UInt64.
* Otherwise throw an exception.
*/
virtual UInt64 getUInt(size_t n) const
virtual UInt64 getUInt(size_t /*n*/) const
{
throw Exception("Method getUInt is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
virtual Int64 getInt(size_t n) const
virtual Int64 getInt(size_t /*n*/) const
{
throw Exception("Method getInt is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
@ -253,7 +253,7 @@ public:
/// Reserves memory for specified amount of elements. If reservation isn't possible, does nothing.
/// It affects performance only (not correctness).
virtual void reserve(size_t n) {};
virtual void reserve(size_t /*n*/) {};
/// Size of column data in memory (may be approximate) - for profiling. Zero, if could not be determined.
virtual size_t byteSize() const = 0;

View File

@ -128,7 +128,7 @@ class AIOContextPool : public ext::singleton<AIOContextPool>
int getCompletionEvents(io_event events[], const int max_events)
{
timespec timeout{timeout_sec};
timespec timeout{timeout_sec, 0};
auto num_events = 0;

View File

@ -30,7 +30,7 @@ static inline String formatQuotedWithPrefix(T x, const char * prefix)
}
String FieldVisitorDump::operator() (const Null & x) const { return "NULL"; }
String FieldVisitorDump::operator() (const Null &) const { return "NULL"; }
String FieldVisitorDump::operator() (const UInt64 & x) const { return formatQuotedWithPrefix(x, "UInt64_"); }
String FieldVisitorDump::operator() (const Int64 & x) const { return formatQuotedWithPrefix(x, "Int64_"); }
String FieldVisitorDump::operator() (const Float64 & x) const { return formatQuotedWithPrefix(x, "Float64_"); }
@ -99,7 +99,7 @@ static String formatFloat(const Float64 x)
}
String FieldVisitorToString::operator() (const Null & x) const { return "NULL"; }
String FieldVisitorToString::operator() (const Null &) const { return "NULL"; }
String FieldVisitorToString::operator() (const UInt64 & x) const { return formatQuoted(x); }
String FieldVisitorToString::operator() (const Int64 & x) const { return formatQuoted(x); }
String FieldVisitorToString::operator() (const Float64 & x) const { return formatFloat(x); }
@ -142,7 +142,7 @@ String FieldVisitorToString::operator() (const Tuple & x_def) const
FieldVisitorHash::FieldVisitorHash(SipHash & hash) : hash(hash) {}
void FieldVisitorHash::operator() (const Null & x) const
void FieldVisitorHash::operator() (const Null &) const
{
UInt8 type = Field::Types::Null;
hash.update(reinterpret_cast<const char *>(&type), sizeof(type));

View File

@ -133,22 +133,22 @@ template <typename T>
class FieldVisitorConvertToNumber : public StaticVisitor<T>
{
public:
T operator() (const Null & x) const
T operator() (const Null &) const
{
throw Exception("Cannot convert NULL to " + String(TypeName<T>::get()), ErrorCodes::CANNOT_CONVERT_TYPE);
}
T operator() (const String & x) const
T operator() (const String &) const
{
throw Exception("Cannot convert String to " + String(TypeName<T>::get()), ErrorCodes::CANNOT_CONVERT_TYPE);
}
T operator() (const Array & x) const
T operator() (const Array &) const
{
throw Exception("Cannot convert Array to " + String(TypeName<T>::get()), ErrorCodes::CANNOT_CONVERT_TYPE);
}
T operator() (const Tuple & x) const
T operator() (const Tuple &) const
{
throw Exception("Cannot convert Tuple to " + String(TypeName<T>::get()), ErrorCodes::CANNOT_CONVERT_TYPE);
}
@ -183,120 +183,120 @@ public:
class FieldVisitorAccurateEquals : public StaticVisitor<bool>
{
public:
bool operator() (const Null & l, const Null & r) const { return true; }
bool operator() (const Null & l, const UInt64 & r) const { return false; }
bool operator() (const Null & l, const Int64 & r) const { return false; }
bool operator() (const Null & l, const Float64 & r) const { return false; }
bool operator() (const Null & l, const String & r) const { return false; }
bool operator() (const Null & l, const Array & r) const { return false; }
bool operator() (const Null & l, const Tuple & r) const { return false; }
bool operator() (const Null &, const Null &) const { return true; }
bool operator() (const Null &, const UInt64 &) const { return false; }
bool operator() (const Null &, const Int64 &) const { return false; }
bool operator() (const Null &, const Float64 &) const { return false; }
bool operator() (const Null &, const String &) const { return false; }
bool operator() (const Null &, const Array &) const { return false; }
bool operator() (const Null &, const Tuple &) const { return false; }
bool operator() (const UInt64 & l, const Null & r) const { return false; }
bool operator() (const UInt64 &, const Null &) const { return false; }
bool operator() (const UInt64 & l, const UInt64 & r) const { return l == r; }
bool operator() (const UInt64 & l, const Int64 & r) const { return accurate::equalsOp(l, r); }
bool operator() (const UInt64 & l, const Float64 & r) const { return accurate::equalsOp(l, r); }
bool operator() (const UInt64 & l, const String & r) const { return false; }
bool operator() (const UInt64 & l, const Array & r) const { return false; }
bool operator() (const UInt64 & l, const Tuple & r) const { return false; }
bool operator() (const UInt64 &, const String &) const { return false; }
bool operator() (const UInt64 &, const Array &) const { return false; }
bool operator() (const UInt64 &, const Tuple &) const { return false; }
bool operator() (const Int64 & l, const Null & r) const { return false; }
bool operator() (const Int64 &, const Null &) const { return false; }
bool operator() (const Int64 & l, const UInt64 & r) const { return accurate::equalsOp(l, r); }
bool operator() (const Int64 & l, const Int64 & r) const { return l == r; }
bool operator() (const Int64 & l, const Float64 & r) const { return accurate::equalsOp(l, r); }
bool operator() (const Int64 & l, const String & r) const { return false; }
bool operator() (const Int64 & l, const Array & r) const { return false; }
bool operator() (const Int64 & l, const Tuple & r) const { return false; }
bool operator() (const Int64 &, const String &) const { return false; }
bool operator() (const Int64 &, const Array &) const { return false; }
bool operator() (const Int64 &, const Tuple &) const { return false; }
bool operator() (const Float64 & l, const Null & r) const { return false; }
bool operator() (const Float64 &, const Null &) const { return false; }
bool operator() (const Float64 & l, const UInt64 & r) const { return accurate::equalsOp(l, r); }
bool operator() (const Float64 & l, const Int64 & r) const { return accurate::equalsOp(l, r); }
bool operator() (const Float64 & l, const Float64 & r) const { return l == r; }
bool operator() (const Float64 & l, const String & r) const { return false; }
bool operator() (const Float64 & l, const Array & r) const { return false; }
bool operator() (const Float64 & l, const Tuple & r) const { return false; }
bool operator() (const Float64 &, const String &) const { return false; }
bool operator() (const Float64 &, const Array &) const { return false; }
bool operator() (const Float64 &, const Tuple &) const { return false; }
bool operator() (const String & l, const Null & r) const { return false; }
bool operator() (const String & l, const UInt64 & r) const { return false; }
bool operator() (const String & l, const Int64 & r) const { return false; }
bool operator() (const String & l, const Float64 & r) const { return false; }
bool operator() (const String &, const Null &) const { return false; }
bool operator() (const String &, const UInt64 &) const { return false; }
bool operator() (const String &, const Int64 &) const { return false; }
bool operator() (const String &, const Float64 &) const { return false; }
bool operator() (const String & l, const String & r) const { return l == r; }
bool operator() (const String & l, const Array & r) const { return false; }
bool operator() (const String & l, const Tuple & r) const { return false; }
bool operator() (const String &, const Array &) const { return false; }
bool operator() (const String &, const Tuple &) const { return false; }
bool operator() (const Array & l, const Null & r) const { return false; }
bool operator() (const Array & l, const UInt64 & r) const { return false; }
bool operator() (const Array & l, const Int64 & r) const { return false; }
bool operator() (const Array & l, const Float64 & r) const { return false; }
bool operator() (const Array & l, const String & r) const { return false; }
bool operator() (const Array &, const Null &) const { return false; }
bool operator() (const Array &, const UInt64 &) const { return false; }
bool operator() (const Array &, const Int64 &) const { return false; }
bool operator() (const Array &, const Float64 &) const { return false; }
bool operator() (const Array &, const String &) const { return false; }
bool operator() (const Array & l, const Array & r) const { return l == r; }
bool operator() (const Array & l, const Tuple & r) const { return false; }
bool operator() (const Array &, const Tuple &) const { return false; }
bool operator() (const Tuple & l, const Null & r) const { return false; }
bool operator() (const Tuple & l, const UInt64 & r) const { return false; }
bool operator() (const Tuple & l, const Int64 & r) const { return false; }
bool operator() (const Tuple & l, const Float64 & r) const { return false; }
bool operator() (const Tuple & l, const String & r) const { return false; }
bool operator() (const Tuple & l, const Array & r) const { return false; }
bool operator() (const Tuple &, const Null &) const { return false; }
bool operator() (const Tuple &, const UInt64 &) const { return false; }
bool operator() (const Tuple &, const Int64 &) const { return false; }
bool operator() (const Tuple &, const Float64 &) const { return false; }
bool operator() (const Tuple &, const String &) const { return false; }
bool operator() (const Tuple &, const Array &) const { return false; }
bool operator() (const Tuple & l, const Tuple & r) const { return l == r; }
};
class FieldVisitorAccurateLess : public StaticVisitor<bool>
{
public:
bool operator() (const Null & l, const Null & r) const { return false; }
bool operator() (const Null & l, const UInt64 & r) const { return true; }
bool operator() (const Null & l, const Int64 & r) const { return true; }
bool operator() (const Null & l, const Float64 & r) const { return true; }
bool operator() (const Null & l, const String & r) const { return true; }
bool operator() (const Null & l, const Array & r) const { return true; }
bool operator() (const Null & l, const Tuple & r) const { return true; }
bool operator() (const Null &, const Null &) const { return false; }
bool operator() (const Null &, const UInt64 &) const { return true; }
bool operator() (const Null &, const Int64 &) const { return true; }
bool operator() (const Null &, const Float64 &) const { return true; }
bool operator() (const Null &, const String &) const { return true; }
bool operator() (const Null &, const Array &) const { return true; }
bool operator() (const Null &, const Tuple &) const { return true; }
bool operator() (const UInt64 & l, const Null & r) const { return false; }
bool operator() (const UInt64 &, const Null &) const { return false; }
bool operator() (const UInt64 & l, const UInt64 & r) const { return l < r; }
bool operator() (const UInt64 & l, const Int64 & r) const { return accurate::lessOp(l, r); }
bool operator() (const UInt64 & l, const Float64 & r) const { return accurate::lessOp(l, r); }
bool operator() (const UInt64 & l, const String & r) const { return true; }
bool operator() (const UInt64 & l, const Array & r) const { return true; }
bool operator() (const UInt64 & l, const Tuple & r) const { return true; }
bool operator() (const UInt64 &, const String &) const { return true; }
bool operator() (const UInt64 &, const Array &) const { return true; }
bool operator() (const UInt64 &, const Tuple &) const { return true; }
bool operator() (const Int64 & l, const Null & r) const { return false; }
bool operator() (const Int64 &, const Null &) const { return false; }
bool operator() (const Int64 & l, const UInt64 & r) const { return accurate::lessOp(l, r); }
bool operator() (const Int64 & l, const Int64 & r) const { return l < r; }
bool operator() (const Int64 & l, const Float64 & r) const { return accurate::lessOp(l, r); }
bool operator() (const Int64 & l, const String & r) const { return true; }
bool operator() (const Int64 & l, const Array & r) const { return true; }
bool operator() (const Int64 & l, const Tuple & r) const { return true; }
bool operator() (const Int64 &, const String &) const { return true; }
bool operator() (const Int64 &, const Array &) const { return true; }
bool operator() (const Int64 &, const Tuple &) const { return true; }
bool operator() (const Float64 & l, const Null & r) const { return false; }
bool operator() (const Float64 &, const Null &) const { return false; }
bool operator() (const Float64 & l, const UInt64 & r) const { return accurate::lessOp(l, r); }
bool operator() (const Float64 & l, const Int64 & r) const { return accurate::lessOp(l, r); }
bool operator() (const Float64 & l, const Float64 & r) const { return l < r; }
bool operator() (const Float64 & l, const String & r) const { return true; }
bool operator() (const Float64 & l, const Array & r) const { return true; }
bool operator() (const Float64 & l, const Tuple & r) const { return true; }
bool operator() (const Float64 &, const String &) const { return true; }
bool operator() (const Float64 &, const Array &) const { return true; }
bool operator() (const Float64 &, const Tuple &) const { return true; }
bool operator() (const String & l, const Null & r) const { return false; }
bool operator() (const String & l, const UInt64 & r) const { return false; }
bool operator() (const String & l, const Int64 & r) const { return false; }
bool operator() (const String & l, const Float64 & r) const { return false; }
bool operator() (const String &, const Null &) const { return false; }
bool operator() (const String &, const UInt64 &) const { return false; }
bool operator() (const String &, const Int64 &) const { return false; }
bool operator() (const String &, const Float64 &) const { return false; }
bool operator() (const String & l, const String & r) const { return l < r; }
bool operator() (const String & l, const Array & r) const { return true; }
bool operator() (const String & l, const Tuple & r) const { return true; }
bool operator() (const String &, const Array &) const { return true; }
bool operator() (const String &, const Tuple &) const { return true; }
bool operator() (const Array & l, const Null & r) const { return false; }
bool operator() (const Array & l, const UInt64 & r) const { return false; }
bool operator() (const Array & l, const Int64 & r) const { return false; }
bool operator() (const Array & l, const Float64 & r) const { return false; }
bool operator() (const Array & l, const String & r) const { return false; }
bool operator() (const Array &, const Null &) const { return false; }
bool operator() (const Array &, const UInt64 &) const { return false; }
bool operator() (const Array &, const Int64 &) const { return false; }
bool operator() (const Array &, const Float64 &) const { return false; }
bool operator() (const Array &, const String &) const { return false; }
bool operator() (const Array & l, const Array & r) const { return l < r; }
bool operator() (const Array & l, const Tuple & r) const { return false; }
bool operator() (const Array &, const Tuple &) const { return false; }
bool operator() (const Tuple & l, const Null & r) const { return false; }
bool operator() (const Tuple & l, const UInt64 & r) const { return false; }
bool operator() (const Tuple & l, const Int64 & r) const { return false; }
bool operator() (const Tuple & l, const Float64 & r) const { return false; }
bool operator() (const Tuple & l, const String & r) const { return false; }
bool operator() (const Tuple & l, const Array & r) const { return false; }
bool operator() (const Tuple &, const Null &) const { return false; }
bool operator() (const Tuple &, const UInt64 &) const { return false; }
bool operator() (const Tuple &, const Int64 &) const { return false; }
bool operator() (const Tuple &, const Float64 &) const { return false; }
bool operator() (const Tuple &, const String &) const { return false; }
bool operator() (const Tuple &, const Array &) const { return false; }
bool operator() (const Tuple & l, const Tuple & r) const { return l < r; }
};
@ -314,9 +314,9 @@ public:
bool operator() (Int64 & x) const { x += get<Int64>(rhs); return x != 0; }
bool operator() (Float64 & x) const { x += get<Float64>(rhs); return x != 0; }
bool operator() (Null & x) const { throw Exception("Cannot sum Nulls", ErrorCodes::LOGICAL_ERROR); }
bool operator() (String & x) const { throw Exception("Cannot sum Strings", ErrorCodes::LOGICAL_ERROR); }
bool operator() (Array & x) const { throw Exception("Cannot sum Arrays", ErrorCodes::LOGICAL_ERROR); }
bool operator() (Null &) const { throw Exception("Cannot sum Nulls", ErrorCodes::LOGICAL_ERROR); }
bool operator() (String &) const { throw Exception("Cannot sum Strings", ErrorCodes::LOGICAL_ERROR); }
bool operator() (Array &) const { throw Exception("Cannot sum Arrays", ErrorCodes::LOGICAL_ERROR); }
};
}

View File

@ -16,7 +16,7 @@ namespace DB
template <typename T>
struct TrivialWeightFunction
{
size_t operator()(const T & x) const
size_t operator()(const T &) const
{
return 1;
}
@ -188,14 +188,14 @@ private:
InsertTokenHolder() = default;
void acquire(const Key * key_, const std::shared_ptr<InsertToken> & token_, std::lock_guard<std::mutex> & cache_lock)
void acquire(const Key * key_, const std::shared_ptr<InsertToken> & token_, [[maybe_unused]] std::lock_guard<std::mutex> & cache_lock)
{
key = key_;
token = token_;
++token->refcount;
}
void cleanup(std::lock_guard<std::mutex> & token_lock, std::lock_guard<std::mutex> & cache_lock)
void cleanup([[maybe_unused]] std::lock_guard<std::mutex> & token_lock, [[maybe_unused]] std::lock_guard<std::mutex> & cache_lock)
{
token->cache.insert_tokens.erase(*key);
token->cleaned_up = true;
@ -260,7 +260,7 @@ private:
WeightFunction weight_function;
MappedPtr getImpl(const Key & key, std::lock_guard<std::mutex> & cache_lock)
MappedPtr getImpl(const Key & key, [[maybe_unused]] std::lock_guard<std::mutex> & cache_lock)
{
auto it = cells.find(key);
if (it == cells.end())
@ -277,7 +277,7 @@ private:
return cell.value;
}
void setImpl(const Key & key, const MappedPtr & mapped, std::lock_guard<std::mutex> & cache_lock)
void setImpl(const Key & key, const MappedPtr & mapped, [[maybe_unused]] std::lock_guard<std::mutex> & cache_lock)
{
auto res = cells.emplace(std::piecewise_construct,
std::forward_as_tuple(key),

View File

@ -60,7 +60,7 @@ struct WatchContext
}
};
void ZooKeeper::processCallback(zhandle_t * zh, int type, int state, const char * path, void * watcher_ctx)
void ZooKeeper::processCallback(zhandle_t *, int type, int state, const char * path, void * watcher_ctx)
{
WatchContext * context = static_cast<WatchContext *>(watcher_ctx);
context->process(type, state, path);

View File

@ -9,7 +9,7 @@ int func(int x, int y)
}
int main(int argc, char ** argv)
int main(int, char **)
{
SimpleCache<decltype(func), &func> func_cached;

View File

@ -108,7 +108,7 @@ public:
template <typename T>
Field(T && rhs,
typename std::enable_if<!std::is_same<typename std::decay<T>::type, Field>::value, void>::type * unused = nullptr)
[[maybe_unused]] typename std::enable_if<!std::is_same<typename std::decay<T>::type, Field>::value, void>::type * unused = nullptr)
{
createConcrete(std::forward<T>(rhs));
}
@ -523,25 +523,25 @@ class WriteBuffer;
/// It is assumed that all elements of the array have the same type.
void readBinary(Array & x, ReadBuffer & buf);
inline void readText(Array & x, ReadBuffer & buf) { throw Exception("Cannot read Array.", ErrorCodes::NOT_IMPLEMENTED); }
inline void readQuoted(Array & x, ReadBuffer & buf) { throw Exception("Cannot read Array.", ErrorCodes::NOT_IMPLEMENTED); }
inline void readText(Array &, ReadBuffer &) { throw Exception("Cannot read Array.", ErrorCodes::NOT_IMPLEMENTED); }
inline void readQuoted(Array &, ReadBuffer &) { throw Exception("Cannot read Array.", ErrorCodes::NOT_IMPLEMENTED); }
/// It is assumed that all elements of the array have the same type.
void writeBinary(const Array & x, WriteBuffer & buf);
void writeText(const Array & x, WriteBuffer & buf);
inline void writeQuoted(const Array & x, WriteBuffer & buf) { throw Exception("Cannot write Array quoted.", ErrorCodes::NOT_IMPLEMENTED); }
inline void writeQuoted(const Array &, WriteBuffer &) { throw Exception("Cannot write Array quoted.", ErrorCodes::NOT_IMPLEMENTED); }
void readBinary(Tuple & x, ReadBuffer & buf);
inline void readText(Tuple & x, ReadBuffer & buf) { throw Exception("Cannot read Tuple.", ErrorCodes::NOT_IMPLEMENTED); }
inline void readQuoted(Tuple & x, ReadBuffer & buf) { throw Exception("Cannot read Tuple.", ErrorCodes::NOT_IMPLEMENTED); }
inline void readText(Tuple &, ReadBuffer &) { throw Exception("Cannot read Tuple.", ErrorCodes::NOT_IMPLEMENTED); }
inline void readQuoted(Tuple &, ReadBuffer &) { throw Exception("Cannot read Tuple.", ErrorCodes::NOT_IMPLEMENTED); }
void writeBinary(const Tuple & x, WriteBuffer & buf);
void writeText(const Tuple & x, WriteBuffer & buf);
inline void writeQuoted(const Tuple & x, WriteBuffer & buf) { throw Exception("Cannot write Tuple quoted.", ErrorCodes::NOT_IMPLEMENTED); }
inline void writeQuoted(const Tuple &, WriteBuffer &) { throw Exception("Cannot write Tuple quoted.", ErrorCodes::NOT_IMPLEMENTED); }
}

View File

@ -247,7 +247,7 @@ struct Factory3 : IFactory
};
int main(int argc, char ** argv)
int main(int, char **)
{
srand(time(0));

View File

@ -124,7 +124,7 @@ void DistinctBlockInputStream::buildFilter(
method.data.emplace(key, it, inserted);
if (inserted)
method.onNewKey(*it, columns.size(), i, variants.string_pool);
method.onNewKey(*it, columns.size(), variants.string_pool);
/// Emit the record if there is no such key in the current set yet.
/// Skip it otherwise.

View File

@ -147,7 +147,7 @@ bool DistinctSortedBlockInputStream::buildFilter(
if (inserted)
{
method.onNewKey(*it, columns.size(), i, variants.string_pool);
method.onNewKey(*it, columns.size(), variants.string_pool);
has_new_data = true;
}

View File

@ -47,10 +47,10 @@ template <> inline WriteBuffer & operator<< (WriteBuffer & buf, const cha
inline WriteBuffer & operator<< (WriteBuffer & buf, const char * x) { writeCString(x, buf); return buf; }
inline EscapeManipWriteBuffer & operator<< (WriteBuffer & buf, EscapeManip x) { return static_cast<EscapeManipWriteBuffer &>(buf); }
inline QuoteManipWriteBuffer & operator<< (WriteBuffer & buf, QuoteManip x) { return static_cast<QuoteManipWriteBuffer &>(buf); }
inline DoubleQuoteManipWriteBuffer & operator<< (WriteBuffer & buf, DoubleQuoteManip x) { return static_cast<DoubleQuoteManipWriteBuffer &>(buf); }
inline BinaryManipWriteBuffer & operator<< (WriteBuffer & buf, BinaryManip x) { return static_cast<BinaryManipWriteBuffer &>(buf); }
inline EscapeManipWriteBuffer & operator<< (WriteBuffer & buf, EscapeManip) { return static_cast<EscapeManipWriteBuffer &>(buf); }
inline QuoteManipWriteBuffer & operator<< (WriteBuffer & buf, QuoteManip) { return static_cast<QuoteManipWriteBuffer &>(buf); }
inline DoubleQuoteManipWriteBuffer & operator<< (WriteBuffer & buf, DoubleQuoteManip) { return static_cast<DoubleQuoteManipWriteBuffer &>(buf); }
inline BinaryManipWriteBuffer & operator<< (WriteBuffer & buf, BinaryManip) { return static_cast<BinaryManipWriteBuffer &>(buf); }
template <typename T> WriteBuffer & operator<< (EscapeManipWriteBuffer & buf, const T & x) { writeText(x, buf); return buf; }
template <typename T> WriteBuffer & operator<< (QuoteManipWriteBuffer & buf, const T & x) { writeQuoted(x, buf); return buf; }
@ -65,7 +65,7 @@ inline WriteBuffer & operator<< (BinaryManipWriteBuffer & buf, const char *
/// The manipulator calls the WriteBuffer method `next` - this makes the buffer reset. For nested buffers, the reset is not recursive.
enum FlushManip { flush };
inline WriteBuffer & operator<< (WriteBuffer & buf, FlushManip x) { buf.next(); return buf; }
inline WriteBuffer & operator<< (WriteBuffer & buf, FlushManip) { buf.next(); return buf; }
template <typename T> ReadBuffer & operator>> (ReadBuffer & buf, T & x) { readText(x, buf); return buf; }
@ -75,10 +75,10 @@ template <> inline ReadBuffer & operator>> (ReadBuffer & buf, char & x)
/// If you specify a string literal for reading, this will mean - make sure there is a sequence of bytes and skip it.
inline ReadBuffer & operator>> (ReadBuffer & buf, const char * x) { assertString(x, buf); return buf; }
inline EscapeManipReadBuffer & operator>> (ReadBuffer & buf, EscapeManip x) { return static_cast<EscapeManipReadBuffer &>(buf); }
inline QuoteManipReadBuffer & operator>> (ReadBuffer & buf, QuoteManip x) { return static_cast<QuoteManipReadBuffer &>(buf); }
inline DoubleQuoteManipReadBuffer & operator>> (ReadBuffer & buf, DoubleQuoteManip x) { return static_cast<DoubleQuoteManipReadBuffer &>(buf); }
inline BinaryManipReadBuffer & operator>> (ReadBuffer & buf, BinaryManip x) { return static_cast<BinaryManipReadBuffer &>(buf); }
inline EscapeManipReadBuffer & operator>> (ReadBuffer & buf, EscapeManip) { return static_cast<EscapeManipReadBuffer &>(buf); }
inline QuoteManipReadBuffer & operator>> (ReadBuffer & buf, QuoteManip) { return static_cast<QuoteManipReadBuffer &>(buf); }
inline DoubleQuoteManipReadBuffer & operator>> (ReadBuffer & buf, DoubleQuoteManip) { return static_cast<DoubleQuoteManipReadBuffer &>(buf); }
inline BinaryManipReadBuffer & operator>> (ReadBuffer & buf, BinaryManip) { return static_cast<BinaryManipReadBuffer &>(buf); }
template <typename T> ReadBuffer & operator>> (EscapeManipReadBuffer & buf, T & x) { readText(x, buf); return buf; }
template <typename T> ReadBuffer & operator>> (QuoteManipReadBuffer & buf, T & x) { readQuoted(x, buf); return buf; }

View File

@ -9,7 +9,7 @@
namespace DB
{
void Progress::read(ReadBuffer & in, UInt64 server_revision)
void Progress::read(ReadBuffer & in, UInt64 /*server_revision*/)
{
size_t new_rows = 0;
size_t new_bytes = 0;
@ -25,7 +25,7 @@ void Progress::read(ReadBuffer & in, UInt64 server_revision)
}
void Progress::write(WriteBuffer & out, UInt64 client_revision) const
void Progress::write(WriteBuffer & out, UInt64 /*client_revision*/) const
{
writeVarUInt(rows.load(), out);
writeVarUInt(bytes.load(), out);

View File

@ -63,7 +63,6 @@ ReadBufferFromFile::ReadBufferFromFile(
int fd,
const std::string & original_file_name,
size_t buf_size,
int flags,
char * existing_memory,
size_t alignment)
:

View File

@ -29,7 +29,7 @@ public:
char * existing_memory = nullptr, size_t alignment = 0);
/// Use pre-opened file descriptor.
ReadBufferFromFile(int fd, const std::string & original_file_name = {}, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, int flags = -1,
ReadBufferFromFile(int fd, const std::string & original_file_name = {}, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
char * existing_memory = nullptr, size_t alignment = 0);
ReadBufferFromFile(ReadBufferFromFile &&) = default;

View File

@ -730,7 +730,7 @@ inline void readText(String & x, ReadBuffer & buf) { readEscapedString(x, buf);
inline void readText(LocalDate & x, ReadBuffer & buf) { readDateText(x, buf); }
inline void readText(LocalDateTime & x, ReadBuffer & buf) { readDateTimeText(x, buf); }
inline void readText(UUID & x, ReadBuffer & buf) { readUUIDText(x, buf); }
inline void readText(UInt128 & x, ReadBuffer & buf)
inline void readText(UInt128 &, ReadBuffer &)
{
/** Because UInt128 isn't a natural type, without arithmetic operator and only use as an intermediary type -for UUID-
* it should never arrive here. But because we used the DataTypeNumber class we should have at least a definition of it.
@ -825,7 +825,7 @@ inline void readCSV(String & x, ReadBuffer & buf, const char delimiter = ',') {
inline void readCSV(LocalDate & x, ReadBuffer & buf) { readCSVSimple(x, buf); }
inline void readCSV(LocalDateTime & x, ReadBuffer & buf) { readCSVSimple(x, buf); }
inline void readCSV(UUID & x, ReadBuffer & buf) { readCSVSimple(x, buf); }
inline void readCSV(UInt128 & x, ReadBuffer & buf)
inline void readCSV(UInt128 &, ReadBuffer &)
{
/** Because UInt128 isn't a natural type, without arithmetic operator and only use as an intermediary type -for UUID-
* it should never arrive here. But because we used the DataTypeNumber class we should have at least a definition of it.

View File

@ -671,7 +671,7 @@ inline void writeText(const char * x, size_t size, WriteBuffer & buf) { writeEsc
inline void writeText(const LocalDate & x, WriteBuffer & buf) { writeDateText(x, buf); }
inline void writeText(const LocalDateTime & x, WriteBuffer & buf) { writeDateTimeText(x, buf); }
inline void writeText(const UUID & x, WriteBuffer & buf) { writeUUIDText(x, buf); }
inline void writeText(const UInt128 & x, WriteBuffer & buf)
inline void writeText(const UInt128 &, WriteBuffer &)
{
/** Because UInt128 isn't a natural type, without arithmetic operator and only use as an intermediary type -for UUID-
* it should never arrive here. But because we used the DataTypeNumber class we should have at least a definition of it.
@ -739,7 +739,7 @@ inline void writeCSV(const String & x, WriteBuffer & buf) { writeCSVString<>(x,
inline void writeCSV(const LocalDate & x, WriteBuffer & buf) { writeDoubleQuoted(x, buf); }
inline void writeCSV(const LocalDateTime & x, WriteBuffer & buf) { writeDoubleQuoted(x, buf); }
inline void writeCSV(const UUID & x, WriteBuffer & buf) { writeDoubleQuoted(x, buf); }
inline void writeCSV(const UInt128, WriteBuffer & buf)
inline void writeCSV(const UInt128, WriteBuffer &)
{
/** Because UInt128 isn't a natural type, without arithmetic operator and only use as an intermediary type -for UUID-
* it should never arrive here. But because we used the DataTypeNumber class we should have at least a definition of it.

View File

@ -211,7 +211,7 @@ static inline UInt128 ALWAYS_INLINE hash128(
/// Copy keys to the pool. Then put into pool StringRefs to them and return the pointer to the first.
static inline StringRef * ALWAYS_INLINE placeKeysInPool(
size_t i, size_t keys_size, StringRefs & keys, Arena & pool)
size_t keys_size, StringRefs & keys, Arena & pool)
{
for (size_t j = 0; j < keys_size; ++j)
{
@ -291,7 +291,7 @@ inline StringRef ALWAYS_INLINE extractKeysAndPlaceInPoolContiguous(
/** Serialize keys into a continuous chunk of memory.
*/
static inline StringRef ALWAYS_INLINE serializeKeysToPoolContiguous(
size_t i, size_t keys_size, const ConstColumnPlainPtrs & key_columns, StringRefs & keys, Arena & pool)
size_t i, size_t keys_size, const ConstColumnPlainPtrs & key_columns, Arena & pool)
{
const char * begin = nullptr;

View File

@ -663,7 +663,7 @@ void NO_INLINE Aggregator::executeImplCase(
/// exception-safety - if you can not allocate memory or create states, then destructors will not be called.
aggregate_data = nullptr;
method.onNewKey(*it, params.keys_size, i, keys, *aggregates_pool);
method.onNewKey(*it, params.keys_size, keys, *aggregates_pool);
AggregateDataPtr place = aggregates_pool->alloc(total_size_of_aggregate_states);
createAggregateStates(place);
@ -1944,7 +1944,7 @@ void NO_INLINE Aggregator::mergeStreamsImplCase(
AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second);
aggregate_data = nullptr;
method.onNewKey(*it, params.keys_size, i, keys, *aggregates_pool);
method.onNewKey(*it, params.keys_size, keys, *aggregates_pool);
AggregateDataPtr place = aggregates_pool->alloc(total_size_of_aggregate_states);
createAggregateStates(place);

View File

@ -137,7 +137,7 @@ struct AggregationMethodOneNumber
/** Place additional data, if necessary, in case a new key was inserted into the hash table.
*/
static void onNewKey(typename Data::value_type & value, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
static void onNewKey(typename Data::value_type & value, size_t keys_size, StringRefs & keys, Arena & pool)
{
}
@ -205,7 +205,7 @@ struct AggregationMethodString
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
static void onNewKey(typename Data::value_type & value, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
static void onNewKey(typename Data::value_type & value, size_t keys_size, StringRefs & keys, Arena & pool)
{
value.first.data = pool.insert(value.first.data, value.first.size);
}
@ -266,7 +266,7 @@ struct AggregationMethodFixedString
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
static void onNewKey(typename Data::value_type & value, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
static void onNewKey(typename Data::value_type & value, size_t keys_size, StringRefs & keys, Arena & pool)
{
value.first.data = pool.insert(value.first.data, value.first.size);
}
@ -428,7 +428,7 @@ struct AggregationMethodKeysFixed
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
static void onNewKey(typename Data::value_type & value, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
static void onNewKey(typename Data::value_type & value, size_t keys_size, StringRefs & keys, Arena & pool)
{
}
@ -525,7 +525,7 @@ struct AggregationMethodConcat
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
static void onNewKey(typename Data::value_type & value, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
static void onNewKey(typename Data::value_type & value, size_t keys_size, StringRefs & keys, Arena & pool)
{
}
@ -603,14 +603,14 @@ struct AggregationMethodSerialized
StringRefs & keys,
Arena & pool) const
{
return serializeKeysToPoolContiguous(i, keys_size, key_columns, keys, pool);
return serializeKeysToPoolContiguous(i, keys_size, key_columns, pool);
}
};
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
static void onNewKey(typename Data::value_type & value, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
static void onNewKey(typename Data::value_type & value, size_t keys_size, StringRefs & keys, Arena & pool)
{
}
@ -669,9 +669,9 @@ struct AggregationMethodHashed
static AggregateDataPtr & getAggregateData(Mapped & value) { return value.second; }
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value.second; }
static void onNewKey(typename Data::value_type & value, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
static void onNewKey(typename Data::value_type & value, size_t keys_size, StringRefs & keys, Arena & pool)
{
value.second.first = placeKeysInPool(i, keys_size, keys, pool);
value.second.first = placeKeysInPool(keys_size, keys, pool);
}
static void onExistingKey(const Key & key, StringRefs & keys, Arena & pool) {}

View File

@ -87,7 +87,7 @@ void NO_INLINE Set::insertFromBlockImplCase(
method.data.emplace(key, it, inserted);
if (inserted)
method.onNewKey(*it, keys_size, i, variants.string_pool);
method.onNewKey(*it, keys_size, variants.string_pool);
}
}

View File

@ -53,7 +53,7 @@ struct SetMethodOneNumber
/** Place additional data, if necessary, in case a new key was inserted into the hash table.
*/
static void onNewKey(typename Data::value_type & value, size_t keys_size, size_t i, Arena & pool) {}
static void onNewKey(typename Data::value_type & value, size_t keys_size, Arena & pool) {}
};
/// For the case where there is one string key.
@ -90,7 +90,7 @@ struct SetMethodString
}
};
static void onNewKey(typename Data::value_type & value, size_t keys_size, size_t i, Arena & pool)
static void onNewKey(typename Data::value_type & value, size_t keys_size, Arena & pool)
{
value.data = pool.insert(value.data, value.size);
}
@ -128,7 +128,7 @@ struct SetMethodFixedString
}
};
static void onNewKey(typename Data::value_type & value, size_t keys_size, size_t i, Arena & pool)
static void onNewKey(typename Data::value_type & value, size_t keys_size, Arena & pool)
{
value.data = pool.insert(value.data, value.size);
}
@ -268,7 +268,7 @@ struct SetMethodKeysFixed
}
};
static void onNewKey(typename Data::value_type & value, size_t keys_size, size_t i, Arena & pool) {}
static void onNewKey(typename Data::value_type & value, size_t keys_size, Arena & pool) {}
};
/// For other cases. 128 bit hash from the key.
@ -296,7 +296,7 @@ struct SetMethodHashed
}
};
static void onNewKey(typename Data::value_type & value, size_t keys_size, size_t i, Arena & pool) {}
static void onNewKey(typename Data::value_type & value, size_t keys_size, Arena & pool) {}
};

View File

@ -190,7 +190,7 @@ void NO_INLINE Aggregator::executeSpecializedCase(
AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second);
aggregate_data = nullptr;
method.onNewKey(*it, params.keys_size, i, keys, *aggregates_pool);
method.onNewKey(*it, params.keys_size, keys, *aggregates_pool);
AggregateDataPtr place = aggregates_pool->alloc(total_size_of_aggregate_states);

View File

@ -72,7 +72,7 @@ void * ClickHouseDictionary_v1_loadIds(
return nullptr;
}
void * ClickHouseDictionary_v1_loadAll(void * data_ptr, ClickHouseLibrary::CStrings * settings, ClickHouseLibrary::CStrings * columns)
void * ClickHouseDictionary_v1_loadAll(void * data_ptr, ClickHouseLibrary::CStrings * settings, ClickHouseLibrary::CStrings * /*columns*/)
{
auto ptr = static_cast<DataHolder *>(data_ptr);
std::cerr << "loadAll lib call ptr=" << data_ptr << " => " << ptr << "\n";