github.com/dolthub/dolt/go@v0.40.5-0.20240520175717-68db7794bea6/libraries/doltcore/sqle/index/prolly_row_iter.go (about)

     1  // Copyright 2021 Dolthub, Inc.
     2  //
     3  // Licensed under the Apache License, Version 2.0 (the "License");
     4  // you may not use this file except in compliance with the License.
     5  // You may obtain a copy of the License at
     6  //
     7  //     http://www.apache.org/licenses/LICENSE-2.0
     8  //
     9  // Unless required by applicable law or agreed to in writing, software
    10  // distributed under the License is distributed on an "AS IS" BASIS,
    11  // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    12  // See the License for the specific language governing permissions and
    13  // limitations under the License.
    14  
    15  package index
    16  
    17  import (
    18  	"github.com/dolthub/go-mysql-server/sql"
    19  
    20  	"github.com/dolthub/dolt/go/libraries/doltcore/schema"
    21  	"github.com/dolthub/dolt/go/store/prolly"
    22  	"github.com/dolthub/dolt/go/store/prolly/tree"
    23  	"github.com/dolthub/dolt/go/store/val"
    24  )
    25  
    26  type prollyRowIter struct {
    27  	iter prolly.MapIter
    28  	ns   tree.NodeStore
    29  
    30  	keyDesc val.TupleDesc
    31  	valDesc val.TupleDesc
    32  
    33  	keyProj []int
    34  	valProj []int
    35  	// ordProj is a concatenated list of output ordinals for |keyProj| and |valProj|
    36  	ordProj []int
    37  	rowLen  int
    38  }
    39  
    40  var _ sql.RowIter = prollyRowIter{}
    41  
    42  func NewProllyRowIterForMap(sch schema.Schema, rows prolly.Map, iter prolly.MapIter, projections []uint64) sql.RowIter {
    43  	if projections == nil {
    44  		projections = sch.GetAllCols().Tags
    45  	}
    46  
    47  	kd, vd := rows.Descriptors()
    48  	ns := rows.NodeStore()
    49  
    50  	return NewProllyRowIterForSchema(sch, iter, kd, vd, projections, ns)
    51  }
    52  
    53  func NewProllyRowIterForSchema(
    54  	sch schema.Schema,
    55  	iter prolly.MapIter,
    56  	kd val.TupleDesc,
    57  	vd val.TupleDesc,
    58  	projections []uint64,
    59  	ns tree.NodeStore,
    60  ) sql.RowIter {
    61  	if schema.IsKeyless(sch) {
    62  		return NewKeylessProllyRowIter(sch, iter, vd, projections, ns)
    63  	}
    64  
    65  	return NewKeyedProllyRowIter(sch, iter, kd, vd, projections, ns)
    66  }
    67  
    68  func NewKeyedProllyRowIter(
    69  	sch schema.Schema,
    70  	iter prolly.MapIter,
    71  	kd val.TupleDesc,
    72  	vd val.TupleDesc,
    73  	projections []uint64,
    74  	ns tree.NodeStore,
    75  ) sql.RowIter {
    76  	keyProj, valProj, ordProj := projectionMappings(sch, projections)
    77  
    78  	return prollyRowIter{
    79  		iter:    iter,
    80  		keyDesc: kd,
    81  		valDesc: vd,
    82  		keyProj: keyProj,
    83  		valProj: valProj,
    84  		ordProj: ordProj,
    85  		rowLen:  len(projections),
    86  		ns:      ns,
    87  	}
    88  }
    89  
    90  func NewKeylessProllyRowIter(
    91  	sch schema.Schema,
    92  	iter prolly.MapIter,
    93  	vd val.TupleDesc,
    94  	projections []uint64,
    95  	ns tree.NodeStore,
    96  ) sql.RowIter {
    97  	_, valProj, ordProj := projectionMappings(sch, projections)
    98  
    99  	return &prollyKeylessIter{
   100  		iter:    iter,
   101  		valDesc: vd,
   102  		valProj: valProj,
   103  		ordProj: ordProj,
   104  		rowLen:  len(projections),
   105  		ns:      ns,
   106  	}
   107  }
   108  
   109  // projectionMappings returns data structures that specify 1) which fields we read
   110  // from key and value tuples, and 2) the position of those fields in the output row.
   111  func projectionMappings(sch schema.Schema, projections []uint64) (keyMap, valMap, ordMap val.OrdinalMapping) {
   112  	keyMap, valMap, ordMap = projectionMappingsForIndex(sch, projections)
   113  	adjustOffsetsForKeylessTable(sch, keyMap, valMap)
   114  	return keyMap, valMap, ordMap
   115  }
   116  
   117  func adjustOffsetsForKeylessTable(sch schema.Schema, keyMap val.OrdinalMapping, valMap val.OrdinalMapping) {
   118  	if schema.IsKeyless(sch) {
   119  		// skip the cardinality value, increment every index
   120  		for i := range keyMap {
   121  			keyMap[i]++
   122  		}
   123  		for i := range valMap {
   124  			valMap[i]++
   125  		}
   126  	}
   127  }
   128  
   129  func projectionMappingsForIndex(sch schema.Schema, projections []uint64) (keyMap, valMap, ordMap val.OrdinalMapping) {
   130  	pks := sch.GetPKCols()
   131  	nonPks := sch.GetNonPKCols()
   132  
   133  	numPhysicalColumns := len(projections)
   134  	if schema.IsVirtual(sch) {
   135  		numPhysicalColumns = 0
   136  		for _, t := range projections {
   137  			if idx, ok := sch.GetAllCols().TagToIdx[t]; ok && !sch.GetAllCols().GetByIndex(idx).Virtual {
   138  				numPhysicalColumns++
   139  			}
   140  		}
   141  	}
   142  
   143  	// Build a slice of positional values. For a set of P projections, for K key columns and N=P-K non-key columns,
   144  	// we'll generate a slice 2P long structured as follows:
   145  	// [K key projections, // list of tuple indexes to read for key columns
   146  	//  N non-key projections, // list of tuple indexes to read for non-key columns, ordered backward from end
   147  	//  P output ordinals]  // list of output column ordinals for each projection
   148  	// Afterward we slice this into three separate mappings to return.
   149  	allMap := make([]int, 2*numPhysicalColumns)
   150  	keyIdx := 0
   151  	nonKeyIdx := numPhysicalColumns - 1
   152  	for projNum, tag := range projections {
   153  		if idx, ok := pks.StoredIndexByTag(tag); ok && !pks.GetByStoredIndex(idx).Virtual {
   154  			allMap[keyIdx] = idx
   155  			allMap[numPhysicalColumns+keyIdx] = projNum
   156  			keyIdx++
   157  		} else if idx, ok := nonPks.StoredIndexByTag(tag); ok && !nonPks.GetByStoredIndex(idx).Virtual {
   158  			allMap[nonKeyIdx] = idx
   159  			allMap[numPhysicalColumns+nonKeyIdx] = projNum
   160  			nonKeyIdx--
   161  		}
   162  	}
   163  
   164  	keyMap = allMap[:keyIdx]
   165  	valMap = allMap[keyIdx:numPhysicalColumns]
   166  	ordMap = allMap[numPhysicalColumns:]
   167  	return keyMap, valMap, ordMap
   168  }
   169  
   170  func (it prollyRowIter) Next(ctx *sql.Context) (sql.Row, error) {
   171  	key, value, err := it.iter.Next(ctx)
   172  	if err != nil {
   173  		return nil, err
   174  	}
   175  
   176  	row := make(sql.Row, it.rowLen)
   177  	for i, idx := range it.keyProj {
   178  		outputIdx := it.ordProj[i]
   179  		row[outputIdx], err = tree.GetField(ctx, it.keyDesc, idx, key, it.ns)
   180  		if err != nil {
   181  			return nil, err
   182  		}
   183  	}
   184  	for i, idx := range it.valProj {
   185  		outputIdx := it.ordProj[len(it.keyProj)+i]
   186  		row[outputIdx], err = tree.GetField(ctx, it.valDesc, idx, value, it.ns)
   187  		if err != nil {
   188  			return nil, err
   189  		}
   190  	}
   191  	return row, nil
   192  }
   193  
   194  func (it prollyRowIter) Close(ctx *sql.Context) error {
   195  	return nil
   196  }
   197  
   198  type prollyKeylessIter struct {
   199  	iter prolly.MapIter
   200  	ns   tree.NodeStore
   201  
   202  	valDesc val.TupleDesc
   203  	valProj []int
   204  	ordProj []int
   205  	rowLen  int
   206  
   207  	curr sql.Row
   208  	card uint64
   209  }
   210  
   211  var _ sql.RowIter = &prollyKeylessIter{}
   212  
   213  //var _ sql.RowIter2 = prollyKeylessIter{}
   214  
   215  func (it *prollyKeylessIter) Next(ctx *sql.Context) (sql.Row, error) {
   216  	if it.card == 0 {
   217  		if err := it.nextTuple(ctx); err != nil {
   218  			return nil, err
   219  		}
   220  	}
   221  
   222  	it.card--
   223  
   224  	return it.curr, nil
   225  }
   226  
   227  func (it *prollyKeylessIter) nextTuple(ctx *sql.Context) error {
   228  	_, value, err := it.iter.Next(ctx)
   229  	if err != nil {
   230  		return err
   231  	}
   232  
   233  	it.card = val.ReadKeylessCardinality(value)
   234  	it.curr = make(sql.Row, it.rowLen)
   235  
   236  	for i, idx := range it.valProj {
   237  		outputIdx := it.ordProj[i]
   238  		it.curr[outputIdx], err = tree.GetField(ctx, it.valDesc, idx, value, it.ns)
   239  		if err != nil {
   240  			return err
   241  		}
   242  	}
   243  	return nil
   244  }
   245  
   246  func (it *prollyKeylessIter) Close(ctx *sql.Context) error {
   247  	return nil
   248  }