Style [#CLICKHOUSE-2]

This commit is contained in:
Alexey Milovidov 2018-07-24 21:46:23 +03:00
parent d17aa47250
commit 7c07f46d50
8 changed files with 23 additions and 16 deletions

View File

@ -58,13 +58,13 @@ It is designed to retain the following properties of data:
Most of the properties above are viable for performance testing:
- reading data, filtering, aggregation and sorting will work at almost the same speed
as on original data due to saved cardinalities, magnitudes, compression ratios, etc.
as on original data due to saved cardinalities, magnitudes, compression ratios, etc.
It works in deterministic fashion: you define a seed value and transform is totally determined by input data and by seed.
Some transforms are one to one and could be reversed, so you need to have large enough seed and keep it in secret.
It use some cryptographic primitives to transform data, but from the cryptographic point of view,
it doesn't do anything properly and you should never consider the result as secure, unless you have other reasons for it.
it doesn't do anything properly and you should never consider the result as secure, unless you have other reasons for it.
It may retain some data you don't want to publish.
@ -74,7 +74,7 @@ So, the user will be able to count exact ratio of mobile traffic.
Another example, suppose you have some private data in your table, like user email and you don't want to publish any single email address.
If your table is large enough and contain multiple different emails and there is no email that have very high frequency than all others,
it will perfectly anonymize all data. But if you have small amount of different values in a column, it can possibly reproduce some of them.
it will perfectly anonymize all data. But if you have small amount of different values in a column, it can possibly reproduce some of them.
And you should take care and look at exact algorithm, how this tool works, and probably fine tune some of it command line parameters.
This tool works fine only with reasonable amount of data (at least 1000s of rows).

View File

@ -128,7 +128,8 @@ void BackgroundSchedulePool::TaskInfo::execute()
zkutil::WatchCallback BackgroundSchedulePool::TaskInfo::getWatchCallback()
{
return [t=shared_from_this()](const ZooKeeperImpl::ZooKeeper::WatchResponse &) {
return [t = shared_from_this()](const ZooKeeperImpl::ZooKeeper::WatchResponse &)
{
t->schedule();
};
}

View File

@ -1286,12 +1286,12 @@ DataTypePtr FunctionArrayDistinct::getReturnTypeImpl(const DataTypes & arguments
{
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(arguments[0].get());
if (!array_type)
throw Exception("Argument for function " + getName() + " must be array but it "
throw Exception("Argument for function " + getName() + " must be array but it "
" has type " + arguments[0]->getName() + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
auto nested_type = removeNullable(array_type->getNestedType());
return std::make_shared<DataTypeArray>(nested_type);
}
@ -1307,7 +1307,7 @@ void FunctionArrayDistinct::executeImpl(Block & block, const ColumnNumbers & arg
const IColumn & src_data = array->getData();
const ColumnArray::Offsets & offsets = array->getOffsets();
ColumnRawPtrs original_data_columns;
original_data_columns.push_back(&src_data);
@ -1416,7 +1416,7 @@ bool FunctionArrayDistinct::executeString(
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>;
const PaddedPODArray<UInt8> * src_null_map = nullptr;
if (nullable_col)
{
src_null_map = &static_cast<const ColumnUInt8 *>(&nullable_col->getNullMapColumn())->getData();
@ -1471,7 +1471,7 @@ void FunctionArrayDistinct::executeHashed(
res_data_col.insertFrom(*columns[0], j);
}
}
res_offsets.emplace_back(set.size() + prev_off);
prev_off = off;
}

View File

@ -1250,7 +1250,7 @@ private:
IColumn & res_data_col,
ColumnArray::Offsets & res_offsets,
const ColumnNullable * nullable_col);
void executeHashed(
const ColumnArray::Offsets & offsets,
const ColumnRawPtrs & columns,

View File

@ -7,6 +7,7 @@
namespace DB
{
NamesAndTypesList StorageSystemClusters::getNamesAndTypes()
{
return {
@ -25,7 +26,8 @@ NamesAndTypesList StorageSystemClusters::getNamesAndTypes()
void StorageSystemClusters::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const
{
auto updateColumns = [&](const std::string & cluster_name, const Cluster::ShardInfo & shard_info, const Cluster::Address & address) {
auto updateColumns = [&](const std::string & cluster_name, const Cluster::ShardInfo & shard_info, const Cluster::Address & address)
{
size_t i = 0;
res_columns[i++]->insert(cluster_name);
res_columns[i++]->insert(static_cast<UInt64>(shard_info.shard_num));

View File

@ -15,7 +15,8 @@
namespace DB
{
NamesAndTypesList StorageSystemColumns::getNamesAndTypes() {
NamesAndTypesList StorageSystemColumns::getNamesAndTypes()
{
return {
{ "database", std::make_shared<DataTypeString>() },
{ "table", std::make_shared<DataTypeString>() },

View File

@ -43,7 +43,8 @@ public:
const std::string & name,
const Context & context) const;
const TableFunctions & getAllTableFunctions() const {
const TableFunctions & getAllTableFunctions() const
{
return functions;
}

View File

@ -62,7 +62,8 @@ void MakeColumnsFromVector(DataHolder * ptr)
ptr->ctable.data = ptr->rowHolder.get();
}
extern "C" {
extern "C"
{
void * ClickHouseDictionary_v3_loadIds(void * data_ptr,
ClickHouseLibrary::CStrings * settings,
@ -151,7 +152,8 @@ void * ClickHouseDictionary_v3_loadKeys(void * data_ptr, ClickHouseLibrary::CStr
if (requested_keys)
{
LOG(ptr->lib->log, "requested_keys columns passed: " << requested_keys->size);
for (size_t i = 0; i < requested_keys->size; ++i) {
for (size_t i = 0; i < requested_keys->size; ++i)
{
LOG(ptr->lib->log, "requested_keys at column " << i << " passed: " << requested_keys->data[i].size);
}
}