mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge pull request #27690 from kitaisreal/remove-dense-hash-map-set
Removed DenseHashMap, DenseHashSet
This commit is contained in:
commit
76b0502482
@ -1,29 +0,0 @@
|
|||||||
#pragma once
|
|
||||||
#include <unordered_map>
|
|
||||||
|
|
||||||
/// DenseHashMap is a wrapper for google::dense_hash_map.
|
|
||||||
/// Some hacks are needed to make it work in "Arcadia".
|
|
||||||
/// "Arcadia" is a proprietary monorepository in Yandex.
|
|
||||||
/// It uses slightly changed version of sparsehash with a different set of hash functions (which we don't need).
|
|
||||||
/// Those defines are needed to make it compile.
|
|
||||||
#if defined(ARCADIA_BUILD)
|
|
||||||
#define HASH_FUN_H <unordered_map>
|
|
||||||
template <typename T>
|
|
||||||
struct THash;
|
|
||||||
#endif
|
|
||||||
|
|
||||||
#include <sparsehash/dense_hash_map>
|
|
||||||
|
|
||||||
#if !defined(ARCADIA_BUILD)
|
|
||||||
template <class Key, class T, class HashFcn = std::hash<Key>,
|
|
||||||
class EqualKey = std::equal_to<Key>,
|
|
||||||
class Alloc = google::libc_allocator_with_realloc<std::pair<const Key, T>>>
|
|
||||||
using DenseHashMap = google::dense_hash_map<Key, T, HashFcn, EqualKey, Alloc>;
|
|
||||||
#else
|
|
||||||
template <class Key, class T, class HashFcn = std::hash<Key>,
|
|
||||||
class EqualKey = std::equal_to<Key>,
|
|
||||||
class Alloc = google::sparsehash::libc_allocator_with_realloc<std::pair<const Key, T>>>
|
|
||||||
using DenseHashMap = google::sparsehash::dense_hash_map<Key, T, HashFcn, EqualKey, Alloc>;
|
|
||||||
|
|
||||||
#undef THash
|
|
||||||
#endif
|
|
@ -1,25 +0,0 @@
|
|||||||
#pragma once
|
|
||||||
|
|
||||||
/// DenseHashSet is a wrapper for google::dense_hash_set.
|
|
||||||
/// See comment in DenseHashMap.h
|
|
||||||
#if defined(ARCADIA_BUILD)
|
|
||||||
#define HASH_FUN_H <unordered_map>
|
|
||||||
template <typename T>
|
|
||||||
struct THash;
|
|
||||||
#endif
|
|
||||||
|
|
||||||
#include <sparsehash/dense_hash_set>
|
|
||||||
|
|
||||||
#if !defined(ARCADIA_BUILD)
|
|
||||||
template <class Value, class HashFcn = std::hash<Value>,
|
|
||||||
class EqualKey = std::equal_to<Value>,
|
|
||||||
class Alloc = google::libc_allocator_with_realloc<Value>>
|
|
||||||
using DenseHashSet = google::dense_hash_set<Value, HashFcn, EqualKey, Alloc>;
|
|
||||||
#else
|
|
||||||
template <class Value, class HashFcn = std::hash<Value>,
|
|
||||||
class EqualKey = std::equal_to<Value>,
|
|
||||||
class Alloc = google::sparsehash::libc_allocator_with_realloc<Value>>
|
|
||||||
using DenseHashSet = google::sparsehash::dense_hash_set<Value, HashFcn, EqualKey, Alloc>;
|
|
||||||
|
|
||||||
#undef THash
|
|
||||||
#endif
|
|
@ -1,7 +1,6 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
/// SparseHashMap is a wrapper for google::sparse_hash_map.
|
/// SparseHashMap is a wrapper for google::sparse_hash_map.
|
||||||
/// See comment in DenseHashMap.h
|
|
||||||
#if defined(ARCADIA_BUILD)
|
#if defined(ARCADIA_BUILD)
|
||||||
#define HASH_FUN_H <unordered_map>
|
#define HASH_FUN_H <unordered_map>
|
||||||
template <typename T>
|
template <typename T>
|
||||||
|
@ -1,4 +1,5 @@
|
|||||||
#include <Core/NamesAndTypes.h>
|
#include <Core/NamesAndTypes.h>
|
||||||
|
#include <Common/HashTable/HashMap.h>
|
||||||
#include <DataTypes/DataTypeFactory.h>
|
#include <DataTypes/DataTypeFactory.h>
|
||||||
#include <IO/ReadBuffer.h>
|
#include <IO/ReadBuffer.h>
|
||||||
#include <IO/WriteBuffer.h>
|
#include <IO/WriteBuffer.h>
|
||||||
@ -6,7 +7,6 @@
|
|||||||
#include <IO/WriteHelpers.h>
|
#include <IO/WriteHelpers.h>
|
||||||
#include <IO/ReadBufferFromString.h>
|
#include <IO/ReadBufferFromString.h>
|
||||||
#include <IO/WriteBufferFromString.h>
|
#include <IO/WriteBufferFromString.h>
|
||||||
#include <Common/DenseHashMap.h>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -163,8 +163,7 @@ NamesAndTypesList NamesAndTypesList::filter(const Names & names) const
|
|||||||
NamesAndTypesList NamesAndTypesList::addTypes(const Names & names) const
|
NamesAndTypesList NamesAndTypesList::addTypes(const Names & names) const
|
||||||
{
|
{
|
||||||
/// NOTE: It's better to make a map in `IStorage` than to create it here every time again.
|
/// NOTE: It's better to make a map in `IStorage` than to create it here every time again.
|
||||||
DenseHashMap<StringRef, const DataTypePtr *, StringRefHash> types;
|
HashMapWithSavedHash<StringRef, const DataTypePtr *, StringRefHash> types;
|
||||||
types.set_empty_key(StringRef());
|
|
||||||
|
|
||||||
for (const auto & column : *this)
|
for (const auto & column : *this)
|
||||||
types[column.name] = &column.type;
|
types[column.name] = &column.type;
|
||||||
@ -172,10 +171,11 @@ NamesAndTypesList NamesAndTypesList::addTypes(const Names & names) const
|
|||||||
NamesAndTypesList res;
|
NamesAndTypesList res;
|
||||||
for (const String & name : names)
|
for (const String & name : names)
|
||||||
{
|
{
|
||||||
auto it = types.find(name);
|
const auto * it = types.find(name);
|
||||||
if (it == types.end())
|
if (it == types.end())
|
||||||
throw Exception("No column " + name, ErrorCodes::THERE_IS_NO_COLUMN);
|
throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "No column {}", name);
|
||||||
res.emplace_back(name, *it->second);
|
|
||||||
|
res.emplace_back(name, *it->getMapped());
|
||||||
}
|
}
|
||||||
|
|
||||||
return res;
|
return res;
|
||||||
|
@ -48,7 +48,6 @@ IMergeTreeReader::IMergeTreeReader(
|
|||||||
part_columns = Nested::collect(part_columns);
|
part_columns = Nested::collect(part_columns);
|
||||||
}
|
}
|
||||||
|
|
||||||
columns_from_part.set_empty_key(StringRef());
|
|
||||||
for (const auto & column_from_part : part_columns)
|
for (const auto & column_from_part : part_columns)
|
||||||
columns_from_part[column_from_part.name] = &column_from_part.type;
|
columns_from_part[column_from_part.name] = &column_from_part.type;
|
||||||
}
|
}
|
||||||
@ -213,7 +212,7 @@ NameAndTypePair IMergeTreeReader::getColumnFromPart(const NameAndTypePair & requ
|
|||||||
{
|
{
|
||||||
auto name_in_storage = required_column.getNameInStorage();
|
auto name_in_storage = required_column.getNameInStorage();
|
||||||
|
|
||||||
decltype(columns_from_part.begin()) it;
|
ColumnsFromPart::ConstLookupResult it;
|
||||||
if (alter_conversions.isColumnRenamed(name_in_storage))
|
if (alter_conversions.isColumnRenamed(name_in_storage))
|
||||||
{
|
{
|
||||||
String old_name = alter_conversions.getColumnOldName(name_in_storage);
|
String old_name = alter_conversions.getColumnOldName(name_in_storage);
|
||||||
@ -227,7 +226,7 @@ NameAndTypePair IMergeTreeReader::getColumnFromPart(const NameAndTypePair & requ
|
|||||||
if (it == columns_from_part.end())
|
if (it == columns_from_part.end())
|
||||||
return required_column;
|
return required_column;
|
||||||
|
|
||||||
const auto & type = *it->second;
|
const DataTypePtr & type = *it->getMapped();
|
||||||
if (required_column.isSubcolumn())
|
if (required_column.isSubcolumn())
|
||||||
{
|
{
|
||||||
auto subcolumn_name = required_column.getSubcolumnName();
|
auto subcolumn_name = required_column.getSubcolumnName();
|
||||||
@ -236,10 +235,10 @@ NameAndTypePair IMergeTreeReader::getColumnFromPart(const NameAndTypePair & requ
|
|||||||
if (!subcolumn_type)
|
if (!subcolumn_type)
|
||||||
return required_column;
|
return required_column;
|
||||||
|
|
||||||
return {String(it->first), subcolumn_name, type, subcolumn_type};
|
return {String(it->getKey()), subcolumn_name, type, subcolumn_type};
|
||||||
}
|
}
|
||||||
|
|
||||||
return {String(it->first), type};
|
return {String(it->getKey()), type};
|
||||||
}
|
}
|
||||||
|
|
||||||
void IMergeTreeReader::performRequiredConversions(Columns & res_columns)
|
void IMergeTreeReader::performRequiredConversions(Columns & res_columns)
|
||||||
|
@ -1,7 +1,7 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Core/NamesAndTypes.h>
|
#include <Core/NamesAndTypes.h>
|
||||||
#include <Common/DenseHashMap.h>
|
#include <Common/HashTable/HashMap.h>
|
||||||
#include <Storages/MergeTree/MergeTreeReaderStream.h>
|
#include <Storages/MergeTree/MergeTreeReaderStream.h>
|
||||||
#include <Storages/MergeTree/MergeTreeBlockReadUtils.h>
|
#include <Storages/MergeTree/MergeTreeBlockReadUtils.h>
|
||||||
|
|
||||||
@ -95,7 +95,8 @@ private:
|
|||||||
|
|
||||||
/// Actual data type of columns in part
|
/// Actual data type of columns in part
|
||||||
|
|
||||||
DenseHashMap<StringRef, const DataTypePtr *, StringRefHash> columns_from_part;
|
using ColumnsFromPart = HashMapWithSavedHash<StringRef, const DataTypePtr *, StringRefHash>;
|
||||||
|
ColumnsFromPart columns_from_part;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -1,7 +1,7 @@
|
|||||||
#include <Storages/StorageInMemoryMetadata.h>
|
#include <Storages/StorageInMemoryMetadata.h>
|
||||||
|
|
||||||
#include <Common/DenseHashMap.h>
|
#include <Common/HashTable/HashMap.h>
|
||||||
#include <Common/DenseHashSet.h>
|
#include <Common/HashTable/HashSet.h>
|
||||||
#include <Common/quoteString.h>
|
#include <Common/quoteString.h>
|
||||||
#include <Common/StringUtils/StringUtils.h>
|
#include <Common/StringUtils/StringUtils.h>
|
||||||
#include <Core/ColumnWithTypeAndName.h>
|
#include <Core/ColumnWithTypeAndName.h>
|
||||||
@ -320,8 +320,7 @@ Block StorageInMemoryMetadata::getSampleBlockForColumns(
|
|||||||
{
|
{
|
||||||
Block res;
|
Block res;
|
||||||
|
|
||||||
DenseHashMap<StringRef, const DataTypePtr *, StringRefHash> virtuals_map;
|
HashMapWithSavedHash<StringRef, const DataTypePtr *, StringRefHash> virtuals_map;
|
||||||
virtuals_map.set_empty_key(StringRef());
|
|
||||||
|
|
||||||
/// Virtual columns must be appended after ordinary, because user can
|
/// Virtual columns must be appended after ordinary, because user can
|
||||||
/// override them.
|
/// override them.
|
||||||
@ -335,9 +334,9 @@ Block StorageInMemoryMetadata::getSampleBlockForColumns(
|
|||||||
{
|
{
|
||||||
res.insert({column->type->createColumn(), column->type, column->name});
|
res.insert({column->type->createColumn(), column->type, column->name});
|
||||||
}
|
}
|
||||||
else if (auto it = virtuals_map.find(name); it != virtuals_map.end())
|
else if (auto * it = virtuals_map.find(name); it != virtuals_map.end())
|
||||||
{
|
{
|
||||||
const auto & type = *it->second;
|
const auto & type = *it->getMapped();
|
||||||
res.insert({type->createColumn(), type, name});
|
res.insert({type->createColumn(), type, name});
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
@ -470,8 +469,8 @@ bool StorageInMemoryMetadata::hasSelectQuery() const
|
|||||||
|
|
||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
using NamesAndTypesMap = DenseHashMap<StringRef, const IDataType *, StringRefHash>;
|
using NamesAndTypesMap = HashMapWithSavedHash<StringRef, const IDataType *, StringRefHash>;
|
||||||
using UniqueStrings = DenseHashSet<StringRef, StringRefHash>;
|
using UniqueStrings = HashSetWithSavedHash<StringRef, StringRefHash>;
|
||||||
|
|
||||||
String listOfColumns(const NamesAndTypesList & available_columns)
|
String listOfColumns(const NamesAndTypesList & available_columns)
|
||||||
{
|
{
|
||||||
@ -488,20 +487,12 @@ namespace
|
|||||||
NamesAndTypesMap getColumnsMap(const NamesAndTypesList & columns)
|
NamesAndTypesMap getColumnsMap(const NamesAndTypesList & columns)
|
||||||
{
|
{
|
||||||
NamesAndTypesMap res;
|
NamesAndTypesMap res;
|
||||||
res.set_empty_key(StringRef());
|
|
||||||
|
|
||||||
for (const auto & column : columns)
|
for (const auto & column : columns)
|
||||||
res.insert({column.name, column.type.get()});
|
res.insert({column.name, column.type.get()});
|
||||||
|
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
UniqueStrings initUniqueStrings()
|
|
||||||
{
|
|
||||||
UniqueStrings strings;
|
|
||||||
strings.set_empty_key(StringRef());
|
|
||||||
return strings;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
void StorageInMemoryMetadata::check(const Names & column_names, const NamesAndTypesList & virtuals, const StorageID & storage_id) const
|
void StorageInMemoryMetadata::check(const Names & column_names, const NamesAndTypesList & virtuals, const StorageID & storage_id) const
|
||||||
@ -514,11 +505,12 @@ void StorageInMemoryMetadata::check(const Names & column_names, const NamesAndTy
|
|||||||
}
|
}
|
||||||
|
|
||||||
const auto virtuals_map = getColumnsMap(virtuals);
|
const auto virtuals_map = getColumnsMap(virtuals);
|
||||||
auto unique_names = initUniqueStrings();
|
UniqueStrings unique_names;
|
||||||
|
|
||||||
for (const auto & name : column_names)
|
for (const auto & name : column_names)
|
||||||
{
|
{
|
||||||
bool has_column = getColumns().hasColumnOrSubcolumn(ColumnsDescription::AllPhysical, name) || virtuals_map.count(name);
|
bool has_column = getColumns().hasColumnOrSubcolumn(ColumnsDescription::AllPhysical, name)
|
||||||
|
|| virtuals_map.find(name) != nullptr;
|
||||||
|
|
||||||
if (!has_column)
|
if (!has_column)
|
||||||
{
|
{
|
||||||
@ -540,23 +532,31 @@ void StorageInMemoryMetadata::check(const NamesAndTypesList & provided_columns)
|
|||||||
const NamesAndTypesList & available_columns = getColumns().getAllPhysical();
|
const NamesAndTypesList & available_columns = getColumns().getAllPhysical();
|
||||||
const auto columns_map = getColumnsMap(available_columns);
|
const auto columns_map = getColumnsMap(available_columns);
|
||||||
|
|
||||||
auto unique_names = initUniqueStrings();
|
UniqueStrings unique_names;
|
||||||
|
|
||||||
for (const NameAndTypePair & column : provided_columns)
|
for (const NameAndTypePair & column : provided_columns)
|
||||||
{
|
{
|
||||||
auto it = columns_map.find(column.name);
|
const auto * it = columns_map.find(column.name);
|
||||||
if (columns_map.end() == it)
|
if (columns_map.end() == it)
|
||||||
throw Exception(
|
throw Exception(
|
||||||
"There is no column with name " + column.name + ". There are columns: " + listOfColumns(available_columns),
|
ErrorCodes::NO_SUCH_COLUMN_IN_TABLE,
|
||||||
ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
|
"There is no column with name {}. There are columns: {}",
|
||||||
|
column.name,
|
||||||
|
listOfColumns(available_columns));
|
||||||
|
|
||||||
if (!column.type->equals(*it->second))
|
if (!column.type->equals(*it->getMapped()))
|
||||||
throw Exception(
|
throw Exception(
|
||||||
"Type mismatch for column " + column.name + ". Column has type " + it->second->getName() + ", got type "
|
ErrorCodes::TYPE_MISMATCH,
|
||||||
+ column.type->getName(),
|
"Type mismatch for column {}. Column has type {}, got type {}",
|
||||||
ErrorCodes::TYPE_MISMATCH);
|
column.name,
|
||||||
|
it->getMapped()->getName(),
|
||||||
|
column.type->getName());
|
||||||
|
|
||||||
if (unique_names.end() != unique_names.find(column.name))
|
if (unique_names.end() != unique_names.find(column.name))
|
||||||
throw Exception("Column " + column.name + " queried more than once", ErrorCodes::COLUMN_QUERIED_MORE_THAN_ONCE);
|
throw Exception(ErrorCodes::COLUMN_QUERIED_MORE_THAN_ONCE,
|
||||||
|
"Column {} queried more than once",
|
||||||
|
column.name);
|
||||||
|
|
||||||
unique_names.insert(column.name);
|
unique_names.insert(column.name);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -572,26 +572,38 @@ void StorageInMemoryMetadata::check(const NamesAndTypesList & provided_columns,
|
|||||||
"Empty list of columns queried. There are columns: " + listOfColumns(available_columns),
|
"Empty list of columns queried. There are columns: " + listOfColumns(available_columns),
|
||||||
ErrorCodes::EMPTY_LIST_OF_COLUMNS_QUERIED);
|
ErrorCodes::EMPTY_LIST_OF_COLUMNS_QUERIED);
|
||||||
|
|
||||||
auto unique_names = initUniqueStrings();
|
UniqueStrings unique_names;
|
||||||
|
|
||||||
for (const String & name : column_names)
|
for (const String & name : column_names)
|
||||||
{
|
{
|
||||||
auto it = provided_columns_map.find(name);
|
const auto * it = provided_columns_map.find(name);
|
||||||
if (provided_columns_map.end() == it)
|
if (provided_columns_map.end() == it)
|
||||||
continue;
|
continue;
|
||||||
|
|
||||||
auto jt = available_columns_map.find(name);
|
const auto * jt = available_columns_map.find(name);
|
||||||
if (available_columns_map.end() == jt)
|
if (available_columns_map.end() == jt)
|
||||||
throw Exception(
|
throw Exception(
|
||||||
"There is no column with name " + name + ". There are columns: " + listOfColumns(available_columns),
|
ErrorCodes::NO_SUCH_COLUMN_IN_TABLE,
|
||||||
ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
|
"There is no column with name {}. There are columns: {}",
|
||||||
|
name,
|
||||||
|
listOfColumns(available_columns));
|
||||||
|
|
||||||
if (!it->second->equals(*jt->second))
|
const auto & provided_column_type = *it->getMapped();
|
||||||
|
const auto & available_column_type = *jt->getMapped();
|
||||||
|
|
||||||
|
if (!provided_column_type.equals(available_column_type))
|
||||||
throw Exception(
|
throw Exception(
|
||||||
"Type mismatch for column " + name + ". Column has type " + jt->second->getName() + ", got type " + it->second->getName(),
|
ErrorCodes::TYPE_MISMATCH,
|
||||||
ErrorCodes::TYPE_MISMATCH);
|
"Type mismatch for column {}. Column has type {}, got type {}",
|
||||||
|
name,
|
||||||
|
provided_column_type.getName(),
|
||||||
|
available_column_type.getName());
|
||||||
|
|
||||||
if (unique_names.end() != unique_names.find(name))
|
if (unique_names.end() != unique_names.find(name))
|
||||||
throw Exception("Column " + name + " queried more than once", ErrorCodes::COLUMN_QUERIED_MORE_THAN_ONCE);
|
throw Exception(ErrorCodes::COLUMN_QUERIED_MORE_THAN_ONCE,
|
||||||
|
"Column {} queried more than once",
|
||||||
|
name);
|
||||||
|
|
||||||
unique_names.insert(name);
|
unique_names.insert(name);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -612,17 +624,21 @@ void StorageInMemoryMetadata::check(const Block & block, bool need_all) const
|
|||||||
|
|
||||||
names_in_block.insert(column.name);
|
names_in_block.insert(column.name);
|
||||||
|
|
||||||
auto it = columns_map.find(column.name);
|
const auto * it = columns_map.find(column.name);
|
||||||
if (columns_map.end() == it)
|
if (columns_map.end() == it)
|
||||||
throw Exception(
|
throw Exception(
|
||||||
"There is no column with name " + column.name + ". There are columns: " + listOfColumns(available_columns),
|
ErrorCodes::NO_SUCH_COLUMN_IN_TABLE,
|
||||||
ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
|
"There is no column with name {}. There are columns: {}",
|
||||||
|
column.name,
|
||||||
|
listOfColumns(available_columns));
|
||||||
|
|
||||||
if (!column.type->equals(*it->second))
|
if (!column.type->equals(*it->getMapped()))
|
||||||
throw Exception(
|
throw Exception(
|
||||||
"Type mismatch for column " + column.name + ". Column has type " + it->second->getName() + ", got type "
|
ErrorCodes::TYPE_MISMATCH,
|
||||||
+ column.type->getName(),
|
"Type mismatch for column {}. Column has type {}, got type {}",
|
||||||
ErrorCodes::TYPE_MISMATCH);
|
column.name,
|
||||||
|
it->getMapped()->getName(),
|
||||||
|
column.type->getName());
|
||||||
}
|
}
|
||||||
|
|
||||||
if (need_all && names_in_block.size() < columns_map.size())
|
if (need_all && names_in_block.size() < columns_map.size())
|
||||||
|
Loading…
Reference in New Issue
Block a user