Merge pull request #27690 from kitaisreal/remove-dense-hash-map-set

Removed DenseHashMap, DenseHashSet
This commit is contained in:
alexey-milovidov 2021-08-16 02:39:41 +03:00 committed by GitHub
commit 76b0502482
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 72 additions and 111 deletions

View File

@ -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

View File

@ -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

View File

@ -1,7 +1,6 @@
#pragma once
/// SparseHashMap is a wrapper for google::sparse_hash_map.
/// See comment in DenseHashMap.h
#if defined(ARCADIA_BUILD)
#define HASH_FUN_H <unordered_map>
template <typename T>

View File

@ -1,4 +1,5 @@
#include <Core/NamesAndTypes.h>
#include <Common/HashTable/HashMap.h>
#include <DataTypes/DataTypeFactory.h>
#include <IO/ReadBuffer.h>
#include <IO/WriteBuffer.h>
@ -6,7 +7,6 @@
#include <IO/WriteHelpers.h>
#include <IO/ReadBufferFromString.h>
#include <IO/WriteBufferFromString.h>
#include <Common/DenseHashMap.h>
namespace DB
@ -163,8 +163,7 @@ NamesAndTypesList NamesAndTypesList::filter(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.
DenseHashMap<StringRef, const DataTypePtr *, StringRefHash> types;
types.set_empty_key(StringRef());
HashMapWithSavedHash<StringRef, const DataTypePtr *, StringRefHash> types;
for (const auto & column : *this)
types[column.name] = &column.type;
@ -172,10 +171,11 @@ NamesAndTypesList NamesAndTypesList::addTypes(const Names & names) const
NamesAndTypesList res;
for (const String & name : names)
{
auto it = types.find(name);
const auto * it = types.find(name);
if (it == types.end())
throw Exception("No column " + name, ErrorCodes::THERE_IS_NO_COLUMN);
res.emplace_back(name, *it->second);
throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "No column {}", name);
res.emplace_back(name, *it->getMapped());
}
return res;

View File

@ -48,7 +48,6 @@ IMergeTreeReader::IMergeTreeReader(
part_columns = Nested::collect(part_columns);
}
columns_from_part.set_empty_key(StringRef());
for (const auto & column_from_part : part_columns)
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();
decltype(columns_from_part.begin()) it;
ColumnsFromPart::ConstLookupResult it;
if (alter_conversions.isColumnRenamed(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())
return required_column;
const auto & type = *it->second;
const DataTypePtr & type = *it->getMapped();
if (required_column.isSubcolumn())
{
auto subcolumn_name = required_column.getSubcolumnName();
@ -236,10 +235,10 @@ NameAndTypePair IMergeTreeReader::getColumnFromPart(const NameAndTypePair & requ
if (!subcolumn_type)
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)

View File

@ -1,7 +1,7 @@
#pragma once
#include <Core/NamesAndTypes.h>
#include <Common/DenseHashMap.h>
#include <Common/HashTable/HashMap.h>
#include <Storages/MergeTree/MergeTreeReaderStream.h>
#include <Storages/MergeTree/MergeTreeBlockReadUtils.h>
@ -95,7 +95,8 @@ private:
/// 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;
};
}

View File

@ -1,7 +1,7 @@
#include <Storages/StorageInMemoryMetadata.h>
#include <Common/DenseHashMap.h>
#include <Common/DenseHashSet.h>
#include <Common/HashTable/HashMap.h>
#include <Common/HashTable/HashSet.h>
#include <Common/quoteString.h>
#include <Common/StringUtils/StringUtils.h>
#include <Core/ColumnWithTypeAndName.h>
@ -320,8 +320,7 @@ Block StorageInMemoryMetadata::getSampleBlockForColumns(
{
Block res;
DenseHashMap<StringRef, const DataTypePtr *, StringRefHash> virtuals_map;
virtuals_map.set_empty_key(StringRef());
HashMapWithSavedHash<StringRef, const DataTypePtr *, StringRefHash> virtuals_map;
/// Virtual columns must be appended after ordinary, because user can
/// override them.
@ -335,9 +334,9 @@ Block StorageInMemoryMetadata::getSampleBlockForColumns(
{
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});
}
else
@ -470,8 +469,8 @@ bool StorageInMemoryMetadata::hasSelectQuery() const
namespace
{
using NamesAndTypesMap = DenseHashMap<StringRef, const IDataType *, StringRefHash>;
using UniqueStrings = DenseHashSet<StringRef, StringRefHash>;
using NamesAndTypesMap = HashMapWithSavedHash<StringRef, const IDataType *, StringRefHash>;
using UniqueStrings = HashSetWithSavedHash<StringRef, StringRefHash>;
String listOfColumns(const NamesAndTypesList & available_columns)
{
@ -488,20 +487,12 @@ namespace
NamesAndTypesMap getColumnsMap(const NamesAndTypesList & columns)
{
NamesAndTypesMap res;
res.set_empty_key(StringRef());
for (const auto & column : columns)
res.insert({column.name, column.type.get()});
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
@ -514,11 +505,12 @@ void StorageInMemoryMetadata::check(const Names & column_names, const NamesAndTy
}
const auto virtuals_map = getColumnsMap(virtuals);
auto unique_names = initUniqueStrings();
UniqueStrings unique_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)
{
@ -540,23 +532,31 @@ void StorageInMemoryMetadata::check(const NamesAndTypesList & provided_columns)
const NamesAndTypesList & available_columns = getColumns().getAllPhysical();
const auto columns_map = getColumnsMap(available_columns);
auto unique_names = initUniqueStrings();
UniqueStrings unique_names;
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)
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(
"Type mismatch for column " + column.name + ". Column has type " + it->second->getName() + ", got type "
+ column.type->getName(),
ErrorCodes::TYPE_MISMATCH);
ErrorCodes::TYPE_MISMATCH,
"Type mismatch for column {}. Column has type {}, got type {}",
column.name,
it->getMapped()->getName(),
column.type->getName());
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);
}
}
@ -572,26 +572,38 @@ void StorageInMemoryMetadata::check(const NamesAndTypesList & provided_columns,
"Empty list of columns queried. There are columns: " + listOfColumns(available_columns),
ErrorCodes::EMPTY_LIST_OF_COLUMNS_QUERIED);
auto unique_names = initUniqueStrings();
UniqueStrings unique_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)
continue;
auto jt = available_columns_map.find(name);
const auto * jt = available_columns_map.find(name);
if (available_columns_map.end() == jt)
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(
"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))
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);
}
}
@ -612,17 +624,21 @@ void StorageInMemoryMetadata::check(const Block & block, bool need_all) const
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)
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(
"Type mismatch for column " + column.name + ". Column has type " + it->second->getName() + ", got type "
+ column.type->getName(),
ErrorCodes::TYPE_MISMATCH);
ErrorCodes::TYPE_MISMATCH,
"Type mismatch for column {}. Column has type {}, got type {}",
column.name,
it->getMapped()->getName(),
column.type->getName());
}
if (need_all && names_in_block.size() < columns_map.size())