MySQL 9.2.0
Source Code Documentation
All Classes Namespaces Files Functions Variables Typedefs Enumerations Enumerator Friends Macros Modules Pages Concepts
composite_iterators.h
Go to the documentation of this file.
1#ifndef SQL_ITERATORS_COMPOSITE_ITERATORS_H_
2#define SQL_ITERATORS_COMPOSITE_ITERATORS_H_
3
4/* Copyright (c) 2018, 2024, Oracle and/or its affiliates.
5
6 This program is free software; you can redistribute it and/or modify
7 it under the terms of the GNU General Public License, version 2.0,
8 as published by the Free Software Foundation.
9
10 This program is designed to work with certain software (including
11 but not limited to OpenSSL) that is licensed under separate terms,
12 as designated in a particular file or component or in included license
13 documentation. The authors of MySQL hereby grant you an additional
14 permission to link the program and your derivative works with the
15 separately licensed software that they have either included with
16 the program or referenced in the documentation.
17
18 This program is distributed in the hope that it will be useful,
19 but WITHOUT ANY WARRANTY; without even the implied warranty of
20 MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
21 GNU General Public License, version 2.0, for more details.
22
23 You should have received a copy of the GNU General Public License
24 along with this program; if not, write to the Free Software
25 Foundation, Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA */
26
27/**
28 @file composite_iterators.h
29
30 A composite row iterator is one that takes in one or more existing iterators
31 and processes their rows in some interesting way. They are usually not bound
32 to a single table or similar, but are the inner (non-leaf) nodes of the
33 iterator execution tree. They consistently own their source iterator, although
34 not its memory (since we never allocate row iterators on the heap--usually on
35 a MEM_ROOT>). This means that in the end, you'll end up with a single root
36 iterator which then owns everything else recursively.
37
38 SortingIterator and the two window iterators are also composite iterators,
39 but are defined in their own files.
40 */
41
42#include <assert.h>
43#include <stddef.h>
44#include <stdint.h>
45#include <sys/types.h>
46#include <memory>
47#include <span>
48#include <string>
49#include <utility>
50#include <vector>
51
52#include "my_alloc.h"
53#include "my_base.h"
54#include "my_inttypes.h"
55#include "my_table_map.h"
58#include "sql/join_type.h"
59#include "sql/mem_root_array.h"
60#include "sql/pack_rows.h"
61#include "sql/sql_array.h"
62#include "sql_string.h"
63
64class Cached_item;
66class Item;
67class JOIN;
68class KEY;
70class SJ_TMP_TABLE;
71class Table_ref;
72class THD;
73class Table_function;
75struct TABLE;
76
77/**
78 An iterator that takes in a stream of rows and passes through only those that
79 meet some criteria (i.e., a condition evaluates to true). This is typically
80 used for WHERE/HAVING.
81 */
82class FilterIterator final : public RowIterator {
83 public:
85 Item *condition)
86 : RowIterator(thd), m_source(std::move(source)), m_condition(condition) {}
87
88 bool Init() override { return m_source->Init(); }
89
90 int Read() override;
91
92 void SetNullRowFlag(bool is_null_row) override {
93 m_source->SetNullRowFlag(is_null_row);
94 }
95
96 void StartPSIBatchMode() override { m_source->StartPSIBatchMode(); }
97 void EndPSIBatchModeIfStarted() override {
98 m_source->EndPSIBatchModeIfStarted();
99 }
100 void UnlockRow() override { m_source->UnlockRow(); }
101
102 private:
105};
106
107/**
108 Handles LIMIT and/or OFFSET; Init() eats the first "offset" rows, and Read()
109 stops as soon as it's seen "limit" rows (including any skipped by offset).
110 */
111class LimitOffsetIterator final : public RowIterator {
112 public:
113 /**
114 @param thd Thread context
115 @param source Row source
116 @param limit Maximum number of rows to read, including the ones skipped by
117 offset. Can be HA_POS_ERROR for no limit.
118 @param offset Number of initial rows to skip. Can be 0 for no offset.
119 @param count_all_rows If true, the query will run to completion to get
120 more accurate numbers for skipped_rows, so you will not get any
121 performance benefits of early end.
122 @param reject_multiple_rows True if a derived table transformed from a
123 scalar subquery needs a run-time cardinality check
124 @param skipped_rows If not nullptr, is incremented for each row skipped by
125 offset or limit.
126 */
128 ha_rows limit, ha_rows offset, bool count_all_rows,
129 bool reject_multiple_rows, ha_rows *skipped_rows)
130 : RowIterator(thd),
131 m_source(std::move(source)),
132 m_limit(limit),
133 m_offset(offset),
134 m_count_all_rows(count_all_rows),
135 m_reject_multiple_rows(reject_multiple_rows),
136 m_skipped_rows(skipped_rows) {
137 if (count_all_rows) {
138 assert(m_skipped_rows != nullptr);
139 }
140 }
141
142 bool Init() override;
143
144 int Read() override;
145
146 void SetNullRowFlag(bool is_null_row) override {
147 m_source->SetNullRowFlag(is_null_row);
148 }
149
150 void StartPSIBatchMode() override { m_source->StartPSIBatchMode(); }
151 void EndPSIBatchModeIfStarted() override {
152 m_source->EndPSIBatchModeIfStarted();
153 }
154 void UnlockRow() override { m_source->UnlockRow(); }
155
156 private:
158
159 // Note: The number of seen rows starts off at m_limit if we have OFFSET,
160 // which means we don't need separate LIMIT and OFFSET tests on the
161 // fast path of Read().
163
164 /**
165 Whether we have OFFSET rows that we still need to skip.
166 */
168
173};
174
175/**
176 Handles aggregation (typically used for GROUP BY) for the case where the rows
177 are already properly grouped coming in, ie., all rows that are supposed to be
178 part of the same group are adjacent in the input stream. (This could be
179 because they were sorted earlier, because we are scanning an index that
180 already gives us the rows in a group-compatible order, or because there is no
181 grouping.)
182
183 AggregateIterator needs to be able to save and restore rows; it doesn't know
184 when a group ends until it's seen the first row that is part of the _next_
185 group. When that happens, it needs to tuck away that next row, and then
186 restore the previous row so that the output row gets the correct grouped
187 values. A simple example, doing SELECT a, SUM(b) FROM t1 GROUP BY a:
188
189 t1.a t1.b SUM(b)
190 1 1 <-- first row, save it 1
191 1 2 3
192 1 3 6
193 2 1 <-- group changed, save row
194 [1 1] <-- restore first row, output 6
195 reset aggregate --> 0
196 [2 1] <-- restore new row, process it 1
197 2 10 11
198 <-- EOF, output 11
199
200 To save and restore rows like this, it uses the infrastructure from
201 pack_rows.h to pack and unpack all relevant rows into record[0] of every input
202 table. (Currently, there can only be one input table, but this may very well
203 change in the future.) It would be nice to have a more abstract concept of
204 sending a row around and taking copies of it if needed, as opposed to it
205 implicitly staying in the table's buffer. (This would also solve some
206 issues in EQRefIterator and when synthesizing NULL rows for outer joins.)
207 However, that's a large refactoring.
208 */
209class AggregateIterator final : public RowIterator {
210 public:
213 std::span<AccessPath *> single_row_index_lookups,
214 bool rollup);
215
216 bool Init() override;
217 int Read() override;
218 void SetNullRowFlag(bool is_null_row) override {
219 m_source->SetNullRowFlag(is_null_row);
220 }
221
222 void StartPSIBatchMode() override { m_source->StartPSIBatchMode(); }
223 void EndPSIBatchModeIfStarted() override {
224 m_source->EndPSIBatchModeIfStarted();
225 }
226 void UnlockRow() override {
227 // Most likely, HAVING failed. Ideally, we'd like to backtrack and
228 // unlock all rows that went into this aggregate, but we can't do that,
229 // and we also can't unlock the _current_ row, since that belongs to a
230 // different group. Thus, do nothing.
231 }
232
233 private:
234 enum {
240
242
243 /**
244 The join we are part of. It would be nicer not to rely on this,
245 but we need a large number of members from there, like which
246 aggregate functions we have, the THD, temporary table parameters
247 and so on.
248 */
249 JOIN *m_join = nullptr;
250
251 /// Whether we have seen the last input row.
253
254 /**
255 Used to save NULL information in the specific case where we have
256 zero input rows.
257 */
259
260 /// Whether this is a rollup query.
261 const bool m_rollup;
262
263 /**
264 For rollup: The index of the first group item that did _not_ change when we
265 last switched groups. E.g., if we have group fields A,B,C,D and then switch
266 to group A,B,E,D, this value will become 1 (which means that we need
267 to output rollup rows for 2 -- A,B,E,NULL -- and then 1 -- A,B,NULL,NULL).
268 m_current_rollup_position will count down from the end until it becomes
269 less than this value.
270
271 If we do not have rollup, this value is perennially zero.
272 */
274
275 /**
276 If we are in state OUTPUTTING_ROLLUP_ROWS, where we are in the iteration.
277 This value will start at the index of the last group expression and then
278 count backwards down to and including m_last_unchanged_group_item_idx.
279 It is used to communicate to the rollup group items whether to turn
280 themselves into NULLs, and the sum items which of their sums to output.
281 */
283
284 /**
285 The list of tables we are reading from; they are the ones for which we need
286 to save and restore rows.
287 */
289
290 /// Packed version of the first row in the group we are currently processing.
292
293 /**
294 If applicable, packed version of the first row in the _next_ group. This is
295 used only in the LAST_ROW_STARTED_NEW_GROUP state; we just saw a row that
296 didn't belong to the current group, so we saved it here and went to output
297 a group. On the next Read() call, we need to process this deferred row
298 first of all.
299
300 Even when not in use, this string contains a buffer that is large enough to
301 pack a full row into, sans blobs. (If blobs are present,
302 StoreFromTableBuffers() will automatically allocate more space if needed.)
303 */
305
306 /// All the single-row index lookups that provide rows to this iterator.
307 std::span<AccessPath *> m_single_row_index_lookups;
308
309 /**
310 The slice we're setting when returning rows. See the comment in the
311 constructor.
312 */
314
315 void SetRollupLevel(int level);
316};
317
318/**
319 A simple nested loop join, taking in two iterators (left/outer and
320 right/inner) and joining them together. This may, of course, scan the inner
321 iterator many times. It is currently the only form of join we have.
322
323 The iterator works as a state machine, where the state records whether we need
324 to read a new outer row or not, and whether we've seen any rows from the inner
325 iterator at all (if not, an outer join need to synthesize a new NULL row).
326
327 The iterator takes care of activating performance schema batch mode on the
328 right iterator if needed; this is typically only used if it is the innermost
329 table in the entire join (where the gains from turning on batch mode is the
330 largest, and the accuracy loss from turning it off are the least critical).
331 */
332class NestedLoopIterator final : public RowIterator {
333 public:
337 JoinType join_type, bool pfs_batch_mode)
338 : RowIterator(thd),
339 m_source_outer(std::move(source_outer)),
340 m_source_inner(std::move(source_inner)),
342 m_pfs_batch_mode(pfs_batch_mode) {
343 assert(m_source_outer != nullptr);
344 assert(m_source_inner != nullptr);
345
346 // Batch mode makes no sense for anti- or semijoins, since they should only
347 // be reading one row.
349 assert(!pfs_batch_mode);
350 }
351 }
352
353 bool Init() override;
354
355 int Read() override;
356
357 void SetNullRowFlag(bool is_null_row) override {
358 // TODO: write something here about why we can't do this lazily.
359 m_source_outer->SetNullRowFlag(is_null_row);
360 m_source_inner->SetNullRowFlag(is_null_row);
361 }
362
363 void EndPSIBatchModeIfStarted() override {
364 m_source_outer->EndPSIBatchModeIfStarted();
365 m_source_inner->EndPSIBatchModeIfStarted();
366 }
367
368 void UnlockRow() override {
369 // Since we don't know which condition that caused the row to be rejected,
370 // we can't know whether we could also unlock the outer row
371 // (it may still be used as parts of other joined rows).
373 m_source_inner->UnlockRow();
374 }
375 }
376
377 private:
378 enum {
384
388
389 /** Whether to use batch mode when scanning the inner iterator. */
391};
392
393/**
394 An iterator that helps invalidating caches. Every time a row passes through it
395 or it changes state in any other way, it increments its “generation” counter.
396 This allows MaterializeIterator to see whether any of its dependencies has
397 changed, and then force a rematerialization -- this is typically used for
398 LATERAL tables, where we're joining in a derived table that depends on
399 something earlier in the join.
400 */
402 public:
405 const std::string &name)
406 : RowIterator(thd),
407 m_source_iterator(std::move(source_iterator)),
408 m_name(name) {}
409
410 bool Init() override {
411 ++m_generation;
412 return m_source_iterator->Init();
413 }
414
415 int Read() override {
416 ++m_generation;
417 return m_source_iterator->Read();
418 }
419
420 void SetNullRowFlag(bool is_null_row) override {
421 ++m_generation;
422 m_source_iterator->SetNullRowFlag(is_null_row);
423 }
424
425 void UnlockRow() override { m_source_iterator->UnlockRow(); }
426
427 int64_t generation() const { return m_generation; }
428 std::string name() const { return m_name; }
429
430 private:
432 int64_t m_generation = 0;
433 std::string m_name;
434};
435
437/**
438 An operand (query block) to be materialized by MaterializeIterator.
439 (@see MaterializeIterator for details.)
440*/
441struct Operand {
442 /// The iterator to read the actual rows from.
444
445 /// Used only for optimizer trace.
447
448 /// The JOIN that this query block represents. Used for performance
449 /// schema batch mode: When materializing a query block that consists of
450 /// a single table, MaterializeIterator needs to set up schema batch mode,
451 /// since there is no nested loop iterator to do it. (This is similar to
452 /// what ExecuteIteratorQuery() needs to do at the top level.)
454
455 /// If true, de-duplication checking via hash key is disabled
456 /// when materializing this query block (ie., we simply avoid calling
457 /// check_unique_fields() for each row). Used when materializing
458 /// UNION DISTINCT and UNION ALL parts into the same table.
459 /// We'd like to just use a unique constraint via unique index instead,
460 /// but there might be other indexes on the destination table
461 /// that we'd like to keep, and the implementation doesn't allow
462 /// disabling only one index.
463 ///
464 /// If you use this on a query block, doing_hash_deduplication()
465 /// must be true.
467
468 /// If set to false, the Field objects in the output row are
469 /// presumed already to be filled out. This is the case iff
470 /// there's a windowing iterator earlier in the chain.
472
473 /// The number of operands (i.e. blocks) involved in the set operation:
474 /// used for INTERSECT to determine if a value is present in all operands
476 /// The current operand (i.e. block) number, starting at zero. We use this
477 /// for INTERSECT and EXCEPT materialization operand.
479 /// Used for EXCEPT computation: the index of the first operand involved in
480 /// a N-ary except operation which has DISTINCT. This is significant for
481 /// calculating whether to set the counter to zero or just decrement it
482 /// when we see a right side operand.
484
485 /// If copy_items is true, used for copying the Field objects
486 /// into the temporary table row. Otherwise unused.
488
489 // Whether this query block is a recursive reference back to the
490 // output of the materialization.
492
493 // If is_recursive_reference is true, contains the FollowTailIterator
494 // in the query block (there can be at most one recursive reference
495 // in a join list, as per the SQL standard, so there should be exactly one).
496 // Used for informing the iterators about various shared state in the
497 // materialization (including coordinating rematerializations).
499
500 /// The estimated number of rows produced by this block
502};
503
504/**
505 Create an iterator that materializes a set of row into a temporary table
506 and sets up a (pre-existing) iterator to access that.
507 @see MaterializeIterator.
508
509 @param thd Thread handler.
510 @param operands List of operands (query blocks) to materialize.
511 @param path_params MaterializePath settings.
512 @param table_iterator Iterator used for accessing the temporary table
513 after materialization.
514 @param join
515 When materializing within the same JOIN (e.g., into a temporary table
516 before sorting), as opposed to a derived table or a CTE, we may need
517 to change the slice on the join before returning rows from the result
518 table. If so, join and ref_slice would need to be set, and
519 query_blocks_to_materialize should contain only one member, with the same
520 join.
521 @return the iterator.
522*/
525 const MaterializePathParameters *path_params,
527
528} // namespace materialize_iterator
529
531/**
532 Create an iterator that aggregates the output rows from another iterator
533 into a temporary table and then sets up a (pre-existing) iterator to
534 access the temporary table.
535 @see TemptableAggregateIterator.
536
537 @param thd Thread handler.
538 @param subquery_iterator input to aggregation.
539 @param temp_table_param temporary table settings.
540 @param table_iterator Iterator used for scanning the temporary table
541 after materialization.
542 @param table the temporary table.
543 @param join the JOIN in which we aggregate.
544 @param ref_slice the slice to set when accessing temporary table;
545 used if anything upstream wants to evaluate values based on its contents.
546 @return the iterator.
547*/
549 THD *thd, unique_ptr_destroy_only<RowIterator> subquery_iterator,
550 Temp_table_param *temp_table_param, TABLE *table,
552 int ref_slice);
553
554} // namespace temptable_aggregate_iterator
555
556/**
557 StreamingIterator is a minimal version of MaterializeIterator that does not
558 actually materialize; instead, every Read() just forwards the call to the
559 subquery iterator and does the required copying from one set of fields to
560 another.
561
562 It is used for when the optimizer would normally set up a materialization,
563 but you don't actually need one, ie. you don't want to read the rows multiple
564 times after writing them, and you don't want to access them by index (only
565 a single table scan). It also takes care of setting the NULL row flag
566 on the temporary table.
567 */
569 public:
570 /**
571 @param thd Thread handle.
572 @param subquery_iterator The iterator to read rows from.
573 @param temp_table_param Parameters for the temp table.
574 @param table The table we are streaming through. Will never actually
575 be written to, but its fields will be used.
576 @param provide_rowid If true, generate a row ID for each row we stream.
577 This is used if the parent needs row IDs for deduplication, in particular
578 weedout.
579 @param join See MaterializeIterator.
580 @param ref_slice See MaterializeIterator.
581 */
583 unique_ptr_destroy_only<RowIterator> subquery_iterator,
584 Temp_table_param *temp_table_param, TABLE *table,
585 bool provide_rowid, JOIN *join, int ref_slice);
586
587 bool Init() override;
588
589 int Read() override;
590
591 void StartPSIBatchMode() override {
592 m_subquery_iterator->StartPSIBatchMode();
593 }
594 void EndPSIBatchModeIfStarted() override {
595 m_subquery_iterator->EndPSIBatchModeIfStarted();
596 }
597 void UnlockRow() override { m_subquery_iterator->UnlockRow(); }
598
599 private:
603 JOIN *const m_join;
604 const int m_output_slice;
606
607 // Whether the iterator should generate and provide a row ID. Only true if the
608 // iterator is part of weedout, where the iterator will create a fake row ID
609 // to uniquely identify the rows it produces.
610 const bool m_provide_rowid;
611};
612
613/**
614 An iterator that wraps a Table_function (e.g. JSON_TABLE) and allows you to
615 iterate over the materialized temporary table. The table is materialized anew
616 for every Init().
617
618 TODO: Just wrapping it is probably not the optimal thing to do;
619 Table_function is highly oriented around materialization, but never caches.
620 Thus, perhaps we should rewrite Table_function to return a RowIterator
621 instead of going through a temporary table.
622 */
624 public:
626 THD *thd, Table_function *table_function, TABLE *table,
628
629 bool Init() override;
630 int Read() override { return m_table_iterator->Read(); }
631 void SetNullRowFlag(bool is_null_row) override {
632 m_table_iterator->SetNullRowFlag(is_null_row);
633 }
634
635 void StartPSIBatchMode() override { m_table_iterator->StartPSIBatchMode(); }
636 void EndPSIBatchModeIfStarted() override {
637 m_table_iterator->EndPSIBatchModeIfStarted();
638 }
639
640 // The temporary table is private to us, so there's no need to worry about
641 // locks to other transactions.
642 void UnlockRow() override {}
643
644 private:
646
648};
649
650/**
651 Like semijoin materialization, weedout works on the basic idea that a semijoin
652 is just like an inner join as we long as we can get rid of the duplicates
653 somehow. (This is advantageous, because inner joins can be reordered, whereas
654 semijoins generally can't.) However, unlike semijoin materialization, weedout
655 removes duplicates after the join, not before it. Consider something like
656
657 SELECT * FROM t1 WHERE a IN ( SELECT b FROM t2 );
658
659 Semijoin materialization solves this by materializing t2, with deduplication,
660 and then joining. Weedout joins t1 to t2 and then leaves only one output row
661 per t1 row. The disadvantage is that this potentially needs to discard more
662 rows; the (potential) advantage is that we deduplicate on t1 instead of t2.
663
664 Weedout, unlike materialization, works in a streaming fashion; rows are output
665 (or discarded) as they come in, with a temporary table used for recording the
666 row IDs we've seen before. (We need to deduplicate on t1's row IDs, not its
667 contents.) See SJ_TMP_TABLE for details about the table format.
668 */
669class WeedoutIterator final : public RowIterator {
670 public:
672 SJ_TMP_TABLE *sj, table_map tables_to_get_rowid_for);
673
674 bool Init() override;
675 int Read() override;
676
677 void SetNullRowFlag(bool is_null_row) override {
678 m_source->SetNullRowFlag(is_null_row);
679 }
680
681 void EndPSIBatchModeIfStarted() override {
682 m_source->EndPSIBatchModeIfStarted();
683 }
684 void UnlockRow() override { m_source->UnlockRow(); }
685
686 private:
690};
691
692/**
693 An iterator that removes consecutive rows that are the same according to
694 a set of items (typically the join key), so-called “loose scan”
695 (not to be confused with “loose index scan”, which is made by the
696 range optimizer). This is similar in spirit to WeedoutIterator above
697 (removing duplicates allows us to treat the semijoin as a normal join),
698 but is much cheaper if the data is already ordered/grouped correctly,
699 as the removal can happen before the join, and it does not need a
700 temporary table.
701 */
703 public:
706 JOIN *join, Item **group_items,
707 int group_items_size);
708
709 bool Init() override;
710 int Read() override;
711
712 void SetNullRowFlag(bool is_null_row) override {
713 m_source->SetNullRowFlag(is_null_row);
714 }
715
716 void StartPSIBatchMode() override { m_source->StartPSIBatchMode(); }
717 void EndPSIBatchModeIfStarted() override {
718 m_source->EndPSIBatchModeIfStarted();
719 }
720 void UnlockRow() override { m_source->UnlockRow(); }
721
722 private:
726};
727
728/**
729 Much like RemoveDuplicatesIterator, but works on the basis of a given index
730 (or more accurately, its keypart), not an arbitrary list of grouped fields.
731 This is only used in the non-hypergraph optimizer; the hypergraph optimizer
732 can deal with groupings that come from e.g. sorts.
733 */
735 public:
738 const TABLE *table, KEY *key, size_t key_len);
739
740 bool Init() override;
741 int Read() override;
742
743 void SetNullRowFlag(bool is_null_row) override {
744 m_source->SetNullRowFlag(is_null_row);
745 }
746
747 void StartPSIBatchMode() override { m_source->StartPSIBatchMode(); }
748 void EndPSIBatchModeIfStarted() override {
749 m_source->EndPSIBatchModeIfStarted();
750 }
751 void UnlockRow() override { m_source->UnlockRow(); }
752
753 private:
757 uchar *m_key_buf; // Owned by the THD's MEM_ROOT.
758 const size_t m_key_len;
760};
761
762/**
763 An iterator that is semantically equivalent to a semijoin NestedLoopIterator
764 immediately followed by a RemoveDuplicatesOnIndexIterator. It is used to
765 implement the “loose scan” strategy in queries with multiple tables on the
766 inside of a semijoin, like
767
768 ... FROM t1 WHERE ... IN ( SELECT ... FROM t2 JOIN t3 ... )
769
770 In this case, the query tree without this iterator would ostensibly look like
771
772 -> Nested loop join
773 -> Table scan on t1
774 -> Remove duplicates on t2_idx
775 -> Nested loop semijoin
776 -> Index scan on t2 using t2_idx
777 -> Filter (e.g. t3.a = t2.a)
778 -> Table scan on t3
779
780 (t3 will be marked as “first match” on t2 when implementing loose scan,
781 thus the semijoin.)
782
783 First note that we can't put the duplicate removal directly on t2 in this
784 case, as the first t2 row doesn't necessarily match anything in t3, so it
785 needs to be above. However, this is wasteful, because once we find a matching
786 t2/t3 pair, we should stop scanning t3 until we have a new t2.
787
788 NestedLoopSemiJoinWithDuplicateRemovalIterator solves the problem by doing
789 exactly this; it gets a row from the outer side, gets exactly one row from the
790 inner side, and then skips over rows from the outer side (_without_ scanning
791 the inner side) until its keypart changes.
792 */
794 : public RowIterator {
795 public:
799 KEY *key, size_t key_len);
800
801 bool Init() override;
802
803 int Read() override;
804
805 void SetNullRowFlag(bool is_null_row) override {
806 m_source_outer->SetNullRowFlag(is_null_row);
807 m_source_inner->SetNullRowFlag(is_null_row);
808 }
809
810 void EndPSIBatchModeIfStarted() override {
811 m_source_outer->EndPSIBatchModeIfStarted();
812 m_source_inner->EndPSIBatchModeIfStarted();
813 }
814
815 void UnlockRow() override {
816 m_source_outer->UnlockRow();
817 m_source_inner->UnlockRow();
818 }
819
820 private:
823
826 uchar *m_key_buf; // Owned by the THD's MEM_ROOT.
827 const size_t m_key_len;
829};
830
831/**
832 MaterializeInformationSchemaTableIterator makes sure a given I_S temporary
833 table is materialized (filled out) before we try to scan it.
834 */
836 public:
839 Table_ref *table_list, Item *condition);
840
841 bool Init() override;
842 int Read() override { return m_table_iterator->Read(); }
843
844 void SetNullRowFlag(bool is_null_row) override {
845 m_table_iterator->SetNullRowFlag(is_null_row);
846 }
847
848 void StartPSIBatchMode() override { m_table_iterator->StartPSIBatchMode(); }
849 void EndPSIBatchModeIfStarted() override {
850 m_table_iterator->EndPSIBatchModeIfStarted();
851 }
852
853 // The temporary table is private to us, so there's no need to worry about
854 // locks to other transactions.
855 void UnlockRow() override {}
856
857 private:
858 /// The iterator that reads from the materialized table.
862};
863
864/**
865 Takes in two or more iterators and output rows from them sequentially
866 (first all rows from the first one, the all from the second one, etc.).
867 Used for implementing UNION ALL, typically together with StreamingIterator.
868 */
869class AppendIterator final : public RowIterator {
870 public:
872 THD *thd,
874
875 bool Init() override;
876 int Read() override;
877
878 void StartPSIBatchMode() override;
879 void EndPSIBatchModeIfStarted() override;
880
881 void SetNullRowFlag(bool is_null_row) override;
882 void UnlockRow() override;
883
884 private:
885 std::vector<unique_ptr_destroy_only<RowIterator>> m_sub_iterators;
888};
889
890#endif // SQL_ITERATORS_COMPOSITE_ITERATORS_H_
Handles aggregation (typically used for GROUP BY) for the case where the rows are already properly gr...
Definition: composite_iterators.h:209
void UnlockRow() override
Definition: composite_iterators.h:226
int m_current_rollup_position
If we are in state OUTPUTTING_ROLLUP_ROWS, where we are in the iteration.
Definition: composite_iterators.h:282
void StartPSIBatchMode() override
Start performance schema batch mode, if supported (otherwise ignored).
Definition: composite_iterators.h:222
int Read() override
Read a single row.
Definition: composite_iterators.cc:260
JOIN * m_join
The join we are part of.
Definition: composite_iterators.h:249
bool m_seen_eof
Whether we have seen the last input row.
Definition: composite_iterators.h:252
AggregateIterator(THD *thd, unique_ptr_destroy_only< RowIterator > source, JOIN *join, pack_rows::TableCollection tables, std::span< AccessPath * > single_row_index_lookups, bool rollup)
Definition: composite_iterators.cc:196
pack_rows::TableCollection m_tables
The list of tables we are reading from; they are the ones for which we need to save and restore rows.
Definition: composite_iterators.h:288
String m_first_row_this_group
Packed version of the first row in the group we are currently processing.
Definition: composite_iterators.h:291
String m_first_row_next_group
If applicable, packed version of the first row in the next group.
Definition: composite_iterators.h:304
table_map m_save_nullinfo
Used to save NULL information in the specific case where we have zero input rows.
Definition: composite_iterators.h:258
@ LAST_ROW_STARTED_NEW_GROUP
Definition: composite_iterators.h:236
@ READING_FIRST_ROW
Definition: composite_iterators.h:235
@ OUTPUTTING_ROLLUP_ROWS
Definition: composite_iterators.h:237
@ DONE_OUTPUTTING_ROWS
Definition: composite_iterators.h:238
bool Init() override
Initialize or reinitialize the iterator.
Definition: composite_iterators.cc:216
unique_ptr_destroy_only< RowIterator > m_source
Definition: composite_iterators.h:241
int m_output_slice
The slice we're setting when returning rows.
Definition: composite_iterators.h:313
void EndPSIBatchModeIfStarted() override
Ends performance schema batch mode, if started.
Definition: composite_iterators.h:223
const bool m_rollup
Whether this is a rollup query.
Definition: composite_iterators.h:261
std::span< AccessPath * > m_single_row_index_lookups
All the single-row index lookups that provide rows to this iterator.
Definition: composite_iterators.h:307
enum AggregateIterator::@62 m_state
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...
Definition: composite_iterators.h:218
void SetRollupLevel(int level)
Definition: composite_iterators.cc:482
int m_last_unchanged_group_item_idx
For rollup: The index of the first group item that did not change when we last switched groups.
Definition: composite_iterators.h:273
Takes in two or more iterators and output rows from them sequentially (first all rows from the first ...
Definition: composite_iterators.h:869
size_t m_current_iterator_index
Definition: composite_iterators.h:886
AppendIterator(THD *thd, std::vector< unique_ptr_destroy_only< RowIterator > > &&sub_iterators)
Definition: composite_iterators.cc:4463
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...
Definition: composite_iterators.cc:4500
bool Init() override
Initialize or reinitialize the iterator.
Definition: composite_iterators.cc:4469
void EndPSIBatchModeIfStarted() override
Ends performance schema batch mode, if started.
Definition: composite_iterators.cc:4510
void UnlockRow() override
Definition: composite_iterators.cc:4518
std::vector< unique_ptr_destroy_only< RowIterator > > m_sub_iterators
Definition: composite_iterators.h:885
int Read() override
Read a single row.
Definition: composite_iterators.cc:4475
bool m_pfs_batch_mode_enabled
Definition: composite_iterators.h:887
void StartPSIBatchMode() override
Start performance schema batch mode, if supported (otherwise ignored).
Definition: composite_iterators.cc:4505
A wrapper class which provides array bounds checking.
Definition: sql_array.h:47
An iterator that helps invalidating caches.
Definition: composite_iterators.h:401
bool Init() override
Initialize or reinitialize the iterator.
Definition: composite_iterators.h:410
CacheInvalidatorIterator(THD *thd, unique_ptr_destroy_only< RowIterator > source_iterator, const std::string &name)
Definition: composite_iterators.h:403
std::string m_name
Definition: composite_iterators.h:433
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...
Definition: composite_iterators.h:420
unique_ptr_destroy_only< RowIterator > m_source_iterator
Definition: composite_iterators.h:431
void UnlockRow() override
Definition: composite_iterators.h:425
int64_t m_generation
Definition: composite_iterators.h:432
int Read() override
Read a single row.
Definition: composite_iterators.h:415
std::string name() const
Definition: composite_iterators.h:428
int64_t generation() const
Definition: composite_iterators.h:427
This is used for segregating rows in groups (e.g.
Definition: item.h:6485
An iterator that takes in a stream of rows and passes through only those that meet some criteria (i....
Definition: composite_iterators.h:82
void EndPSIBatchModeIfStarted() override
Ends performance schema batch mode, if started.
Definition: composite_iterators.h:97
int Read() override
Read a single row.
Definition: composite_iterators.cc:95
void UnlockRow() override
Definition: composite_iterators.h:100
bool Init() override
Initialize or reinitialize the iterator.
Definition: composite_iterators.h:88
unique_ptr_destroy_only< RowIterator > m_source
Definition: composite_iterators.h:103
FilterIterator(THD *thd, unique_ptr_destroy_only< RowIterator > source, Item *condition)
Definition: composite_iterators.h:84
void StartPSIBatchMode() override
Start performance schema batch mode, if supported (otherwise ignored).
Definition: composite_iterators.h:96
Item * m_condition
Definition: composite_iterators.h:104
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...
Definition: composite_iterators.h:92
FollowTailIterator is a special version of TableScanIterator that is used as part of WITH RECURSIVE q...
Definition: basic_row_iterators.h:470
Base class that is used to represent any kind of expression in a relational query.
Definition: item.h:930
Definition: sql_optimizer.h:133
Definition: key.h:113
Handles LIMIT and/or OFFSET; Init() eats the first "offset" rows, and Read() stops as soon as it's se...
Definition: composite_iterators.h:111
void UnlockRow() override
Definition: composite_iterators.h:154
LimitOffsetIterator(THD *thd, unique_ptr_destroy_only< RowIterator > source, ha_rows limit, ha_rows offset, bool count_all_rows, bool reject_multiple_rows, ha_rows *skipped_rows)
Definition: composite_iterators.h:127
bool Init() override
Initialize or reinitialize the iterator.
Definition: composite_iterators.cc:120
ha_rows m_seen_rows
Definition: composite_iterators.h:162
const bool m_count_all_rows
Definition: composite_iterators.h:170
ha_rows * m_skipped_rows
Definition: composite_iterators.h:172
void EndPSIBatchModeIfStarted() override
Ends performance schema batch mode, if started.
Definition: composite_iterators.h:151
const ha_rows m_limit
Definition: composite_iterators.h:169
const ha_rows m_offset
Definition: composite_iterators.h:169
bool m_needs_offset
Whether we have OFFSET rows that we still need to skip.
Definition: composite_iterators.h:167
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...
Definition: composite_iterators.h:146
unique_ptr_destroy_only< RowIterator > m_source
Definition: composite_iterators.h:157
int Read() override
Read a single row.
Definition: composite_iterators.cc:134
const bool m_reject_multiple_rows
Definition: composite_iterators.h:171
void StartPSIBatchMode() override
Start performance schema batch mode, if supported (otherwise ignored).
Definition: composite_iterators.h:150
MaterializeInformationSchemaTableIterator makes sure a given I_S temporary table is materialized (fil...
Definition: composite_iterators.h:835
Item * m_condition
Definition: composite_iterators.h:861
void UnlockRow() override
Definition: composite_iterators.h:855
MaterializeInformationSchemaTableIterator(THD *thd, unique_ptr_destroy_only< RowIterator > table_iterator, Table_ref *table_list, Item *condition)
Definition: composite_iterators.cc:4438
Table_ref * m_table_list
Definition: composite_iterators.h:860
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...
Definition: composite_iterators.h:844
void EndPSIBatchModeIfStarted() override
Ends performance schema batch mode, if started.
Definition: composite_iterators.h:849
int Read() override
Read a single row.
Definition: composite_iterators.h:842
void StartPSIBatchMode() override
Start performance schema batch mode, if supported (otherwise ignored).
Definition: composite_iterators.h:848
unique_ptr_destroy_only< RowIterator > m_table_iterator
The iterator that reads from the materialized table.
Definition: composite_iterators.h:859
bool Init() override
Initialize or reinitialize the iterator.
Definition: composite_iterators.cc:4446
An iterator that wraps a Table_function (e.g.
Definition: composite_iterators.h:623
Table_function * m_table_function
Definition: composite_iterators.h:647
unique_ptr_destroy_only< RowIterator > m_table_iterator
Definition: composite_iterators.h:645
void StartPSIBatchMode() override
Start performance schema batch mode, if supported (otherwise ignored).
Definition: composite_iterators.h:635
void UnlockRow() override
Definition: composite_iterators.h:642
int Read() override
Read a single row.
Definition: composite_iterators.h:630
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...
Definition: composite_iterators.h:631
void EndPSIBatchModeIfStarted() override
Ends performance schema batch mode, if started.
Definition: composite_iterators.h:636
MaterializedTableFunctionIterator(THD *thd, Table_function *table_function, TABLE *table, unique_ptr_destroy_only< RowIterator > table_iterator)
Definition: composite_iterators.cc:4195
bool Init() override
Initialize or reinitialize the iterator.
Definition: composite_iterators.cc:4202
A typesafe replacement for DYNAMIC_ARRAY.
Definition: mem_root_array.h:432
A simple nested loop join, taking in two iterators (left/outer and right/inner) and joining them toge...
Definition: composite_iterators.h:332
bool Init() override
Initialize or reinitialize the iterator.
Definition: composite_iterators.cc:494
@ END_OF_ROWS
Definition: composite_iterators.h:382
@ READING_INNER_ROWS
Definition: composite_iterators.h:381
@ NEEDS_OUTER_ROW
Definition: composite_iterators.h:379
@ READING_FIRST_INNER_ROW
Definition: composite_iterators.h:380
int Read() override
Read a single row.
Definition: composite_iterators.cc:505
void UnlockRow() override
Definition: composite_iterators.h:368
const bool m_pfs_batch_mode
Whether to use batch mode when scanning the inner iterator.
Definition: composite_iterators.h:390
NestedLoopIterator(THD *thd, unique_ptr_destroy_only< RowIterator > source_outer, unique_ptr_destroy_only< RowIterator > source_inner, JoinType join_type, bool pfs_batch_mode)
Definition: composite_iterators.h:334
unique_ptr_destroy_only< RowIterator > const m_source_inner
Definition: composite_iterators.h:386
void EndPSIBatchModeIfStarted() override
Ends performance schema batch mode, if started.
Definition: composite_iterators.h:363
const JoinType m_join_type
Definition: composite_iterators.h:387
enum NestedLoopIterator::@63 m_state
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...
Definition: composite_iterators.h:357
unique_ptr_destroy_only< RowIterator > const m_source_outer
Definition: composite_iterators.h:385
An iterator that is semantically equivalent to a semijoin NestedLoopIterator immediately followed by ...
Definition: composite_iterators.h:794
int Read() override
Read a single row.
Definition: composite_iterators.cc:4382
void UnlockRow() override
Definition: composite_iterators.h:815
KEY * m_key
Definition: composite_iterators.h:825
unique_ptr_destroy_only< RowIterator > const m_source_outer
Definition: composite_iterators.h:821
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...
Definition: composite_iterators.h:805
const size_t m_key_len
Definition: composite_iterators.h:827
bool m_deduplicate_against_previous_row
Definition: composite_iterators.h:828
uchar * m_key_buf
Definition: composite_iterators.h:826
unique_ptr_destroy_only< RowIterator > const m_source_inner
Definition: composite_iterators.h:822
NestedLoopSemiJoinWithDuplicateRemovalIterator(THD *thd, unique_ptr_destroy_only< RowIterator > source_outer, unique_ptr_destroy_only< RowIterator > source_inner, const TABLE *table, KEY *key, size_t key_len)
Definition: composite_iterators.cc:4359
void EndPSIBatchModeIfStarted() override
Ends performance schema batch mode, if started.
Definition: composite_iterators.h:810
const TABLE * m_table_outer
Definition: composite_iterators.h:824
bool Init() override
Initialize or reinitialize the iterator.
Definition: composite_iterators.cc:4374
An iterator that removes consecutive rows that are the same according to a set of items (typically th...
Definition: composite_iterators.h:702
void StartPSIBatchMode() override
Start performance schema batch mode, if supported (otherwise ignored).
Definition: composite_iterators.h:716
void EndPSIBatchModeIfStarted() override
Ends performance schema batch mode, if started.
Definition: composite_iterators.h:717
void UnlockRow() override
Definition: composite_iterators.h:720
int Read() override
Read a single row.
Definition: composite_iterators.cc:4293
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...
Definition: composite_iterators.h:712
bool m_first_row
Definition: composite_iterators.h:725
bool Init() override
Initialize or reinitialize the iterator.
Definition: composite_iterators.cc:4288
Bounds_checked_array< Cached_item * > m_caches
Definition: composite_iterators.h:724
RemoveDuplicatesIterator(THD *thd, unique_ptr_destroy_only< RowIterator > source, JOIN *join, Item **group_items, int group_items_size)
Definition: composite_iterators.cc:4276
unique_ptr_destroy_only< RowIterator > m_source
Definition: composite_iterators.h:723
Much like RemoveDuplicatesIterator, but works on the basis of a given index (or more accurately,...
Definition: composite_iterators.h:734
bool Init() override
Initialize or reinitialize the iterator.
Definition: composite_iterators.cc:4330
void UnlockRow() override
Definition: composite_iterators.h:751
uchar * m_key_buf
Definition: composite_iterators.h:757
bool m_first_row
Definition: composite_iterators.h:759
const TABLE * m_table
Definition: composite_iterators.h:755
void StartPSIBatchMode() override
Start performance schema batch mode, if supported (otherwise ignored).
Definition: composite_iterators.h:747
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...
Definition: composite_iterators.h:743
const size_t m_key_len
Definition: composite_iterators.h:758
void EndPSIBatchModeIfStarted() override
Ends performance schema batch mode, if started.
Definition: composite_iterators.h:748
RemoveDuplicatesOnIndexIterator(THD *thd, unique_ptr_destroy_only< RowIterator > source, const TABLE *table, KEY *key, size_t key_len)
Definition: composite_iterators.cc:4320
KEY * m_key
Definition: composite_iterators.h:756
int Read() override
Read a single row.
Definition: composite_iterators.cc:4335
unique_ptr_destroy_only< RowIterator > m_source
Definition: composite_iterators.h:754
A context for reading through a single table using a chosen access method: index read,...
Definition: row_iterator.h:82
THD * thd() const
Definition: row_iterator.h:228
Definition: sql_executor.h:95
StreamingIterator is a minimal version of MaterializeIterator that does not actually materialize; ins...
Definition: composite_iterators.h:568
JOIN *const m_join
Definition: composite_iterators.h:603
void UnlockRow() override
Definition: composite_iterators.h:597
void StartPSIBatchMode() override
Start performance schema batch mode, if supported (otherwise ignored).
Definition: composite_iterators.h:591
int Read() override
Read a single row.
Definition: composite_iterators.cc:3787
StreamingIterator(THD *thd, unique_ptr_destroy_only< RowIterator > subquery_iterator, Temp_table_param *temp_table_param, TABLE *table, bool provide_rowid, JOIN *join, int ref_slice)
Definition: composite_iterators.cc:3735
Temp_table_param * m_temp_table_param
Definition: composite_iterators.h:601
const bool m_provide_rowid
Definition: composite_iterators.h:610
const int m_output_slice
Definition: composite_iterators.h:604
void EndPSIBatchModeIfStarted() override
Ends performance schema batch mode, if started.
Definition: composite_iterators.h:594
bool Init() override
Initialize or reinitialize the iterator.
Definition: composite_iterators.cc:3764
int m_input_slice
Definition: composite_iterators.h:605
unique_ptr_destroy_only< RowIterator > m_subquery_iterator
Definition: composite_iterators.h:600
ha_rows m_row_number
Definition: composite_iterators.h:602
Using this class is fraught with peril, and you need to be very careful when doing so.
Definition: sql_string.h:167
For each client connection we create a separate thread with THD serving as a thread/connection descri...
Definition: sql_lexer_thd.h:36
Definition: row_iterator.h:234
TABLE * table() const
Definition: row_iterator.h:246
Class representing a table function.
Definition: table_function.h:53
Definition: table.h:2900
Object containing parameters used when creating and using temporary tables.
Definition: temp_table_param.h:97
Like semijoin materialization, weedout works on the basic idea that a semijoin is just like an inner ...
Definition: composite_iterators.h:669
unique_ptr_destroy_only< RowIterator > m_source
Definition: composite_iterators.h:687
void EndPSIBatchModeIfStarted() override
Ends performance schema batch mode, if started.
Definition: composite_iterators.h:681
WeedoutIterator(THD *thd, unique_ptr_destroy_only< RowIterator > source, SJ_TMP_TABLE *sj, table_map tables_to_get_rowid_for)
Definition: composite_iterators.cc:4215
int Read() override
Read a single row.
Definition: composite_iterators.cc:4245
const table_map m_tables_to_get_rowid_for
Definition: composite_iterators.h:689
SJ_TMP_TABLE * m_sj
Definition: composite_iterators.h:688
void UnlockRow() override
Definition: composite_iterators.h:684
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...
Definition: composite_iterators.h:677
bool Init() override
Initialize or reinitialize the iterator.
Definition: composite_iterators.cc:4228
A structure that contains a list of input tables for a hash join operation, BKA join operation or a s...
Definition: pack_rows.h:84
JoinType
Definition: join_type.h:28
@ ANTI
Left antijoin, i.e.
@ SEMI
Left semijoin, i.e.
This file follows Google coding style, except for the name MEM_ROOT (which is kept for historical rea...
std::unique_ptr< T, Destroy_only< T > > unique_ptr_destroy_only
std::unique_ptr, but only destroying.
Definition: my_alloc.h:480
This file includes constants used by all storage engines.
my_off_t ha_rows
Definition: my_base.h:1141
Some integer typedefs for easier portability.
unsigned long long int ulonglong
Definition: my_inttypes.h:56
unsigned char uchar
Definition: my_inttypes.h:52
uint64_t table_map
Definition: my_table_map.h:30
static PFS_engine_table_share_proxy table
Definition: pfs.cc:61
Definition: composite_iterators.h:436
RowIterator * CreateIterator(THD *thd, Mem_root_array< materialize_iterator::Operand > operands, const MaterializePathParameters *path_params, unique_ptr_destroy_only< RowIterator > table_iterator, JOIN *join)
Create an iterator that materializes a set of row into a temporary table and sets up a (pre-existing)...
Definition: composite_iterators.cc:3712
std::string join(const detail::range auto &rng, std::string_view delim)
join elements of a range into a string separated by a delimiter.
Definition: string.h:74
Definition: gcs_xcom_synode.h:64
Definition: composite_iterators.h:530
RowIterator * CreateIterator(THD *thd, unique_ptr_destroy_only< RowIterator > subquery_iterator, Temp_table_param *temp_table_param, TABLE *table, unique_ptr_destroy_only< RowIterator > table_iterator, JOIN *join, int ref_slice)
Create an iterator that aggregates the output rows from another iterator into a temporary table and t...
Definition: composite_iterators.cc:4167
std::vector< T, ut::allocator< T > > vector
Specialization of vector which uses allocator.
Definition: ut0new.h:2876
Generic routines for packing rows (possibly from multiple tables at the same time) into strings,...
required string key
Definition: replication_asynchronous_connection_failover.proto:60
repeated Source source
Definition: replication_asynchronous_connection_failover.proto:42
join_type
Definition: sql_opt_exec_shared.h:186
Our own string classes, used pervasively throughout the executor.
Definition: materialize_path_parameters.h:40
Definition: table.h:1421
An operand (query block) to be materialized by MaterializeIterator.
Definition: composite_iterators.h:441
unique_ptr_destroy_only< RowIterator > subquery_iterator
The iterator to read the actual rows from.
Definition: composite_iterators.h:443
bool copy_items
If set to false, the Field objects in the output row are presumed already to be filled out.
Definition: composite_iterators.h:471
Temp_table_param * temp_table_param
If copy_items is true, used for copying the Field objects into the temporary table row.
Definition: composite_iterators.h:487
double m_estimated_output_rows
The estimated number of rows produced by this block.
Definition: composite_iterators.h:501
ulonglong m_operand_idx
The current operand (i.e.
Definition: composite_iterators.h:478
bool is_recursive_reference
Definition: composite_iterators.h:491
FollowTailIterator * recursive_reader
Definition: composite_iterators.h:498
int select_number
Used only for optimizer trace.
Definition: composite_iterators.h:446
ulonglong m_total_operands
The number of operands (i.e.
Definition: composite_iterators.h:475
uint m_first_distinct
Used for EXCEPT computation: the index of the first operand involved in a N-ary except operation whic...
Definition: composite_iterators.h:483
bool disable_deduplication_by_hash_field
If true, de-duplication checking via hash key is disabled when materializing this query block (ie....
Definition: composite_iterators.h:466
JOIN * join
The JOIN that this query block represents.
Definition: composite_iterators.h:453