OmniSciDB  72c90bc290
 All Classes Namespaces Files Functions Variables Typedefs Enumerations Enumerator Friends Macros Groups Pages
QueryMemoryDescriptor.h
Go to the documentation of this file.
1 /*
2  * Copyright 2022 HEAVY.AI, Inc.
3  *
4  * Licensed under the Apache License, Version 2.0 (the "License");
5  * you may not use this file except in compliance with the License.
6  * You may obtain a copy of the License at
7  *
8  * http://www.apache.org/licenses/LICENSE-2.0
9  *
10  * Unless required by applicable law or agreed to in writing, software
11  * distributed under the License is distributed on an "AS IS" BASIS,
12  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13  * See the License for the specific language governing permissions and
14  * limitations under the License.
15  */
16 
23 #ifndef QUERYENGINE_QUERYMEMORYDESCRIPTOR_H
24 #define QUERYENGINE_QUERYMEMORYDESCRIPTOR_H
25 
26 #include "../CompilationOptions.h"
27 #include "../CountDistinct.h"
28 #include "ColSlotContext.h"
29 #include "Types.h"
30 
31 #include <boost/optional.hpp>
32 #include "Logger/Logger.h"
33 
34 #include <algorithm>
35 #include <cstddef>
36 #include <cstdint>
37 #include <memory>
38 #include <numeric>
39 #include <unordered_map>
40 #include <vector>
41 
42 #include <Shared/SqlTypesLayout.h>
43 #include <Shared/TargetInfo.h>
44 
45 extern bool g_cluster;
46 
47 class Executor;
49 class RenderInfo;
50 class RowSetMemoryOwner;
51 struct InputTableInfo;
52 struct RelAlgExecutionUnit;
53 class TResultSetBufferDescriptor;
55 struct ColRangeInfo;
56 struct KeylessInfo;
57 
58 class StreamingTopNOOM : public std::runtime_error {
59  public:
60  StreamingTopNOOM(const size_t heap_size_bytes)
61  : std::runtime_error("Unable to use streaming top N due to required heap size of " +
62  std::to_string(heap_size_bytes) +
63  " bytes exceeding maximum slab size.") {}
64 };
65 
67  public:
69 
70  // constructor for init call
71  QueryMemoryDescriptor(const Executor* executor,
72  const RelAlgExecutionUnit& ra_exe_unit,
73  const std::vector<InputTableInfo>& query_infos,
74  const bool allow_multifrag,
75  const bool keyless_hash,
76  const bool interleaved_bins_on_gpu,
77  const int32_t idx_target_as_key,
78  const ColRangeInfo& col_range_info,
79  const ColSlotContext& col_slot_context,
80  const std::vector<int8_t>& group_col_widths,
81  const int8_t group_col_compact_width,
82  const std::vector<int64_t>& target_groupby_indices,
83  const size_t entry_count,
84  const CountDistinctDescriptors count_distinct_descriptors,
85  const bool sort_on_gpu_hint,
86  const bool output_columnar,
87  const bool render_output,
88  const bool must_use_baseline_sort,
89  const bool use_streaming_top_n,
90  const bool threads_can_reuse_group_by_buffers);
91 
92  QueryMemoryDescriptor(const Executor* executor,
93  const size_t entry_count,
94  const QueryDescriptionType query_desc_type);
95 
96  QueryMemoryDescriptor(const QueryDescriptionType query_desc_type,
97  const int64_t min_val,
98  const int64_t max_val,
99  const bool has_nulls,
100  const std::vector<int8_t>& group_col_widths);
101 
102  // Serialization
103  QueryMemoryDescriptor(const TResultSetBufferDescriptor& thrift_query_memory_descriptor);
104  static TResultSetBufferDescriptor toThrift(const QueryMemoryDescriptor&);
105 
106  bool operator==(const QueryMemoryDescriptor& other) const;
107 
108  static std::unique_ptr<QueryMemoryDescriptor> init(
109  const Executor* executor,
110  const RelAlgExecutionUnit& ra_exe_unit,
111  const std::vector<InputTableInfo>& query_infos,
112  const ColRangeInfo& col_range_info,
113  const KeylessInfo& keyless_info,
114  const bool allow_multifrag,
115  const ExecutorDeviceType device_type,
116  const int8_t crt_min_byte_width,
117  const bool sort_on_gpu_hint,
118  const size_t shard_count,
119  const size_t max_groups_buffer_entry_count,
120  RenderInfo* render_info,
121  const CountDistinctDescriptors count_distinct_descriptors,
122  const bool must_use_baseline_sort,
123  const bool output_columnar_hint,
124  const bool streaming_top_n_hint,
125  const bool threads_can_reuse_group_by_buffers);
126 
127  std::unique_ptr<QueryExecutionContext> getQueryExecutionContext(
128  const RelAlgExecutionUnit&,
129  const Executor* executor,
130  const ExecutorDeviceType device_type,
131  const ExecutorDispatchMode dispatch_mode,
132  const int device_id,
133  const shared::TableKey& outer_table_key,
134  const int64_t num_rows,
135  const std::vector<std::vector<const int8_t*>>& col_buffers,
136  const std::vector<std::vector<uint64_t>>& frag_offsets,
137  std::shared_ptr<RowSetMemoryOwner>,
138  const bool output_columnar,
139  const bool sort_on_gpu,
140  const size_t thread_idx,
141  RenderInfo*) const;
142 
143  static bool many_entries(const int64_t max_val,
144  const int64_t min_val,
145  const int64_t bucket) {
146  return max_val - min_val > 10000 * std::max(bucket, int64_t(1));
147  }
148 
150  const CountDistinctDescriptors& count_distinct_descriptors) {
151  return std::all_of(count_distinct_descriptors.begin(),
152  count_distinct_descriptors.end(),
153  [](const CountDistinctDescriptor& desc) {
154  return desc.impl_type_ == CountDistinctImplType::Invalid;
155  });
156  }
157 
160  }
161 
162  static int8_t pick_target_compact_width(const RelAlgExecutionUnit& ra_exe_unit,
163  const std::vector<InputTableInfo>& query_infos,
164  const int8_t crt_min_byte_width);
165 
166  // Getters and Setters
167  const Executor* getExecutor() const { return executor_; }
168 
173  getGroupbyColCount() == 1;
174  }
175 
176  bool hasKeylessHash() const { return keyless_hash_; }
177  void setHasKeylessHash(const bool val) { keyless_hash_ = val; }
178 
180  void setHasInterleavedBinsOnGpu(const bool val) { interleaved_bins_on_gpu_ = val; }
181 
182  int32_t getTargetIdxForKey() const { return idx_target_as_key_; }
183  void setTargetIdxForKey(const int32_t val) { idx_target_as_key_ = val; }
184 
185  int8_t groupColWidth(const size_t key_idx) const {
186  CHECK_LT(key_idx, group_col_widths_.size());
187  return group_col_widths_[key_idx];
188  }
189  size_t getPrependedGroupColOffInBytes(const size_t group_idx) const;
190  size_t getPrependedGroupBufferSizeInBytes() const;
191 
192  const auto groupColWidthsBegin() const { return group_col_widths_.begin(); }
193  const auto groupColWidthsEnd() const { return group_col_widths_.end(); }
195 
196  bool isGroupBy() const { return !group_col_widths_.empty(); }
197 
198  void setGroupColCompactWidth(const int8_t val) { group_col_compact_width_ = val; }
199 
200  size_t getColCount() const;
201  size_t getSlotCount() const;
202 
203  const int8_t getPaddedSlotWidthBytes(const size_t slot_idx) const;
204  const int8_t getLogicalSlotWidthBytes(const size_t slot_idx) const;
205 
206  void setPaddedSlotWidthBytes(const size_t slot_idx, const int8_t bytes);
207 
208  const int8_t getSlotIndexForSingleSlotCol(const size_t col_idx) const;
209 
210  size_t getPaddedColWidthForRange(const size_t offset, const size_t range) const {
211  size_t ret = 0;
212  for (size_t i = offset; i < offset + range; i++) {
213  ret += static_cast<size_t>(getPaddedSlotWidthBytes(i));
214  }
215  return ret;
216  }
217 
218  void useConsistentSlotWidthSize(const int8_t slot_width_size);
219  size_t getRowWidth() const;
220 
221  int8_t updateActualMinByteWidth(const int8_t actual_min_byte_width) const;
222 
223  void addColSlotInfo(const std::vector<std::tuple<int8_t, int8_t>>& slots_for_col);
224 
225  // FlatBuffer support:
226  void addColSlotInfoFlatBuffer(const int64_t flatbuffer_size);
227  int64_t getFlatBufferSize(const size_t slot_idx) const {
228  return col_slot_context_.getFlatBufferSize(slot_idx);
229  }
230  bool checkSlotUsesFlatBufferFormat(const size_t slot_idx) const {
232  }
233  int64_t getPaddedSlotBufferSize(const size_t slot_idx) const;
234 
235  void clearSlotInfo();
236 
237  void alignPaddedSlots();
238 
239  int64_t getTargetGroupbyIndex(const size_t target_idx) const {
240  CHECK_LT(target_idx, target_groupby_indices_.size());
241  return target_groupby_indices_[target_idx];
242  }
243 
244  void setAllTargetGroupbyIndices(std::vector<int64_t> group_by_indices) {
245  target_groupby_indices_ = group_by_indices;
246  }
247 
248  size_t targetGroupbyIndicesSize() const { return target_groupby_indices_.size(); }
250  return std::count_if(
251  target_groupby_indices_.begin(),
253  [](const int64_t& target_group_by_index) { return target_group_by_index < 0; });
254  }
256 
257  size_t getEntryCount() const { return entry_count_; }
258  void setEntryCount(const size_t val) { entry_count_ = val; }
259 
260  int64_t getMinVal() const { return min_val_; }
261  int64_t getMaxVal() const { return max_val_; }
262  int64_t getBucket() const { return bucket_; }
263 
264  bool hasNulls() const { return has_nulls_; }
265 
266  const CountDistinctDescriptor& getCountDistinctDescriptor(const size_t idx) const {
268  return count_distinct_descriptors_[idx];
269  }
271  return count_distinct_descriptors_.size();
272  }
273 
274  bool sortOnGpu() const { return sort_on_gpu_; }
275 
276  bool canOutputColumnar() const;
277  bool didOutputColumnar() const { return output_columnar_; }
278  void setOutputColumnar(const bool val);
279 
280  bool useStreamingTopN() const { return use_streaming_top_n_; }
281 
282  bool isLogicalSizedColumnsAllowed() const;
283 
285 
288  }
289 
290  void setThreadsCanReuseGroupByBuffers(const bool val) {
292  }
293 
294  // TODO(adb): remove and store this info more naturally in another
295  // member
296  bool forceFourByteFloat() const { return force_4byte_float_; }
297  void setForceFourByteFloat(const bool val) { force_4byte_float_ = val; }
298 
299  // Getters derived from state
300  size_t getGroupbyColCount() const { return group_col_widths_.size(); }
301  size_t getKeyCount() const { return keyless_hash_ ? 0 : getGroupbyColCount(); }
302  size_t getBufferColSlotCount() const;
303 
304  size_t getBufferSizeBytes(const RelAlgExecutionUnit& ra_exe_unit,
305  const unsigned thread_count,
306  const ExecutorDeviceType device_type) const;
307  size_t getBufferSizeBytes(const ExecutorDeviceType device_type) const;
308  size_t getBufferSizeBytes(const ExecutorDeviceType device_type,
309  const size_t override_entry_count) const;
310 
312 
313  // TODO(alex): remove
314  bool usesGetGroupValueFast() const;
315 
316  bool blocksShareMemory() const;
317  bool threadsShareMemory() const;
318 
319  bool lazyInitGroups(const ExecutorDeviceType) const;
320 
321  bool interleavedBins(const ExecutorDeviceType) const;
322 
323  size_t getColOffInBytes(const size_t col_idx) const;
324  size_t getColOffInBytesInNextBin(const size_t col_idx) const;
325  size_t getNextColOffInBytes(const int8_t* col_ptr,
326  const size_t bin,
327  const size_t col_idx) const;
328 
329  // returns the ptr offset of the next column, 64-bit aligned
330  size_t getNextColOffInBytesRowOnly(const int8_t* col_ptr, const size_t col_idx) const;
331  // returns the ptr offset of the current column, 64-bit aligned
332  size_t getColOnlyOffInBytes(const size_t col_idx) const;
333  size_t getRowSize() const;
334  size_t getColsSize() const;
335  size_t getWarpCount() const;
336 
337  size_t getCompactByteWidth() const;
338 
339  inline size_t getEffectiveKeyWidth() const {
340  return group_col_compact_width_ ? group_col_compact_width_ : sizeof(int64_t);
341  }
342 
343  bool isWarpSyncRequired(const ExecutorDeviceType) const;
344 
345  std::string queryDescTypeToString() const;
346  std::string toString() const;
347 
348  std::string reductionKey() const;
349 
351 
352  // returns a value if the buffer can be a fixed size; otherwise, we will need to use the
353  // bump allocator
354  std::optional<size_t> varlenOutputBufferElemSize() const;
355 
356  // returns the number of bytes needed for all slots preceeding slot_idx. Used to compute
357  // the offset into the varlen buffer for each projected target in a given row.
358  size_t varlenOutputRowSizeToSlot(const size_t slot_idx) const;
359 
360  bool slotIsVarlenOutput(const size_t slot_idx) const {
361  return col_slot_context_.slotIsVarlen(slot_idx);
362  }
363 
365 
366  void setAvailableCpuThreads(size_t num_available_threads) const {
367  num_available_threads_ = num_available_threads;
368  }
369 
370  std::optional<size_t> getMaxPerDeviceCardinality(
371  const RelAlgExecutionUnit& ra_exe_unit) const;
372 
373  bool canUsePerDeviceCardinality(const RelAlgExecutionUnit& ra_exe_unit) const;
374 
375  protected:
376  void resetGroupColWidths(const std::vector<int8_t>& new_group_col_widths) {
377  group_col_widths_ = new_group_col_widths;
378  }
379 
380  private:
386  int32_t idx_target_as_key_; // If keyless_hash_ enabled, then represents what target
387  // expression should be used to identify the key (e.g., in
388  // locating empty entries). Currently only valid with
389  // keyless_hash_ and single-column GroupByPerfectHash
390  std::vector<int8_t> group_col_widths_;
391  int8_t group_col_compact_width_; // compact width for all group
392  // cols if able to be consistent
393  // otherwise 0
394  std::vector<int64_t> target_groupby_indices_;
395  size_t entry_count_; // the number of entries in the main buffer
396  int64_t min_val_; // meaningful for OneColKnownRange,
397  // MultiColPerfectHash only
398  int64_t max_val_;
399  int64_t bucket_;
409 
411 
412  // # available CPU threads can be used for this query kernel, i.e., to parallelize rest
413  // of query initialization step its default value is one which means we do not
414  // parallelize for the query kernel, and it will be updated to a proper value before
415  // performing the query initialization
416  mutable size_t num_available_threads_{1};
417 
418  size_t getTotalBytesOfColumnarBuffers() const;
419  size_t getTotalBytesOfColumnarBuffers(const size_t num_entries_per_column) const;
420  size_t getTotalBytesOfColumnarProjections(const size_t projection_count) const;
421 
422  friend class ResultSet;
423  friend class QueryExecutionContext;
424 };
425 
426 inline void set_notnull(TargetInfo& target, const bool not_null) {
427  target.skip_null_val = !not_null;
428  auto new_type = get_compact_type(target);
429  new_type.set_notnull(not_null);
430  set_compact_type(target, new_type);
431 }
432 
433 std::vector<TargetInfo> target_exprs_to_infos(
434  const std::vector<Analyzer::Expr*>& targets,
436 
437 #endif // QUERYENGINE_QUERYMEMORYDESCRIPTOR_H
size_t varlenOutputRowSizeToSlot(const size_t slot_idx) const
void set_compact_type(TargetInfo &target, const SQLTypeInfo &new_type)
static bool many_entries(const int64_t max_val, const int64_t min_val, const int64_t bucket)
void addColSlotInfoFlatBuffer(const int64_t flatbuffer_size)
bool canUsePerDeviceCardinality(const RelAlgExecutionUnit &ra_exe_unit) const
size_t getBufferSizeBytes(const RelAlgExecutionUnit &ra_exe_unit, const unsigned thread_count, const ExecutorDeviceType device_type) const
bool slotIsVarlenOutput(const size_t slot_idx) const
bool countDistinctDescriptorsLogicallyEmpty() const
bool slotIsVarlen(const size_t slot_idx) const
size_t getTotalBytesOfColumnarProjections(const size_t projection_count) const
void setEntryCount(const size_t val)
size_t getAvailableCpuThreads() const
int64_t getTargetGroupbyIndex(const size_t target_idx) const
void sort_on_gpu(int64_t *val_buff, int32_t *idx_buff, const uint64_t entry_count, const bool desc, const uint32_t chosen_bytes, ThrustAllocator &alloc, const int device_id)
std::string toString() const
bool isLogicalSizedColumnsAllowed() const
void setHasKeylessHash(const bool val)
void setGroupColCompactWidth(const int8_t val)
void setThreadsCanReuseGroupByBuffers(const bool val)
void setOutputColumnar(const bool val)
static std::unique_ptr< QueryMemoryDescriptor > init(const Executor *executor, const RelAlgExecutionUnit &ra_exe_unit, const std::vector< InputTableInfo > &query_infos, const ColRangeInfo &col_range_info, const KeylessInfo &keyless_info, const bool allow_multifrag, const ExecutorDeviceType device_type, const int8_t crt_min_byte_width, const bool sort_on_gpu_hint, const size_t shard_count, const size_t max_groups_buffer_entry_count, RenderInfo *render_info, const CountDistinctDescriptors count_distinct_descriptors, const bool must_use_baseline_sort, const bool output_columnar_hint, const bool streaming_top_n_hint, const bool threads_can_reuse_group_by_buffers)
size_t getNextColOffInBytes(const int8_t *col_ptr, const size_t bin, const size_t col_idx) const
size_t getEffectiveKeyWidth() const
bool use_streaming_top_n(const RelAlgExecutionUnit &ra_exe_unit, const bool output_columnar)
bool hasVarlenOutput() const
bool skip_null_val
Definition: TargetInfo.h:54
void setQueryDescriptionType(const QueryDescriptionType val)
size_t targetGroupbyNegativeIndicesSize() const
bool hasInterleavedBinsOnGpu() const
ExecutorDeviceType
std::string to_string(char const *&&v)
const auto groupColWidthsBegin() const
std::optional< size_t > getMaxPerDeviceCardinality(const RelAlgExecutionUnit &ra_exe_unit) const
void useConsistentSlotWidthSize(const int8_t slot_width_size)
size_t getColOnlyOffInBytes(const size_t col_idx) const
ExecutorDispatchMode
const SQLTypeInfo get_compact_type(const TargetInfo &target)
const auto groupColWidthsEnd() const
int8_t groupColWidth(const size_t key_idx) const
void setTargetIdxForKey(const int32_t val)
std::vector< CountDistinctDescriptor > CountDistinctDescriptors
Definition: CountDistinct.h:34
Provides column info and slot info for the output buffer and some metadata helpers.
size_t getGroupbyColCount() const
bool lazyInitGroups(const ExecutorDeviceType) const
size_t targetGroupbyIndicesSize() const
bool threadsCanReuseGroupByBuffers() const
size_t getPrependedGroupBufferSizeInBytes() const
size_t getTotalBytesOfColumnarBuffers() const
std::vector< int64_t > target_groupby_indices_
static int8_t pick_target_compact_width(const RelAlgExecutionUnit &ra_exe_unit, const std::vector< InputTableInfo > &query_infos, const int8_t crt_min_byte_width)
CountDistinctDescriptors count_distinct_descriptors_
size_t getPaddedColWidthForRange(const size_t offset, const size_t range) const
const int8_t getPaddedSlotWidthBytes(const size_t slot_idx) const
int64_t getPaddedSlotBufferSize(const size_t slot_idx) const
static TResultSetBufferDescriptor toThrift(const QueryMemoryDescriptor &)
size_t getCountDistinctDescriptorsSize() const
QueryDescriptionType getQueryDescriptionType() const
const CountDistinctDescriptor & getCountDistinctDescriptor(const size_t idx) const
std::optional< size_t > varlenOutputBufferElemSize() const
#define CHECK_LT(x, y)
Definition: Logger.h:303
bool isSingleColumnGroupByWithPerfectHash() const
size_t getNextColOffInBytesRowOnly(const int8_t *col_ptr, const size_t col_idx) const
bool checkSlotUsesFlatBufferFormat(const size_t slot_idx) const
StreamingTopNOOM(const size_t heap_size_bytes)
QueryDescriptionType query_desc_type_
void setAvailableCpuThreads(size_t num_available_threads) const
int8_t updateActualMinByteWidth(const int8_t actual_min_byte_width) const
bool operator==(const QueryMemoryDescriptor &other) const
void setForceFourByteFloat(const bool val)
bool isWarpSyncRequired(const ExecutorDeviceType) const
bool interleavedBins(const ExecutorDeviceType) const
const ColSlotContext & getColSlotContext() const
std::vector< int8_t > group_col_widths_
void setAllTargetGroupbyIndices(std::vector< int64_t > group_by_indices)
QueryDescriptionType
Definition: Types.h:29
bool g_cluster
void setPaddedSlotWidthBytes(const size_t slot_idx, const int8_t bytes)
void resetGroupColWidths(const std::vector< int8_t > &new_group_col_widths)
std::vector< TargetInfo > target_exprs_to_infos(const std::vector< Analyzer::Expr * > &targets, const QueryMemoryDescriptor &query_mem_desc)
std::string queryDescTypeToString() const
void addColSlotInfo(const std::vector< std::tuple< int8_t, int8_t >> &slots_for_col)
void setHasInterleavedBinsOnGpu(const bool val)
static bool countDescriptorsLogicallyEmpty(const CountDistinctDescriptors &count_distinct_descriptors)
int64_t getFlatBufferSize(const size_t slot_idx) const
const int8_t getSlotIndexForSingleSlotCol(const size_t col_idx) const
bool checkSlotUsesFlatBufferFormat(const size_t slot_idx) const
const int8_t getLogicalSlotWidthBytes(const size_t slot_idx) const
size_t getColOffInBytes(const size_t col_idx) const
int64_t getFlatBufferSize(const size_t slot_idx) const
size_t getColOffInBytesInNextBin(const size_t col_idx) const
std::unique_ptr< QueryExecutionContext > getQueryExecutionContext(const RelAlgExecutionUnit &, const Executor *executor, const ExecutorDeviceType device_type, const ExecutorDispatchMode dispatch_mode, const int device_id, const shared::TableKey &outer_table_key, const int64_t num_rows, const std::vector< std::vector< const int8_t * >> &col_buffers, const std::vector< std::vector< uint64_t >> &frag_offsets, std::shared_ptr< RowSetMemoryOwner >, const bool output_columnar, const bool sort_on_gpu, const size_t thread_idx, RenderInfo *) const
std::string reductionKey() const
const Executor * getExecutor() const
void set_notnull(TargetInfo &target, const bool not_null)
int32_t getTargetIdxForKey() const
size_t getPrependedGroupColOffInBytes(const size_t group_idx) const