OmniSciDB
a5dc49c757
|
#include <PerfectJoinHashTable.h>
Classes | |
struct | AlternativeCacheKeyForPerfectHashJoin |
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 () |
BucketizedHashEntryInfo | getHashEntryInfo () const |
size_t | getNormalizedHashEntryCount () const |
virtual | ~PerfectJoinHashTable () |
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 < PerfectJoinHashTable > | 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_hints, const TableIdToNodeMap &table_id_to_node_map) |
Make hash table from an in-flight SQL query's parse tree etc. More... | |
static HashtableRecycler * | getHashTableCache () |
static HashingSchemeRecycler * | getHashingSchemeCache () |
static void | invalidateCache () |
static void | markCachedItemAsDirty (size_t table_key) |
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) |
Private Member Functions | |
bool | isOneToOneHashPossible (const std::vector< ColumnsForDevice > &columns_per_device) const |
ColumnsForDevice | fetchColumnsForDevice (const std::vector< Fragmenter_Namespace::FragmentInfo > &fragments, const int device_id, DeviceAllocator *dev_buff_owner) |
void | reifyForDevice (const ChunkKey &hash_table_key, const ColumnsForDevice &columns_for_device, const HashType layout, const int device_id, const logger::ThreadLocalIds) |
int | initHashTableForDevice (const ChunkKey &chunk_key, const JoinColumn &join_column, const InnerOuter &cols, const HashType layout, const Data_Namespace::MemoryLevel effective_memory_level, const int device_id) |
Data_Namespace::MemoryLevel | getEffectiveMemoryLevel (const std::vector< InnerOuter > &inner_outer_pairs) const |
PerfectJoinHashTable (const std::shared_ptr< Analyzer::BinOper > qual_bin_oper, const Analyzer::ColumnVar *col_var, const std::vector< InputTableInfo > &query_infos, const Data_Namespace::MemoryLevel memory_level, const JoinType join_type, const HashType preferred_hash_type, const ExpressionRange &col_range, const ExpressionRange &rhs_source_col_range, const BucketizedHashEntryInfo hash_entry_info, ColumnCacheMap &column_cache, Executor *executor, const int device_count, const RegisteredQueryHint &query_hints, const HashTableBuildDagMap &hashtable_build_dag_map, const TableIdToNodeMap &table_id_to_node_map, const size_t rowid_size, const InnerOuterStringOpInfos &inner_outer_string_op_infos={}) | |
ChunkKey | genChunkKey (const std::vector< Fragmenter_Namespace::FragmentInfo > &fragments, const Analyzer::Expr *outer_col, const Analyzer::ColumnVar *inner_col) const |
void | reify () |
std::shared_ptr< PerfectHashTable > | initHashTableOnCpuFromCache (QueryPlanHash key, CacheItemType item_type, DeviceIdentifier device_identifier) |
void | putHashTableOnCpuToCache (QueryPlanHash key, CacheItemType item_type, std::shared_ptr< PerfectHashTable > hashtable_ptr, DeviceIdentifier device_identifier, size_t hashtable_building_time) |
const InputTableInfo & | getInnerQueryInfo (const Analyzer::ColumnVar *inner_col) const |
size_t | shardCount () const |
llvm::Value * | codegenHashTableLoad (const size_t table_idx) |
std::vector< llvm::Value * > | getHashJoinArgs (llvm::Value *hash_ptr, llvm::Value *key_lvs, const Analyzer::Expr *key_col, const int shard_count, const CompilationOptions &co) |
bool | isBitwiseEq () const override |
size_t | getComponentBufferSize () const noexceptoverride |
HashTable * | getHashTableForDevice (const size_t device_id) const |
void | copyCpuHashTableToGpu (std::shared_ptr< PerfectHashTable > &cpu_hash_table, const PerfectHashTableEntryInfo hash_table_entry_info, const int device_id, Data_Namespace::DataMgr *data_mgr) |
Static Private Member Functions | |
static QueryPlanHash | getAlternativeCacheKey (AlternativeCacheKeyForPerfectHashJoin &info) |
Static Private 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 |
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) |
Protected Attributes inherited from HashJoin | |
std::vector< std::shared_ptr < HashTable > > | hash_tables_for_device_ |
Definition at line 50 of file PerfectJoinHashTable.h.
|
inlinevirtual |
Definition at line 145 of file PerfectJoinHashTable.h.
|
inlineprivate |
Definition at line 175 of file PerfectJoinHashTable.h.
Referenced by getInstance().
|
private |
Definition at line 989 of file PerfectJoinHashTable.cpp.
References AUTOMATIC_IR_METADATA, CHECK, HashJoin::codegenHashTableLoad(), executor_, and get_arg_by_name().
Referenced by codegenMatchingSet(), and codegenSlot().
|
overridevirtual |
Implements HashJoin.
Definition at line 1055 of file PerfectJoinHashTable.cpp.
References AUTOMATIC_IR_METADATA, CHECK, HashJoin::codegenColOrStringOper(), codegenHashTableLoad(), HashJoin::codegenMatchingSet(), executor_, anonymous_namespace{HashJoin.cpp}::get_cols(), get_max_rte_scan_table(), getComponentBufferSize(), getHashJoinArgs(), inner_outer_string_op_infos_, isBitwiseEq(), kDATE, qual_bin_oper_, self_join_not_covered_by_left_deep_tree(), and shardCount().
|
overridevirtual |
Implements HashJoin.
Definition at line 1234 of file PerfectJoinHashTable.cpp.
References AUTOMATIC_IR_METADATA, CHECK, HashJoin::codegenColOrStringOper(), codegenHashTableLoad(), executor_, anonymous_namespace{HashJoin.cpp}::get_cols(), get_max_rte_scan_table(), Analyzer::Expr::get_type_info(), getHashJoinArgs(), getHashType(), isBitwiseEq(), kDATE, OneToOne, qual_bin_oper_, self_join_not_covered_by_left_deep_tree(), and shardCount().
|
private |
Definition at line 1129 of file PerfectJoinHashTable.cpp.
References PerfectJoinHashTableBuilder::allocateDeviceMemory(), CHECK, CHECK_EQ, CHECK_LT, CPU, cpu_hash_table_buff_mutex_, device_count_, executor_, PerfectJoinHashTableBuilder::getHashTable(), getQueryEngineCudaStreamForDevice(), Data_Namespace::GPU_LEVEL, hash_entry_info_, HashJoin::hash_tables_for_device_, memory_level_, and shardCount().
Referenced by initHashTableForDevice(), and reify().
|
overridevirtualnoexcept |
Implements HashJoin.
Definition at line 1104 of file PerfectJoinHashTable.cpp.
References getComponentBufferSize().
Referenced by toSet(), and toString().
|
private |
Definition at line 720 of file PerfectJoinHashTable.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 reify().
|
private |
Definition at line 927 of file PerfectJoinHashTable.cpp.
References gpu_enabled::accumulate(), CHECK, CHECK_EQ, shared::ColumnKey::db_id, Analyzer::Expr::get_type_info(), Analyzer::ColumnVar::getColumnKey(), getInnerQueryInfo(), InputTableInfo::info, and kENCODING_DICT.
Referenced by reify().
|
inlinestaticprivate |
Definition at line 264 of file PerfectJoinHashTable.h.
References PerfectJoinHashTable::AlternativeCacheKeyForPerfectHashJoin::chunk_key, PerfectJoinHashTable::AlternativeCacheKeyForPerfectHashJoin::col_range, Analyzer::Expr::get_type_info(), hash_value(), PerfectJoinHashTable::AlternativeCacheKeyForPerfectHashJoin::inner_col, PerfectJoinHashTable::AlternativeCacheKeyForPerfectHashJoin::inner_outer_string_op_infos, SQLTypeInfo::is_string(), PerfectJoinHashTable::AlternativeCacheKeyForPerfectHashJoin::join_type, PerfectJoinHashTable::AlternativeCacheKeyForPerfectHashJoin::num_elements, PerfectJoinHashTable::AlternativeCacheKeyForPerfectHashJoin::optype, PerfectJoinHashTable::AlternativeCacheKeyForPerfectHashJoin::outer_col, toString(), ExpressionRange::toString(), and Analyzer::ColumnVar::toString().
Referenced by reify().
|
overrideprivatevirtualnoexcept |
Implements HashJoin.
Definition at line 1112 of file PerfectJoinHashTable.cpp.
References HashJoin::hash_tables_for_device_, and OneToMany.
Referenced by codegenMatchingSet(), countBufferOff(), and payloadBufferOff().
|
inlineoverridevirtualnoexcept |
Implements HashJoin.
Definition at line 92 of file PerfectJoinHashTable.h.
References device_count_.
|
private |
Definition at line 710 of file PerfectJoinHashTable.cpp.
References Data_Namespace::CPU_LEVEL, executor_, inner_outer_string_op_infos_, memory_level_, needs_dict_translation_, and needs_dictionary_translation().
Referenced by reify().
|
inline |
Definition at line 139 of file PerfectJoinHashTable.h.
References hash_entry_info_.
|
inlinestatic |
Definition at line 106 of file PerfectJoinHashTable.h.
References CHECK, and hash_table_layout_cache_.
|
private |
Definition at line 1001 of file PerfectJoinHashTable.cpp.
References AUTOMATIC_IR_METADATA, BucketizedHashEntryInfo::bucket_normalization, CHECK, col_range_, device_count_, executor_, anonymous_namespace{PerfectJoinHashTable.cpp}::get_hash_entry_count(), get_logical_type_info(), SQLTypeInfo::get_type(), Analyzer::Expr::get_type_info(), ExpressionRange::getIntMax(), ExpressionRange::getIntMin(), hash_entry_info_, inline_fixed_encoding_null_val(), isBitwiseEq(), and kDATE.
Referenced by codegenMatchingSet(), and codegenSlot().
|
inlinefinalvirtual |
Implements HashJoin.
Definition at line 100 of file PerfectJoinHashTable.h.
|
inlinestatic |
Definition at line 102 of file PerfectJoinHashTable.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().
|
private |
Definition at line 1124 of file PerfectJoinHashTable.cpp.
References CHECK_LT, and HashJoin::hash_tables_for_device_.
Referenced by toSet(), and toString().
|
inlineoverridevirtualnoexcept |
Implements HashJoin.
Definition at line 86 of file PerfectJoinHashTable.h.
References hash_type_.
Referenced by codegenSlot().
|
private |
Definition at line 1290 of file PerfectJoinHashTable.cpp.
References get_inner_query_info(), Analyzer::ColumnVar::getTableKey(), and query_infos_.
Referenced by genChunkKey(), and reify().
|
inlineoverridevirtualnoexcept |
Implements HashJoin.
Definition at line 78 of file PerfectJoinHashTable.h.
References col_var_.
Referenced by reify().
|
inlineoverridevirtualnoexcept |
Implements HashJoin.
Definition at line 82 of file PerfectJoinHashTable.h.
References col_var_.
|
static |
Make hash table from an in-flight SQL query's parse tree etc.
Definition at line 168 of file PerfectJoinHashTable.cpp.
References CHECK, CHECK_EQ, RegisteredQueryHint::force_one_to_many_hash_join, g_is_test_env, g_num_tuple_threshold_switch_to_baseline, g_ratio_num_hash_entry_to_num_tuple_switch_to_baseline, HashJoin::generateTooManyHashEntriesErrMsg(), anonymous_namespace{PerfectJoinHashTable.cpp}::get_bucketized_hash_entry_info(), anonymous_namespace{HashJoin.cpp}::get_cols(), get_inner_query_info(), getExpressionRange(), HashJoin::getHashTypeString(), HashJoin::getMaximumNumHashEntriesCanHold(), BucketizedHashEntryInfo::getNormalizedHashEntryCount(), logger::INFO, Invalid, IS_EQUIVALENCE, kBW_EQ, LOG, ExpressionRange::makeIntRange(), OneToMany, PerfectJoinHashTable(), VLOG, and VLOGGING.
Referenced by HashJoin::getInstance().
|
inlineoverridevirtualnoexcept |
Implements HashJoin.
Definition at line 88 of file PerfectJoinHashTable.h.
References memory_level_.
|
inline |
Definition at line 141 of file PerfectJoinHashTable.h.
References BucketizedHashEntryInfo::getNormalizedHashEntryCount(), and hash_entry_info_.
Referenced by initHashTableForDevice(), and isOneToOneHashPossible().
|
inline |
Definition at line 137 of file PerfectJoinHashTable.h.
References query_hints_.
|
private |
Definition at line 796 of file PerfectJoinHashTable.cpp.
References PerfectJoinHashTableBuilder::allocateDeviceMemory(), CHECK, CHECK_EQ, CHECK_LT, col_range_, copyCpuHashTableToGpu(), CPU, DataRecyclerUtil::CPU_DEVICE_IDENTIFIER, cpu_hash_table_buff_mutex_, Data_Namespace::CPU_LEVEL, DEBUG_TIMER, device_count_, executor_, PerfectJoinHashTableBuilder::getHashTable(), getNormalizedHashEntryCount(), Data_Namespace::GPU_LEVEL, hash_entry_info_, HashJoin::hash_tables_for_device_, hashtable_cache_key_, PerfectJoinHashTableBuilder::initOneToManyHashTableOnCpu(), PerfectJoinHashTableBuilder::initOneToOneHashTableOnCpu(), inner_outer_string_op_infos_, isBitwiseEq(), RegisteredQueryHint::isHintRegistered(), HashtableRecycler::isSafeToCacheHashtable(), join_type_, kMaxJoinHashTableSize, RegisteredQueryHint::max_join_hash_table_size, memory_level_, needs_dict_translation_, JoinColumn::num_elems, OneToOne, PERFECT_HT, putHashTableOnCpuToCache(), query_hints_, rowid_size_, shardCount(), str_proxy_translation_map_, table_id_to_node_map_, UNREACHABLE, and WINDOW_FUNCTION_FRAMING.
Referenced by reifyForDevice().
|
private |
Definition at line 957 of file PerfectJoinHashTable.cpp.
References CHECK, DEBUG_TIMER, hash_table_cache_, and VLOG.
Referenced by reify().
|
inlinestatic |
Definition at line 111 of file PerfectJoinHashTable.h.
References CHECK, hash_table_cache_, and hash_table_layout_cache_.
|
overrideprivatevirtual |
Implements HashJoin.
Definition at line 1316 of file PerfectJoinHashTable.cpp.
References kBW_EQ, and qual_bin_oper_.
Referenced by codegenMatchingSet(), codegenSlot(), fetchColumnsForDevice(), getHashJoinArgs(), initHashTableForDevice(), and reify().
|
private |
Definition at line 376 of file PerfectJoinHashTable.cpp.
References CHECK, anonymous_namespace{PerfectJoinHashTable.cpp}::get_bucketized_hash_entry_info(), getNormalizedHashEntryCount(), ExpressionRange::hasNulls(), inner_outer_pairs_, kBW_EQ, qual_bin_oper_, rhs_source_col_range_, and VLOG.
Referenced by reify().
|
inlinestatic |
Definition at line 119 of file PerfectJoinHashTable.h.
References CHECK, DataRecyclerUtil::CPU_DEVICE_IDENTIFIER, hash_table_cache_, hash_table_layout_cache_, HT_HASHING_SCHEME, and PERFECT_HT.
|
overridevirtualnoexcept |
Implements HashJoin.
Definition at line 1100 of file PerfectJoinHashTable.cpp.
Referenced by toSet(), and toString().
|
overridevirtualnoexcept |
Implements HashJoin.
Definition at line 1108 of file PerfectJoinHashTable.cpp.
References getComponentBufferSize().
Referenced by toSet(), and toString().
|
private |
Definition at line 972 of file PerfectJoinHashTable.cpp.
References CHECK, CPU, and hash_table_cache_.
Referenced by initHashTableForDevice().
|
private |
Definition at line 399 of file PerfectJoinHashTable.cpp.
References gpu_enabled::accumulate(), anonymous_namespace{QueryMemoryDescriptor.cpp}::any_of(), threading_serial::async(), HashJoin::canAccessHashTable(), CHECK, CHECK_EQ, CHECK_GE, CHECK_LT, HashJoin::checkHashJoinReplicationConstraint(), col_range_, copyCpuHashTableToGpu(), DataRecyclerUtil::CPU_DEVICE_IDENTIFIER, Data_Namespace::CPU_LEVEL, DEBUG_TIMER, device_count_, executor_, fetchColumnsForDevice(), HashJoin::freeHashBufferMemory(), genChunkKey(), anonymous_namespace{PerfectJoinHashTable.cpp}::get_bucketized_hash_entry_info(), anonymous_namespace{HashJoin.cpp}::get_cols(), get_shard_count(), getAlternativeCacheKey(), DataRecyclerUtil::getAlternativeTableKeys(), getEffectiveMemoryLevel(), HashtableRecycler::getHashtableAccessPathInfo(), getInnerQueryInfo(), getInnerTableId(), getQueryEngineCudaStreamForDevice(), Data_Namespace::GPU_LEVEL, hash_entry_info_, hash_table_cache_, hash_table_layout_cache_, HashJoin::hash_tables_for_device_, hash_type_, hashtable_build_dag_map_, hashtable_cache_key_, hashtable_cache_meta_info_, HT_HASHING_SCHEME, InputTableInfo::info, initHashTableOnCpuFromCache(), inner_outer_pairs_, inner_outer_string_op_infos_, isBitwiseEq(), HashtableRecycler::isInvalidHashTableCacheKey(), isOneToOneHashPossible(), HashtableRecycler::isSafeToCacheHashtable(), join_type_, HashJoin::MAX_NUM_HASH_ENTRIES, memory_level_, needs_dict_translation_, needs_dictionary_translation(), OneToMany, OneToOne, only_shards_for_device(), PERFECT_HT, qual_bin_oper_, reifyForDevice(), shardCount(), str_proxy_translation_map_, str_proxy_translation_mutex_, shared::TableKey::table_id, table_id_to_node_map_, table_keys_, logger::thread_local_ids(), HashJoin::translateInnerToOuterStrDictProxies(), UNREACHABLE, and VLOG.
|
private |
Definition at line 758 of file PerfectJoinHashTable.cpp.
References CHECK_EQ, DEBUG_TIMER_NEW_THREAD, get_effective_memory_level(), initHashTableForDevice(), inner_outer_pairs_, ColumnsForDevice::join_columns, memory_level_, needs_dict_translation_, OneToMany, OneToOne, logger::ThreadLocalIds::setNewThreadId(), logger::ThreadLocalIds::thread_id_, and to_string().
Referenced by reify().
|
private |
Definition at line 1310 of file PerfectJoinHashTable.cpp.
References executor_, get_shard_count(), Data_Namespace::GPU_LEVEL, memory_level_, and qual_bin_oper_.
Referenced by codegenMatchingSet(), codegenSlot(), copyCpuHashTableToGpu(), initHashTableForDevice(), and reify().
|
overridevirtual |
Implements HashJoin.
Definition at line 1201 of file PerfectJoinHashTable.cpp.
References countBufferOff(), executor_, getHashTableForDevice(), HashJoin::getJoinHashBuffer(), HashJoin::getJoinHashBufferSize(), getQueryEngineCudaStreamForDevice(), GPU, offsetBufferOff(), payloadBufferOff(), and HashTable::toSet().
|
overridevirtual |
Implements HashJoin.
Definition at line 1162 of file PerfectJoinHashTable.cpp.
References countBufferOff(), executor_, getHashTableForDevice(), HashJoin::getHashTypeString(), HashJoin::getJoinHashBuffer(), HashJoin::getJoinHashBufferSize(), getQueryEngineCudaStreamForDevice(), GPU, hash_type_, offsetBufferOff(), payloadBufferOff(), and HashTable::toString().
Referenced by getAlternativeCacheKey().
|
private |
Definition at line 288 of file PerfectJoinHashTable.h.
Referenced by getHashJoinArgs(), initHashTableForDevice(), and reify().
|
private |
Definition at line 281 of file PerfectJoinHashTable.h.
Referenced by getInnerTableId(), and getInnerTableRteIdx().
|
private |
Definition at line 292 of file PerfectJoinHashTable.h.
Referenced by fetchColumnsForDevice().
|
private |
Definition at line 285 of file PerfectJoinHashTable.h.
Referenced by copyCpuHashTableToGpu(), and initHashTableForDevice().
|
private |
Definition at line 293 of file PerfectJoinHashTable.h.
Referenced by copyCpuHashTableToGpu(), getDeviceCount(), getHashJoinArgs(), initHashTableForDevice(), and reify().
|
private |
Definition at line 291 of file PerfectJoinHashTable.h.
Referenced by codegenHashTableLoad(), codegenMatchingSet(), codegenSlot(), copyCpuHashTableToGpu(), fetchColumnsForDevice(), getEffectiveMemoryLevel(), getHashJoinArgs(), initHashTableForDevice(), reify(), shardCount(), toSet(), and toString().
|
mutableprivate |
Definition at line 290 of file PerfectJoinHashTable.h.
Referenced by copyCpuHashTableToGpu(), getHashEntryInfo(), getHashJoinArgs(), getNormalizedHashEntryCount(), initHashTableForDevice(), and reify().
|
staticprivate |
Definition at line 305 of file PerfectJoinHashTable.h.
Referenced by getHashTableCache(), initHashTableOnCpuFromCache(), invalidateCache(), markCachedItemAsDirty(), putHashTableOnCpuToCache(), and reify().
|
staticprivate |
Definition at line 306 of file PerfectJoinHashTable.h.
Referenced by getHashingSchemeCache(), invalidateCache(), markCachedItemAsDirty(), and reify().
|
private |
Definition at line 284 of file PerfectJoinHashTable.h.
Referenced by getHashType(), reify(), and toString().
|
private |
Definition at line 296 of file PerfectJoinHashTable.h.
Referenced by reify().
|
private |
Definition at line 298 of file PerfectJoinHashTable.h.
Referenced by initHashTableForDevice(), and reify().
|
private |
Definition at line 299 of file PerfectJoinHashTable.h.
Referenced by reify().
|
private |
Definition at line 173 of file PerfectJoinHashTable.h.
Referenced by fetchColumnsForDevice(), isOneToOneHashPossible(), reify(), and reifyForDevice().
|
private |
Definition at line 302 of file PerfectJoinHashTable.h.
Referenced by codegenMatchingSet(), getEffectiveMemoryLevel(), initHashTableForDevice(), and reify().
|
private |
Definition at line 280 of file PerfectJoinHashTable.h.
Referenced by initHashTableForDevice(), and reify().
|
private |
Definition at line 283 of file PerfectJoinHashTable.h.
Referenced by copyCpuHashTableToGpu(), fetchColumnsForDevice(), getEffectiveMemoryLevel(), getMemoryLevel(), initHashTableForDevice(), reify(), reifyForDevice(), and shardCount().
|
mutableprivate |
Definition at line 295 of file PerfectJoinHashTable.h.
Referenced by fetchColumnsForDevice(), getEffectiveMemoryLevel(), initHashTableForDevice(), reify(), and reifyForDevice().
|
private |
Definition at line 279 of file PerfectJoinHashTable.h.
Referenced by codegenMatchingSet(), codegenSlot(), isBitwiseEq(), isOneToOneHashPossible(), reify(), and shardCount().
|
private |
Definition at line 294 of file PerfectJoinHashTable.h.
Referenced by getRegisteredQueryHint(), and initHashTableForDevice().
|
private |
Definition at line 282 of file PerfectJoinHashTable.h.
Referenced by getInnerQueryInfo().
|
private |
Definition at line 289 of file PerfectJoinHashTable.h.
Referenced by isOneToOneHashPossible().
|
private |
Definition at line 303 of file PerfectJoinHashTable.h.
Referenced by initHashTableForDevice().
|
private |
Definition at line 287 of file PerfectJoinHashTable.h.
Referenced by initHashTableForDevice(), and reify().
|
private |
Definition at line 286 of file PerfectJoinHashTable.h.
Referenced by reify().
|
private |
Definition at line 301 of file PerfectJoinHashTable.h.
Referenced by initHashTableForDevice(), and reify().
|
private |
Definition at line 300 of file PerfectJoinHashTable.h.
Referenced by reify().