OmniSciDB
a5dc49c757
|
#include <BaselineJoinHashTable.h>
Classes | |
struct | AlternativeCacheKeyForBaselineHashJoin |
Public Member Functions | |
std::string | toString (const ExecutorDeviceType device_type, const int device_id=0, bool raw=false) const override |
std::set < DecodedJoinHashBufferEntry > | toSet (const ExecutorDeviceType device_type, const int device_id) const override |
llvm::Value * | codegenSlot (const CompilationOptions &, const size_t) override |
HashJoinMatchingSet | codegenMatchingSet (const CompilationOptions &, const size_t) override |
shared::TableKey | getInnerTableId () const noexceptoverride |
int | getInnerTableRteIdx () const noexceptoverride |
HashType | getHashType () const noexceptoverride |
Data_Namespace::MemoryLevel | getMemoryLevel () const noexceptoverride |
int | getDeviceCount () const noexceptoverride |
size_t | offsetBufferOff () const noexceptoverride |
size_t | countBufferOff () const noexceptoverride |
size_t | payloadBufferOff () const noexceptoverride |
std::string | getHashJoinType () const final |
const RegisteredQueryHint & | getRegisteredQueryHint () |
virtual | ~BaselineJoinHashTable () |
Public Member Functions inherited from HashJoin | |
virtual std::string | toStringFlat64 (const ExecutorDeviceType device_type, const int device_id) const |
virtual std::string | toStringFlat32 (const ExecutorDeviceType device_type, const int device_id) const |
JoinColumn | fetchJoinColumn (const Analyzer::ColumnVar *hash_col, const std::vector< Fragmenter_Namespace::FragmentInfo > &fragment_info, const Data_Namespace::MemoryLevel effective_memory_level, const int device_id, std::vector< std::shared_ptr< Chunk_NS::Chunk >> &chunks_owner, DeviceAllocator *dev_buff_owner, std::vector< std::shared_ptr< void >> &malloc_owner, Executor *executor, ColumnCacheMap *column_cache) |
HashTable * | getHashTableForDevice (const size_t device_id) const |
size_t | getJoinHashBufferSize (const ExecutorDeviceType device_type) |
size_t | getJoinHashBufferSize (const ExecutorDeviceType device_type, const int device_id) const |
int8_t * | getJoinHashBuffer (const ExecutorDeviceType device_type, const int device_id) const |
void | freeHashBufferMemory () |
Static Public Member Functions | |
static std::shared_ptr < BaselineJoinHashTable > | getInstance (const std::shared_ptr< Analyzer::BinOper > condition, const std::vector< InputTableInfo > &query_infos, const Data_Namespace::MemoryLevel memory_level, const JoinType join_type, const HashType preferred_hash_type, const int device_count, ColumnCacheMap &column_cache, Executor *executor, const HashTableBuildDagMap &hashtable_build_dag_map, const RegisteredQueryHint &query_hints, const TableIdToNodeMap &table_id_to_node_map) |
Make hash table from an in-flight SQL query's parse tree etc. More... | |
static size_t | getShardCountForCondition (const Analyzer::BinOper *condition, const Executor *executor, const std::vector< InnerOuter > &inner_outer_pairs) |
static void | invalidateCache () |
static void | markCachedItemAsDirty (size_t table_key) |
static HashtableRecycler * | getHashTableCache () |
static HashingSchemeRecycler * | getHashingSchemeCache () |
Static Public Member Functions inherited from HashJoin | |
static size_t | getMaximumNumHashEntriesCanHold (MemoryLevel memory_level, const Executor *executor, size_t rowid_size) noexcept |
static std::string | generateTooManyHashEntriesErrMsg (size_t num_entries, size_t threshold, MemoryLevel memory_level) |
static bool | layoutRequiresAdditionalBuffers (HashType layout) noexcept |
static std::string | getHashTypeString (HashType ht) noexcept |
static HashJoinMatchingSet | codegenMatchingSet (const std::vector< llvm::Value * > &hash_join_idx_args_in, const bool is_sharded, const bool col_is_nullable, const bool is_bw_eq, const int64_t sub_buff_size, Executor *executor, const bool is_bucketized=false) |
static llvm::Value * | codegenHashTableLoad (const size_t table_idx, Executor *executor) |
static std::shared_ptr< HashJoin > | getInstance (const std::shared_ptr< Analyzer::BinOper > qual_bin_oper, const std::vector< InputTableInfo > &query_infos, const Data_Namespace::MemoryLevel memory_level, const JoinType join_type, const HashType preferred_hash_type, const int device_count, ColumnCacheMap &column_cache, Executor *executor, const HashTableBuildDagMap &hashtable_build_dag_map, const RegisteredQueryHint &query_hint, const TableIdToNodeMap &table_id_to_node_map) |
Make hash table from an in-flight SQL query's parse tree etc. More... | |
static std::shared_ptr< HashJoin > | getSyntheticInstance (std::string_view table1, std::string_view column1, const Catalog_Namespace::Catalog &catalog1, std::string_view table2, std::string_view column2, const Catalog_Namespace::Catalog &catalog2, const Data_Namespace::MemoryLevel memory_level, const HashType preferred_hash_type, const int device_count, ColumnCacheMap &column_cache, Executor *executor) |
Make hash table from named tables and columns (such as for testing). More... | |
static std::shared_ptr< HashJoin > | getSyntheticInstance (const std::shared_ptr< Analyzer::BinOper > qual_bin_oper, const Data_Namespace::MemoryLevel memory_level, const HashType preferred_hash_type, const int device_count, ColumnCacheMap &column_cache, Executor *executor) |
Make hash table from named tables and columns (such as for testing). More... | |
static std::pair< std::string, std::shared_ptr< HashJoin > > | getSyntheticInstance (std::vector< std::shared_ptr< Analyzer::BinOper >>, const Data_Namespace::MemoryLevel memory_level, const HashType preferred_hash_type, const int device_count, ColumnCacheMap &column_cache, Executor *executor) |
static shared::TableKey | getInnerTableId (const std::vector< InnerOuter > &inner_outer_pairs) |
static bool | canAccessHashTable (bool allow_hash_table_recycling, bool invalid_cache_key, JoinType join_type) |
static void | checkHashJoinReplicationConstraint (const shared::TableKey &table_key, const size_t shard_count, const Executor *executor) |
static std::pair< InnerOuter, InnerOuterStringOpInfos > | normalizeColumnPair (const Analyzer::Expr *lhs, const Analyzer::Expr *rhs, const TemporaryTables *temporary_tables, const bool is_bbox_intersect=false) |
template<typename T > | |
static const T * | getHashJoinColumn (const Analyzer::Expr *expr) |
static std::pair< std::vector < InnerOuter >, std::vector < InnerOuterStringOpInfos > > | normalizeColumnPairs (const Analyzer::BinOper *condition, const TemporaryTables *temporary_tables) |
static std::vector< int > | collectFragmentIds (const std::vector< Fragmenter_Namespace::FragmentInfo > &fragments) |
static CompositeKeyInfo | getCompositeKeyInfo (const std::vector< InnerOuter > &inner_outer_pairs, const Executor *executor, const std::vector< InnerOuterStringOpInfos > &inner_outer_string_op_infos_pairs={}) |
static std::vector< const StringDictionaryProxy::IdMap * > | translateCompositeStrDictProxies (const CompositeKeyInfo &composite_key_info, const std::vector< InnerOuterStringOpInfos > &string_op_infos_for_keys, const Executor *executor) |
static std::pair< const StringDictionaryProxy *, StringDictionaryProxy * > | getStrDictProxies (const InnerOuter &cols, const Executor *executor, const bool has_string_ops) |
static const StringDictionaryProxy::IdMap * | translateInnerToOuterStrDictProxies (const InnerOuter &cols, const InnerOuterStringOpInfos &inner_outer_string_op_infos, ExpressionRange &old_col_range, const Executor *executor) |
Protected Member Functions | |
BaselineJoinHashTable (const std::shared_ptr< Analyzer::BinOper > condition, const JoinType join_type, const std::vector< InputTableInfo > &query_infos, const Data_Namespace::MemoryLevel memory_level, ColumnCacheMap &column_cache, Executor *executor, const std::vector< InnerOuter > &inner_outer_pairs, const std::vector< InnerOuterStringOpInfos > &col_pairs_string_op_infos, const int device_count, const RegisteredQueryHint &query_hints, const HashTableBuildDagMap &hashtable_build_dag_map, const TableIdToNodeMap &table_id_to_node_map) | |
size_t | getComponentBufferSize () const noexceptoverride |
size_t | getKeyBufferSize () const noexcept |
virtual void | reifyWithLayout (const HashType layout) |
virtual ColumnsForDevice | fetchColumnsForDevice (const std::vector< Fragmenter_Namespace::FragmentInfo > &fragments, const int device_id, DeviceAllocator *dev_buff_owner) |
virtual std::pair< size_t, size_t > | approximateTupleCount (const std::vector< ColumnsForDevice > &) const |
virtual size_t | getKeyComponentWidth () const |
virtual size_t | getKeyComponentCount () const |
virtual llvm::Value * | codegenKey (const CompilationOptions &) |
size_t | shardCount () const |
Data_Namespace::MemoryLevel | getEffectiveMemoryLevel (const std::vector< InnerOuter > &inner_outer_pairs) const |
void | reify (const HashType preferred_layout) |
void | copyCpuHashTableToGpu (std::shared_ptr< BaselineHashTable > &cpu_hash_table, const int device_id, Data_Namespace::DataMgr *data_mgr) |
virtual void | reifyForDevice (const ColumnsForDevice &columns_for_device, const HashType layout, const int device_id, const BaselineHashTableEntryInfo hash_table_entry_info, const logger::ThreadLocalIds parent_thread_local_ids) |
virtual int | initHashTableForDevice (const std::vector< JoinColumn > &join_columns, const std::vector< JoinColumnTypeInfo > &join_column_types, const std::vector< JoinBucketInfo > &join_buckets, const HashType layout, const Data_Namespace::MemoryLevel effective_memory_level, const BaselineHashTableEntryInfo hash_table_entry_info, const int device_id) |
llvm::Value * | hashPtr (const size_t index) |
std::shared_ptr< HashTable > | initHashTableOnCpuFromCache (QueryPlanHash key, CacheItemType item_type, DeviceIdentifier device_identifier) |
void | putHashTableOnCpuToCache (QueryPlanHash key, CacheItemType item_type, std::shared_ptr< HashTable > hashtable_ptr, DeviceIdentifier device_identifier, size_t hashtable_building_time) |
bool | isBitwiseEq () const override |
ChunkKey | genChunkKey (const std::vector< Fragmenter_Namespace::FragmentInfo > &fragments) const |
Static Protected Member Functions | |
static shared::TableKey | getInnerTableId (const std::vector< InnerOuter > &inner_outer_pairs) |
static QueryPlanHash | getAlternativeCacheKey (AlternativeCacheKeyForBaselineHashJoin &info) |
Static Protected Member Functions inherited from HashJoin | |
static llvm::Value * | codegenColOrStringOper (const Analyzer::Expr *col_or_string_oper, const std::vector< StringOps_Namespace::StringOpInfo > &string_op_infos, CodeGenerator &code_generator, const CompilationOptions &co) |
Static Protected Attributes | |
static std::unique_ptr < HashtableRecycler > | hash_table_cache_ |
static std::unique_ptr < HashingSchemeRecycler > | hash_table_layout_cache_ |
Additional Inherited Members | |
Static Public Attributes inherited from HashJoin | |
static constexpr size_t | MAX_NUM_HASH_ENTRIES = size_t(1) << 31 |
Definition at line 49 of file BaselineJoinHashTable.h.
|
inlinevirtual |
Definition at line 139 of file BaselineJoinHashTable.h.
|
protected |
Definition at line 121 of file BaselineJoinHashTable.cpp.
References CHECK_GT, device_count_, and HashJoin::hash_tables_for_device_.
Referenced by getInstance().
|
protectedvirtual |
Definition at line 524 of file BaselineJoinHashTable.cpp.
References approximate_distinct_tuples(), approximate_distinct_tuples_on_device(), threading_serial::async(), Bitmap, CHECK, CHECK_EQ, CPU, Data_Namespace::CPU_LEVEL, cpu_threads(), device_count_, executor_, getEffectiveMemoryLevel(), getQueryEngineCudaStreamForDevice(), GPU, Data_Namespace::GPU_LEVEL, hll_size(), hll_unify(), inner_outer_pairs_, transfer_flat_object_to_gpu(), transfer_vector_of_flat_objects_to_gpu(), and UNREACHABLE.
Referenced by reifyWithLayout().
|
protectedvirtual |
Definition at line 1014 of file BaselineJoinHashTable.cpp.
References AUTOMATIC_IR_METADATA, CHECK, HashJoin::codegenColOrStringOper(), executor_, get_int_type(), get_max_rte_scan_table(), getKeyComponentCount(), getKeyComponentWidth(), inner_outer_pairs_, inner_outer_string_op_infos_pairs_, LL_BUILDER, LL_CONTEXT, LL_INT, and self_join_not_covered_by_left_deep_tree().
Referenced by codegenMatchingSet(), and codegenSlot().
|
overridevirtual |
Implements HashJoin.
Definition at line 932 of file BaselineJoinHashTable.cpp.
References AUTOMATIC_IR_METADATA, CHECK, HashJoin::codegenHashTableLoad(), codegenKey(), HashJoin::codegenMatchingSet(), executor_, get_int_type(), getComponentBufferSize(), HashJoin::getHashTableForDevice(), getHashType(), getKeyComponentCount(), getKeyComponentWidth(), LL_BUILDER, LL_CONTEXT, LL_INT, offsetBufferOff(), OneToMany, and to_string().
|
overridevirtual |
Implements HashJoin.
Definition at line 914 of file BaselineJoinHashTable.cpp.
References AUTOMATIC_IR_METADATA, CHECK, codegenKey(), executor_, get_int_type(), HashJoin::getHashTableForDevice(), getHashType(), getKeyComponentCount(), getKeyComponentWidth(), hashPtr(), LL_BUILDER, LL_CONTEXT, LL_INT, OneToOne, and to_string().
|
protected |
Definition at line 728 of file BaselineJoinHashTable.cpp.
References BaselineJoinHashTableBuilder::allocateDeviceMemory(), CHECK, CHECK_LT, CPU, executor_, BaselineJoinHashTableBuilder::getHashTable(), getQueryEngineCudaStreamForDevice(), HashJoin::hash_tables_for_device_, and query_hints_.
Referenced by initHashTableForDevice(), and reifyWithLayout().
|
overridevirtualnoexcept |
Implements HashJoin.
Definition at line 980 of file BaselineJoinHashTable.cpp.
References getComponentBufferSize(), getHashType(), getKeyBufferSize(), HashJoin::layoutRequiresAdditionalBuffers(), and offsetBufferOff().
Referenced by payloadBufferOff(), toSet(), and toString().
|
protectedvirtual |
Definition at line 628 of file BaselineJoinHashTable.cpp.
References column_cache_, executor_, HashJoin::fetchJoinColumn(), get_column_descriptor_maybe(), get_effective_memory_level(), get_join_column_type_kind(), inline_fixed_encoding_null_val(), inner_outer_pairs_, isBitwiseEq(), memory_level_, and needs_dict_translation_.
Referenced by reifyWithLayout().
|
protected |
Definition at line 1142 of file BaselineJoinHashTable.cpp.
Referenced by reifyWithLayout().
|
inlinestaticprotected |
Definition at line 232 of file BaselineJoinHashTable.h.
References hash_value(), BaselineJoinHashTable::AlternativeCacheKeyForBaselineHashJoin::inner_outer_pairs, BaselineJoinHashTable::AlternativeCacheKeyForBaselineHashJoin::inner_outer_string_op_infos_pairs, BaselineJoinHashTable::AlternativeCacheKeyForBaselineHashJoin::join_type, BaselineJoinHashTable::AlternativeCacheKeyForBaselineHashJoin::num_elements, BaselineJoinHashTable::AlternativeCacheKeyForBaselineHashJoin::optype, and toString().
Referenced by reifyWithLayout().
|
overrideprotectedvirtualnoexcept |
Implements HashJoin.
Definition at line 1009 of file BaselineJoinHashTable.cpp.
References HashJoin::getHashTableForDevice().
Referenced by codegenMatchingSet(), countBufferOff(), and payloadBufferOff().
|
inlineoverridevirtualnoexcept |
Implements HashJoin.
Definition at line 92 of file BaselineJoinHashTable.h.
References device_count_.
|
protected |
Definition at line 718 of file BaselineJoinHashTable.cpp.
References Data_Namespace::CPU_LEVEL, executor_, inner_outer_string_op_infos_pairs_, memory_level_, needs_dict_translation_, and needs_dictionary_translation().
Referenced by approximateTupleCount(), and reifyWithLayout().
|
inlinestatic |
Definition at line 132 of file BaselineJoinHashTable.h.
References CHECK, and hash_table_layout_cache_.
|
inlinefinalvirtual |
Implements HashJoin.
Definition at line 100 of file BaselineJoinHashTable.h.
|
inlinestatic |
Definition at line 128 of file BaselineJoinHashTable.h.
References CHECK, and hash_table_cache_.
Referenced by QueryRunner::QueryRunner::getCachedHashtableWithoutCacheKey(), QueryRunner::QueryRunner::getCacheItemMetric(), QueryRunner::QueryRunner::getNumberOfCachedItem(), anonymous_namespace{DBHandler.cpp}::log_cache_size(), and CommandLineOptions::parse_command_line().
|
overridevirtualnoexcept |
Implements HashJoin.
Definition at line 1094 of file BaselineJoinHashTable.cpp.
References CHECK, HashJoin::getHashTableForDevice(), and layout_override_.
Referenced by codegenMatchingSet(), codegenSlot(), countBufferOff(), payloadBufferOff(), and toString().
|
overridevirtualnoexcept |
Implements HashJoin.
Definition at line 1079 of file BaselineJoinHashTable.cpp.
References CHECK, and inner_outer_pairs_.
Referenced by initHashTableForDevice(), reify(), and reifyWithLayout().
|
staticprotected |
|
overridevirtualnoexcept |
Implements HashJoin.
Definition at line 1088 of file BaselineJoinHashTable.cpp.
References CHECK, and inner_outer_pairs_.
|
static |
Make hash table from an in-flight SQL query's parse tree etc.
Definition at line 41 of file BaselineJoinHashTable.cpp.
References BaselineJoinHashTable(), RegisteredQueryHint::force_one_to_many_hash_join, HashJoin::getHashTypeString(), logger::INFO, LOG, HashJoin::normalizeColumnPairs(), OneToMany, VLOG, and VLOGGING.
Referenced by HashJoin::getInstance().
|
protectednoexcept |
Definition at line 996 of file BaselineJoinHashTable.cpp.
References CHECK, HashJoin::getHashTableForDevice(), getKeyComponentCount(), getKeyComponentWidth(), and HashJoin::layoutRequiresAdditionalBuffers().
Referenced by countBufferOff(), offsetBufferOff(), and payloadBufferOff().
|
protectedvirtual |
Definition at line 714 of file BaselineJoinHashTable.cpp.
References inner_outer_pairs_.
Referenced by codegenKey(), codegenMatchingSet(), codegenSlot(), getKeyBufferSize(), initHashTableForDevice(), reifyWithLayout(), toSet(), and toString().
|
protectedvirtual |
Definition at line 699 of file BaselineJoinHashTable.cpp.
References CHECK_EQ, and inner_outer_pairs_.
Referenced by codegenKey(), codegenMatchingSet(), codegenSlot(), getKeyBufferSize(), reifyWithLayout(), toSet(), and toString().
|
inlineoverridevirtualnoexcept |
Implements HashJoin.
Definition at line 88 of file BaselineJoinHashTable.h.
References memory_level_.
|
inline |
Definition at line 137 of file BaselineJoinHashTable.h.
References query_hints_.
|
static |
Definition at line 152 of file BaselineJoinHashTable.cpp.
References get_shard_count().
Referenced by RangeJoinHashTable::getInstance(), reify(), shardCount(), BoundingBoxIntersectJoinHashTable::shardCount(), and Executor::skipFragmentPair().
|
protected |
Definition at line 1065 of file BaselineJoinHashTable.cpp.
References AUTOMATIC_IR_METADATA, HashJoin::codegenHashTableLoad(), executor_, LL_BUILDER, and LL_CONTEXT.
Referenced by codegenSlot().
|
protectedvirtual |
Definition at line 774 of file BaselineJoinHashTable.cpp.
References BASELINE_HT, CHECK, CHECK_EQ, CHECK_LT, copyCpuHashTableToGpu(), CPU, DataRecyclerUtil::CPU_DEVICE_IDENTIFIER, cpu_hash_table_buff_mutex_, Data_Namespace::CPU_LEVEL, DEBUG_TIMER, decomposeStrDictTranslationMaps(), executor_, HashJoin::getCompositeKeyInfo(), BaselineJoinHashTableBuilder::getHashTable(), getInnerTableId(), getKeyComponentCount(), getQueryEngineCudaStreamForDevice(), Data_Namespace::GPU_LEVEL, hash_table_layout_cache_, HashJoin::hash_tables_for_device_, hashtable_cache_key_, HT_HASHING_SCHEME, BaselineJoinHashTableBuilder::initHashTableOnCpu(), BaselineJoinHashTableBuilder::initHashTableOnGpu(), inner_outer_pairs_, inner_outer_string_op_infos_pairs_, HashtableRecycler::isSafeToCacheHashtable(), join_type_, memory_level_, needs_dict_translation_, putHashTableOnCpuToCache(), query_hints_, str_proxy_translation_maps_, table_id_to_node_map_, transfer_vector_of_flat_objects_to_gpu(), and UNREACHABLE.
Referenced by reifyForDevice().
|
protected |
Definition at line 1111 of file BaselineJoinHashTable.cpp.
References CHECK, DEBUG_TIMER, hash_table_cache_, and VLOG.
Referenced by reifyWithLayout().
|
inlinestatic |
Definition at line 102 of file BaselineJoinHashTable.h.
References CHECK, hash_table_cache_, and hash_table_layout_cache_.
|
overrideprotectedvirtual |
Implements HashJoin.
Definition at line 1138 of file BaselineJoinHashTable.cpp.
References condition_, and kBW_EQ.
Referenced by fetchColumnsForDevice().
|
inlinestatic |
Definition at line 110 of file BaselineJoinHashTable.h.
References BASELINE_HT, CHECK, DataRecyclerUtil::CPU_DEVICE_IDENTIFIER, hash_table_cache_, hash_table_layout_cache_, and HT_HASHING_SCHEME.
|
overridevirtualnoexcept |
Implements HashJoin.
Definition at line 976 of file BaselineJoinHashTable.cpp.
References getKeyBufferSize().
Referenced by codegenMatchingSet(), countBufferOff(), toSet(), and toString().
|
overridevirtualnoexcept |
Implements HashJoin.
Definition at line 988 of file BaselineJoinHashTable.cpp.
References countBufferOff(), getComponentBufferSize(), getHashType(), getKeyBufferSize(), and HashJoin::layoutRequiresAdditionalBuffers().
Referenced by toSet(), and toString().
|
protected |
Definition at line 1121 of file BaselineJoinHashTable.cpp.
References CHECK, CPU, and hash_table_cache_.
Referenced by initHashTableForDevice().
|
protected |
Definition at line 259 of file BaselineJoinHashTable.cpp.
References CHECK_EQ, CHECK_LT, HashJoin::checkHashJoinReplicationConstraint(), condition_, DEBUG_TIMER, device_count_, executor_, HashJoin::freeHashBufferMemory(), getInnerTableId(), getShardCountForCondition(), inner_outer_pairs_, inner_outer_string_op_infos_pairs_, ManyToMany, OneToMany, reifyWithLayout(), and VLOG.
|
protectedvirtual |
Definition at line 667 of file BaselineJoinHashTable.cpp.
References DEBUG_TIMER_NEW_THREAD, get_effective_memory_level(), initHashTableForDevice(), ColumnsForDevice::join_buckets, ColumnsForDevice::join_column_types, ColumnsForDevice::join_columns, memory_level_, needs_dict_translation_, logger::ThreadLocalIds::setNewThreadId(), logger::ThreadLocalIds::thread_id_, and to_string().
Referenced by reifyWithLayout().
|
protectedvirtual |
Definition at line 316 of file BaselineJoinHashTable.cpp.
References gpu_enabled::accumulate(), approximateTupleCount(), threading_serial::async(), BASELINE_HT, HashJoin::canAccessHashTable(), CHECK, CHECK_EQ, condition_, copyCpuHashTableToGpu(), DataRecyclerUtil::CPU_DEVICE_IDENTIFIER, Data_Namespace::CPU_LEVEL, device_count_, executor_, fetchColumnsForDevice(), genChunkKey(), get_entries_per_device(), get_inner_query_info(), getAlternativeCacheKey(), DataRecyclerUtil::getAlternativeTableKeys(), HashJoin::getCompositeKeyInfo(), getEffectiveMemoryLevel(), HashtableRecycler::getHashtableAccessPathInfo(), getInnerTableId(), getKeyComponentCount(), getKeyComponentWidth(), Fragmenter_Namespace::TableInfo::getNumTuplesUpperBound(), getQueryEngineCudaStreamForDevice(), Data_Namespace::GPU_LEVEL, hash_table_cache_, HashJoin::hash_tables_for_device_, hashtable_build_dag_map_, hashtable_cache_key_, hashtable_cache_meta_info_, InputTableInfo::info, initHashTableOnCpuFromCache(), inner_outer_pairs_, inner_outer_string_op_infos_pairs_, HashtableRecycler::isInvalidHashTableCacheKey(), HashtableRecycler::isSafeToCacheHashtable(), join_type_, HashJoin::MAX_NUM_HASH_ENTRIES, memory_level_, needs_dict_translation_, HashJoin::normalizeColumnPairs(), OneToMany, only_shards_for_device(), query_infos_, reifyForDevice(), rowid_size_, shardCount(), str_proxy_translation_maps_, str_proxy_translation_mutex_, shared::TableKey::table_id, table_id_to_node_map_, table_keys_, logger::thread_local_ids(), HashJoin::translateCompositeStrDictProxies(), UNREACHABLE, and WINDOW_FUNCTION_FRAMING.
Referenced by reify().
|
protected |
Definition at line 691 of file BaselineJoinHashTable.cpp.
References condition_, executor_, getShardCountForCondition(), Data_Namespace::GPU_LEVEL, inner_outer_pairs_, and memory_level_.
Referenced by reifyWithLayout().
|
overridevirtual |
Implements HashJoin.
Definition at line 209 of file BaselineJoinHashTable.cpp.
References CHECK, countBufferOff(), executor_, HashJoin::getHashTableForDevice(), HashJoin::getJoinHashBuffer(), getKeyComponentCount(), getKeyComponentWidth(), getQueryEngineCudaStreamForDevice(), GPU, offsetBufferOff(), OneToOne, payloadBufferOff(), and HashTable::toSet().
|
overridevirtual |
Implements HashJoin.
Definition at line 165 of file BaselineJoinHashTable.cpp.
References CHECK, CHECK_LT, countBufferOff(), executor_, getHashType(), HashJoin::getHashTypeString(), HashJoin::getJoinHashBuffer(), getKeyComponentCount(), getKeyComponentWidth(), getQueryEngineCudaStreamForDevice(), GPU, HashJoin::hash_tables_for_device_, offsetBufferOff(), OneToOne, payloadBufferOff(), and HashTable::toString().
Referenced by getAlternativeCacheKey().
|
protected |
Definition at line 257 of file BaselineJoinHashTable.h.
Referenced by fetchColumnsForDevice().
|
protected |
Definition at line 252 of file BaselineJoinHashTable.h.
Referenced by isBitwiseEq(), reify(), reifyWithLayout(), and shardCount().
|
protected |
Definition at line 258 of file BaselineJoinHashTable.h.
Referenced by initHashTableForDevice().
|
protected |
Definition at line 264 of file BaselineJoinHashTable.h.
Referenced by approximateTupleCount(), BaselineJoinHashTable(), getDeviceCount(), reify(), and reifyWithLayout().
|
protected |
Definition at line 256 of file BaselineJoinHashTable.h.
Referenced by approximateTupleCount(), codegenKey(), codegenMatchingSet(), codegenSlot(), copyCpuHashTableToGpu(), fetchColumnsForDevice(), getEffectiveMemoryLevel(), hashPtr(), initHashTableForDevice(), reify(), reifyWithLayout(), shardCount(), toSet(), and toString().
|
staticprotected |
Definition at line 278 of file BaselineJoinHashTable.h.
Referenced by getHashTableCache(), initHashTableOnCpuFromCache(), invalidateCache(), markCachedItemAsDirty(), putHashTableOnCpuToCache(), and reifyWithLayout().
|
staticprotected |
Definition at line 279 of file BaselineJoinHashTable.h.
Referenced by getHashingSchemeCache(), initHashTableForDevice(), invalidateCache(), and markCachedItemAsDirty().
|
protected |
Definition at line 270 of file BaselineJoinHashTable.h.
Referenced by reifyWithLayout().
|
protected |
Definition at line 272 of file BaselineJoinHashTable.h.
Referenced by initHashTableForDevice(), and reifyWithLayout().
|
protected |
Definition at line 273 of file BaselineJoinHashTable.h.
Referenced by reifyWithLayout().
|
protected |
Definition at line 262 of file BaselineJoinHashTable.h.
Referenced by approximateTupleCount(), codegenKey(), fetchColumnsForDevice(), getInnerTableId(), getInnerTableRteIdx(), getKeyComponentCount(), getKeyComponentWidth(), initHashTableForDevice(), reify(), reifyWithLayout(), and shardCount().
|
protected |
Definition at line 263 of file BaselineJoinHashTable.h.
Referenced by codegenKey(), getEffectiveMemoryLevel(), initHashTableForDevice(), reify(), and reifyWithLayout().
|
protected |
Definition at line 253 of file BaselineJoinHashTable.h.
Referenced by initHashTableForDevice(), and reifyWithLayout().
|
protected |
Definition at line 268 of file BaselineJoinHashTable.h.
Referenced by getHashType().
|
protected |
Definition at line 255 of file BaselineJoinHashTable.h.
Referenced by fetchColumnsForDevice(), getEffectiveMemoryLevel(), getMemoryLevel(), initHashTableForDevice(), reifyForDevice(), reifyWithLayout(), and shardCount().
|
mutableprotected |
Definition at line 266 of file BaselineJoinHashTable.h.
Referenced by fetchColumnsForDevice(), getEffectiveMemoryLevel(), initHashTableForDevice(), reifyForDevice(), and reifyWithLayout().
|
protected |
Definition at line 265 of file BaselineJoinHashTable.h.
Referenced by copyCpuHashTableToGpu(), getRegisteredQueryHint(), and initHashTableForDevice().
|
protected |
Definition at line 254 of file BaselineJoinHashTable.h.
Referenced by reifyWithLayout().
|
protected |
Definition at line 276 of file BaselineJoinHashTable.h.
Referenced by reifyWithLayout().
|
protected |
Definition at line 260 of file BaselineJoinHashTable.h.
Referenced by initHashTableForDevice(), and reifyWithLayout().
|
protected |
Definition at line 259 of file BaselineJoinHashTable.h.
Referenced by reifyWithLayout().
|
protected |
Definition at line 275 of file BaselineJoinHashTable.h.
Referenced by initHashTableForDevice(), and reifyWithLayout().
|
protected |
Definition at line 274 of file BaselineJoinHashTable.h.
Referenced by reifyWithLayout().