Better [#CLICKHOUSE-2].

This commit is contained in:
Alexey Milovidov 2017-12-01 22:34:51 +03:00
parent 5bbd3a3a48
commit bdd25b15bd
23 changed files with 170 additions and 184 deletions

View File

@ -60,7 +60,7 @@ public:
* If no parameters are provided, or the passed parameters are not valid, throw an exception.
* If there are parameters - it is necessary to call before other calls, otherwise - do not call.
*/
virtual void setParameters(const Array & params)
virtual void setParameters(const Array & /*params*/)
{
throw Exception("Aggregate function " + getName() + " doesn't allow parameters.",
ErrorCodes::AGGREGATE_FUNCTION_DOESNT_ALLOW_PARAMETERS);

View File

@ -182,8 +182,7 @@ void createASTsForAllColumnsInTable(const CollectTables::TableInfo & table, ASTs
}
ASTs expandUnqualifiedAsterisk(
AnalyzeColumns::Columns & columns, const CollectAliases & aliases, const CollectTables & tables)
ASTs expandUnqualifiedAsterisk(const CollectTables & tables)
{
ASTs res;
for (const auto & table : tables.tables)
@ -193,7 +192,7 @@ ASTs expandUnqualifiedAsterisk(
ASTs expandQualifiedAsterisk(
const IAST & ast, AnalyzeColumns::Columns & columns, const CollectAliases & aliases, const CollectTables & tables)
const IAST & ast, const CollectTables & tables)
{
if (ast.children.size() != 1)
throw Exception("Logical error: AST node for qualified asterisk has number of children not equal to one", ErrorCodes::LOGICAL_ERROR);
@ -353,13 +352,13 @@ void processImpl(ASTPtr & ast, AnalyzeColumns::Columns & columns, const CollectA
{
if (typeid_cast<ASTAsterisk *>(asts[i].get()))
{
ASTs expanded = expandUnqualifiedAsterisk(columns, aliases, tables);
ASTs expanded = expandUnqualifiedAsterisk(tables);
asts.erase(asts.begin() + i);
asts.insert(asts.begin() + i, expanded.begin(), expanded.end());
}
else if (ASTQualifiedAsterisk * asterisk = typeid_cast<ASTQualifiedAsterisk *>(asts[i].get()))
{
ASTs expanded = expandQualifiedAsterisk(*asterisk, columns, aliases, tables);
ASTs expanded = expandQualifiedAsterisk(*asterisk, tables);
asts.erase(asts.begin() + i);
asts.insert(asts.begin() + i, expanded.begin(), expanded.end());
}

View File

@ -92,7 +92,7 @@ static CollectTables::TableInfo processSubquery(ASTPtr & ast_subquery, const Con
}
void CollectTables::process(ASTPtr & ast, const Context & context, const CollectAliases & aliases, ExecuteTableFunctions & table_functions)
void CollectTables::process(ASTPtr & ast, const Context & context, const CollectAliases & /*aliases*/, ExecuteTableFunctions & table_functions)
{
const ASTSelectQuery * select = typeid_cast<const ASTSelectQuery *>(ast.get());
if (!select)

View File

@ -318,7 +318,7 @@ void processScalarSubquery(const String & column_name, ASTPtr & ast, TypeAndCons
}
void processHigherOrderFunction(const String & column_name,
void processHigherOrderFunction(
ASTPtr & ast, const Context & context,
CollectAliases & aliases, const AnalyzeColumns & columns,
TypeAndConstantInference::Info & info,
@ -459,7 +459,7 @@ void processImpl(
{
/// If this is higher-order function, determine types of lambda arguments and infer types of subexpressions inside lambdas.
if (lambdas.higher_order_functions.end() != std::find(lambdas.higher_order_functions.begin(), lambdas.higher_order_functions.end(), ast))
processHigherOrderFunction(column_name, ast, context, aliases, columns, info, lambdas, table_functions);
processHigherOrderFunction(ast, context, aliases, columns, info, lambdas, table_functions);
processFunction(column_name, ast, info, context);
}

View File

@ -153,7 +153,7 @@ public:
void gather(ColumnGathererStream & gatherer_stream) override;
int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override
int compareAt(size_t, size_t, const IColumn &, int) const override
{
return 0;
}

View File

@ -82,57 +82,57 @@ public:
return s;
}
Field operator[](size_t n) const override
Field operator[](size_t) const override
{
return (*data)[0];
}
void get(size_t n, Field & res) const override
void get(size_t, Field & res) const override
{
data->get(0, res);
}
StringRef getDataAt(size_t n) const override
StringRef getDataAt(size_t) const override
{
return data->getDataAt(0);
}
StringRef getDataAtWithTerminatingZero(size_t n) const override
StringRef getDataAtWithTerminatingZero(size_t) const override
{
return data->getDataAtWithTerminatingZero(0);
}
UInt64 get64(size_t n) const override
UInt64 get64(size_t) const override
{
return data->get64(0);
}
UInt64 getUInt(size_t n) const override
UInt64 getUInt(size_t) const override
{
return data->getUInt(0);
}
Int64 getInt(size_t n) const override
Int64 getInt(size_t) const override
{
return data->getInt(0);
}
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override
void insertRangeFrom(const IColumn &, size_t /*start*/, size_t length) override
{
s += length;
}
void insert(const Field & x) override
void insert(const Field &) override
{
++s;
}
void insertData(const char * pos, size_t length) override
void insertData(const char *, size_t) override
{
++s;
}
void insertFrom(const IColumn & src, size_t n) override
void insertFrom(const IColumn &, size_t) override
{
++s;
}
@ -147,7 +147,7 @@ public:
s -= n;
}
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override
StringRef serializeValueIntoArena(size_t, Arena & arena, char const *& begin) const override
{
return data->serializeValueIntoArena(0, arena, begin);
}
@ -160,12 +160,12 @@ public:
return res;
}
void updateHashWithValue(size_t n, SipHash & hash) const override
void updateHashWithValue(size_t, SipHash & hash) const override
{
data->updateHashWithValue(0, hash);
}
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override
ColumnPtr filter(const Filter & filt, ssize_t /*result_size_hint*/) const override
{
if (s != filt.size())
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
@ -205,12 +205,12 @@ public:
return std::make_shared<ColumnConst>(data, limit);
}
int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override
int compareAt(size_t, size_t, const IColumn & rhs, int nan_direction_hint) const override
{
return data->compareAt(0, 0, *static_cast<const ColumnConst &>(rhs).data, nan_direction_hint);
}
void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override
void getPermutation(bool /*reverse*/, size_t /*limit*/, int /*nan_direction_hint*/, Permutation & res) const override
{
res.resize(s);
for (size_t i = 0; i < s; ++i)

View File

@ -208,7 +208,7 @@ public:
offsets.push_back(offsets.size() == 0 ? 1 : (offsets.back() + 1));
}
int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override
int compareAt(size_t n, size_t m, const IColumn & rhs_, int /*nan_direction_hint*/) const override
{
const ColumnString & rhs = static_cast<const ColumnString &>(rhs_);

View File

@ -36,7 +36,7 @@ struct ClearableHashTableCell : public BaseCell
UInt32 version;
bool isZero(const State & state) const { return version != state.version; }
static bool isZero(const Key & key, const State & state) { return false; }
static bool isZero(const Key & /*key*/, const State & /*state*/) { return false; }
/// Set the key value to zero.
void setZero() { version = 0; }

View File

@ -47,7 +47,7 @@ std::ostream & operator<<(std::ostream & stream, const DB::IStorage & what)
return stream;
}
std::ostream & operator<<(std::ostream & stream, const DB::TableStructureReadLock & what)
std::ostream & operator<<(std::ostream & stream, const DB::TableStructureReadLock &)
{
stream << "TableStructureReadLock()";
return stream;

View File

@ -43,14 +43,14 @@ public:
/** Methods to set additional information for output in formats, that support it.
*/
virtual void setRowsBeforeLimit(size_t rows_before_limit) {}
virtual void setTotals(const Block & totals) {}
virtual void setExtremes(const Block & extremes) {}
virtual void setRowsBeforeLimit(size_t /*rows_before_limit*/) {}
virtual void setTotals(const Block & /*totals*/) {}
virtual void setExtremes(const Block & /*extremes*/) {}
/** Notify about progress. Method could be called from different threads.
* Passed value are delta, that must be summarized.
*/
virtual void onProgress(const Progress & progress) {}
virtual void onProgress(const Progress & /*progress*/) {}
/** Content-Type to set when sending HTTP response. */
virtual String getContentType() const { return "text/plain; charset=UTF-8"; }

View File

@ -48,7 +48,7 @@ NativeBlockInputStream::NativeBlockInputStream(
void NativeBlockInputStream::readData(const IDataType & type, IColumn & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint)
{
IDataType::InputStreamGetter input_stream_getter = [&] (const IDataType::SubstreamPath & path) { return &istr; };
IDataType::InputStreamGetter input_stream_getter = [&] (const IDataType::SubstreamPath &) { return &istr; };
type.deserializeBinaryBulkWithMultipleStreams(column, input_stream_getter, rows, avg_value_size_hint, false, {});
if (column.size() != rows)

View File

@ -60,7 +60,7 @@ void NativeBlockOutputStream::writeData(const IDataType & type, const ColumnPtr
else
full_column = column;
IDataType::OutputStreamGetter output_stream_getter = [&] (const IDataType::SubstreamPath & path) { return &ostr; };
IDataType::OutputStreamGetter output_stream_getter = [&] (const IDataType::SubstreamPath &) { return &ostr; };
type.serializeBinaryBulkWithMultipleStreams(*full_column, output_stream_getter, offset, limit, false, {});
}

View File

@ -11,7 +11,7 @@ namespace DB
class NullBlockOutputStream : public IBlockOutputStream
{
public:
void write(const Block & block) override {}
void write(const Block &) override {}
};
}

View File

@ -45,22 +45,22 @@ enum class StreamUnionMode
struct ParallelInputsHandler
{
/// Processing the data block.
void onBlock(Block & block, size_t thread_num) {}
void onBlock(Block & /*block*/, size_t /*thread_num*/) {}
/// Processing the data block + additional information.
void onBlock(Block & block, BlockExtraInfo & extra_info, size_t thread_num) {}
void onBlock(Block & /*block*/, BlockExtraInfo & /*extra_info*/, size_t /*thread_num*/) {}
/// Called for each thread, when the thread has nothing else to do.
/// Due to the fact that part of the sources has run out, and now there are fewer sources left than streams.
/// Called if the `onException` method does not throw an exception; is called before the `onFinish` method.
void onFinishThread(size_t thread_num) {}
void onFinishThread(size_t /*thread_num*/) {}
/// Blocks are over. Due to the fact that all sources ran out or because of the cancellation of work.
/// This method is always called exactly once, at the end of the work, if the `onException` method does not throw an exception.
void onFinish() {}
/// Exception handling. It is reasonable to call the ParallelInputsProcessor::cancel method in this method, and also pass the exception to the main thread.
void onException(std::exception_ptr & exception, size_t thread_num) {}
void onException(std::exception_ptr & /*exception*/, size_t /*thread_num*/) {}
};
@ -158,19 +158,15 @@ private:
InputData(const BlockInputStreamPtr & in_, size_t i_) : in(in_), i(i_) {}
};
template <StreamUnionMode mode2 = mode>
void publishPayload(BlockInputStreamPtr & stream, Block & block, size_t thread_num,
typename std::enable_if<mode2 == StreamUnionMode::Basic>::type * = nullptr)
void publishPayload(BlockInputStreamPtr & stream, Block & block, size_t thread_num)
{
handler.onBlock(block, thread_num);
}
template <StreamUnionMode mode2 = mode>
void publishPayload(BlockInputStreamPtr & stream, Block & block, size_t thread_num,
typename std::enable_if<mode2 == StreamUnionMode::ExtraInfo>::type * = nullptr)
{
BlockExtraInfo extra_info = stream->getBlockExtraInfo();
handler.onBlock(block, extra_info, thread_num);
if constexpr (mode == StreamUnionMode::Basic)
handler.onBlock(block, thread_num);
else
{
BlockExtraInfo extra_info = stream->getBlockExtraInfo();
handler.onBlock(block, extra_info, thread_num);
}
}
void thread(MemoryTracker * memory_tracker, size_t thread_num)

View File

@ -76,8 +76,7 @@ void PrettyCompactBlockOutputStream::writeRow(
size_t row_num,
const Block & block,
const WidthsPerColumn & widths,
const Widths & max_widths,
const Widths & name_widths)
const Widths & max_widths)
{
size_t columns = max_widths.size();
@ -112,7 +111,7 @@ void PrettyCompactBlockOutputStream::write(const Block & block)
writeHeader(block, max_widths, name_widths);
for (size_t i = 0; i < rows && total_rows + i < max_rows; ++i)
writeRow(i, block, widths, max_widths, name_widths);
writeRow(i, block, widths, max_widths);
writeBottom(max_widths);

View File

@ -19,7 +19,7 @@ public:
protected:
void writeHeader(const Block & block, const Widths & max_widths, const Widths & name_widths);
void writeBottom(const Widths & max_widths);
void writeRow(size_t row_num, const Block & block, const WidthsPerColumn & widths, const Widths & max_widths, const Widths & name_widths);
void writeRow(size_t row_num, const Block & block, const WidthsPerColumn & widths, const Widths & max_widths);
};
}

View File

@ -30,29 +30,21 @@ private:
}
public:
void serializeBinary(const Field & field, WriteBuffer & ostr) const override { throwNoSerialization(); }
void deserializeBinary(Field & field, ReadBuffer & istr) const override { throwNoSerialization(); }
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override { throwNoSerialization(); }
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override { throwNoSerialization(); }
void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr,
size_t offset, size_t limit) const override { throwNoSerialization(); }
void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override { throwNoSerialization(); }
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override { throwNoSerialization(); }
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override { throwNoSerialization(); }
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr) const override { throwNoSerialization(); }
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override { throwNoSerialization(); }
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr) const override { throwNoSerialization(); }
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettingsJSON &) const override { throwNoSerialization(); }
void deserializeTextJSON(IColumn & column, ReadBuffer & istr) const override { throwNoSerialization(); }
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override { throwNoSerialization(); }
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const char delimiter) const override { throwNoSerialization(); }
void serializeBinary(const Field &, WriteBuffer &) const override { throwNoSerialization(); }
void deserializeBinary(Field &, ReadBuffer &) const override { throwNoSerialization(); }
void serializeBinary(const IColumn &, size_t, WriteBuffer &) const override { throwNoSerialization(); }
void deserializeBinary(IColumn &, ReadBuffer &) const override { throwNoSerialization(); }
void serializeBinaryBulk(const IColumn &, WriteBuffer &, size_t, size_t) const override { throwNoSerialization(); }
void deserializeBinaryBulk(IColumn &, ReadBuffer &, size_t, double) const override { throwNoSerialization(); }
void serializeText(const IColumn &, size_t, WriteBuffer &) const override { throwNoSerialization(); }
void serializeTextEscaped(const IColumn &, size_t, WriteBuffer &) const override { throwNoSerialization(); }
void deserializeTextEscaped(IColumn &, ReadBuffer &) const override { throwNoSerialization(); }
void serializeTextQuoted(const IColumn &, size_t, WriteBuffer &) const override { throwNoSerialization(); }
void deserializeTextQuoted(IColumn &, ReadBuffer &) const override { throwNoSerialization(); }
void serializeTextJSON(const IColumn &, size_t, WriteBuffer &, const FormatSettingsJSON &) const override { throwNoSerialization(); }
void deserializeTextJSON(IColumn &, ReadBuffer &) const override { throwNoSerialization(); }
void serializeTextCSV(const IColumn &, size_t, WriteBuffer &) const override { throwNoSerialization(); }
void deserializeTextCSV(IColumn &, ReadBuffer &, const char) const override { throwNoSerialization(); }
ColumnPtr createColumn() const override
{
@ -64,7 +56,7 @@ public:
throw Exception("Method getDefault() is not implemented for data type " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
void insertDefaultInto(IColumn & column) const override
void insertDefaultInto(IColumn &) const override
{
throw Exception("Method insertDefaultInto() is not implemented for data type " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}

View File

@ -84,7 +84,7 @@ public:
*
* sample_block should contain data types of arguments and values of constants, if relevant.
*/
virtual bool isInjective(const Block & sample_block) { return false; }
virtual bool isInjective(const Block & /*sample_block*/) { return false; }
/** Function is called "deterministic", if it returns same result for same values of arguments.
* Most of functions are deterministic. Notable counterexample is rand().
@ -98,7 +98,7 @@ public:
/// Overloading for those who do not need prerequisites and values of constant arguments. Not called from outside.
DataTypePtr getReturnType(const DataTypes & arguments) const;
virtual DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const
virtual DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const
{
throw Exception("getReturnType is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
@ -117,7 +117,7 @@ public:
virtual void getReturnTypeAndPrerequisitesImpl(
const ColumnsWithTypeAndName & arguments,
DataTypePtr & out_return_type,
std::vector<ExpressionAction> & out_prerequisites)
std::vector<ExpressionAction> & /*out_prerequisites*/)
{
DataTypes types(arguments.size());
for (size_t i = 0; i < arguments.size(); ++i)
@ -130,7 +130,7 @@ public:
/// This function will replace it with DataTypeExpression containing actual types.
void getLambdaArgumentTypes(DataTypes & arguments) const;
virtual void getLambdaArgumentTypesImpl(DataTypes & arguments) const
virtual void getLambdaArgumentTypesImpl(DataTypes & /*arguments*/) const
{
throw Exception("Function " + getName() + " can't have lambda-expressions as arguments", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
@ -143,12 +143,12 @@ public:
/// `prerequisites` go in the same order as `out_prerequisites` obtained from getReturnTypeAndPrerequisites.
void execute(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & prerequisites, size_t result);
virtual void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
virtual void executeImpl(Block & /*block*/, const ColumnNumbers & /*arguments*/, size_t /*result*/)
{
throw Exception("executeImpl is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
virtual void executeImpl(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & prerequisites, size_t result)
virtual void executeImpl(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & /*prerequisites*/, size_t result)
{
executeImpl(block, arguments, result);
}
@ -194,7 +194,7 @@ public:
/** Get information about monotonicity on a range of values. Call only if hasInformationAboutMonotonicity.
* NULL can be passed as one of the arguments. This means that the corresponding range is unlimited on the left or on the right.
*/
virtual Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const
virtual Monotonicity getMonotonicityForRange(const IDataType & /*type*/, const Field & /*left*/, const Field & /*right*/) const
{
throw Exception("Function " + getName() + " has no information about its monotonicity.", ErrorCodes::NOT_IMPLEMENTED);
}

View File

@ -120,12 +120,12 @@ struct AggregationMethodOneNumber
/// Get the key from the key columns for insertion into the hash table.
Key getKey(
const ConstColumnPlainPtrs & key_columns, /// Key columns.
size_t keys_size, /// Number of key columns.
const ConstColumnPlainPtrs & /*key_columns*/,
size_t /*keys_size*/, /// Number of key columns.
size_t i, /// From which row of the block, get the key.
const Sizes & key_sizes, /// If the keys of a fixed length - their lengths. It is not used in aggregation methods for variable length keys.
StringRefs & keys, /// Here references to key data in columns can be written. They can be used in the future.
Arena & pool) const
const Sizes & /*key_sizes*/, /// If the keys of a fixed length - their lengths. It is not used in aggregation methods for variable length keys.
StringRefs & /*keys*/, /// Here references to key data in columns can be written. They can be used in the future.
Arena & /*pool*/) const
{
return unionCastToUInt64(vec[i]);
}
@ -137,13 +137,13 @@ 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, StringRefs & keys, Arena & pool)
static void onNewKey(typename Data::value_type & /*value*/, size_t /*keys_size*/, StringRefs & /*keys*/, Arena & /*pool*/)
{
}
/** The action to be taken if the key is not new. For example, roll back the memory allocation in the pool.
*/
static void onExistingKey(const Key & key, StringRefs & keys, Arena & pool) {}
static void onExistingKey(const Key & /*key*/, StringRefs & /*keys*/, Arena & /*pool*/) {}
/** Do not use optimization for consecutive keys.
*/
@ -151,7 +151,7 @@ struct AggregationMethodOneNumber
/** Insert the key from the hash table into columns.
*/
static void insertKeyIntoColumns(const typename Data::value_type & value, ColumnPlainPtrs & key_columns, size_t keys_size, const Sizes & key_sizes)
static void insertKeyIntoColumns(const typename Data::value_type & value, ColumnPlainPtrs & key_columns, size_t /*keys_size*/, const Sizes & /*key_sizes*/)
{
static_cast<ColumnVector<FieldType> *>(key_columns[0])->insertData(reinterpret_cast<const char *>(&value.first), sizeof(value.first));
}
@ -189,12 +189,12 @@ struct AggregationMethodString
}
Key getKey(
const ConstColumnPlainPtrs & key_columns,
size_t keys_size,
const ConstColumnPlainPtrs & /*key_columns*/,
size_t /*keys_size*/,
size_t i,
const Sizes & key_sizes,
StringRefs & keys,
Arena & pool) const
const Sizes & /*key_sizes*/,
StringRefs & /*keys*/,
Arena & /*pool*/) const
{
return StringRef(
&(*chars)[i == 0 ? 0 : (*offsets)[i - 1]],
@ -205,16 +205,16 @@ 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, 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);
}
static void onExistingKey(const Key & key, StringRefs & keys, Arena & pool) {}
static void onExistingKey(const Key & /*key*/, StringRefs & /*keys*/, Arena & /*pool*/) {}
static const bool no_consecutive_keys_optimization = false;
static void insertKeyIntoColumns(const typename Data::value_type & value, ColumnPlainPtrs & key_columns, size_t keys_size, const Sizes & key_sizes)
static void insertKeyIntoColumns(const typename Data::value_type & value, ColumnPlainPtrs & key_columns, size_t, const Sizes &)
{
key_columns[0]->insertData(value.first.data, value.first.size);
}
@ -252,12 +252,12 @@ struct AggregationMethodFixedString
}
Key getKey(
const ConstColumnPlainPtrs & key_columns,
size_t keys_size,
const ConstColumnPlainPtrs &,
size_t,
size_t i,
const Sizes & key_sizes,
StringRefs & keys,
Arena & pool) const
const Sizes &,
StringRefs &,
Arena &) const
{
return StringRef(&(*chars)[i * n], n);
}
@ -266,16 +266,16 @@ 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, StringRefs & keys, Arena & pool)
static void onNewKey(typename Data::value_type & value, size_t, StringRefs &, Arena & pool)
{
value.first.data = pool.insert(value.first.data, value.first.size);
}
static void onExistingKey(const Key & key, StringRefs & keys, Arena & pool) {}
static void onExistingKey(const Key &, StringRefs &, Arena &) {}
static const bool no_consecutive_keys_optimization = false;
static void insertKeyIntoColumns(const typename Data::value_type & value, ColumnPlainPtrs & key_columns, size_t keys_size, const Sizes & key_sizes)
static void insertKeyIntoColumns(const typename Data::value_type & value, ColumnPlainPtrs & key_columns, size_t, const Sizes &)
{
key_columns[0]->insertData(value.first.data, value.first.size);
}
@ -357,7 +357,7 @@ template <typename Key>
class BaseStateKeysFixed<Key, false>
{
protected:
void init(const ConstColumnPlainPtrs & key_columns)
void init(const ConstColumnPlainPtrs &)
{
throw Exception{"Internal error: calling init() for non-nullable"
" keys is forbidden", ErrorCodes::LOGICAL_ERROR};
@ -369,7 +369,7 @@ protected:
" keys is forbidden", ErrorCodes::LOGICAL_ERROR};
}
KeysNullMap<Key> createBitmap(size_t row) const
KeysNullMap<Key> createBitmap(size_t) const
{
throw Exception{"Internal error: calling createBitmap() for non-nullable keys"
" is forbidden", ErrorCodes::LOGICAL_ERROR};
@ -412,8 +412,8 @@ struct AggregationMethodKeysFixed
size_t keys_size,
size_t i,
const Sizes & key_sizes,
StringRefs & keys,
Arena & pool) const
StringRefs &,
Arena &) const
{
if (has_nullable_keys)
{
@ -428,11 +428,11 @@ 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, StringRefs & keys, Arena & pool)
static void onNewKey(typename Data::value_type &, size_t, StringRefs &, Arena &)
{
}
static void onExistingKey(const Key & key, StringRefs & keys, Arena & pool) {}
static void onExistingKey(const Key &, StringRefs &, Arena &) {}
static const bool no_consecutive_keys_optimization = false;
@ -506,7 +506,7 @@ struct AggregationMethodConcat
struct State
{
void init(ConstColumnPlainPtrs & key_columns)
void init(ConstColumnPlainPtrs &)
{
}
@ -514,7 +514,7 @@ struct AggregationMethodConcat
const ConstColumnPlainPtrs & key_columns,
size_t keys_size,
size_t i,
const Sizes & key_sizes,
const Sizes &,
StringRefs & keys,
Arena & pool) const
{
@ -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, StringRefs & keys, Arena & pool)
static void onNewKey(typename Data::value_type &, size_t, StringRefs &, Arena &)
{
}
@ -544,7 +544,7 @@ struct AggregationMethodConcat
private:
/// Insert the values of the specified keys into the corresponding columns.
static void insertKeyIntoColumnsImpl(const typename Data::value_type & value, ColumnPlainPtrs & key_columns, size_t keys_size, const Sizes & key_sizes)
static void insertKeyIntoColumnsImpl(const typename Data::value_type & value, ColumnPlainPtrs & key_columns, size_t keys_size, const Sizes &)
{
/// See function extractKeysAndPlaceInPoolContiguous.
const StringRef * key_refs = reinterpret_cast<const StringRef *>(value.first.data + value.first.size);
@ -591,7 +591,7 @@ struct AggregationMethodSerialized
struct State
{
void init(ConstColumnPlainPtrs & key_columns)
void init(ConstColumnPlainPtrs &)
{
}
@ -599,8 +599,8 @@ struct AggregationMethodSerialized
const ConstColumnPlainPtrs & key_columns,
size_t keys_size,
size_t i,
const Sizes & key_sizes,
StringRefs & keys,
const Sizes &,
StringRefs &,
Arena & pool) const
{
return serializeKeysToPoolContiguous(i, keys_size, key_columns, pool);
@ -610,11 +610,11 @@ struct AggregationMethodSerialized
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, StringRefs & keys, Arena & pool)
static void onNewKey(typename Data::value_type &, size_t, StringRefs &, Arena &)
{
}
static void onExistingKey(const Key & key, StringRefs & keys, Arena & pool)
static void onExistingKey(const Key & key, StringRefs &, Arena & pool)
{
pool.rollback(key.size);
}
@ -622,7 +622,7 @@ struct AggregationMethodSerialized
/// If the key already was, it is removed from the pool (overwritten), and the next key can not be compared with it.
static const bool no_consecutive_keys_optimization = true;
static void insertKeyIntoColumns(const typename Data::value_type & value, ColumnPlainPtrs & key_columns, size_t keys_size, const Sizes & key_sizes)
static void insertKeyIntoColumns(const typename Data::value_type & value, ColumnPlainPtrs & key_columns, size_t keys_size, const Sizes &)
{
auto pos = value.first.data;
for (size_t i = 0; i < keys_size; ++i)
@ -650,7 +650,7 @@ struct AggregationMethodHashed
struct State
{
void init(ConstColumnPlainPtrs & key_columns)
void init(ConstColumnPlainPtrs &)
{
}
@ -658,9 +658,9 @@ struct AggregationMethodHashed
const ConstColumnPlainPtrs & key_columns,
size_t keys_size,
size_t i,
const Sizes & key_sizes,
const Sizes &,
StringRefs & keys,
Arena & pool) const
Arena &) const
{
return hash128(i, keys_size, key_columns, keys);
}
@ -674,11 +674,11 @@ struct AggregationMethodHashed
value.second.first = placeKeysInPool(keys_size, keys, pool);
}
static void onExistingKey(const Key & key, StringRefs & keys, Arena & pool) {}
static void onExistingKey(const Key &, StringRefs &, Arena &) {}
static const bool no_consecutive_keys_optimization = false;
static void insertKeyIntoColumns(const typename Data::value_type & value, ColumnPlainPtrs & key_columns, size_t keys_size, const Sizes & key_sizes)
static void insertKeyIntoColumns(const typename Data::value_type & value, ColumnPlainPtrs & key_columns, size_t keys_size, const Sizes &)
{
for (size_t i = 0; i < keys_size; ++i)
key_columns[i]->insertDataWithTerminatingZero(value.second.first[i].data, value.second.first[i].size);

View File

@ -39,16 +39,16 @@ struct JoinKeyGetterOneNumber
}
Key getKey(
const ConstColumnPlainPtrs & key_columns,
size_t keys_size, /// number of key columns.
size_t i, /// row number to get key from.
const Sizes & key_sizes) const /// If keys are of fixed size - their sizes. Not used for methods with variable-length keys.
const ConstColumnPlainPtrs & /*key_columns*/,
size_t /*keys_size*/, /// number of key columns.
size_t i, /// row number to get key from.
const Sizes & /*key_sizes*/) const /// If keys are of fixed size - their sizes. Not used for methods with variable-length keys.
{
return unionCastToUInt64(vec[i]);
}
/// Place additional data into memory pool, if needed, when new key was inserted into hash table.
static void onNewKey(Key & key, Arena & pool) {}
static void onNewKey(Key & /*key*/, Arena & /*pool*/) {}
};
/// For single String key.
@ -68,10 +68,10 @@ struct JoinKeyGetterString
}
Key getKey(
const ConstColumnPlainPtrs & key_columns,
size_t keys_size,
const ConstColumnPlainPtrs &,
size_t,
size_t i,
const Sizes & key_sizes) const
const Sizes &) const
{
return StringRef(
&(*chars)[i == 0 ? 0 : (*offsets)[i - 1]],
@ -101,10 +101,10 @@ struct JoinKeyGetterFixedString
}
Key getKey(
const ConstColumnPlainPtrs & key_columns,
size_t keys_size,
const ConstColumnPlainPtrs &,
size_t,
size_t i,
const Sizes & key_sizes) const
const Sizes &) const
{
return StringRef(&(*chars)[i * n], n);
}
@ -121,7 +121,7 @@ struct JoinKeyGetterFixed
{
using Key = TKey;
JoinKeyGetterFixed(const ConstColumnPlainPtrs & key_columns)
JoinKeyGetterFixed(const ConstColumnPlainPtrs &)
{
}
@ -134,7 +134,7 @@ struct JoinKeyGetterFixed
return packFixed<Key>(i, keys_size, key_columns, key_sizes);
}
static void onNewKey(Key & key, Arena & pool) {}
static void onNewKey(Key &, Arena &) {}
};
/// Generic method, use crypto hash function.
@ -142,7 +142,7 @@ struct JoinKeyGetterHashed
{
using Key = UInt128;
JoinKeyGetterHashed(const ConstColumnPlainPtrs & key_columns)
JoinKeyGetterHashed(const ConstColumnPlainPtrs &)
{
}
@ -150,12 +150,12 @@ struct JoinKeyGetterHashed
const ConstColumnPlainPtrs & key_columns,
size_t keys_size,
size_t i,
const Sizes & key_sizes) const
const Sizes &) const
{
return hash128(i, keys_size, key_columns);
}
static void onNewKey(Key & key, Arena & pool) {}
static void onNewKey(Key &, Arena &) {}
};

View File

@ -42,10 +42,10 @@ struct SetMethodOneNumber
/// Get key from key columns for insertion into hash table.
Key getKey(
const ConstColumnPlainPtrs & key_columns, /// Key columns.
size_t keys_size, /// Number of key columns.
size_t i, /// From what row of the block I get the key.
const Sizes & key_sizes) const /// If keys of a fixed length - their lengths. Not used in methods for variable length keys.
const ConstColumnPlainPtrs & /*key_columns*/,
size_t /*keys_size*/, /// Number of key columns.
size_t i, /// From what row of the block I get the key.
const Sizes & /*key_sizes*/) const /// If keys of a fixed length - their lengths. Not used in methods for variable length keys.
{
return unionCastToUInt64(vec[i]);
}
@ -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, 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.
@ -79,10 +79,10 @@ struct SetMethodString
}
Key getKey(
const ConstColumnPlainPtrs & key_columns,
size_t keys_size,
const ConstColumnPlainPtrs &,
size_t,
size_t i,
const Sizes & key_sizes) const
const Sizes &) const
{
return StringRef(
&(*chars)[i == 0 ? 0 : (*offsets)[i - 1]],
@ -90,7 +90,7 @@ struct SetMethodString
}
};
static void onNewKey(typename Data::value_type & value, size_t keys_size, Arena & pool)
static void onNewKey(typename Data::value_type & value, size_t, Arena & pool)
{
value.data = pool.insert(value.data, value.size);
}
@ -119,16 +119,16 @@ struct SetMethodFixedString
}
Key getKey(
const ConstColumnPlainPtrs & key_columns,
size_t keys_size,
const ConstColumnPlainPtrs &,
size_t,
size_t i,
const Sizes & key_sizes) const
const Sizes &) const
{
return StringRef(&(*chars)[i * n], n);
}
};
static void onNewKey(typename Data::value_type & value, size_t keys_size, Arena & pool)
static void onNewKey(typename Data::value_type & value, size_t, Arena & pool)
{
value.data = pool.insert(value.data, value.size);
}
@ -210,7 +210,7 @@ template <typename Key>
class BaseStateKeysFixed<Key, false>
{
protected:
void init(const ConstColumnPlainPtrs & key_columns)
void init(const ConstColumnPlainPtrs &)
{
throw Exception{"Internal error: calling init() for non-nullable"
" keys is forbidden", ErrorCodes::LOGICAL_ERROR};
@ -222,7 +222,7 @@ protected:
" keys is forbidden", ErrorCodes::LOGICAL_ERROR};
}
KeysNullMap<Key> createBitmap(size_t row) const
KeysNullMap<Key> createBitmap(size_t) const
{
throw Exception{"Internal error: calling createBitmap() for non-nullable keys"
" is forbidden", ErrorCodes::LOGICAL_ERROR};
@ -268,7 +268,7 @@ struct SetMethodKeysFixed
}
};
static void onNewKey(typename Data::value_type & value, size_t keys_size, Arena & pool) {}
static void onNewKey(typename Data::value_type &, size_t, Arena &) {}
};
/// For other cases. 128 bit hash from the key.
@ -282,7 +282,7 @@ struct SetMethodHashed
struct State
{
void init(const ConstColumnPlainPtrs & key_columns)
void init(const ConstColumnPlainPtrs &)
{
}
@ -290,13 +290,13 @@ struct SetMethodHashed
const ConstColumnPlainPtrs & key_columns,
size_t keys_size,
size_t i,
const Sizes & key_sizes) const
const Sizes &) const
{
return hash128(i, keys_size, key_columns);
}
};
static void onNewKey(typename Data::value_type & value, size_t keys_size, Arena & pool) {}
static void onNewKey(typename Data::value_type &, size_t, Arena &) {}
};

View File

@ -169,12 +169,12 @@ public:
* It is guaranteed that the structure of the table will not change over the lifetime of the returned streams (that is, there will not be ALTER, RENAME and DROP).
*/
virtual BlockInputStreams read(
const Names & column_names,
const SelectQueryInfo & query_info,
const Context & context,
QueryProcessingStage::Enum & processed_stage,
size_t max_block_size,
unsigned num_streams)
const Names & /*column_names*/,
const SelectQueryInfo & /*query_info*/,
const Context & /*context*/,
QueryProcessingStage::Enum & /*processed_stage*/,
size_t /*max_block_size*/,
unsigned /*num_streams*/)
{
throw Exception("Method read is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
@ -186,8 +186,8 @@ public:
* It is guaranteed that the table structure will not change over the lifetime of the returned streams (that is, there will not be ALTER, RENAME and DROP).
*/
virtual BlockOutputStreamPtr write(
const ASTPtr & query,
const Settings & settings)
const ASTPtr & /*query*/,
const Settings & /*settings*/)
{
throw Exception("Method write is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
@ -202,7 +202,7 @@ public:
* In this function, you need to rename the directory with the data, if any.
* Called when the table structure is locked for write.
*/
virtual void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name)
virtual void rename(const String & /*new_path_to_db*/, const String & /*new_database_name*/, const String & /*new_table_name*/)
{
throw Exception("Method rename is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
@ -211,41 +211,41 @@ public:
* This method must fully execute the ALTER query, taking care of the locks itself.
* To update the table metadata on disk, this method should call InterpreterAlterQuery::updateMetadata.
*/
virtual void alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context)
virtual void alter(const AlterCommands & /*params*/, const String & /*database_name*/, const String & /*table_name*/, const Context & /*context*/)
{
throw Exception("Method alter is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
/** Execute CLEAR COLUMN ... IN PARTITION query which removes column from given partition. */
virtual void clearColumnInPartition(const ASTPtr & partition, const Field & column_name, const Context & context)
virtual void clearColumnInPartition(const ASTPtr & /*partition*/, const Field & /*column_name*/, const Context & /*context*/)
{
throw Exception("Method dropColumnFromPartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
/** Run the query (DROP|DETACH) PARTITION.
*/
virtual void dropPartition(const ASTPtr & query, const ASTPtr & partition, bool detach, const Context & context)
virtual void dropPartition(const ASTPtr & /*query*/, const ASTPtr & /*partition*/, bool /*detach*/, const Context & /*context*/)
{
throw Exception("Method dropPartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
/** Run the ATTACH request (PART|PARTITION).
*/
virtual void attachPartition(const ASTPtr & partition, bool part, const Context & context)
virtual void attachPartition(const ASTPtr & /*partition*/, bool /*part*/, const Context & /*context*/)
{
throw Exception("Method attachPartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
/** Run the FETCH PARTITION query.
*/
virtual void fetchPartition(const ASTPtr & partition, const String & from, const Context & context)
virtual void fetchPartition(const ASTPtr & /*partition*/, const String & /*from*/, const Context & /*context*/)
{
throw Exception("Method fetchPartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
/** Run the FREEZE PARTITION request. That is, create a local backup (snapshot) of data using the `localBackup` function (see localBackup.h)
*/
virtual void freezePartition(const ASTPtr & partition, const String & with_name, const Context & context)
virtual void freezePartition(const ASTPtr & /*partition*/, const String & /*with_name*/, const Context & /*context*/)
{
throw Exception("Method freezePartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
@ -253,7 +253,7 @@ public:
/** Perform any background work. For example, combining parts in a MergeTree type table.
* Returns whether any work has been done.
*/
virtual bool optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context)
virtual bool optimize(const ASTPtr & /*query*/, const ASTPtr & /*partition*/, bool /*final*/, bool /*deduplicate*/, const Context & /*context*/)
{
throw Exception("Method optimize is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}

View File

@ -117,7 +117,7 @@ std::ostream & operator<<(std::ostream & stream, const std::list<K> & what)
#include <ratio>
template <std::intmax_t Num, std::intmax_t Denom>
std::ostream & operator<<(std::ostream & stream, const std::ratio<Num, Denom> & what)
std::ostream & operator<<(std::ostream & stream, const std::ratio<Num, Denom> &)
{
stream << "ratio<Num=" << Num << ", Denom=" << Denom << ">";
return stream;