Merge branch 'master' into table_name_in_istorage

This commit is contained in:
Alexander Tokmakov 2019-12-30 21:30:24 +03:00
commit 3687900848
11 changed files with 108 additions and 34 deletions

View File

@ -1,3 +1,38 @@
## ClickHouse release v19.17.6.36, 2019-12-27
### Bug Fix
* Fixed potential buffer overflow in decompress. Malicious user can pass fabricated compressed data that could cause read after buffer. This issue was found by Eldar Zaitov from Yandex information security team. [#8404](https://github.com/ClickHouse/ClickHouse/pull/8404) ([alexey-milovidov](https://github.com/alexey-milovidov))
* Fixed possible server crash (`std::terminate`) when the server cannot send or write data in JSON or XML format with values of String data type (that require UTF-8 validation) or when compressing result data with Brotli algorithm or in some other rare cases. [#8384](https://github.com/ClickHouse/ClickHouse/pull/8384) ([alexey-milovidov](https://github.com/alexey-milovidov))
* Fixed dictionaries with source from a clickhouse `VIEW`, now reading such dictionaries doesn't cause the error `There is no query`. [#8351](https://github.com/ClickHouse/ClickHouse/pull/8351) ([Nikolai Kochetov](https://github.com/KochetovNicolai))
* Fixed checking if a client host is allowed by host_regexp specified in users.xml. [#8241](https://github.com/ClickHouse/ClickHouse/pull/8241), [#8342](https://github.com/ClickHouse/ClickHouse/pull/8342) ([Vitaly Baranov](https://github.com/vitlibar))
* `RENAME TABLE` for a distributed table now renames the folder containing inserted data before sending to shards. This fixes an issue with successive renames `tableA->tableB`, `tableC->tableA`. [#8306](https://github.com/ClickHouse/ClickHouse/pull/8306) ([tavplubix](https://github.com/tavplubix))
* `range_hashed` external dictionaries created by DDL queries now allow ranges of arbitrary numeric types. [#8275](https://github.com/ClickHouse/ClickHouse/pull/8275) ([alesapin](https://github.com/alesapin))
* Fixed `INSERT INTO table SELECT ... FROM mysql(...)` table function. [#8234](https://github.com/ClickHouse/ClickHouse/pull/8234) ([tavplubix](https://github.com/tavplubix))
* Fixed segfault in `INSERT INTO TABLE FUNCTION file()` while inserting into a file which doesn't exist. Now in this case file would be created and then insert would be processed. [#8177](https://github.com/ClickHouse/ClickHouse/pull/8177) ([Olga Khvostikova](https://github.com/stavrolia))
* Fixed bitmapAnd error when intersecting an aggregated bitmap and a scalar bitmap. [#8082](https://github.com/ClickHouse/ClickHouse/pull/8082) ([Yue Huang](https://github.com/moon03432))
* Fixed segfault when `EXISTS` query was used without `TABLE` or `DICTIONARY` qualifier, just like `EXISTS t`. [#8213](https://github.com/ClickHouse/ClickHouse/pull/8213) ([alexey-milovidov](https://github.com/alexey-milovidov))
* Fixed return type for functions `rand` and `randConstant` in case of nullable argument. Now functions always return `UInt32` and never `Nullable(UInt32)`. [#8204](https://github.com/ClickHouse/ClickHouse/pull/8204) ([Nikolai Kochetov](https://github.com/KochetovNicolai))
* Fixed `DROP DICTIONARY IF EXISTS db.dict`, now it doesn't throw exception if `db` doesn't exist. [#8185](https://github.com/ClickHouse/ClickHouse/pull/8185) ([Vitaly Baranov](https://github.com/vitlibar))
* If a table wasn't completely dropped because of server crash, the server will try to restore and load it [#8176](https://github.com/ClickHouse/ClickHouse/pull/8176) ([tavplubix](https://github.com/tavplubix))
* Fixed a trivial count query for a distributed table if there are more than two shard local table. [#8164](https://github.com/ClickHouse/ClickHouse/pull/8164) ([小路](https://github.com/nicelulu))
* Fixed bug that lead to a data race in DB::BlockStreamProfileInfo::calculateRowsBeforeLimit() [#8143](https://github.com/ClickHouse/ClickHouse/pull/8143) ([Alexander Kazakov](https://github.com/Akazz))
* Fixed `ALTER table MOVE part` executed immediately after merging the specified part, which could cause moving a part which the specified part merged into. Now it correctly moves the specified part. [#8104](https://github.com/ClickHouse/ClickHouse/pull/8104) ([Vladimir Chebotarev](https://github.com/excitoon))
* Expressions for dictionaries can be specified as strings now. This is useful for calculation of attributes while extracting data from non-ClickHouse sources because it allows to use non-ClickHouse syntax for those expressions. [#8098](https://github.com/ClickHouse/ClickHouse/pull/8098) ([alesapin](https://github.com/alesapin))
* Fixed a very rare race in `clickhouse-copier` because of an overflow in ZXid. [#8088](https://github.com/ClickHouse/ClickHouse/pull/8088) ([Ding Xiang Fei](https://github.com/dingxiangfei2009))
* Fixed the bug when after the query failed (due to "Too many simultaneous queries" for example) it would not read external tables info, and the
next request would interpret this info as the beginning of the next query causing an error like `Unknown packet from client`. [#8084](https://github.com/ClickHouse/ClickHouse/pull/8084) ([Azat Khuzhin](https://github.com/azat))
* Avoid null dereference after "Unknown packet X from server" [#8071](https://github.com/ClickHouse/ClickHouse/pull/8071) ([Azat Khuzhin](https://github.com/azat))
* Restore support of all ICU locales, add the ability to apply collations for constant expressions and add language name to system.collations table. [#8051](https://github.com/ClickHouse/ClickHouse/pull/8051) ([alesapin](https://github.com/alesapin))
* Number of streams for read from `StorageFile` and `StorageHDFS` is now limited, to avoid exceeding the memory limit. [#7981](https://github.com/ClickHouse/ClickHouse/pull/7981) ([alesapin](https://github.com/alesapin))
* Fixed `CHECK TABLE` query for `*MergeTree` tables without key. [#7979](https://github.com/ClickHouse/ClickHouse/pull/7979) ([alesapin](https://github.com/alesapin))
* Removed the mutation number from a part name in case there were no mutations. This removing improved the compatibility with older versions. [#8250](https://github.com/ClickHouse/ClickHouse/pull/8250) ([alesapin](https://github.com/alesapin))
* Fixed the bug that mutations are skipped for some attached parts due to their data_version are larger than the table mutation version. [#7812](https://github.com/ClickHouse/ClickHouse/pull/7812) ([Zhichang Yu](https://github.com/yuzhichang))
* Allow starting the server with redundant copies of parts after moving them to another device. [#7810](https://github.com/ClickHouse/ClickHouse/pull/7810) ([Vladimir Chebotarev](https://github.com/excitoon))
* Fixed the error "Sizes of columns doesn't match" that might appear when using aggregate function columns. [#7790](https://github.com/ClickHouse/ClickHouse/pull/7790) ([Boris Granveaud](https://github.com/bgranvea))
* Now an exception will be thrown in case of using WITH TIES alongside LIMIT BY. And now it's possible to use TOP with LIMIT BY. [#7637](https://github.com/ClickHouse/ClickHouse/pull/7637) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov))
* Fix dictionary reload if it has `invalidate_query`, which stopped updates and some exception on previous update tries. [#8029](https://github.com/ClickHouse/ClickHouse/pull/8029) ([alesapin](https://github.com/alesapin))
## ClickHouse release v19.17.4.11, 2019-11-22
### Backward Incompatible Change

View File

@ -76,7 +76,7 @@ void LocalServer::initialize(Poco::Util::Application & self)
if (config().has("logger") || config().has("logger.level") || config().has("logger.log"))
{
// sensitive data rules are not used here
buildLoggers(config(), logger());
buildLoggers(config(), logger(), self.commandName());
}
else
{

View File

@ -124,7 +124,7 @@ void ODBCBridge::initialize(Application & self)
config().setString("logger", "ODBCBridge");
buildLoggers(config(), logger());
buildLoggers(config(), logger(), self.commandName());
log = &logger();
hostname = config().getString("listen-host", "localhost");

View File

@ -12,6 +12,7 @@
#include "IFunctionImpl.h"
#include <Common/intExp.h>
#include <Common/assert_cast.h>
#include <Core/Defines.h>
#include <cmath>
#include <type_traits>
#include <array>
@ -702,7 +703,7 @@ private:
}
template <typename Container>
void executeImplNumToNum(const Container & src, Container & dst, const Array & boundaries)
void NO_INLINE executeImplNumToNum(const Container & src, Container & dst, const Array & boundaries)
{
using ValueType = typename Container::value_type;
std::vector<ValueType> boundary_values(boundaries.size());
@ -714,6 +715,38 @@ private:
size_t size = src.size();
dst.resize(size);
if (boundary_values.size() < 32) /// Just a guess
{
/// Linear search with value on previous iteration as a hint.
/// Not optimal if the size of list is large and distribution of values is uniform random.
auto begin = boundary_values.begin();
auto end = boundary_values.end();
auto it = begin + (end - begin) / 2;
for (size_t i = 0; i < size; ++i)
{
auto value = src[i];
if (*it < value)
{
while (it != end && *it <= value)
++it;
if (it != begin)
--it;
}
else
{
while (*it > value && it != begin)
--it;
}
dst[i] = *it;
}
}
else
{
for (size_t i = 0; i < size; ++i)
{
auto it = std::upper_bound(boundary_values.begin(), boundary_values.end(), src[i]);
@ -731,6 +764,7 @@ private:
}
}
}
}
private:
const Context & context;

View File

@ -106,6 +106,7 @@ BlockIO InterpreterAlterQuery::execute()
StorageInMemoryMetadata metadata = table->getInMemoryMetadata();
alter_commands.validate(metadata, context);
alter_commands.prepare(metadata, context);
table->checkAlterIsPossible(alter_commands, context.getSettingsRef());
table->alter(alter_commands, context, table_lock_holder);
}

View File

@ -392,8 +392,9 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata) const
for (const auto & change : settings_changes)
{
auto finder = [&change](const SettingChange & c) { return c.name == change.name; };
if (auto it = std::find_if(settings_from_storage.begin(), settings_from_storage.end(), finder);
it != settings_from_storage.end())
auto it = std::find_if(settings_from_storage.begin(), settings_from_storage.end(), finder);
if (it != settings_from_storage.end())
it->value = change.value;
else
settings_from_storage.push_back(change);
@ -644,11 +645,6 @@ void AlterCommands::prepare(const StorageInMemoryMetadata & metadata, const Cont
void AlterCommands::validate(const StorageInMemoryMetadata & metadata, const Context & context) const
{
/// We will save ALTER ADD/MODIFY command indices (only the last for each column) for possible modification
/// (we might need to add deduced types or modify default expressions).
/// Saving indices because we can add new commands later and thus cause vector resize.
std::unordered_map<String, size_t> column_to_command_idx;
for (size_t i = 0; i < size(); ++i)
{
auto & command = (*this)[i];

View File

@ -96,32 +96,45 @@ struct AlterCommand
/// in each part on disk (it's not lightweight alter).
bool isModifyingData() const;
/// checks that only settings changed by alter
/// Checks that only settings changed by alter
bool isSettingsAlter() const;
/// Checks that only comment changed by alter
bool isCommentAlter() const;
};
/// Return string representation of AlterCommand::Type
String alterTypeToString(const AlterCommand::Type type);
class Context;
/// Vector of AlterCommand with several additional functions
class AlterCommands : public std::vector<AlterCommand>
{
private:
bool prepared = false;
public:
void apply(StorageInMemoryMetadata & metadata) const;
void prepare(const StorageInMemoryMetadata & metadata, const Context & context);
/// Validate that commands can be applied to metadata.
/// Checks that all columns exist and dependecies between them.
/// This check is lightweight and base only on metadata.
/// More accurate check have to be performed with storage->checkAlterIsPossible.
void validate(const StorageInMemoryMetadata & metadata, const Context & context) const;
/// Prepare alter commands. Set ignore flag to some of them
/// and additional commands for dependent columns.
void prepare(const StorageInMemoryMetadata & metadata, const Context & context);
/// Apply all alter command in sequential order to storage metadata.
/// Commands have to be prepared before apply.
void apply(StorageInMemoryMetadata & metadata) const;
/// At least one command modify data on disk.
bool isModifyingData() const;
/// At least one command modify settings.
bool isSettingsAlter() const;
/// At least one command modify comments.
bool isCommentAlter() const;
};

View File

@ -381,10 +381,10 @@ StorageInMemoryMetadata IStorage::getInMemoryMetadata() const
void IStorage::alter(
const AlterCommands & params,
const Context & context,
TableStructureWriteLockHolder & /*table_lock_holder*/)
TableStructureWriteLockHolder & table_lock_holder)
{
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId());
auto table_id = getStorageID();
checkAlterIsPossible(params, context.getSettingsRef());
StorageInMemoryMetadata metadata = getInMemoryMetadata();
params.apply(metadata);
context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, metadata);

View File

@ -251,8 +251,6 @@ void StorageMergeTree::alter(
lockNewDataStructureExclusively(table_lock_holder, context.getCurrentQueryId());
checkAlterIsPossible(params, context.getSettingsRef());
StorageInMemoryMetadata metadata = getInMemoryMetadata();
params.apply(metadata);

View File

@ -3211,9 +3211,6 @@ void StorageReplicatedMergeTree::alter(
auto table_id = getStorageID();
checkAlterIsPossible(params, query_context.getSettingsRef());
/// We cannot check this alter commands with method isModifyingData()
/// because ReplicatedMergeTree stores both columns and metadata for
/// each replica. So we have to wait AlterThread even with lightweight

View File

@ -686,7 +686,7 @@ void BaseDaemon::initialize(Application & self)
}
// sensitive data masking rules are not used here
buildLoggers(config(), logger());
buildLoggers(config(), logger(), self.commandName());
if (is_daemon)
{