MySQL 8.3.0
Source Code Documentation
MaterializeIterator< Profiler > Class Template Referencefinal

Handles materialization; the first call to Init() will scan the given iterator to the end, store the results in a temporary table (optionally with deduplication), and then Read() will allow you to read that table repeatedly without the cost of executing the given subquery many times (unless you ask for rematerialization). More...

Inheritance diagram for MaterializeIterator< Profiler >:
[legend]

Classes

struct  Invalidator
 

Public Member Functions

 MaterializeIterator (THD *thd, materialize_iterator::Operands operands, const MaterializePathParameters *path_params, unique_ptr_destroy_only< RowIterator > table_iterator, JOIN *join)
 
bool Init () override
 Initialize or reinitialize the iterator. More...
 
int Read () override
 Read a single row. More...
 
void SetNullRowFlag (bool is_null_row) override
 Mark the current row buffer as containing a NULL row or not, so that if you read from it and the flag is true, you'll get only NULLs no matter what is actually in the buffer (typically some old leftover row). More...
 
void StartPSIBatchMode () override
 Start performance schema batch mode, if supported (otherwise ignored). More...
 
void EndPSIBatchModeIfStarted () override
 Ends performance schema batch mode, if started. More...
 
void UnlockRow () override
 
const IteratorProfilerGetProfiler () const override
 Get profiling data for this iterator (for 'EXPLAIN ANALYZE'). More...
 
const Profiler * GetTableIterProfiler () const
 
- Public Member Functions inherited from TableRowIterator
 TableRowIterator (THD *thd, TABLE *table)
 
void UnlockRow () override
 The default implementation of unlock-row method of RowIterator, used in all access methods except EQRefIterator. More...
 
void SetNullRowFlag (bool is_null_row) override
 Mark the current row buffer as containing a NULL row or not, so that if you read from it and the flag is true, you'll get only NULLs no matter what is actually in the buffer (typically some old leftover row). More...
 
void StartPSIBatchMode () override
 Start performance schema batch mode, if supported (otherwise ignored). More...
 
void EndPSIBatchModeIfStarted () override
 Ends performance schema batch mode, if started. More...
 
- Public Member Functions inherited from RowIterator
 RowIterator (THD *thd)
 
virtual ~RowIterator ()=default
 
 RowIterator (const RowIterator &)=delete
 
 RowIterator (RowIterator &&)=default
 
virtual void SetOverrideProfiler ([[maybe_unused]] const IteratorProfiler *profiler)
 
virtual RowIteratorreal_iterator ()
 If this iterator is wrapping a different iterator (e.g. More...
 
virtual const RowIteratorreal_iterator () const
 

Private Types

enum  Operand_type { LEFT_OPERAND , RIGHT_OPERAND }
 
using hash_map_type = robin_hood::unordered_flat_map< ImmutableStringWithLength, LinkedImmutableString, hash_join_buffer::KeyHasher, hash_join_buffer::KeyEquals >
 

Private Member Functions

bool doing_hash_deduplication () const
 Whether we are deduplicating using a hash field on the temporary table. More...
 
bool MaterializeRecursive ()
 Recursive materialization happens much like regular materialization, but some steps are repeated multiple times. More...
 
bool MaterializeOperand (const materialize_iterator::Operand &operand, ha_rows *stored_rows)
 
int read_next_row (const materialize_iterator::Operand &operand)
 
bool check_unique_fields_hash_map (TABLE *t, bool write, bool *found, bool *spill)
 Check presence of row in hash map, and make hash map iterator ready for writing value. More...
 
void backup_or_restore_blob_pointers (bool backup)
 Save (or restore) blob pointers in Field::m_blob_backup. More...
 
void update_row_in_hash_map ()
 
bool store_row_in_hash_map (Operand_type type=LEFT_OPERAND)
 Store the current row image into the hash map. More...
 
bool handle_hash_map_full (const materialize_iterator::Operand &operand, ha_rows *stored_rows)
 
bool process_row (const materialize_iterator::Operand &operand, materialize_iterator::Operands &operands, TABLE *t, uchar *set_counter_0, uchar *set_counter_1, bool *read_next)
 
bool process_row_hash (const materialize_iterator::Operand &operand, TABLE *t, ha_rows *stored_rows)
 
bool materialize_hash_map (TABLE *t, ha_rows *stored_rows)
 Walk through de-duplicated rows from in-memory hash table and/or spilled overflow HF chunks [1] and write said rows to table t, updating stored_rows counter. More...
 
bool load_HF_row_into_hash_map ()
 We just read a row from a HF chunk file. More...
 

Private Attributes

materialize_iterator::Operands m_operands
 
unique_ptr_destroy_only< RowIteratorm_table_iterator
 
Common_table_exprm_cte
 If we are materializing a CTE, points to it (otherwise nullptr). More...
 
Query_expressionm_query_expression
 The query expression we are materializing. More...
 
JOIN *const m_join
 See constructor. More...
 
const int m_ref_slice
 The slice to set when accessing temporary table; used if anything upstream (e.g. More...
 
const bool m_rematerialize
 If true, we need to materialize anew for each Init() (because the contents of the table will depend on some outer non-constant value). More...
 
const bool m_reject_multiple_rows
 See constructor. More...
 
const ha_rows m_limit_rows
 See constructor. More...
 
Mem_root_array< Invalidatorm_invalidators
 
Profiler m_profiler
 Profiling data for this iterator. More...
 
Profiler m_table_iter_profiler
 Profiling data for m_table_iterator. More...
 
bool m_use_hash_map {true}
 Use a hash map to implement row matching for set operations if true. More...
 
unique_ptr_destroy_only< MEM_ROOTm_mem_root
 
MEM_ROOTm_overflow_mem_root {nullptr}
 
size_t m_row_size_upper_bound
 
std::unique_ptr< hash_map_typem_hash_map
 
size_t m_rows_in_hash_map {0}
 
size_t m_read_rows_before_dedup {0}
 
hash_map_type::iterator m_hash_map_iterator
 Used to keep track of the current hash table entry focus after insertion or lookup. More...
 
LinkedImmutableString m_next_ptr {nullptr}
 Holds encoded row (if any) stored in the hash table. More...
 
LinkedImmutableString m_last_row {nullptr}
 last found row in hash map More...
 
TableCollection m_table_collection
 Needed for interfacing hash join function by hash set ops. More...
 
materialize_iterator::SpillState m_spill_state
 Spill to disk state for set operation: when in-memory hash map overflows, this keeps track of state. More...
 

Friends

class materialize_iterator::SpillState
 

Additional Inherited Members

- Protected Member Functions inherited from TableRowIterator
int HandleError (int error)
 
void PrintError (int error)
 
TABLEtable () const
 
- Protected Member Functions inherited from RowIterator
THDthd () const
 

Detailed Description

template<typename Profiler>
class MaterializeIterator< Profiler >

Handles materialization; the first call to Init() will scan the given iterator to the end, store the results in a temporary table (optionally with deduplication), and then Read() will allow you to read that table repeatedly without the cost of executing the given subquery many times (unless you ask for rematerialization).

When materializing, MaterializeIterator takes care of evaluating any items that need so, and storing the results in the fields of the outgoing table – which items is governed by the temporary table parameters.

Conceptually (although not performance-wise!), the MaterializeIterator is a no-op if you don't ask for deduplication[1], and in some cases (e.g. when scanning a table only once), we elide it. However, it's not necessarily straightforward to do so by just not inserting the iterator, as the optimizer will have set up everything (e.g., read sets, or what table upstream items will read from) assuming the materialization will happen, so the realistic option is setting up everything as if materialization would happen but not actually write to the table; see StreamingIterator for details.

[1] if we have a UNION DISTINCT or INTERSECT or EXCEPT it is not a no-op

  • for UNION DISTINCT MaterializeIterator de-duplicates rows via a key on the materialized table in two ways: a) a unique key if possible or a non-unique key on a hash of the row, if not. For details, see create_tmp_table.
  • INTERSECT and EXCEPE use two ways: a) using in-memory hashing (with posible spill to disk), in which case the materialized table is keyless, or if this approach overflows, b) using a non-unique key on the materialized table, the keys being the hash of the rows.

MaterializeIterator conceptually materializes iterators, not JOINs or Query_expressions. However, there are many details that leak out (e.g., setting performance schema batch mode, slices, reusing CTEs, etc.), so we need to send them in anyway.

'Profiler' should be 'IteratorProfilerImpl' for 'EXPLAIN ANALYZE' and 'DummyIteratorProfiler' otherwise. It is implemented as a a template parameter rather than a pointer to a base class in order to minimize the impact this probe has on normal query execution.

Member Typedef Documentation

◆ hash_map_type

template<typename Profiler >
using MaterializeIterator< Profiler >::hash_map_type = robin_hood::unordered_flat_map< ImmutableStringWithLength, LinkedImmutableString, hash_join_buffer::KeyHasher, hash_join_buffer::KeyEquals>
private

Member Enumeration Documentation

◆ Operand_type

template<typename Profiler >
enum MaterializeIterator::Operand_type
private
Enumerator
LEFT_OPERAND 
RIGHT_OPERAND 

Constructor & Destructor Documentation

◆ MaterializeIterator()

template<typename Profiler >
MaterializeIterator< Profiler >::MaterializeIterator ( THD thd,
materialize_iterator::Operands  operands,
const MaterializePathParameters path_params,
unique_ptr_destroy_only< RowIterator table_iterator,
JOIN join 
)
Parameters
thdThread handler.
operandsList of operands (aka query blocks) to materialize.
path_paramsMaterializePath settings.
table_iteratorIterator used for scanning the temporary table after materialization.
joinWhen materializing within the same JOIN (e.g., into a temporary table before sorting), as opposed to a derived table or a CTE, we may need to change the slice on the join before returning rows from the result table. If so, join and ref_slice would need to be set, and query_blocks_to_materialize should contain only one member, with the same join.

Member Function Documentation

◆ backup_or_restore_blob_pointers()

template<typename Profiler >
void MaterializeIterator< Profiler >::backup_or_restore_blob_pointers ( bool  backup)
private

Save (or restore) blob pointers in Field::m_blob_backup.

We need to have two full copies of a record for comparison, so we save record[0] to record[1] before reading from the hash table with LoadImmutableStringIntoTableBuffers. This will only work correctly for blobs if we also save the blob pointers lest they be clobbered when reading from the hash table, which reestablishes a full record in record[0] and resets Field blob pointers based on record[0]'s blob pointers. By saving them here we make sure that record[1]'s blob pointers do not point to overwritten or deallocated space.

Parameters
backupIf true, do backup, else restore blob pointers

◆ check_unique_fields_hash_map()

template<typename Profiler >
bool MaterializeIterator< Profiler >::check_unique_fields_hash_map ( TABLE t,
bool  write,
bool *  found,
bool *  spill 
)
private

Check presence of row in hash map, and make hash map iterator ready for writing value.

If we find the row or prepare a write, we set

  • m_hash_map_iterator
  • m_last_row
  • m_next_ptr

for use by companion method store_row_in_hash_map.

Parameters
[in]tthe source table
[in]writeif true, prepare a new write of row in hash map if not already there (left block only)
[out]foundset to true if the row was found in hash map
[out]spillset to true of we ran out of space
Returns
true on error

◆ doing_hash_deduplication()

template<typename Profiler >
bool MaterializeIterator< Profiler >::doing_hash_deduplication ( ) const
inlineprivate

Whether we are deduplicating using a hash field on the temporary table.

(This condition mirrors check_unique_fields().) If so, we compute a hash value for every row, look up all rows with the same hash and manually compare them to the row we are trying to insert.

Note that this is not the common way of deduplicating as we go. The common method is to have a regular index on the table over the right columns, and in that case, ha_write_row() will fail with an ignorable error, so that the row is ignored even though check_unique_fields() is not called. However, B-tree indexes have limitations, in particular on length, that sometimes require us to do this instead. See create_tmp_table() for details.

◆ EndPSIBatchModeIfStarted()

template<typename Profiler >
void MaterializeIterator< Profiler >::EndPSIBatchModeIfStarted
overridevirtual

Ends performance schema batch mode, if started.

It's always safe to call this.

Iterators that have children (composite iterators) must forward the EndPSIBatchModeIfStarted() call to every iterator they could conceivably have called StartPSIBatchMode() on. This ensures that after such a call to on the root iterator, all handlers are out of batch mode.

Reimplemented from RowIterator.

◆ GetProfiler()

template<typename Profiler >
const IteratorProfiler * MaterializeIterator< Profiler >::GetProfiler ( ) const
inlineoverridevirtual

Get profiling data for this iterator (for 'EXPLAIN ANALYZE').

Valid for TimingIterator, MaterializeIterator and TemptableAggregateIterator only.

Reimplemented from RowIterator.

◆ GetTableIterProfiler()

template<typename Profiler >
const Profiler * MaterializeIterator< Profiler >::GetTableIterProfiler ( ) const
inline

◆ handle_hash_map_full()

template<typename Profiler >
bool MaterializeIterator< Profiler >::handle_hash_map_full ( const materialize_iterator::Operand operand,
ha_rows stored_rows 
)
private

◆ Init()

template<typename Profiler >
bool MaterializeIterator< Profiler >::Init ( )
overridevirtual

Initialize or reinitialize the iterator.

You must always call Init() before trying a Read() (but Init() does not imply Read()).

You can call Init() multiple times; subsequent calls will rewind the iterator (or reposition it, depending on whether the iterator takes in e.g. a Index_lookup) and allow you to read the records anew.

Implements RowIterator.

◆ load_HF_row_into_hash_map()

template<typename Profiler >
bool MaterializeIterator< Profiler >::load_HF_row_into_hash_map
private

We just read a row from a HF chunk file.

Now, insert it into the hash map to prepare for the set operation with another operand, in IF chunk files.

Returns
true on error

◆ materialize_hash_map()

template<typename Profiler >
bool MaterializeIterator< Profiler >::materialize_hash_map ( TABLE t,
ha_rows stored_rows 
)
private

Walk through de-duplicated rows from in-memory hash table and/or spilled overflow HF chunks [1] and write said rows to table t, updating stored_rows counter.

[1] Depending on spill state. We have three cases:

a) No spill to disk: write rows from in-memory hash table. b) Spill to disk: write completed HF chunks, all chunks exist in the same generation >= 2 (the number is the same as the number of set operands). c) We saw secondary overflow during spill processing and must recover: write completed HF chunks (mix of 1. and 2.generation) and write the in-memory hash table

Parameters
toutput table
stored_rowscounter for # of rows stored in output table
Returns
true on error

◆ MaterializeOperand()

template<typename Profiler >
bool MaterializeIterator< Profiler >::MaterializeOperand ( const materialize_iterator::Operand operand,
ha_rows stored_rows 
)
private

◆ MaterializeRecursive()

template<typename Profiler >
bool MaterializeIterator< Profiler >::MaterializeRecursive
private

Recursive materialization happens much like regular materialization, but some steps are repeated multiple times.

Our general strategy is:

  1. Materialize all non-recursive query blocks, once.
  2. Materialize all recursive query blocks in turn.
  3. Repeat #2 until no query block writes any more rows (ie., we have converged) – for UNION DISTINCT queries, rows removed by deduplication do not count. Each materialization sees only rows that were newly added since the previous iteration; see FollowTailIterator for more details on the implementation.

Note that the result table is written to while other iterators are still reading from it; again, see FollowTailIterator. This means that each run of #2 can potentially run many actual CTE iterations – possibly the entire query to completion if we have only one query block.

This is not how the SQL standard specifies recursive CTE execution (it assumes building up the new result set from scratch for each iteration, using the previous iteration's results), but it is equivalent, and more efficient for the class of queries we support, since we don't need to re-create the same rows over and over again.

◆ process_row()

template<typename Profiler >
bool MaterializeIterator< Profiler >::process_row ( const materialize_iterator::Operand operand,
materialize_iterator::Operands operands,
TABLE t,
uchar set_counter_0,
uchar set_counter_1,
bool *  read_next 
)
private

Read the value of TABLE::m_set_counter from record[1]. The value can be found there after a call to check_unique_fields if the row was found. Note that m_set_counter a priori points to record[0], which is used when writing and updating the counter.

◆ process_row_hash()

template<typename Profiler >
bool MaterializeIterator< Profiler >::process_row_hash ( const materialize_iterator::Operand operand,
TABLE t,
ha_rows stored_rows 
)
private

◆ Read()

template<typename Profiler >
int MaterializeIterator< Profiler >::Read ( )
overridevirtual

Read a single row.

The row data is not actually returned from the function; it is put in the table's (or tables', in case of a join) record buffer, ie., table->records[0].

Return values
0OK
-1End of records
1Error

Implements RowIterator.

◆ read_next_row()

template<typename Profiler >
int MaterializeIterator< Profiler >::read_next_row ( const materialize_iterator::Operand operand)
private

◆ SetNullRowFlag()

template<typename Profiler >
void MaterializeIterator< Profiler >::SetNullRowFlag ( bool  is_null_row)
inlineoverridevirtual

Mark the current row buffer as containing a NULL row or not, so that if you read from it and the flag is true, you'll get only NULLs no matter what is actually in the buffer (typically some old leftover row).

This is used for outer joins, when an iterator hasn't produced any rows and we need to produce a NULL-complemented row. Init() or Read() won't necessarily reset this flag, so if you ever set is to true, make sure to also set it to false when needed.

Note that this can be called without Init() having been called first. For example, NestedLoopIterator can hit EOF immediately on the outer iterator, which means the inner iterator doesn't get an Init() call, but will still forward SetNullRowFlag to both inner and outer iterators.

TODO: We shouldn't need this. See the comments on AggregateIterator for a bit more discussion on abstracting out a row interface.

Implements RowIterator.

◆ StartPSIBatchMode()

template<typename Profiler >
void MaterializeIterator< Profiler >::StartPSIBatchMode ( )
inlineoverridevirtual

Start performance schema batch mode, if supported (otherwise ignored).

PFS batch mode is a mitigation to reduce the overhead of performance schema, typically applied at the innermost table of the entire join. If you start it before scanning the table and then end it afterwards, the entire set of handler calls will be timed only once, as a group, and the costs will be distributed evenly out. This reduces timer overhead.

If you start PFS batch mode, you must also take care to end it at the end of the scan, one way or the other. Do note that this is true even if the query ends abruptly (LIMIT is reached, or an error happens). The easiest workaround for this is to simply call EndPSIBatchModeIfStarted() on the root iterator at the end of the scan. See the PFSBatchMode class for a useful helper.

The rules for starting batch and ending mode are:

  1. If you are an iterator with exactly one child (FilterIterator etc.), forward any StartPSIBatchMode() calls to it.
  2. If you drive an iterator (read rows from it using a for loop or similar), use PFSBatchMode as described above.
  3. If you have multiple children, ignore the call and do your own handling of batch mode as appropriate. For materialization, #2 would typically apply. For joins, it depends on the join type (e.g., NestedLoopIterator applies batch mode only when scanning the innermost table).

The upshot of this is that when scanning a single table, batch mode will typically be activated for that table (since we call StartPSIBatchMode() on the root iterator, and it will trickle all the way down to the table iterator), but for a join, the call will be ignored and the join iterator will activate batch mode by itself as needed.

Reimplemented from RowIterator.

◆ store_row_in_hash_map()

template<typename Profiler >
bool MaterializeIterator< Profiler >::store_row_in_hash_map ( Operand_type  type = LEFT_OPERAND)
private

Store the current row image into the hash map.

Presumes the hash map iterator has looked up the (secondary hash), and possibly inserted its key and is positioned on it. Links any existing entry behind it, i.e. we insert at front of the hash bucket, cf. StoreLinkedImmutableStringFromTableBuffers. Update m_rows_in_hash_map.

Parameters
typeindicates whether we are processing the left operand or one of the right operands in the set operation
Returns
true on error

◆ UnlockRow()

template<typename Profiler >
void MaterializeIterator< Profiler >::UnlockRow ( )
inlineoverridevirtual

Implements RowIterator.

◆ update_row_in_hash_map()

template<typename Profiler >
void MaterializeIterator< Profiler >::update_row_in_hash_map
private

Friends And Related Function Documentation

◆ materialize_iterator::SpillState

template<typename Profiler >
friend class materialize_iterator::SpillState
friend

Member Data Documentation

◆ m_cte

template<typename Profiler >
Common_table_expr* MaterializeIterator< Profiler >::m_cte
private

If we are materializing a CTE, points to it (otherwise nullptr).

Used so that we see if some other iterator already materialized the table, avoiding duplicate work.

◆ m_hash_map

template<typename Profiler >
std::unique_ptr<hash_map_type> MaterializeIterator< Profiler >::m_hash_map
private

◆ m_hash_map_iterator

template<typename Profiler >
hash_map_type::iterator MaterializeIterator< Profiler >::m_hash_map_iterator
private

Used to keep track of the current hash table entry focus after insertion or lookup.

◆ m_invalidators

template<typename Profiler >
Mem_root_array<Invalidator> MaterializeIterator< Profiler >::m_invalidators
private

◆ m_join

template<typename Profiler >
JOIN* const MaterializeIterator< Profiler >::m_join
private

See constructor.

◆ m_last_row

template<typename Profiler >
LinkedImmutableString MaterializeIterator< Profiler >::m_last_row {nullptr}
private

last found row in hash map

◆ m_limit_rows

template<typename Profiler >
const ha_rows MaterializeIterator< Profiler >::m_limit_rows
private

See constructor.

◆ m_mem_root

template<typename Profiler >
unique_ptr_destroy_only<MEM_ROOT> MaterializeIterator< Profiler >::m_mem_root
private

◆ m_next_ptr

template<typename Profiler >
LinkedImmutableString MaterializeIterator< Profiler >::m_next_ptr {nullptr}
private

Holds encoded row (if any) stored in the hash table.

◆ m_operands

template<typename Profiler >
materialize_iterator::Operands MaterializeIterator< Profiler >::m_operands
private

◆ m_overflow_mem_root

template<typename Profiler >
MEM_ROOT* MaterializeIterator< Profiler >::m_overflow_mem_root {nullptr}
private

◆ m_profiler

template<typename Profiler >
Profiler MaterializeIterator< Profiler >::m_profiler
private

Profiling data for this iterator.

Used for 'EXPLAIN ANALYZE'. Note that MaterializeIterator merely (re)materializes a set of rows. It delegates the task of iterating over those rows to m_table_iterator. m_profiler thus records:

  • The total number of rows materialized (for the initial materialization and any subsequent rematerialization).
  • The total time spent on all materializations.

It does not measure the time spent accessing the materialized rows. That is handled by m_table_iter_profiler. The example below illustrates what 'EXPLAIN ANALYZE' output will be like. (Cost-data has been removed for the sake of simplicity.) The second line represents the MaterializeIterator that materializes x1, and the first line represents m_table_iterator, which is a TableScanIterator in this example.

-> Table scan on x1 (actual time=t1..t2 rows=r1 loops=l1) -> Materialize CTE x1 if needed (actual time=t3..t4 rows=r2 loops=l2)

t3 is the average time (across l2 materializations) spent materializing x1. Since MaterializeIterator does no iteration, we always set t3=t4. 'actual time' is cumulative, so that the values for an iterator should include the time spent in all its descendants. Therefore we know that t1*l1>=t3*l2 . (Note that t1 may be smaller than t3. We may re-scan x1 repeatedly without rematerializing it. Restarting a scan is quick, bringing the average time for fetching the first row (t1) down.)

◆ m_query_expression

template<typename Profiler >
Query_expression* MaterializeIterator< Profiler >::m_query_expression
private

The query expression we are materializing.

For derived tables, we materialize the entire query expression; for materialization within a query expression (e.g. for sorting or for windowing functions), we materialize only parts of it. Used to clear correlated CTEs within the unit when we rematerialize, since they depend on values from outside the query expression, and those values may have changed since last materialization.

◆ m_read_rows_before_dedup

template<typename Profiler >
size_t MaterializeIterator< Profiler >::m_read_rows_before_dedup {0}
private

◆ m_ref_slice

template<typename Profiler >
const int MaterializeIterator< Profiler >::m_ref_slice
private

The slice to set when accessing temporary table; used if anything upstream (e.g.

WHERE, HAVING) wants to evaluate values based on its contents. See constructor.

◆ m_reject_multiple_rows

template<typename Profiler >
const bool MaterializeIterator< Profiler >::m_reject_multiple_rows
private

See constructor.

◆ m_rematerialize

template<typename Profiler >
const bool MaterializeIterator< Profiler >::m_rematerialize
private

If true, we need to materialize anew for each Init() (because the contents of the table will depend on some outer non-constant value).

◆ m_row_size_upper_bound

template<typename Profiler >
size_t MaterializeIterator< Profiler >::m_row_size_upper_bound
private

◆ m_rows_in_hash_map

template<typename Profiler >
size_t MaterializeIterator< Profiler >::m_rows_in_hash_map {0}
private

◆ m_spill_state

template<typename Profiler >
materialize_iterator::SpillState MaterializeIterator< Profiler >::m_spill_state
private

Spill to disk state for set operation: when in-memory hash map overflows, this keeps track of state.

◆ m_table_collection

template<typename Profiler >
TableCollection MaterializeIterator< Profiler >::m_table_collection
private

Needed for interfacing hash join function by hash set ops.

We only ever have one table (the resulting tmp table of the set operation).

◆ m_table_iter_profiler

template<typename Profiler >
Profiler MaterializeIterator< Profiler >::m_table_iter_profiler
private

Profiling data for m_table_iterator.

'this' is a descendant of m_table_iterator in 'EXPLAIN ANALYZE' output, and 'elapsed time' should be cumulative. Therefore, m_table_iter_profiler will measure the sum of the time spent materializing the result rows and iterating over those rows.

◆ m_table_iterator

template<typename Profiler >
unique_ptr_destroy_only<RowIterator> MaterializeIterator< Profiler >::m_table_iterator
private

◆ m_use_hash_map

template<typename Profiler >
bool MaterializeIterator< Profiler >::m_use_hash_map {true}
private

Use a hash map to implement row matching for set operations if true.


The documentation for this class was generated from the following file: