github.com/grafana/pyroscope@v1.18.0/pkg/phlaredb/query/iters.go (about)

     1  package query
     2  
     3  import (
     4  	"bytes"
     5  	"context"
     6  	"fmt"
     7  	"io"
     8  	"math"
     9  	"strings"
    10  	"sync"
    11  
    12  	"github.com/grafana/dskit/multierror"
    13  	"github.com/opentracing/opentracing-go"
    14  	"github.com/opentracing/opentracing-go/log"
    15  	"github.com/parquet-go/parquet-go"
    16  
    17  	"github.com/grafana/pyroscope/pkg/iter"
    18  )
    19  
    20  const MaxDefinitionLevel = 5
    21  
    22  // RowNumber is the sequence of row numbers uniquely identifying a value
    23  // in a tree of nested columns, starting at the top-level and including
    24  // another row number for each level of nesting. -1 is a placeholder
    25  // for undefined at lower levels.  RowNumbers can be compared for full
    26  // equality using the == operator, or can be compared partially, looking
    27  // for equal lineages down to a certain level.
    28  // For example given the following tree, the row numbers would be:
    29  //
    30  //	A          0, -1, -1
    31  //	  B        0,  0, -1
    32  //	  C        0,  1, -1
    33  //	    D      0,  1,  0
    34  //	  E        0,  2, -1
    35  //
    36  // Currently supports 6 levels of nesting which should be enough for anybody. :)
    37  type RowNumber [MaxDefinitionLevel + 1]int64
    38  
    39  type RowNumberWithDefinitionLevel struct {
    40  	RowNumber       RowNumber
    41  	DefinitionLevel int
    42  }
    43  
    44  // EmptyRowNumber creates an empty invalid row number.
    45  func EmptyRowNumber() RowNumber {
    46  	return RowNumber{-1, -1, -1, -1, -1, -1}
    47  }
    48  
    49  // MaxRowNumber is a helper that represents the maximum(-ish) representable value.
    50  func MaxRowNumber() RowNumber {
    51  	return RowNumber{math.MaxInt64}
    52  }
    53  
    54  // CompareRowNumbers compares the sequences of row numbers in
    55  // a and b for partial equality, descending from top-level
    56  // through the given definition level.
    57  // For example, definition level 1 means that row numbers are compared
    58  // at two levels of nesting, the top-level and 1 level of nesting
    59  // below.
    60  func CompareRowNumbers(upToDefinitionLevel int, a, b RowNumber) int {
    61  	for i := 0; i <= upToDefinitionLevel; i++ {
    62  		if a[i] < b[i] {
    63  			return -1
    64  		}
    65  		if a[i] > b[i] {
    66  			return 1
    67  		}
    68  	}
    69  	return 0
    70  }
    71  
    72  func TruncateRowNumber(t RowNumberWithDefinitionLevel) RowNumber {
    73  	n := EmptyRowNumber()
    74  	for i := 0; i <= t.DefinitionLevel; i++ {
    75  		n[i] = t.RowNumber[i]
    76  	}
    77  	return n
    78  }
    79  
    80  func (t *RowNumber) Valid() bool {
    81  	return t[0] >= 0
    82  }
    83  
    84  // Next increments and resets the row numbers according
    85  // to the given repetition and definition levels. Examples
    86  // from the Dremel whitepaper:
    87  // https://storage.googleapis.com/pub-tools-public-publication-data/pdf/36632.pdf
    88  // Name.Language.Country
    89  // value  | r | d | expected RowNumber
    90  // -------|---|---|-------------------
    91  //
    92  //	|   |   | { -1, -1, -1, -1 }  <-- starting position
    93  //
    94  // us     | 0 | 3 | {  0,  0,  0,  0 }
    95  // null   | 2 | 2 | {  0,  0,  1, -1 }
    96  // null   | 1 | 1 | {  0,  1, -1, -1 }
    97  // gb     | 1 | 3 | {  0,  2,  0,  0 }
    98  // null   | 0 | 1 | {  1,  0, -1, -1 }
    99  func (t *RowNumber) Next(repetitionLevel, definitionLevel int) {
   100  	t[repetitionLevel]++
   101  
   102  	// the following is nextSlow() unrolled
   103  	switch repetitionLevel {
   104  	case 0:
   105  		switch definitionLevel {
   106  		case 0:
   107  			t[1] = -1
   108  			t[2] = -1
   109  			t[3] = -1
   110  			t[4] = -1
   111  			t[5] = -1
   112  		case 1:
   113  			t[1] = 0
   114  			t[2] = -1
   115  			t[3] = -1
   116  			t[4] = -1
   117  			t[5] = -1
   118  		case 2:
   119  			t[1] = 0
   120  			t[2] = 0
   121  			t[3] = -1
   122  			t[4] = -1
   123  			t[5] = -1
   124  		case 3:
   125  			t[1] = 0
   126  			t[2] = 0
   127  			t[3] = 0
   128  			t[4] = -1
   129  			t[5] = -1
   130  		case 4:
   131  			t[1] = 0
   132  			t[2] = 0
   133  			t[3] = 0
   134  			t[4] = 0
   135  			t[5] = -1
   136  		case 5:
   137  			t[1] = 0
   138  			t[2] = 0
   139  			t[3] = 0
   140  			t[4] = 0
   141  			t[5] = 0
   142  		}
   143  	case 1:
   144  		switch definitionLevel {
   145  		case 0:
   146  			t[1] = -1
   147  			t[2] = -1
   148  			t[3] = -1
   149  			t[4] = -1
   150  			t[5] = -1
   151  		case 1:
   152  			t[2] = -1
   153  			t[3] = -1
   154  			t[4] = -1
   155  			t[5] = -1
   156  		case 2:
   157  			t[2] = 0
   158  			t[3] = -1
   159  			t[4] = -1
   160  			t[5] = -1
   161  		case 3:
   162  			t[2] = 0
   163  			t[3] = 0
   164  			t[4] = -1
   165  			t[5] = -1
   166  		case 4:
   167  			t[2] = 0
   168  			t[3] = 0
   169  			t[4] = 0
   170  			t[5] = -1
   171  		case 5:
   172  			t[2] = 0
   173  			t[3] = 0
   174  			t[4] = 0
   175  			t[5] = 0
   176  		}
   177  	case 2:
   178  		switch definitionLevel {
   179  		case 0:
   180  			t[1] = -1
   181  			t[2] = -1
   182  			t[3] = -1
   183  			t[4] = -1
   184  			t[5] = -1
   185  		case 1:
   186  			t[2] = -1
   187  			t[3] = -1
   188  			t[4] = -1
   189  			t[5] = -1
   190  		case 2:
   191  			t[3] = -1
   192  			t[4] = -1
   193  			t[5] = -1
   194  		case 3:
   195  			t[3] = 0
   196  			t[4] = -1
   197  			t[5] = -1
   198  		case 4:
   199  			t[3] = 0
   200  			t[4] = 0
   201  			t[5] = -1
   202  		case 5:
   203  			t[3] = 0
   204  			t[4] = 0
   205  			t[5] = 0
   206  		}
   207  	case 3:
   208  		switch definitionLevel {
   209  		case 0:
   210  			t[1] = -1
   211  			t[2] = -1
   212  			t[3] = -1
   213  			t[4] = -1
   214  			t[5] = -1
   215  		case 1:
   216  			t[2] = -1
   217  			t[3] = -1
   218  			t[4] = -1
   219  			t[5] = -1
   220  		case 2:
   221  			t[3] = -1
   222  			t[4] = -1
   223  			t[5] = -1
   224  		case 3:
   225  			t[4] = -1
   226  			t[5] = -1
   227  		case 4:
   228  			t[4] = 0
   229  			t[5] = -1
   230  		case 5:
   231  			t[4] = 0
   232  			t[5] = 0
   233  		}
   234  	case 4:
   235  		switch definitionLevel {
   236  		case 0:
   237  			t[1] = -1
   238  			t[2] = -1
   239  			t[3] = -1
   240  			t[4] = -1
   241  			t[5] = -1
   242  		case 1:
   243  			t[2] = -1
   244  			t[3] = -1
   245  			t[4] = -1
   246  			t[5] = -1
   247  		case 2:
   248  			t[3] = -1
   249  			t[4] = -1
   250  			t[5] = -1
   251  		case 3:
   252  			t[4] = -1
   253  			t[5] = -1
   254  		case 4:
   255  			t[5] = -1
   256  		case 5:
   257  			t[5] = 0
   258  		}
   259  	case 5:
   260  		switch definitionLevel {
   261  		case 0:
   262  			t[1] = -1
   263  			t[2] = -1
   264  			t[3] = -1
   265  			t[4] = -1
   266  			t[5] = -1
   267  		case 1:
   268  			t[2] = -1
   269  			t[3] = -1
   270  			t[4] = -1
   271  			t[5] = -1
   272  		case 2:
   273  			t[3] = -1
   274  			t[4] = -1
   275  			t[5] = -1
   276  		case 3:
   277  			t[4] = -1
   278  			t[5] = -1
   279  		case 4:
   280  			t[5] = -1
   281  		}
   282  	}
   283  }
   284  
   285  // nextSlow is the original implementation of next. it is kept to test against
   286  // the unrolled version above
   287  func (t *RowNumber) nextSlow(repetitionLevel, definitionLevel int) {
   288  	t[repetitionLevel]++
   289  
   290  	// New children up through the definition level
   291  	for i := repetitionLevel + 1; i <= definitionLevel; i++ {
   292  		t[i] = 0
   293  	}
   294  
   295  	// // Children past the definition level are undefined
   296  	for i := definitionLevel + 1; i < len(t); i++ {
   297  		t[i] = -1
   298  	}
   299  }
   300  
   301  // Skip rows at the root-level.
   302  func (t *RowNumber) Skip(numRows int64) {
   303  	t[0] += numRows
   304  	for i := 1; i < len(t); i++ {
   305  		t[i] = -1
   306  	}
   307  }
   308  
   309  // Preceding returns the largest representable row number that is immediately prior to this
   310  // one. Think of it like math.NextAfter but for segmented row numbers. Examples:
   311  //
   312  //		RowNumber 1000.0.0 (defined at 3 levels) is preceded by 999.max.max
   313  //	    RowNumber 1000.-1.-1 (defined at 1 level) is preceded by 999.-1.-1
   314  func (t RowNumber) Preceding() RowNumber {
   315  	for i := len(t) - 1; i >= 0; i-- {
   316  		switch t[i] {
   317  		case -1:
   318  			continue
   319  		case 0:
   320  			t[i] = math.MaxInt64
   321  		default:
   322  			t[i]--
   323  			return t
   324  		}
   325  	}
   326  	return t
   327  }
   328  
   329  // IteratorResult is a row of data with a row number and named columns of data.
   330  // Internally it has an unstructured list for efficient collection. The ToMap()
   331  // function can be used to make inspection easier.
   332  type IteratorResult struct {
   333  	RowNumber RowNumber
   334  	Entries   []struct {
   335  		k        string
   336  		V        parquet.Value
   337  		RowValue interface{}
   338  	}
   339  }
   340  
   341  func (r *IteratorResult) Reset() {
   342  	r.Entries = r.Entries[:0]
   343  }
   344  
   345  func (r *IteratorResult) Append(rr *IteratorResult) {
   346  	r.Entries = append(r.Entries, rr.Entries...)
   347  }
   348  
   349  func (r *IteratorResult) AppendValue(k string, v parquet.Value) {
   350  	r.Entries = append(r.Entries, struct {
   351  		k        string
   352  		V        parquet.Value
   353  		RowValue interface{}
   354  	}{k, v, nil})
   355  }
   356  
   357  // ToMap converts the unstructured list of data into a map containing an entry
   358  // for each column, and the lists of values.  The order of columns is
   359  // not preseved, but the order of values within each column is.
   360  func (r *IteratorResult) ToMap() map[string][]parquet.Value {
   361  	m := map[string][]parquet.Value{}
   362  	for _, e := range r.Entries {
   363  		m[e.k] = append(m[e.k], e.V)
   364  	}
   365  	return m
   366  }
   367  
   368  // Columns gets the values for each named column. The order of returned values
   369  // matches the order of names given. This is more efficient than converting to a map.
   370  func (r *IteratorResult) Columns(buffer [][]parquet.Value, names ...string) [][]parquet.Value {
   371  	if cap(buffer) < len(names) {
   372  		buffer = make([][]parquet.Value, len(names))
   373  	}
   374  	buffer = buffer[:len(names)]
   375  	for i := range buffer {
   376  		buffer[i] = buffer[i][:0]
   377  	}
   378  
   379  	for _, e := range r.Entries {
   380  		for i := range names {
   381  			if e.k == names[i] {
   382  				buffer[i] = append(buffer[i], e.V)
   383  				break
   384  			}
   385  		}
   386  	}
   387  	return buffer
   388  }
   389  
   390  func (r *IteratorResult) ColumnValue(name string) parquet.Value {
   391  	for _, e := range r.Entries {
   392  		if e.k == name {
   393  			return e.V
   394  		}
   395  	}
   396  	return parquet.Value{}
   397  }
   398  
   399  // iterator - Every iterator follows this interface and can be composed.
   400  type Iterator = iter.SeekIterator[*IteratorResult, RowNumberWithDefinitionLevel]
   401  
   402  func NewErrIterator(err error) Iterator {
   403  	return iter.NewErrSeekIterator[*IteratorResult, RowNumberWithDefinitionLevel](err)
   404  }
   405  
   406  var iteratorResultPool = sync.Pool{
   407  	New: func() interface{} {
   408  		return &IteratorResult{Entries: make([]struct {
   409  			k        string
   410  			V        parquet.Value
   411  			RowValue interface{}
   412  		}, 0, 10)} // For luck
   413  	},
   414  }
   415  
   416  func iteratorResultPoolGet() *IteratorResult {
   417  	res := iteratorResultPool.Get().(*IteratorResult)
   418  	return res
   419  }
   420  
   421  func iteratorResultPoolPut(r *IteratorResult) {
   422  	if r != nil {
   423  		r.Reset()
   424  		iteratorResultPool.Put(r)
   425  	}
   426  }
   427  
   428  type BinaryJoinIterator struct {
   429  	left            Iterator
   430  	right           Iterator
   431  	definitionLevel int
   432  
   433  	err error
   434  	res *IteratorResult
   435  }
   436  
   437  var _ Iterator = (*BinaryJoinIterator)(nil)
   438  
   439  func NewBinaryJoinIterator(definitionLevel int, left, right Iterator) *BinaryJoinIterator {
   440  	return &BinaryJoinIterator{
   441  		left:            left,
   442  		right:           right,
   443  		definitionLevel: definitionLevel,
   444  		res:             iteratorResultPoolGet(),
   445  	}
   446  }
   447  
   448  // nextOrSeek will use next if the iterator is exactly one row aways
   449  func (bj *BinaryJoinIterator) nextOrSeek(to RowNumberWithDefinitionLevel, it Iterator) bool {
   450  	oldResult := it.At()
   451  	defer iteratorResultPoolPut(oldResult)
   452  	// Seek when definition level is higher then 0, there is not previous iteration or when the difference between current position and to is not 1
   453  	if to.DefinitionLevel != 0 || oldResult == nil || to.RowNumber[0] != (oldResult.RowNumber[0]-1) {
   454  		return it.Seek(to)
   455  	}
   456  	return it.Next()
   457  }
   458  
   459  func (bj *BinaryJoinIterator) makeResult() {
   460  	bj.res.Reset()
   461  	bj.res.RowNumber = EmptyRowNumber()
   462  	bj.res.RowNumber[0] = bj.left.At().RowNumber[0]
   463  	bj.res.Append(bj.left.At())
   464  	bj.res.Append(bj.right.At())
   465  }
   466  
   467  func (bj *BinaryJoinIterator) Next() bool {
   468  	var r *IteratorResult
   469  	for {
   470  		if r != nil {
   471  			iteratorResultPoolPut(r)
   472  		}
   473  		r = bj.left.At()
   474  		if !bj.left.Next() {
   475  			bj.err = bj.left.Err()
   476  			return false
   477  		}
   478  
   479  		// now seek the right iterator to the left position
   480  		if !bj.nextOrSeek(RowNumberWithDefinitionLevel{bj.left.At().RowNumber, bj.definitionLevel}, bj.right) {
   481  			bj.err = bj.right.Err()
   482  			return false
   483  		}
   484  
   485  		if cmp := CompareRowNumbers(bj.definitionLevel, bj.left.At().RowNumber, bj.right.At().RowNumber); cmp == 0 {
   486  			// we have a found an element
   487  			bj.makeResult()
   488  			return true
   489  		} else if cmp < 0 {
   490  			// left is smaller, so we need to seek the left iterator to the right position
   491  			if !bj.nextOrSeek(RowNumberWithDefinitionLevel{bj.right.At().RowNumber, bj.definitionLevel}, bj.left) {
   492  				bj.err = bj.left.Err()
   493  				return false
   494  			}
   495  
   496  			if cmp := CompareRowNumbers(bj.definitionLevel, bj.left.At().RowNumber, bj.right.At().RowNumber); cmp == 0 {
   497  				bj.makeResult()
   498  				return true
   499  			}
   500  
   501  		} else {
   502  			panic(fmt.Sprintf(
   503  				"bug in iterator during join: the right iterator cannot be smaller than the left one, as it just has been Seeked beyond left=%v %T right=%v %T",
   504  				bj.left.At().RowNumber[0],
   505  				bj.left,
   506  				bj.right.At().RowNumber[0],
   507  				bj.right,
   508  			))
   509  		}
   510  	}
   511  }
   512  
   513  func (bj *BinaryJoinIterator) At() *IteratorResult {
   514  	return bj.res
   515  }
   516  
   517  func (bj *BinaryJoinIterator) Seek(to RowNumberWithDefinitionLevel) bool {
   518  	if !bj.left.Seek(to) {
   519  		bj.err = bj.left.Err()
   520  		return false
   521  	}
   522  	if !bj.right.Seek(to) {
   523  		bj.err = bj.right.Err()
   524  		return false
   525  	}
   526  
   527  	// if there is a match right away return true
   528  	if cmp := CompareRowNumbers(bj.definitionLevel, bj.left.At().RowNumber, bj.right.At().RowNumber); cmp == 0 {
   529  		bj.makeResult()
   530  		return true
   531  	}
   532  
   533  	// if not look for the next match
   534  	return bj.Next()
   535  }
   536  
   537  func (bj *BinaryJoinIterator) Close() error {
   538  	var merr multierror.MultiError
   539  	merr.Add(bj.left.Close())
   540  	merr.Add(bj.right.Close())
   541  	return merr.Err()
   542  }
   543  
   544  func (bj *BinaryJoinIterator) Err() error {
   545  	return bj.err
   546  }
   547  
   548  // UnionIterator produces all results for all given iterators.  When iterators
   549  // align to the same row, based on the configured definition level, then the results
   550  // are returned together. Else the next matching iterator is returned.
   551  type UnionIterator struct {
   552  	definitionLevel int
   553  	iters           []Iterator
   554  	peeks           []*IteratorResult
   555  	pred            GroupPredicate
   556  	result          *IteratorResult
   557  }
   558  
   559  var _ Iterator = (*UnionIterator)(nil)
   560  
   561  func NewUnionIterator(definitionLevel int, iters []Iterator, pred GroupPredicate) *UnionIterator {
   562  	j := UnionIterator{
   563  		definitionLevel: definitionLevel,
   564  		iters:           iters,
   565  		peeks:           make([]*IteratorResult, len(iters)),
   566  		pred:            pred,
   567  	}
   568  	return &j
   569  }
   570  
   571  func (u *UnionIterator) At() *IteratorResult {
   572  	return u.result
   573  }
   574  
   575  func (u *UnionIterator) Next() bool {
   576  	// Here is the algorithm for unions:  On each pass of the iterators
   577  	// we remember which ones are pointing at the earliest same row. The
   578  	// lowest iterators are then collected and a result is produced. Keep
   579  	// going until all iterators are exhausted.
   580  	for {
   581  		lowestRowNumber := MaxRowNumber()
   582  		lowestIters := make([]int, 0, len(u.iters))
   583  
   584  		for iterNum := range u.iters {
   585  			rn := u.peek(iterNum)
   586  
   587  			// If this iterator is exhausted go to the next one
   588  			if rn == nil {
   589  				continue
   590  			}
   591  
   592  			c := CompareRowNumbers(u.definitionLevel, rn.RowNumber, lowestRowNumber)
   593  			switch c {
   594  			case -1:
   595  				// New lowest
   596  				lowestIters = lowestIters[:0]
   597  				lowestRowNumber = rn.RowNumber
   598  				fallthrough
   599  
   600  			case 0:
   601  				// Same
   602  				lowestIters = append(lowestIters, iterNum)
   603  			}
   604  		}
   605  
   606  		// Consume lowest iterators
   607  		result := u.collect(lowestIters, lowestRowNumber)
   608  
   609  		// After each pass it is guaranteed to have found something
   610  		// from at least one iterator, or all are exhausted
   611  		if len(lowestIters) > 0 {
   612  			if u.pred != nil && !u.pred.KeepGroup(result) {
   613  				continue
   614  			}
   615  
   616  			u.result = result
   617  			return true
   618  		}
   619  
   620  		// All exhausted
   621  		u.result = nil
   622  		return false
   623  	}
   624  }
   625  
   626  func (u *UnionIterator) Seek(to RowNumberWithDefinitionLevel) bool {
   627  	to.RowNumber = TruncateRowNumber(to)
   628  	for iterNum, iter := range u.iters {
   629  		if p := u.peeks[iterNum]; p == nil || CompareRowNumbers(to.DefinitionLevel, p.RowNumber, to.RowNumber) == -1 {
   630  			if iter.Seek(to) {
   631  				u.peeks[iterNum] = iter.At()
   632  			} else {
   633  				u.peeks[iterNum] = nil
   634  			}
   635  		}
   636  	}
   637  	return u.Next()
   638  }
   639  
   640  func (u *UnionIterator) peek(iterNum int) *IteratorResult {
   641  	if u.peeks[iterNum] == nil {
   642  		if u.iters[iterNum].Next() {
   643  			u.peeks[iterNum] = u.iters[iterNum].At()
   644  		}
   645  	}
   646  	return u.peeks[iterNum]
   647  }
   648  
   649  // Collect data from the given iterators until they point at
   650  // the next row (according to the configured definition level)
   651  // or are exhausted.
   652  func (u *UnionIterator) collect(iterNums []int, rowNumber RowNumber) *IteratorResult {
   653  	result := iteratorResultPoolGet()
   654  	result.RowNumber = rowNumber
   655  
   656  	for _, iterNum := range iterNums {
   657  		for u.peeks[iterNum] != nil && CompareRowNumbers(u.definitionLevel, u.peeks[iterNum].RowNumber, rowNumber) == 0 {
   658  
   659  			result.Append(u.peeks[iterNum])
   660  
   661  			iteratorResultPoolPut(u.peeks[iterNum])
   662  
   663  			if u.iters[iterNum].Next() {
   664  				u.peeks[iterNum] = u.iters[iterNum].At()
   665  			}
   666  
   667  		}
   668  	}
   669  
   670  	return result
   671  }
   672  
   673  func (u *UnionIterator) Err() error {
   674  	for _, i := range u.iters {
   675  		if err := i.Err(); err != nil {
   676  			return err
   677  		}
   678  	}
   679  	return nil
   680  }
   681  
   682  func (u *UnionIterator) Close() error {
   683  	var merr multierror.MultiError
   684  	for _, i := range u.iters {
   685  		merr.Add(i.Close())
   686  	}
   687  	return merr.Err()
   688  }
   689  
   690  type GroupPredicate interface {
   691  	KeepGroup(*IteratorResult) bool
   692  }
   693  
   694  // KeyValueGroupPredicate takes key/value pairs and checks if the
   695  // group contains all of them. This is the only predicate/iterator
   696  // that is knowledgable about our trace or search contents. I'd like
   697  // to change that and make it generic, but it's quite complex and not
   698  // figured it out yet.
   699  type KeyValueGroupPredicate struct {
   700  	keys   [][]byte
   701  	vals   [][]byte
   702  	buffer [][]parquet.Value
   703  }
   704  
   705  var _ GroupPredicate = (*KeyValueGroupPredicate)(nil)
   706  
   707  func NewKeyValueGroupPredicate(keys, values []string) *KeyValueGroupPredicate {
   708  	// Pre-convert all to bytes
   709  	p := &KeyValueGroupPredicate{}
   710  	for _, k := range keys {
   711  		p.keys = append(p.keys, []byte(k))
   712  	}
   713  	for _, v := range values {
   714  		p.vals = append(p.vals, []byte(v))
   715  	}
   716  	return p
   717  }
   718  
   719  // KeepGroup checks if the given group contains all of the requested
   720  // key/value pairs.
   721  func (a *KeyValueGroupPredicate) KeepGroup(group *IteratorResult) bool {
   722  	a.buffer = group.Columns(a.buffer, "keys", "values")
   723  
   724  	keys, vals := a.buffer[0], a.buffer[1]
   725  
   726  	if len(keys) < len(a.keys) || len(keys) != len(vals) {
   727  		// Missing data or unsatisfiable condition
   728  		return false
   729  	}
   730  
   731  	for i := 0; i < len(a.keys); i++ {
   732  		k := a.keys[i]
   733  		v := a.vals[i]
   734  
   735  		// Make sure k and v exist somewhere
   736  		found := false
   737  
   738  		for j := 0; j < len(keys) && j < len(vals); j++ {
   739  			if bytes.Equal(k, keys[j].ByteArray()) && bytes.Equal(v, vals[j].ByteArray()) {
   740  				found = true
   741  				break
   742  			}
   743  		}
   744  
   745  		if !found {
   746  			return false
   747  		}
   748  	}
   749  	return true
   750  }
   751  
   752  type RowGetter interface {
   753  	RowNumber() int64
   754  }
   755  
   756  type RowNumberIterator[T any] struct {
   757  	iter.Iterator[T]
   758  	current *IteratorResult
   759  	err     error
   760  
   761  	lastRowNum int64
   762  }
   763  
   764  func NewRowNumberIterator[T any](iter iter.Iterator[T]) *RowNumberIterator[T] {
   765  	return &RowNumberIterator[T]{
   766  		Iterator:   iter,
   767  		lastRowNum: -1,
   768  	}
   769  }
   770  
   771  func (r *RowNumberIterator[T]) Next() bool {
   772  	if !r.Iterator.Next() {
   773  		return false
   774  	}
   775  	r.current = iteratorResultPoolGet()
   776  	r.current.Reset()
   777  	rowGetter, ok := any(r.Iterator.At()).(RowGetter)
   778  	if !ok {
   779  		if r.err == nil {
   780  			r.err = fmt.Errorf("row number iterator: %T does not implement RowGetter", r.Iterator.At())
   781  		}
   782  		return false
   783  	}
   784  
   785  	r.current.RowNumber = RowNumber{rowGetter.RowNumber(), -1, -1, -1, -1, -1}
   786  	if r.lastRowNum >= r.current.RowNumber[0] {
   787  		r.err = fmt.Errorf(
   788  			"row number iterator: %T is not sorted, last_element=%d current_element=%d",
   789  			r.Iterator,
   790  			r.lastRowNum,
   791  			r.current.RowNumber[0],
   792  		)
   793  		return false
   794  	}
   795  	r.lastRowNum = r.current.RowNumber[0]
   796  	r.current.Entries = append(r.current.Entries, struct {
   797  		k        string
   798  		V        parquet.Value
   799  		RowValue interface{}
   800  	}{
   801  		RowValue: r.Iterator.At(),
   802  	})
   803  	return true
   804  }
   805  
   806  func (r *RowNumberIterator[T]) At() *IteratorResult {
   807  	return r.current
   808  }
   809  
   810  func (r *RowNumberIterator[T]) Err() error {
   811  	if r.err != nil {
   812  		return r.err
   813  	}
   814  	return r.Iterator.Err()
   815  }
   816  
   817  func (r *RowNumberIterator[T]) Seek(to RowNumberWithDefinitionLevel) bool {
   818  	if r.current == nil && !r.Next() {
   819  		return false
   820  	}
   821  	for CompareRowNumbers(0, r.current.RowNumber, to.RowNumber) == -1 {
   822  		if !r.Next() {
   823  			return false
   824  		}
   825  	}
   826  	return true
   827  }
   828  
   829  // SyncIterator is a synchronous column iterator. It scans through the given row
   830  // groups and column, and applies the optional predicate to each chunk, page, and value.
   831  // Results are read by calling Next() until it returns nil.
   832  type SyncIterator struct {
   833  	// Config
   834  	column     int
   835  	columnName string
   836  	table      string
   837  	rgs        []parquet.RowGroup
   838  	rgsMin     []RowNumber
   839  	rgsMax     []RowNumber // Exclusive, row number of next one past the row group
   840  	readSize   int
   841  	selectAs   string
   842  	filter     *InstrumentedPredicate
   843  
   844  	// Status
   845  	ctx             context.Context
   846  	cancel          func()
   847  	span            opentracing.Span
   848  	metrics         *Metrics
   849  	curr            RowNumber
   850  	currRowGroup    parquet.RowGroup
   851  	currRowGroupMin RowNumber
   852  	currRowGroupMax RowNumber
   853  	currChunk       parquet.ColumnChunk
   854  	currPages       parquet.Pages
   855  	currPage        parquet.Page
   856  	currPageMax     RowNumber
   857  	currValues      parquet.ValueReader
   858  	currBuf         []parquet.Value
   859  	currBufN        int
   860  
   861  	err error
   862  	res *IteratorResult
   863  }
   864  
   865  var _ Iterator = (*SyncIterator)(nil)
   866  
   867  var syncIteratorPool = sync.Pool{
   868  	New: func() interface{} {
   869  		return []parquet.Value{}
   870  	},
   871  }
   872  
   873  func syncIteratorPoolGet(capacity, len int) []parquet.Value {
   874  	res := syncIteratorPool.Get().([]parquet.Value)
   875  	if cap(res) < capacity {
   876  		res = make([]parquet.Value, capacity)
   877  	}
   878  	res = res[:len]
   879  	return res
   880  }
   881  
   882  func syncIteratorPoolPut(b []parquet.Value) {
   883  	for i := range b {
   884  		b[i] = parquet.Value{}
   885  	}
   886  	syncIteratorPool.Put(b) // nolint: staticcheck
   887  }
   888  
   889  func NewSyncIterator(ctx context.Context, rgs []parquet.RowGroup, column int, columnName string, readSize int, filter Predicate, selectAs string) *SyncIterator {
   890  	// Assign row group bounds.
   891  	// Lower bound is inclusive
   892  	// Upper bound is exclusive, points at the first row of the next group
   893  	rn := EmptyRowNumber()
   894  	rgsMin := make([]RowNumber, len(rgs))
   895  	rgsMax := make([]RowNumber, len(rgs))
   896  	for i, rg := range rgs {
   897  		rgsMin[i] = rn
   898  		rgsMax[i] = rn
   899  		rgsMax[i].Skip(rg.NumRows() + 1)
   900  		rn.Skip(rg.NumRows())
   901  	}
   902  
   903  	span, ctx := opentracing.StartSpanFromContext(ctx, "syncIterator", opentracing.Tags{
   904  		"columnIndex": column,
   905  		"column":      columnName,
   906  	})
   907  
   908  	ctx, cancel := context.WithCancel(ctx)
   909  
   910  	return &SyncIterator{
   911  		table:      strings.ToLower(rgs[0].Schema().Name()) + "s",
   912  		ctx:        ctx,
   913  		cancel:     cancel,
   914  		metrics:    getMetricsFromContext(ctx),
   915  		span:       span,
   916  		column:     column,
   917  		columnName: columnName,
   918  		rgs:        rgs,
   919  		readSize:   readSize,
   920  		selectAs:   selectAs,
   921  		rgsMin:     rgsMin,
   922  		rgsMax:     rgsMax,
   923  		filter:     &InstrumentedPredicate{pred: filter},
   924  		curr:       EmptyRowNumber(),
   925  	}
   926  }
   927  
   928  func (c *SyncIterator) At() *IteratorResult {
   929  	return c.res
   930  }
   931  
   932  func (c *SyncIterator) Next() bool {
   933  	rn, v, err := c.next()
   934  	if err != nil {
   935  		c.res = nil
   936  		c.err = err
   937  		return false
   938  	}
   939  	if !rn.Valid() {
   940  		c.res = nil
   941  		c.err = nil
   942  		return false
   943  	}
   944  	c.res = c.makeResult(rn, v)
   945  	return true
   946  }
   947  
   948  // SeekTo moves this iterator to the next result that is greater than
   949  // or equal to the given row number (and based on the given definition level)
   950  func (c *SyncIterator) Seek(to RowNumberWithDefinitionLevel) bool {
   951  	done, err := c.seekRowGroup(to.RowNumber, to.DefinitionLevel)
   952  	if err != nil {
   953  		c.err = err
   954  		return false
   955  	}
   956  	if done {
   957  		c.res = nil
   958  		c.err = nil
   959  		return false
   960  	}
   961  
   962  	done, err = c.seekPages(to.RowNumber, to.DefinitionLevel)
   963  	if err != nil {
   964  		c.res = nil
   965  		c.err = err
   966  		return false
   967  	}
   968  	if done {
   969  		c.res = nil
   970  		c.err = nil
   971  		return false
   972  	}
   973  
   974  	// The row group and page have been selected to where this value is possibly
   975  	// located. Now scan through the page and look for it.
   976  	for {
   977  		rn, v, err := c.next()
   978  		if err != nil {
   979  			c.res = nil
   980  			c.err = err
   981  			return false
   982  		}
   983  		if !rn.Valid() {
   984  			c.res = nil
   985  			c.err = nil
   986  			return false
   987  		}
   988  
   989  		if CompareRowNumbers(to.DefinitionLevel, rn, to.RowNumber) >= 0 {
   990  			c.res = c.makeResult(rn, v)
   991  			c.err = nil
   992  			return true
   993  		}
   994  	}
   995  }
   996  
   997  func (c *SyncIterator) popRowGroup() (parquet.RowGroup, RowNumber, RowNumber) {
   998  	if len(c.rgs) == 0 {
   999  		return nil, EmptyRowNumber(), EmptyRowNumber()
  1000  	}
  1001  
  1002  	rg := c.rgs[0]
  1003  	min := c.rgsMin[0]
  1004  	max := c.rgsMax[0]
  1005  
  1006  	c.rgs = c.rgs[1:]
  1007  	c.rgsMin = c.rgsMin[1:]
  1008  	c.rgsMax = c.rgsMax[1:]
  1009  
  1010  	return rg, min, max
  1011  }
  1012  
  1013  // seekRowGroup skips ahead to the row group that could contain the value at the
  1014  // desired row number. Does nothing if the current row group is already the correct one.
  1015  func (c *SyncIterator) seekRowGroup(seekTo RowNumber, definitionLevel int) (done bool, err error) {
  1016  	if c.currRowGroup != nil && CompareRowNumbers(definitionLevel, seekTo, c.currRowGroupMax) >= 0 {
  1017  		// Done with this row group
  1018  		c.closeCurrRowGroup()
  1019  	}
  1020  
  1021  	for c.currRowGroup == nil {
  1022  
  1023  		rg, min, max := c.popRowGroup()
  1024  		if rg == nil {
  1025  			return true, nil
  1026  		}
  1027  
  1028  		if CompareRowNumbers(definitionLevel, seekTo, max) != -1 {
  1029  			continue
  1030  		}
  1031  
  1032  		ci, err := rg.ColumnChunks()[c.column].ColumnIndex()
  1033  		if err != nil {
  1034  			return true, err
  1035  		}
  1036  
  1037  		if c.filter != nil && !c.filter.KeepColumnChunk(ci) {
  1038  			continue
  1039  		}
  1040  
  1041  		// This row group matches both row number and filter.
  1042  		c.setRowGroup(rg, min, max)
  1043  	}
  1044  
  1045  	return c.currRowGroup == nil, nil
  1046  }
  1047  
  1048  // seekPages skips ahead in the current row group to the page that could contain the value at
  1049  // the desired row number. Does nothing if the current page is already the correct one.
  1050  func (c *SyncIterator) seekPages(seekTo RowNumber, definitionLevel int) (done bool, err error) {
  1051  	if c.currPage != nil && CompareRowNumbers(definitionLevel, seekTo, c.currPageMax) >= 0 {
  1052  		// Value not in this page
  1053  		c.setPage(nil)
  1054  	}
  1055  
  1056  	if c.currPage == nil {
  1057  		// TODO (mdisibio)   :((((((((
  1058  		//    pages.SeekToRow is more costly than expected.  It doesn't reuse existing i/o
  1059  		// so it can't be called naively every time we swap pages. We need to figure out
  1060  		// a way to determine when it is worth calling here.
  1061  		/*
  1062  			// Seek into the pages. This is relative to the start of the row group
  1063  			if seekTo[0] > 0 {
  1064  				// Determine row delta. We subtract 1 because curr points at the previous row
  1065  				skip := seekTo[0] - c.currRowGroupMin[0] - 1
  1066  				if skip > 0 {
  1067  					if err := c.currPages.SeekToRow(skip); err != nil {
  1068  						return true, err
  1069  					}
  1070  					c.curr.Skip(skip)
  1071  				}
  1072  			}*/
  1073  
  1074  		for c.currPage == nil {
  1075  			pg, err := c.currPages.ReadPage()
  1076  			if pg == nil || err != nil {
  1077  				// No more pages in this column chunk,
  1078  				// cleanup and exit.
  1079  				if err == io.EOF {
  1080  					err = nil
  1081  				}
  1082  				parquet.Release(pg)
  1083  				c.closeCurrRowGroup()
  1084  				return true, err
  1085  			}
  1086  			c.metrics.pageReadsTotal.WithLabelValues(c.table, c.columnName).Add(1)
  1087  			c.span.LogFields(
  1088  				log.String("msg", "reading page (seekPages)"),
  1089  				log.Int64("page_num_values", pg.NumValues()),
  1090  				log.Int64("page_size", pg.Size()),
  1091  			)
  1092  
  1093  			// Skip based on row number?
  1094  			newRN := c.curr
  1095  			newRN.Skip(pg.NumRows() + 1)
  1096  			if CompareRowNumbers(definitionLevel, seekTo, newRN) >= 0 {
  1097  				c.curr.Skip(pg.NumRows())
  1098  				parquet.Release(pg)
  1099  				continue
  1100  			}
  1101  
  1102  			// Skip based on filter?
  1103  			if c.filter != nil && !c.filter.KeepPage(pg) {
  1104  				c.curr.Skip(pg.NumRows())
  1105  				parquet.Release(pg)
  1106  				continue
  1107  			}
  1108  
  1109  			c.setPage(pg)
  1110  		}
  1111  	}
  1112  
  1113  	return false, nil
  1114  }
  1115  
  1116  // next is the core functionality of this iterator and returns the next matching result. This
  1117  // may involve inspecting multiple row groups, pages, and values until a match is found. When
  1118  // we run out of things to inspect, it returns nil. The reason this method is distinct from
  1119  // Next() is because it doesn't wrap the results in an IteratorResult, which is more efficient
  1120  // when being called multiple times and throwing away the results like in SeekTo().
  1121  func (c *SyncIterator) next() (RowNumber, *parquet.Value, error) {
  1122  	for {
  1123  
  1124  		// return if context is cancelled
  1125  		select {
  1126  		case <-c.ctx.Done():
  1127  			return EmptyRowNumber(), nil, c.ctx.Err()
  1128  		default:
  1129  		}
  1130  
  1131  		if c.currRowGroup == nil {
  1132  			rg, min, max := c.popRowGroup()
  1133  			if rg == nil {
  1134  				return EmptyRowNumber(), nil, nil
  1135  			}
  1136  
  1137  			ci, err := rg.ColumnChunks()[c.column].ColumnIndex()
  1138  			if err != nil {
  1139  				return EmptyRowNumber(), nil, err
  1140  			}
  1141  			if c.filter != nil && !c.filter.KeepColumnChunk(ci) {
  1142  				continue
  1143  			}
  1144  
  1145  			c.setRowGroup(rg, min, max)
  1146  		}
  1147  
  1148  		if c.currPage == nil {
  1149  			pg, err := c.currPages.ReadPage()
  1150  			if pg == nil || err == io.EOF {
  1151  				// This row group is exhausted
  1152  				c.closeCurrRowGroup()
  1153  				continue
  1154  			}
  1155  			if err != nil {
  1156  				return EmptyRowNumber(), nil, err
  1157  			}
  1158  			c.metrics.pageReadsTotal.WithLabelValues(c.table, c.columnName).Add(1)
  1159  			c.span.LogFields(
  1160  				log.String("msg", "reading page (next)"),
  1161  				log.Int64("page_num_values", pg.NumValues()),
  1162  				log.Int64("page_size", pg.Size()),
  1163  			)
  1164  
  1165  			if c.filter != nil && !c.filter.KeepPage(pg) {
  1166  				// This page filtered out
  1167  				c.curr.Skip(pg.NumRows())
  1168  				parquet.Release(pg)
  1169  				continue
  1170  			}
  1171  			c.setPage(pg)
  1172  		}
  1173  
  1174  		// Read next batch of values if needed
  1175  		if c.currBuf == nil {
  1176  			c.currBuf = syncIteratorPoolGet(c.readSize, 0)
  1177  		}
  1178  		if c.currBufN >= len(c.currBuf) || len(c.currBuf) == 0 {
  1179  			c.currBuf = c.currBuf[:cap(c.currBuf)]
  1180  			n, err := c.currValues.ReadValues(c.currBuf)
  1181  			if err != nil && err != io.EOF {
  1182  				return EmptyRowNumber(), nil, err
  1183  			}
  1184  			c.currBuf = c.currBuf[:n]
  1185  			c.currBufN = 0
  1186  			if n == 0 {
  1187  				// This value reader and page are exhausted.
  1188  				c.setPage(nil)
  1189  				continue
  1190  			}
  1191  		}
  1192  
  1193  		// Consume current buffer until empty
  1194  		for c.currBufN < len(c.currBuf) {
  1195  			v := &c.currBuf[c.currBufN]
  1196  
  1197  			// Inspect all values to track the current row number,
  1198  			// even if the value is filtered out next.
  1199  			c.curr.Next(v.RepetitionLevel(), v.DefinitionLevel())
  1200  			c.currBufN++
  1201  
  1202  			if c.filter != nil && !c.filter.KeepValue(*v) {
  1203  				continue
  1204  			}
  1205  
  1206  			return c.curr, v, nil
  1207  		}
  1208  	}
  1209  }
  1210  
  1211  func (c *SyncIterator) setRowGroup(rg parquet.RowGroup, min, max RowNumber) {
  1212  	c.closeCurrRowGroup()
  1213  	c.curr = min
  1214  	c.currRowGroup = rg
  1215  	c.currRowGroupMin = min
  1216  	c.currRowGroupMax = max
  1217  	c.currChunk = rg.ColumnChunks()[c.column]
  1218  	c.currPages = c.currChunk.Pages()
  1219  }
  1220  
  1221  func (c *SyncIterator) setPage(pg parquet.Page) {
  1222  	// Handle an outgoing page
  1223  	if c.currPage != nil {
  1224  		c.curr = c.currPageMax.Preceding() // Reposition current row number to end of this page.
  1225  		parquet.Release(c.currPage)
  1226  		c.currPage = nil
  1227  	}
  1228  
  1229  	// Reset value buffers
  1230  	c.currValues = nil
  1231  	c.currPageMax = EmptyRowNumber()
  1232  	c.currBufN = 0
  1233  
  1234  	// If we don't immediately have a new incoming page
  1235  	// then return the buffer to the pool.
  1236  	if pg == nil && c.currBuf != nil {
  1237  		syncIteratorPoolPut(c.currBuf)
  1238  		c.currBuf = nil
  1239  	}
  1240  
  1241  	// Handle an incoming page
  1242  	if pg != nil {
  1243  		rn := c.curr
  1244  		rn.Skip(pg.NumRows() + 1) // Exclusive upper bound, points at the first rownumber in the next page
  1245  		c.currPage = pg
  1246  		c.currPageMax = rn
  1247  		c.currValues = pg.Values()
  1248  	}
  1249  }
  1250  
  1251  func (c *SyncIterator) closeCurrRowGroup() {
  1252  	if c.currPages != nil {
  1253  		c.currPages.Close()
  1254  	}
  1255  
  1256  	c.currRowGroup = nil
  1257  	c.currRowGroupMin = EmptyRowNumber()
  1258  	c.currRowGroupMax = EmptyRowNumber()
  1259  	c.currChunk = nil
  1260  	c.currPages = nil
  1261  	c.setPage(nil)
  1262  }
  1263  
  1264  func (c *SyncIterator) makeResult(t RowNumber, v *parquet.Value) *IteratorResult {
  1265  	r := iteratorResultPoolGet()
  1266  	r.RowNumber = t
  1267  	if c.selectAs != "" {
  1268  		r.AppendValue(c.selectAs, v.Clone())
  1269  	}
  1270  	return r
  1271  }
  1272  
  1273  func (c *SyncIterator) Err() error {
  1274  	return c.err
  1275  }
  1276  
  1277  func (c *SyncIterator) Close() error {
  1278  	c.cancel()
  1279  	c.closeCurrRowGroup()
  1280  
  1281  	c.span.SetTag("inspectedColumnChunks", c.filter.InspectedColumnChunks.Load())
  1282  	c.span.SetTag("inspectedPages", c.filter.InspectedPages.Load())
  1283  	c.span.SetTag("inspectedValues", c.filter.InspectedValues.Load())
  1284  	c.span.SetTag("keptColumnChunks", c.filter.KeptColumnChunks.Load())
  1285  	c.span.SetTag("keptPages", c.filter.KeptPages.Load())
  1286  	c.span.SetTag("keptValues", c.filter.KeptValues.Load())
  1287  	c.span.Finish()
  1288  	return nil
  1289  }