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

     1  // Copyright 2020 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  	"context"
    19  	"io"
    20  
    21  	"github.com/dolthub/go-mysql-server/sql"
    22  	"golang.org/x/sync/errgroup"
    23  
    24  	"github.com/dolthub/dolt/go/libraries/doltcore/doltdb/durable"
    25  	"github.com/dolthub/dolt/go/store/prolly"
    26  	"github.com/dolthub/dolt/go/store/prolly/tree"
    27  	"github.com/dolthub/dolt/go/store/val"
    28  )
    29  
    30  const indexLookupBufSize = 8
    31  
    32  type prollyIndexIter struct {
    33  	idx       DoltIndex
    34  	indexIter prolly.MapIter
    35  	primary   prolly.Map
    36  
    37  	// pkMap transforms indexRows index keys
    38  	// into primary index keys
    39  	pkMap val.OrdinalMapping
    40  	pkBld *val.TupleBuilder
    41  
    42  	// keyMap and valMap transform tuples from
    43  	// primary row storage into sql.Row's
    44  	keyMap, valMap val.OrdinalMapping
    45  	// ordMap are output ordinals for |keyMap| and |valMap| concatenated
    46  	ordMap      val.OrdinalMapping
    47  	projections []uint64
    48  	sqlSch      sql.Schema
    49  }
    50  
    51  var _ sql.RowIter = prollyIndexIter{}
    52  
    53  // NewProllyIndexIter returns a new prollyIndexIter.
    54  func newProllyIndexIter(
    55  	ctx *sql.Context,
    56  	idx DoltIndex,
    57  	rng prolly.Range,
    58  	pkSch sql.PrimaryKeySchema,
    59  	projections []uint64,
    60  	dprimary, dsecondary durable.Index,
    61  ) (prollyIndexIter, error) {
    62  	secondary := durable.ProllyMapFromIndex(dsecondary)
    63  	indexIter, err := secondary.IterRange(ctx, rng)
    64  	if err != nil {
    65  		return prollyIndexIter{}, err
    66  	}
    67  
    68  	primary := durable.ProllyMapFromIndex(dprimary)
    69  	kd, _ := primary.Descriptors()
    70  	pkBld := val.NewTupleBuilder(kd)
    71  	pkMap := ordinalMappingFromIndex(idx)
    72  	keyProj, valProj, ordProj := projectionMappings(idx.Schema(), projections)
    73  
    74  	iter := prollyIndexIter{
    75  		idx:         idx,
    76  		indexIter:   indexIter,
    77  		primary:     primary,
    78  		pkBld:       pkBld,
    79  		pkMap:       pkMap,
    80  		keyMap:      keyProj,
    81  		valMap:      valProj,
    82  		ordMap:      ordProj,
    83  		projections: projections,
    84  		sqlSch:      pkSch.Schema,
    85  	}
    86  
    87  	return iter, nil
    88  }
    89  
    90  // Next returns the next row from the iterator.
    91  func (p prollyIndexIter) Next(ctx *sql.Context) (sql.Row, error) {
    92  	idxKey, _, err := p.indexIter.Next(ctx)
    93  	if err != nil {
    94  		return nil, err
    95  	}
    96  	for to := range p.pkMap {
    97  		from := p.pkMap.MapOrdinal(to)
    98  		p.pkBld.PutRaw(to, idxKey.GetField(from))
    99  	}
   100  	pk := p.pkBld.Build(sharePool)
   101  
   102  	r := make(sql.Row, len(p.projections))
   103  	err = p.primary.Get(ctx, pk, func(key, value val.Tuple) error {
   104  		return p.rowFromTuples(ctx, key, value, r)
   105  	})
   106  	if err != nil {
   107  		return nil, err
   108  	}
   109  	return r, nil
   110  }
   111  
   112  func (p prollyIndexIter) rowFromTuples(ctx context.Context, key, value val.Tuple, r sql.Row) (err error) {
   113  	keyDesc, valDesc := p.primary.Descriptors()
   114  
   115  	for i, idx := range p.keyMap {
   116  		outputIdx := p.ordMap[i]
   117  		r[outputIdx], err = tree.GetField(ctx, keyDesc, idx, key, p.primary.NodeStore())
   118  		if err != nil {
   119  			return err
   120  		}
   121  	}
   122  
   123  	for i, idx := range p.valMap {
   124  		outputIdx := p.ordMap[len(p.keyMap)+i]
   125  		r[outputIdx], err = tree.GetField(ctx, valDesc, idx, value, p.primary.NodeStore())
   126  		if err != nil {
   127  			return err
   128  		}
   129  	}
   130  
   131  	return
   132  }
   133  
   134  func (p prollyIndexIter) Close(*sql.Context) error {
   135  	return nil
   136  }
   137  
   138  func ordinalMappingFromIndex(idx DoltIndex) (m val.OrdinalMapping) {
   139  	def := idx.Schema().Indexes().GetByName(idx.ID())
   140  	pks := def.PrimaryKeyTags()
   141  	if len(pks) == 0 { // keyless index
   142  		m = make(val.OrdinalMapping, 1)
   143  		m[0] = len(def.AllTags())
   144  		return m
   145  	}
   146  
   147  	m = make(val.OrdinalMapping, len(pks))
   148  	for i, pk := range pks {
   149  		for j, tag := range def.AllTags() {
   150  			if tag == pk {
   151  				m[i] = j
   152  				break
   153  			}
   154  		}
   155  	}
   156  	return
   157  }
   158  
   159  type prollyCoveringIndexIter struct {
   160  	idx       DoltIndex
   161  	indexIter prolly.MapIter
   162  	keyDesc   val.TupleDesc
   163  	valDesc   val.TupleDesc
   164  
   165  	ns tree.NodeStore
   166  
   167  	// keyMap transforms secondary index key tuples into SQL tuples.
   168  	// secondary index value tuples are assumed to be empty.
   169  
   170  	// |keyMap| and |valMap| are both of len ==
   171  	keyMap, valMap, ordMap val.OrdinalMapping
   172  	projections            []uint64
   173  	sqlSch                 sql.Schema
   174  }
   175  
   176  var _ sql.RowIter = prollyCoveringIndexIter{}
   177  
   178  func newProllyCoveringIndexIter(
   179  	ctx *sql.Context,
   180  	idx DoltIndex,
   181  	rng prolly.Range,
   182  	pkSch sql.PrimaryKeySchema,
   183  	projections []uint64,
   184  	indexdata durable.Index,
   185  ) (prollyCoveringIndexIter, error) {
   186  	secondary := durable.ProllyMapFromIndex(indexdata)
   187  	indexIter, err := secondary.IterRange(ctx, rng)
   188  	if err != nil {
   189  		return prollyCoveringIndexIter{}, err
   190  	}
   191  	keyDesc, valDesc := secondary.Descriptors()
   192  
   193  	var keyMap, valMap, ordMap val.OrdinalMapping
   194  	if idx.IsPrimaryKey() {
   195  		keyMap, valMap, ordMap = primaryIndexMapping(idx, pkSch, projections)
   196  	} else {
   197  		keyMap, ordMap = coveringIndexMapping(idx, projections)
   198  	}
   199  
   200  	return prollyCoveringIndexIter{
   201  		idx:         idx,
   202  		indexIter:   indexIter,
   203  		keyDesc:     keyDesc,
   204  		valDesc:     valDesc,
   205  		keyMap:      keyMap,
   206  		valMap:      valMap,
   207  		ordMap:      ordMap,
   208  		sqlSch:      pkSch.Schema,
   209  		projections: projections,
   210  		ns:          secondary.NodeStore(),
   211  	}, nil
   212  }
   213  
   214  // Next returns the next row from the iterator.
   215  func (p prollyCoveringIndexIter) Next(ctx *sql.Context) (sql.Row, error) {
   216  	k, v, err := p.indexIter.Next(ctx)
   217  	if err != nil {
   218  		return nil, err
   219  	}
   220  
   221  	r := make(sql.Row, len(p.projections))
   222  	if err := p.writeRowFromTuples(ctx, k, v, r); err != nil {
   223  		return nil, err
   224  	}
   225  
   226  	return r, nil
   227  }
   228  
   229  func (p prollyCoveringIndexIter) writeRowFromTuples(ctx context.Context, key, value val.Tuple, r sql.Row) (err error) {
   230  	for i, idx := range p.keyMap {
   231  		outputIdx := p.ordMap[i]
   232  		r[outputIdx], err = tree.GetField(ctx, p.keyDesc, idx, key, p.ns)
   233  		if err != nil {
   234  			return err
   235  		}
   236  	}
   237  
   238  	for i, idx := range p.valMap {
   239  		outputIdx := p.ordMap[len(p.keyMap)+i]
   240  		r[outputIdx], err = tree.GetField(ctx, p.valDesc, idx, value, p.ns)
   241  		if err != nil {
   242  			return err
   243  		}
   244  	}
   245  	return
   246  }
   247  
   248  func (p prollyCoveringIndexIter) Close(*sql.Context) error {
   249  	return nil
   250  }
   251  
   252  func coveringIndexMapping(d DoltIndex, projections []uint64) (keyMap, ordMap val.OrdinalMapping) {
   253  	idx := d.IndexSchema().GetAllCols()
   254  	allMap := make(val.OrdinalMapping, len(projections)*2)
   255  	var i int
   256  	for _, p := range projections {
   257  		if idx, ok := idx.StoredIndexByTag(p); ok {
   258  			allMap[i] = idx
   259  			allMap[len(projections)+i] = i
   260  			i++
   261  		}
   262  	}
   263  	keyMap = allMap[:len(projections)]
   264  	ordMap = allMap[len(projections):]
   265  	return
   266  }
   267  
   268  func primaryIndexMapping(idx DoltIndex, sqlSch sql.PrimaryKeySchema, projections []uint64) (keyProj, valProj, ordProj val.OrdinalMapping) {
   269  	return projectionMappingsForIndex(idx.Schema(), projections)
   270  }
   271  
   272  type prollyKeylessIndexIter struct {
   273  	idx       DoltIndex
   274  	indexIter prolly.MapIter
   275  	clustered prolly.Map
   276  
   277  	// clusteredMap transforms secondary index keys
   278  	// into clustered index keys
   279  	clusteredMap val.OrdinalMapping
   280  	clusteredBld *val.TupleBuilder
   281  
   282  	eg      *errgroup.Group
   283  	rowChan chan sql.Row
   284  
   285  	// valueMap transforms tuples from the
   286  	// clustered index into sql.Rows
   287  	valueMap  val.OrdinalMapping
   288  	ordMap    val.OrdinalMapping
   289  	valueDesc val.TupleDesc
   290  	sqlSch    sql.Schema
   291  }
   292  
   293  var _ sql.RowIter = prollyKeylessIndexIter{}
   294  
   295  func newProllyKeylessIndexIter(
   296  	ctx *sql.Context,
   297  	idx DoltIndex,
   298  	rng prolly.Range,
   299  	pkSch sql.PrimaryKeySchema,
   300  	projections []uint64,
   301  	rows, dsecondary durable.Index,
   302  ) (prollyKeylessIndexIter, error) {
   303  	secondary := durable.ProllyMapFromIndex(dsecondary)
   304  	indexIter, err := secondary.IterRange(ctx, rng)
   305  	if err != nil {
   306  		return prollyKeylessIndexIter{}, err
   307  	}
   308  
   309  	clustered := durable.ProllyMapFromIndex(rows)
   310  	keyDesc, valDesc := clustered.Descriptors()
   311  	indexMap := ordinalMappingFromIndex(idx)
   312  	keyBld := val.NewTupleBuilder(keyDesc)
   313  	sch := idx.Schema()
   314  	_, vm, om := projectionMappings(sch, projections)
   315  
   316  	eg, c := errgroup.WithContext(ctx)
   317  
   318  	iter := prollyKeylessIndexIter{
   319  		idx:          idx,
   320  		indexIter:    indexIter,
   321  		clustered:    clustered,
   322  		clusteredMap: indexMap,
   323  		clusteredBld: keyBld,
   324  		eg:           eg,
   325  		rowChan:      make(chan sql.Row, indexLookupBufSize),
   326  		valueMap:     vm,
   327  		ordMap:       om,
   328  		valueDesc:    valDesc,
   329  		sqlSch:       pkSch.Schema,
   330  	}
   331  
   332  	eg.Go(func() error {
   333  		return iter.queueRows(c)
   334  	})
   335  
   336  	return iter, nil
   337  }
   338  
   339  // Next returns the next row from the iterator.
   340  func (p prollyKeylessIndexIter) Next(ctx *sql.Context) (sql.Row, error) {
   341  	r, ok := <-p.rowChan
   342  	if ok {
   343  		return r, nil
   344  	}
   345  
   346  	if err := p.eg.Wait(); err != nil {
   347  		return nil, err
   348  	}
   349  
   350  	return nil, io.EOF
   351  }
   352  
   353  func (p prollyKeylessIndexIter) queueRows(ctx context.Context) error {
   354  	defer close(p.rowChan)
   355  
   356  	for {
   357  		idxKey, _, err := p.indexIter.Next(ctx)
   358  		if err != nil {
   359  			return err
   360  		}
   361  
   362  		for to := range p.clusteredMap {
   363  			from := p.clusteredMap.MapOrdinal(to)
   364  			p.clusteredBld.PutRaw(to, idxKey.GetField(from))
   365  		}
   366  		pk := p.clusteredBld.Build(sharePool)
   367  
   368  		var value val.Tuple
   369  		err = p.clustered.Get(ctx, pk, func(k, v val.Tuple) error {
   370  			value = v
   371  			return nil
   372  		})
   373  		if err != nil {
   374  			return err
   375  		}
   376  
   377  		rows, err := p.keylessRowsFromValueTuple(ctx, p.clustered.NodeStore(), value)
   378  		if err != nil {
   379  			return err
   380  		}
   381  
   382  		for i := range rows {
   383  			select {
   384  			case <-ctx.Done():
   385  				return ctx.Err()
   386  			case p.rowChan <- rows[i]:
   387  			}
   388  		}
   389  	}
   390  }
   391  
   392  func (p prollyKeylessIndexIter) keylessRowsFromValueTuple(ctx context.Context, ns tree.NodeStore, value val.Tuple) (rows []sql.Row, err error) {
   393  	card := val.ReadKeylessCardinality(value)
   394  	rows = make([]sql.Row, card)
   395  	rows[0] = make(sql.Row, len(p.valueMap))
   396  
   397  	for i, idx := range p.valueMap {
   398  		outputIdx := p.ordMap[i]
   399  		rows[0][outputIdx], err = tree.GetField(ctx, p.valueDesc, idx, value, ns)
   400  		if err != nil {
   401  			return nil, err
   402  		}
   403  	}
   404  
   405  	// duplicate row |card| times
   406  	for i := 1; i < len(rows); i++ {
   407  		rows[i] = rows[0].Copy()
   408  	}
   409  	return
   410  }
   411  
   412  func (p prollyKeylessIndexIter) Close(*sql.Context) error {
   413  	return nil
   414  }