diff --git a/dbms/programs/obfuscator/Obfuscator.cpp b/dbms/programs/obfuscator/Obfuscator.cpp index febe2b286060..5d649b5f1766 100644 --- a/dbms/programs/obfuscator/Obfuscator.cpp +++ b/dbms/programs/obfuscator/Obfuscator.cpp @@ -670,13 +670,13 @@ class MarkovModel while (pos < end) { - Table::iterator it = table.end(); + Table::MappedPtr it; size_t context_size = params.order; while (true) { it = table.find(hashContext(code_points.data() + code_points.size() - context_size, code_points.data() + code_points.size())); - if (table.end() != it && it->getSecond().total + it->getSecond().count_end != 0) + if (it && it->total + it->count_end != 0) break; if (context_size == 0) @@ -684,7 +684,7 @@ class MarkovModel --context_size; } - if (table.end() == it) + if (!it) throw Exception("Logical error in markov model", ErrorCodes::LOGICAL_ERROR); size_t offset_from_begin_of_string = pos - data; @@ -710,7 +710,7 @@ class MarkovModel if (num_bytes_after_desired_size > 0) end_probability_multiplier = std::pow(1.25, num_bytes_after_desired_size); - CodePoint code = it->getSecond().sample(determinator, end_probability_multiplier); + CodePoint code = it->sample(determinator, end_probability_multiplier); if (code == END) break; diff --git a/dbms/src/Common/ColumnsHashingImpl.h b/dbms/src/Common/ColumnsHashingImpl.h index 89141f0f4d57..98045eb88437 100644 --- a/dbms/src/Common/ColumnsHashingImpl.h +++ b/dbms/src/Common/ColumnsHashingImpl.h @@ -221,20 +221,19 @@ class HashMethodBase } } - auto it = data.find(key); - bool found = it != data.end(); + auto mapped_ptr = data.find(key); if constexpr (consecutive_keys_optimization) { - cache.found = found; + cache.found = mapped_ptr != nullptr; cache.empty = false; if constexpr (has_mapped) { cache.value.first = key; - if (found) + if (mapped_ptr) { - cache.value.second = it->getSecond(); + cache.value.second = *mapped_ptr; } } else @@ -244,9 +243,9 @@ class HashMethodBase } if constexpr (has_mapped) - return FindResult(found ? &it->getSecond() : nullptr, found); + return FindResult(mapped_ptr, mapped_ptr != nullptr); else - return FindResult(found); + return FindResult(mapped_ptr != nullptr); } }; diff --git a/dbms/src/Common/HashTable/FixedHashMap.h b/dbms/src/Common/HashTable/FixedHashMap.h index eb3c798f31d1..f4dbf61fce51 100644 --- a/dbms/src/Common/HashTable/FixedHashMap.h +++ b/dbms/src/Common/HashTable/FixedHashMap.h @@ -90,11 +90,11 @@ class FixedHashMap : public FixedHashTable, A { for (auto it = this->begin(), end = this->end(); it != end; ++it) { - decltype(it) res_it = that.find(it->getFirst(), it.getHash()); - if (res_it == that.end()) + auto res_it = that.find(it->getFirst(), it.getHash()); + if (!res_it) func(it->getSecond(), it->getSecond(), false); else - func(res_it->getSecond(), it->getSecond(), true); + func(*res_it, it->getSecond(), true); } } diff --git a/dbms/src/Common/HashTable/FixedHashTable.h b/dbms/src/Common/HashTable/FixedHashTable.h index 67af43d197b5..fd0e05b72fb5 100644 --- a/dbms/src/Common/HashTable/FixedHashTable.h +++ b/dbms/src/Common/HashTable/FixedHashTable.h @@ -143,6 +143,7 @@ class FixedHashTable : private boost::noncopyable, protected Allocator, protecte using key_type = Key; using value_type = typename Cell::value_type; using MappedPtr = typename Cell::mapped_type *; + using ConstMappedPtr = const typename Cell::mapped_type *; size_t hash(const Key & x) const { return x; } @@ -289,24 +290,24 @@ class FixedHashTable : private boost::noncopyable, protected Allocator, protecte return res; } - iterator ALWAYS_INLINE find(Key x) + MappedPtr ALWAYS_INLINE find(Key x) { - return !buf[x].isZero(*this) ? iterator(this, &buf[x]) : end(); + return !buf[x].isZero(*this) ? buf[x].getMapped() : nullptr; } - const_iterator ALWAYS_INLINE find(Key x) const + ConstMappedPtr ALWAYS_INLINE find(Key x) const { - return !buf[x].isZero(*this) ? const_iterator(this, &buf[x]) : end(); + return const_cast *>(this)->find(x); } - iterator ALWAYS_INLINE find(Key, size_t hash_value) + MappedPtr ALWAYS_INLINE find(Key, size_t hash_value) { - return !buf[hash_value].isZero(*this) ? iterator(this, &buf[hash_value]) : end(); + return !buf[hash_value].isZero(*this) ? buf[hash_value].getMapped() : nullptr; } - const_iterator ALWAYS_INLINE find(Key, size_t hash_value) const + ConstMappedPtr ALWAYS_INLINE find(Key key, size_t hash_value) const { - return !buf[hash_value].isZero(*this) ? const_iterator(this, &buf[hash_value]) : end(); + return const_cast *>(this)->find(key, hash_value); } bool ALWAYS_INLINE has(Key x) const { return !buf[x].isZero(*this); } diff --git a/dbms/src/Common/HashTable/HashMap.h b/dbms/src/Common/HashTable/HashMap.h index c90f91c2d89a..b833a9dbd805 100644 --- a/dbms/src/Common/HashTable/HashMap.h +++ b/dbms/src/Common/HashTable/HashMap.h @@ -173,11 +173,11 @@ class HashMapTable : public HashTable { for (auto it = this->begin(), end = this->end(); it != end; ++it) { - decltype(it) res_it = that.find(it->getFirst(), it.getHash()); - if (res_it == that.end()) + auto res_it = that.find(it->getFirst(), it.getHash()); + if (!res_it) func(it->getSecond(), it->getSecond(), false); else - func(res_it->getSecond(), it->getSecond(), true); + func(*res_it, it->getSecond(), true); } } diff --git a/dbms/src/Common/HashTable/HashTable.h b/dbms/src/Common/HashTable/HashTable.h index b8e5a7e11dec..eeb314e80501 100644 --- a/dbms/src/Common/HashTable/HashTable.h +++ b/dbms/src/Common/HashTable/HashTable.h @@ -484,6 +484,7 @@ class HashTable : using key_type = Key; using value_type = typename Cell::value_type; using MappedPtr = typename Cell::mapped_type *; + using ConstMappedPtr = const typename Cell::mapped_type *; size_t hash(const Key & x) const { return Hash::operator()(x); } @@ -796,48 +797,35 @@ class HashTable : resize(); } - iterator ALWAYS_INLINE find(Key x) - { - if (Cell::isZero(x, *this)) - return this->hasZero() ? iteratorToZero() : end(); - - size_t hash_value = hash(x); - size_t place_value = findCell(x, hash_value, grower.place(hash_value)); - return !buf[place_value].isZero(*this) ? iterator(this, &buf[place_value]) : end(); - } - - const_iterator ALWAYS_INLINE find(Key x) const + MappedPtr ALWAYS_INLINE find(Key x) { if (Cell::isZero(x, *this)) - return this->hasZero() ? iteratorToZero() : end(); + return this->hasZero() ? this->zeroValue()->getMapped() : nullptr; size_t hash_value = hash(x); size_t place_value = findCell(x, hash_value, grower.place(hash_value)); - return !buf[place_value].isZero(*this) ? const_iterator(this, &buf[place_value]) : end(); + return !buf[place_value].isZero(*this) + ? buf[place_value].getMapped() + : nullptr; } - - iterator ALWAYS_INLINE find(Key x, size_t hash_value) + ConstMappedPtr ALWAYS_INLINE find(Key x) const { - if (Cell::isZero(x, *this)) - return this->hasZero() ? iteratorToZero() : end(); - - size_t place_value = findCell(x, hash_value, grower.place(hash_value)); - return !buf[place_value].isZero(*this) ? iterator(this, &buf[place_value]) : end(); + return const_cast *>(this)->find(x); } - - const_iterator ALWAYS_INLINE find(Key x, size_t hash_value) const + MappedPtr ALWAYS_INLINE find(Key x, size_t hash_value) { if (Cell::isZero(x, *this)) - return this->hasZero() ? iteratorToZero() : end(); + return this->hasZero() ? this->zeroValue()->getMapped() : nullptr; size_t place_value = findCell(x, hash_value, grower.place(hash_value)); - return !buf[place_value].isZero(*this) ? const_iterator(this, &buf[place_value]) : end(); + return !buf[place_value].isZero(*this) + ? buf[place_value].getMapped() + : nullptr; } - bool ALWAYS_INLINE has(Key x) const { if (Cell::isZero(x, *this)) diff --git a/dbms/src/Common/HashTable/TwoLevelHashTable.h b/dbms/src/Common/HashTable/TwoLevelHashTable.h index 2d90276bfe42..4b1fed6e0d5f 100644 --- a/dbms/src/Common/HashTable/TwoLevelHashTable.h +++ b/dbms/src/Common/HashTable/TwoLevelHashTable.h @@ -84,6 +84,7 @@ class TwoLevelHashTable : using key_type = typename Impl::key_type; using value_type = typename Impl::value_type; using MappedPtr = typename Impl::MappedPtr; + using ConstMappedPtr = typename Impl::ConstMappedPtr; Impl impls[NUM_BUCKETS]; @@ -255,31 +256,19 @@ class TwoLevelHashTable : it = impl_it; } - - iterator ALWAYS_INLINE find(Key x, size_t hash_value) + MappedPtr ALWAYS_INLINE find(Key x, size_t hash_value) { size_t buck = getBucketFromHash(hash_value); - - typename Impl::iterator found = impls[buck].find(x, hash_value); - return found != impls[buck].end() - ? iterator(this, buck, found) - : end(); + return impls[buck].find(x, hash_value); } - - const_iterator ALWAYS_INLINE find(Key x, size_t hash_value) const + ConstMappedPtr ALWAYS_INLINE find(Key x, size_t hash_value) const { - size_t buck = getBucketFromHash(hash_value); - - typename Impl::const_iterator found = impls[buck].find(x, hash_value); - return found != impls[buck].end() - ? const_iterator(this, buck, found) - : end(); + return const_cast *>(this)->find(x, hash_value); } - - iterator ALWAYS_INLINE find(Key x) { return find(x, hash(x)); } - const_iterator ALWAYS_INLINE find(Key x) const { return find(x, hash(x)); } + MappedPtr ALWAYS_INLINE find(Key x) { return find(x, hash(x)); } + ConstMappedPtr ALWAYS_INLINE find(Key x) const { return find(x, hash(x)); } void write(DB::WriteBuffer & wb) const diff --git a/dbms/src/Common/SpaceSaving.h b/dbms/src/Common/SpaceSaving.h index da7e9293723c..860f19da1758 100644 --- a/dbms/src/Common/SpaceSaving.h +++ b/dbms/src/Common/SpaceSaving.h @@ -366,10 +366,10 @@ class SpaceSaving Counter * findCounter(const TKey & key, size_t hash) { auto it = counter_map.find(key, hash); - if (it == counter_map.end()) + if (!it) return nullptr; - return it->getSecond(); + return *it; } void rebuildCounterMap() diff --git a/dbms/src/Common/tests/parallel_aggregation.cpp b/dbms/src/Common/tests/parallel_aggregation.cpp index 97e7a5a2cd06..11029793e41a 100644 --- a/dbms/src/Common/tests/parallel_aggregation.cpp +++ b/dbms/src/Common/tests/parallel_aggregation.cpp @@ -135,10 +135,10 @@ void aggregate3(Map & local_map, Map & global_map, Mutex & mutex, Source::const_ for (auto it = begin; it != end; ++it) { - Map::iterator found = local_map.find(*it); + auto found = local_map.find(*it); - if (found != local_map.end()) - ++found->getSecond(); + if (found) + ++*found; else if (local_map.size() < threshold) ++local_map[*it]; /// TODO You could do one lookup, not two. else @@ -195,10 +195,10 @@ void aggregate4(Map & local_map, MapTwoLevel & global_map, Mutex * mutexes, Sour { for (; it != block_end; ++it) { - Map::iterator found = local_map.find(*it); + auto found = local_map.find(*it); - if (found != local_map.end()) - ++found->getSecond(); + if (found) + ++*found; else { size_t hash_value = global_map.hash(*it); diff --git a/dbms/src/DataTypes/DataTypeEnum.h b/dbms/src/DataTypes/DataTypeEnum.h index b99e2383860b..521d105ebfe5 100644 --- a/dbms/src/DataTypes/DataTypeEnum.h +++ b/dbms/src/DataTypes/DataTypeEnum.h @@ -77,11 +77,11 @@ class DataTypeEnum final : public IDataTypeEnum FieldType getValue(StringRef field_name) const { - const auto it = name_to_value_map.find(field_name); - if (it == std::end(name_to_value_map)) + const auto value = name_to_value_map.find(field_name); + if (!value) throw Exception{"Unknown element '" + field_name.toString() + "' for type " + getName(), ErrorCodes::LOGICAL_ERROR}; - return it->getSecond(); + return *value; } Field castToName(const Field & value_or_name) const override; diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h index 7c2ba75ba176..2c74dc8a7f1b 100644 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h @@ -469,7 +469,7 @@ class ComplexKeyCacheDictionary final : public IDictionaryBase { const StringRef key = keys_array[row]; const auto it = map.find(key); - const auto string_ref = it != std::end(map) ? it->getSecond() : get_default(row); + const auto string_ref = it ? *it : get_default(row); out->insertData(string_ref.data, string_ref.size); } } diff --git a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp index 586fc5e89f97..a4904e8fc585 100644 --- a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp +++ b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp @@ -357,7 +357,7 @@ void ComplexKeyHashedDictionary::updateData() { const auto s_key = placeKeysInPool(i, saved_key_column_ptrs, keys, temp_key_pool); auto it = update_key_hash.find(s_key); - if (it != std::end(update_key_hash)) + if (it) filter[i] = 0; else filter[i] = 1; @@ -561,7 +561,7 @@ void ComplexKeyHashedDictionary::getItemsImpl( const auto key = placeKeysInPool(i, key_columns, keys, temporary_keys_pool); const auto it = attr.find(key); - set_value(i, it != attr.end() ? static_cast(it->getSecond()) : get_default(i)); + set_value(i, it ? static_cast(*it) : get_default(i)); /// free memory allocated for the key temporary_keys_pool.rollback(key.size); @@ -672,7 +672,7 @@ void ComplexKeyHashedDictionary::has(const Attribute & attribute, const Columns const auto key = placeKeysInPool(i, key_columns, keys, temporary_keys_pool); const auto it = attr.find(key); - out[i] = it != attr.end(); + out[i] = static_cast(it); /// free memory allocated for the key temporary_keys_pool.rollback(key.size); diff --git a/dbms/src/Dictionaries/HashedDictionary.cpp b/dbms/src/Dictionaries/HashedDictionary.cpp index 9b853ac2df54..92538dd47878 100644 --- a/dbms/src/Dictionaries/HashedDictionary.cpp +++ b/dbms/src/Dictionaries/HashedDictionary.cpp @@ -72,8 +72,8 @@ void HashedDictionary::isInImpl(const ChildType & child_ids, const AncestorType while (id != null_value && id != ancestor_id) { auto it = attr.find(id); - if (it != std::end(attr)) - id = it->getSecond(); + if (it) + id = *it; else break; } @@ -555,7 +555,7 @@ void HashedDictionary::getItemsImpl( for (const auto i : ext::range(0, rows)) { const auto it = attr.find(ids[i]); - set_value(i, it != attr.end() ? static_cast(it->getSecond()) : get_default(i)); + set_value(i, it ? static_cast(*it) : get_default(i)); } query_count.fetch_add(rows, std::memory_order_relaxed); @@ -631,7 +631,7 @@ void HashedDictionary::has(const Attribute & attribute, const PaddedPODArray(attr.find(ids[i])); query_count.fetch_add(rows, std::memory_order_relaxed); } diff --git a/dbms/src/Dictionaries/RangeHashedDictionary.cpp b/dbms/src/Dictionaries/RangeHashedDictionary.cpp index ab67ce593715..fa0ef57bf243 100644 --- a/dbms/src/Dictionaries/RangeHashedDictionary.cpp +++ b/dbms/src/Dictionaries/RangeHashedDictionary.cpp @@ -124,10 +124,10 @@ void RangeHashedDictionary::getString( for (const auto i : ext::range(0, ids.size())) { const auto it = attr.find(ids[i]); - if (it != std::end(attr)) + if (it) { const auto date = dates[i]; - const auto & ranges_and_values = it->getSecond(); + const auto & ranges_and_values = *it; const auto val_it = std::find_if(std::begin(ranges_and_values), std::end(ranges_and_values), [date](const Value & v) { @@ -395,10 +395,10 @@ void RangeHashedDictionary::getItemsImpl( for (const auto i : ext::range(0, ids.size())) { const auto it = attr.find(ids[i]); - if (it != std::end(attr)) + if (it) { const auto date = dates[i]; - const auto & ranges_and_values = it->getSecond(); + const auto & ranges_and_values = *it; const auto val_it = std::find_if(std::begin(ranges_and_values), std::end(ranges_and_values), [date](const Value & v) { @@ -423,9 +423,9 @@ void RangeHashedDictionary::setAttributeValueImpl(Attribute & attribute, const K auto & map = *std::get>(attribute.maps); const auto it = map.find(id); - if (it != map.end()) + if (it) { - auto & values = it->getSecond(); + auto & values = *it; const auto insert_it = std::lower_bound(std::begin(values), std::end(values), range, [](const Value & lhs, const Range & rhs_range) @@ -496,9 +496,9 @@ void RangeHashedDictionary::setAttributeValue(Attribute & attribute, const Key i const auto it = map.find(id); - if (it != map.end()) + if (it) { - auto & values = it->getSecond(); + auto & values = *it; const auto insert_it = std::lower_bound( std::begin(values), std::end(values), range, [](const Value & lhs, const Range & rhs_range) diff --git a/dbms/src/Functions/array/arrayDistinct.cpp b/dbms/src/Functions/array/arrayDistinct.cpp index aa4e8aec4f14..4bcd5100b268 100644 --- a/dbms/src/Functions/array/arrayDistinct.cpp +++ b/dbms/src/Functions/array/arrayDistinct.cpp @@ -173,7 +173,7 @@ bool FunctionArrayDistinct::executeNumber( if (nullable_col && (*src_null_map)[j]) continue; - if (set.find(values[j]) == set.end()) + if (!set.find(values[j])) { res_data.emplace_back(values[j]); set.insert(values[j]); @@ -229,7 +229,7 @@ bool FunctionArrayDistinct::executeString( StringRef str_ref = src_data_concrete->getDataAt(j); - if (set.find(str_ref) == set.end()) + if (!set.find(str_ref)) { set.insert(str_ref); res_data_column_string.insertData(str_ref.data, str_ref.size); @@ -279,7 +279,7 @@ void FunctionArrayDistinct::executeHashed( src_data.updateHashWithValue(j, hash_function); hash_function.get128(reinterpret_cast(&hash)); - if (set.find(hash) == set.end()) + if (!set.find(hash)) { set.insert(hash); res_data_col.insertFrom(src_data, j); diff --git a/dbms/src/Functions/transform.cpp b/dbms/src/Functions/transform.cpp index bac5adbb7b9f..9882a7f3d9fd 100644 --- a/dbms/src/Functions/transform.cpp +++ b/dbms/src/Functions/transform.cpp @@ -507,8 +507,8 @@ class FunctionTransform : public IFunction for (size_t i = 0; i < size; ++i) { auto it = table.find(src[i]); - if (it != table.end()) - memcpy(&dst[i], &it->getSecond(), sizeof(dst[i])); /// little endian. + if (it) + memcpy(&dst[i], it, sizeof(dst[i])); /// little endian. else dst[i] = dst_default; } @@ -523,8 +523,8 @@ class FunctionTransform : public IFunction for (size_t i = 0; i < size; ++i) { auto it = table.find(src[i]); - if (it != table.end()) - memcpy(&dst[i], &it->getSecond(), sizeof(dst[i])); /// little endian. + if (it) + memcpy(&dst[i], it, sizeof(dst[i])); /// little endian. else dst[i] = dst_default[i]; } @@ -539,8 +539,8 @@ class FunctionTransform : public IFunction for (size_t i = 0; i < size; ++i) { auto it = table.find(src[i]); - if (it != table.end()) - memcpy(&dst[i], &it->getSecond(), sizeof(dst[i])); + if (it) + memcpy(&dst[i], it, sizeof(dst[i])); else dst[i] = src[i]; } @@ -557,7 +557,7 @@ class FunctionTransform : public IFunction for (size_t i = 0; i < size; ++i) { auto it = table.find(src[i]); - StringRef ref = it != table.end() ? it->getSecond() : dst_default; + StringRef ref = it ? *it : dst_default; dst_data.resize(current_dst_offset + ref.size); memcpy(&dst_data[current_dst_offset], ref.data, ref.size); current_dst_offset += ref.size; @@ -580,8 +580,8 @@ class FunctionTransform : public IFunction auto it = table.find(src[i]); StringRef ref; - if (it != table.end()) - ref = it->getSecond(); + if (it) + ref = *it; else { ref.data = reinterpret_cast(&dst_default_data[current_dst_default_offset]); @@ -610,8 +610,8 @@ class FunctionTransform : public IFunction StringRef ref{&src_data[current_src_offset], src_offsets[i] - current_src_offset}; current_src_offset = src_offsets[i]; auto it = table.find(ref); - if (it != table.end()) - memcpy(&dst[i], &it->getSecond(), sizeof(dst[i])); + if (it) + memcpy(&dst[i], it, sizeof(dst[i])); else dst[i] = dst_default; } @@ -631,8 +631,8 @@ class FunctionTransform : public IFunction StringRef ref{&src_data[current_src_offset], src_offsets[i] - current_src_offset}; current_src_offset = src_offsets[i]; auto it = table.find(ref); - if (it != table.end()) - memcpy(&dst[i], &it->getSecond(), sizeof(dst[i])); + if (it) + memcpy(&dst[i], it, sizeof(dst[i])); else dst[i] = dst_default[i]; } @@ -655,7 +655,7 @@ class FunctionTransform : public IFunction auto it = table.find(src_ref); - StringRef dst_ref = it != table.end() ? it->getSecond() : (with_default ? dst_default : src_ref); + StringRef dst_ref = it ? *it : (with_default ? dst_default : src_ref); dst_data.resize(current_dst_offset + dst_ref.size); memcpy(&dst_data[current_dst_offset], dst_ref.data, dst_ref.size); current_dst_offset += dst_ref.size; @@ -696,8 +696,8 @@ class FunctionTransform : public IFunction auto it = table.find(src_ref); StringRef dst_ref; - if (it != table.end()) - dst_ref = it->getSecond(); + if (it) + dst_ref = *it; else { dst_ref.data = reinterpret_cast(&dst_default_data[current_dst_default_offset]); diff --git a/dbms/src/Interpreters/tests/hash_map_lookup.cpp b/dbms/src/Interpreters/tests/hash_map_lookup.cpp index 62dbd690d936..6310f3a254c4 100644 --- a/dbms/src/Interpreters/tests/hash_map_lookup.cpp +++ b/dbms/src/Interpreters/tests/hash_map_lookup.cpp @@ -63,7 +63,7 @@ void NO_INLINE bench(const std::vector & data, const char * name) for (size_t i = 0, size = data.size(); i < size; ++i) { auto it = map.find(data[i]); - ++it->getSecond(); + ++*it; } watch.stop(); std::cerr << std::fixed << std::setprecision(2) << "HashMap (" << name << "). Size: " << map.size() diff --git a/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index 9730ae3f3cca..99db3b317196 100644 --- a/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -64,9 +64,9 @@ inline size_t JSONEachRowRowInputFormat::columnIndex(const StringRef & name, siz if (prev_positions.size() > key_index && prev_positions[key_index] != name_map.end() - && name == prev_positions[key_index]->getFirst()) + && name == prev_positions[key_index]->first) { - return prev_positions[key_index]->getSecond(); + return prev_positions[key_index]->second; } else { @@ -77,7 +77,7 @@ inline size_t JSONEachRowRowInputFormat::columnIndex(const StringRef & name, siz if (key_index < prev_positions.size()) prev_positions[key_index] = it; - return it->getSecond(); + return it->second; } else return UNKNOWN_FIELD; diff --git a/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h b/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h index 17711b5f27d0..0ecf5d73d32b 100644 --- a/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h +++ b/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h @@ -58,7 +58,7 @@ class JSONEachRowRowInputFormat : public IRowInputFormat std::vector read_columns; /// Hash table match `field name -> position in the block`. NOTE You can use perfect hash map. - using NameMap = HashMap; + using NameMap = std::unordered_map; NameMap name_map; /// Cached search results for previous row (keyed as index in JSON object) - used as a hint. diff --git a/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp index 35a0b4b7a7c7..b0181d2807d8 100644 --- a/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp @@ -118,7 +118,7 @@ bool TSKVRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ex /// and quickly checking for the next expected field, instead of searching the hash table. auto it = name_map.find(name_ref); - if (name_map.end() == it) + if (!it) { if (!format_settings.skip_unknown_fields) throw Exception("Unknown field found while parsing TSKV format: " + name_ref.toString(), ErrorCodes::INCORRECT_DATA); @@ -129,7 +129,7 @@ bool TSKVRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ex } else { - index = it->getSecond(); + index = *it; if (read_columns[index]) throw Exception("Duplicate field found while parsing TSKV format: " + name_ref.toString(), ErrorCodes::INCORRECT_DATA);