chore(dataobj): rename dataset.Reader to RowReader (#20629)

k292
Robert Fratto 3 months ago committed by GitHub
parent 14d2285b76
commit 736ccdbf93
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
  1. 2
      pkg/dataobj/consumer/logsobj/sort.go
  2. 2
      pkg/dataobj/internal/dataset/predicate.go
  3. 94
      pkg/dataobj/internal/dataset/row_reader.go
  4. 54
      pkg/dataobj/internal/dataset/row_reader_basic.go
  5. 32
      pkg/dataobj/internal/dataset/row_reader_basic_test.go
  6. 96
      pkg/dataobj/internal/dataset/row_reader_downloader.go
  7. 38
      pkg/dataobj/internal/dataset/row_reader_test.go
  8. 2
      pkg/dataobj/sections/indexpointers/iter.go
  9. 4
      pkg/dataobj/sections/indexpointers/row_reader.go
  10. 2
      pkg/dataobj/sections/internal/columnar/columnar_test.go
  11. 4
      pkg/dataobj/sections/internal/columnar/reader_adapter.go
  12. 2
      pkg/dataobj/sections/logs/iter.go
  13. 4
      pkg/dataobj/sections/logs/row_reader.go
  14. 6
      pkg/dataobj/sections/logs/table_merge.go
  15. 4
      pkg/dataobj/sections/logs/table_test.go
  16. 2
      pkg/dataobj/sections/pointers/iter.go
  17. 4
      pkg/dataobj/sections/pointers/row_reader.go
  18. 2
      pkg/dataobj/sections/streams/iter.go
  19. 4
      pkg/dataobj/sections/streams/row_reader.go

@ -32,7 +32,7 @@ func sortMergeIterator(ctx context.Context, sections []*dataobj.Section, sort lo
return nil, err
}
r := dataset.NewReader(dataset.ReaderOptions{
r := dataset.NewRowReader(dataset.ReaderOptions{
Dataset: ds,
Columns: columns,
Prefetch: true,

@ -6,7 +6,7 @@ import (
"unsafe"
)
// Predicate is an expression used to filter rows in a [Reader].
// Predicate is an expression used to filter rows in a [RowReader].
type Predicate interface{ isPredicate() }
// Suppported predicates..

@ -13,7 +13,7 @@ import (
"github.com/grafana/loki/v3/pkg/xcap"
)
// ReaderOptions configures how a [Reader] will read [Row]s.
// ReaderOptions configures how a [RowReader] will read [Row]s.
type ReaderOptions struct {
Dataset Dataset // Dataset to read from.
@ -24,8 +24,8 @@ type ReaderOptions struct {
// are considered non-predicate columns.
Columns []Column
// Predicates filter the data returned by a Reader. Predicates are optional; if
// nil, all rows from Columns are returned.
// Predicates filter the data returned by a RowReader. Predicates are
// optional; if nil, all rows from Columns are returned.
//
// Expressions in Predicate may only reference columns in Columns.
// Holds a list of predicates that can be sequentially applied to the dataset.
@ -37,25 +37,25 @@ type ReaderOptions struct {
Prefetch bool
}
// A Reader reads [Row]s from a [Dataset].
type Reader struct {
// A RowReader reads [Row]s from a [Dataset].
type RowReader struct {
opts ReaderOptions
ready bool // ready is true if the Reader has been initialized.
ready bool // ready is true if the RowReader has been initialized.
origColumnLookup map[Column]int // Find the index of a column in opts.Columns.
primaryColumnIndexes []int // Indexes of primary columns in opts.Columns.
dl *readerDownloader // Bulk page download manager.
row int64 // The current row being read.
inner *basicReader // Underlying reader that reads from columns.
ranges rowRanges // Valid ranges to read across the entire dataset.
dl *rowReaderDownloader // Bulk page download manager.
row int64 // The current row being read.
inner *basicRowReader // Underlying reader that reads from columns.
ranges rowRanges // Valid ranges to read across the entire dataset.
region *xcap.Region // Region for recording statistics.
}
// NewReader creates a new Reader from the provided options.
func NewReader(opts ReaderOptions) *Reader {
var r Reader
// NewRowReader creates a new RowReader from the provided options.
func NewRowReader(opts ReaderOptions) *RowReader {
var r RowReader
r.Reset(opts)
return &r
}
@ -63,7 +63,7 @@ func NewReader(opts ReaderOptions) *Reader {
// Read reads up to the next len(s) rows from r and stores them into s. It
// returns the number of rows read and any error encountered. At the end of the
// Dataset, Read returns 0, [io.EOF].
func (r *Reader) Read(ctx context.Context, s []Row) (int, error) {
func (r *RowReader) Read(ctx context.Context, s []Row) (int, error) {
if len(s) == 0 {
return 0, nil
}
@ -102,7 +102,7 @@ func (r *Reader) Read(ctx context.Context, s []Row) (int, error) {
// buffer gets smaller and smaller).
//
// Improvements on this process must consider the trade-offs between less
// calls to [Reader.Read] while permitting the buffer to be as big as
// calls to [RowReader.Read] while permitting the buffer to be as big as
// possible on each call.
row, err := r.alignRow()
@ -164,7 +164,7 @@ func (r *Reader) Read(ctx context.Context, s []Row) (int, error) {
// Fill.
//
// This mask is only needed for secondary filling in our read range, as the
// next call to [Reader.Read] will move the read range forward and these
// next call to [RowReader.Read] will move the read range forward and these
// rows will never be considered.
for maskedRange := range buildMask(readRange, s[:passCount]) {
r.dl.Mask(maskedRange)
@ -200,7 +200,7 @@ func (r *Reader) Read(ctx context.Context, s []Row) (int, error) {
// the columns on the reduced row range.
//
// It returns the max rows read, rows that passed all the predicates, and any error
func (r *Reader) readAndFilterPrimaryColumns(ctx context.Context, readSize int, s []Row) (int, int, error) {
func (r *RowReader) readAndFilterPrimaryColumns(ctx context.Context, readSize int, s []Row) (int, int, error) {
var (
rowsRead int // tracks max rows accessed to move the [r.row] cursor
passCount int // number of rows that passed the predicate
@ -276,7 +276,7 @@ func (r *Reader) readAndFilterPrimaryColumns(ctx context.Context, readSize int,
// alignRow returns r.row if it is a valid row in ranges, or adjusts r.row to
// the next valid row in ranges.
func (r *Reader) alignRow() (uint64, error) {
func (r *RowReader) alignRow() (uint64, error) {
if r.ranges.Includes(uint64(r.row)) {
return uint64(r.row), nil
}
@ -395,9 +395,9 @@ func buildMask(full rowRange, s []Row) iter.Seq[rowRange] {
}
}
// Close closes the Reader. Closed Readers can be reused by calling
// [Reader.Reset].
func (r *Reader) Close() error {
// Close closes the RowReader. Closed RowReaders can be reused by calling
// [RowReader.Reset].
func (r *RowReader) Close() error {
r.region.End()
if r.inner != nil {
@ -407,9 +407,9 @@ func (r *Reader) Close() error {
return nil
}
// Reset discards any state and resets the Reader with a new set of options.
// This permits reusing a Reader rather than allocating a new one.
func (r *Reader) Reset(opts ReaderOptions) {
// Reset discards any state and resets the RowReader with a new set of options.
// This permits reusing a RowReader rather than allocating a new one.
func (r *RowReader) Reset(opts ReaderOptions) {
r.opts = opts
// There's not much work Reset can do without a context, since it needs to
@ -431,11 +431,11 @@ func (r *Reader) Reset(opts ReaderOptions) {
r.region = nil
}
func (r *Reader) init(ctx context.Context) error {
// Reader.init is kept close to the defition of Reader.Reset to make it
func (r *RowReader) init(ctx context.Context) error {
// RowReader.init is kept close to the defition of RowReader.Reset to make it
// easier to follow the correctness of resetting + initializing.
_, r.region = xcap.StartRegion(ctx, "dataset.Reader")
_, r.region = xcap.StartRegion(ctx, "dataset.RowReader")
// r.validatePredicate must be called before initializing anything else; for
// simplicity, other functions assume that the predicate is valid and can
@ -449,7 +449,7 @@ func (r *Reader) init(ctx context.Context) error {
}
if r.inner == nil {
r.inner = newBasicReader(r.allColumns())
r.inner = newBasicRowReader(r.allColumns())
} else {
r.inner.Reset(r.allColumns())
}
@ -459,9 +459,9 @@ func (r *Reader) init(ctx context.Context) error {
}
// allColumns returns the full set of column to read. If r was configured with
// prefetching, wrapped columns from [readerDownloader] are returned. Otherwise,
// prefetching, wrapped columns from [rowReaderDownloader] are returned. Otherwise,
// the columns of the original dataset are returned.
func (r *Reader) allColumns() []Column {
func (r *RowReader) allColumns() []Column {
if r.opts.Prefetch {
return r.dl.AllColumns()
}
@ -469,9 +469,9 @@ func (r *Reader) allColumns() []Column {
}
// primaryColumns returns the primary columns to read. If r was configured with
// prefetching, wrapped columns from [readerDownloader] are returned. Otherwise,
// prefetching, wrapped columns from [rowReaderDownloader] are returned. Otherwise,
// the primary columns of the original dataset are returned.
func (r *Reader) primaryColumns() []Column {
func (r *RowReader) primaryColumns() []Column {
if r.opts.Prefetch {
return r.dl.PrimaryColumns()
}
@ -479,9 +479,9 @@ func (r *Reader) primaryColumns() []Column {
}
// secondaryColumns returns the secondary columns to read. If r was configured with
// prefetching, wrapped columns from [readerDownloader] are returned. Otherwise,
// prefetching, wrapped columns from [rowReaderDownloader] are returned. Otherwise,
// the secondary columns of the original dataset are returned.
func (r *Reader) secondaryColumns() []Column {
func (r *RowReader) secondaryColumns() []Column {
if r.opts.Prefetch {
return r.dl.SecondaryColumns()
}
@ -490,11 +490,11 @@ func (r *Reader) secondaryColumns() []Column {
// validatePredicate ensures that all columns used in a predicate have been
// provided in [ReaderOptions].
func (r *Reader) validatePredicate() error {
func (r *RowReader) validatePredicate() error {
process := func(c Column) error {
_, ok := r.origColumnLookup[c]
if !ok {
return fmt.Errorf("predicate column %v not found in Reader columns", c)
return fmt.Errorf("predicate column %v not found in RowReader columns", c)
}
return nil
}
@ -521,7 +521,7 @@ func (r *Reader) validatePredicate() error {
case AndPredicate, OrPredicate, NotPredicate, TruePredicate, FalsePredicate, nil:
// No columns to process.
default:
panic(fmt.Sprintf("dataset.Reader.validatePredicate: unsupported predicate type %T", p))
panic(fmt.Sprintf("dataset.RowReader.validatePredicate: unsupported predicate type %T", p))
}
return true // Continue walking the Predicate.
@ -531,9 +531,9 @@ func (r *Reader) validatePredicate() error {
return err
}
// initDownloader initializes the reader's [readerDownloader]. initDownloader is
// initDownloader initializes the reader's [rowReaderDownloader]. initDownloader is
// always used to reduce the number of conditions.
func (r *Reader) initDownloader(ctx context.Context) error {
func (r *RowReader) initDownloader(ctx context.Context) error {
// The downloader is initialized in three steps:
//
// 1. Give it the inner dataset.
@ -541,7 +541,7 @@ func (r *Reader) initDownloader(ctx context.Context) error {
// 3. Provide the overall dataset row ranges that will be valid to read.
if r.dl == nil {
r.dl = newReaderDownloader(r.opts.Dataset)
r.dl = newRowReaderDownloader(r.opts.Dataset)
} else {
r.dl.Reset(r.opts.Dataset)
}
@ -599,7 +599,7 @@ func (r *Reader) initDownloader(ctx context.Context) error {
return nil
}
func (r *Reader) fillPrimaryMask(mask *bitmask.Mask) {
func (r *RowReader) fillPrimaryMask(mask *bitmask.Mask) {
process := func(c Column) {
idx, ok := r.origColumnLookup[c]
if !ok {
@ -635,7 +635,7 @@ func (r *Reader) fillPrimaryMask(mask *bitmask.Mask) {
case AndPredicate, OrPredicate, NotPredicate, TruePredicate, FalsePredicate, nil:
// No columns to process.
default:
panic(fmt.Sprintf("dataset.Reader.fillPrimaryMask: unsupported predicate type %T", p))
panic(fmt.Sprintf("dataset.RowReader.fillPrimaryMask: unsupported predicate type %T", p))
}
return true // Continue walking the Predicate.
@ -648,7 +648,7 @@ func (r *Reader) fillPrimaryMask(mask *bitmask.Mask) {
// entire dataset range is valid.
//
// r.dl must be initialized before calling buildPredicateRanges.
func (r *Reader) buildPredicateRanges(ctx context.Context, p Predicate) (rowRanges, error) {
func (r *RowReader) buildPredicateRanges(ctx context.Context, p Predicate) (rowRanges, error) {
// TODO(rfratto): We could be reusing memory for building ranges here.
switch p := p.(type) {
@ -790,12 +790,12 @@ func simplifyNotPredicate(p NotPredicate) (Predicate, error) {
// buildColumnPredicateRanges returns a set of rowRanges that are valid based
// on whether EqualPredicate, InPredicate, GreaterThanPredicate, or LessThanPredicate may be
// true for each page in a column.
func (r *Reader) buildColumnPredicateRanges(ctx context.Context, c Column, p Predicate) (rowRanges, error) {
func (r *RowReader) buildColumnPredicateRanges(ctx context.Context, c Column, p Predicate) (rowRanges, error) {
// Get the wrapped column so that the result of c.ListPages can be cached.
if idx, ok := r.origColumnLookup[c]; ok {
c = r.allColumns()[idx]
} else {
return nil, fmt.Errorf("column %v not found in Reader columns", c)
return nil, fmt.Errorf("column %v not found in RowReader columns", c)
}
var ranges rowRanges
@ -879,7 +879,7 @@ func readMinMax(stats *datasetmd.Statistics) (minValue Value, maxValue Value, er
return
}
func (r *Reader) predicateColumns(p Predicate, keep func(c Column) bool) ([]Column, []int, error) {
func (r *RowReader) predicateColumns(p Predicate, keep func(c Column) bool) ([]Column, []int, error) {
columns := make(map[Column]struct{})
WalkPredicate(p, func(p Predicate) bool {
@ -907,7 +907,7 @@ func (r *Reader) predicateColumns(p Predicate, keep func(c Column) bool) ([]Colu
for c := range columns {
idx, ok := r.origColumnLookup[c]
if !ok {
panic(fmt.Errorf("predicateColumns: column %v not found in Reader columns", c))
panic(fmt.Errorf("predicateColumns: column %v not found in RowReader columns", c))
}
c := r.allColumns()[idx]

@ -11,12 +11,12 @@ import (
"github.com/grafana/loki/v3/pkg/memory"
)
// basicReader is a low-level reader that reads rows from a set of columns.
// basicRowReader is a low-level reader that reads rows from a set of columns.
//
// basicReader lazily reads pages from columns as they are iterated over; see
// [Reader] for a higher-level implementation that supports predicates and
// basicRowReader lazily reads pages from columns as they are iterated over; see
// [RowReader] for a higher-level implementation that supports predicates and
// batching page downloads.
type basicReader struct {
type basicRowReader struct {
columns []Column
readers []*columnReader
columnLookup map[Column]int // Index into columns and readers
@ -26,31 +26,31 @@ type basicReader struct {
nextRow int64
}
// newBasicReader returns a new basicReader that reads rows from the given set
// newBasicRowReader returns a new basicRowReader that reads rows from the given set
// of columns.
func newBasicReader(set []Column) *basicReader {
var br basicReader
func newBasicRowReader(set []Column) *basicRowReader {
var br basicRowReader
br.Reset(set)
return &br
}
// Read is a convenience wrapper around [basicReader.ReadColumns] that reads up
// to the next len(s) rows across the entire column set owned by [basicReader].
func (pr *basicReader) Read(ctx context.Context, s []Row) (n int, err error) {
// Read is a convenience wrapper around [basicRowReader.ReadColumns] that reads up
// to the next len(s) rows across the entire column set owned by [basicRowReader].
func (pr *basicRowReader) Read(ctx context.Context, s []Row) (n int, err error) {
return pr.ReadColumns(ctx, pr.columns, s)
}
// ReadColumns reads up to the next len(s) rows from a subset of columns and
// stores them into s. It returns the number of rows read and any error
// encountered. At the end of the column set used by basicReader, ReadColumns
// encountered. At the end of the column set used by basicRowReader, ReadColumns
// returns 0, [io.EOF].
//
// Row.Values will be populated with one element per column in the order of the
// overall column set owned by basicReader.
// overall column set owned by basicRowReader.
//
// After calling ReadColumns, additional columns in s can be filled using
// [basicReader.Fill].
func (pr *basicReader) ReadColumns(ctx context.Context, columns []Column, s []Row) (n int, err error) {
// [basicRowReader.Fill].
func (pr *basicRowReader) ReadColumns(ctx context.Context, columns []Column, s []Row) (n int, err error) {
if len(columns) == 0 {
return 0, fmt.Errorf("no columns to read")
}
@ -76,7 +76,7 @@ func (pr *basicReader) ReadColumns(ctx context.Context, columns []Column, s []Ro
// values for.
//
// s[i].Values will be populated with one element per column in the order of
// the column set provided to [newBasicReader] or [basicReader.Reset].
// the column set provided to [newBasicRowReader] or [basicRowReader.Reset].
//
// This allows callers to use Fill to implement efficient filtering:
//
@ -89,8 +89,8 @@ func (pr *basicReader) ReadColumns(ctx context.Context, columns []Column, s []Ro
// forward: that is, each filled row is in sorted order with no repeats across
// calls.
//
// Fill does not advance the offset of the basicReader.
func (pr *basicReader) Fill(ctx context.Context, columns []Column, s []Row) (n int, err error) {
// Fill does not advance the offset of the basicRowReader.
func (pr *basicRowReader) Fill(ctx context.Context, columns []Column, s []Row) (n int, err error) {
if len(columns) == 0 {
return 0, fmt.Errorf("no columns to fill")
}
@ -133,7 +133,7 @@ func partitionRows(s []Row) iter.Seq[[]Row] {
// fill implements fill for a single slice of rows that are consecutive and
// have no gaps between them.
func (pr *basicReader) fill(ctx context.Context, columns []Column, s []Row) (n int, err error) {
func (pr *basicRowReader) fill(ctx context.Context, columns []Column, s []Row) (n int, err error) {
if len(s) == 0 {
return 0, nil
}
@ -162,7 +162,7 @@ func (pr *basicReader) fill(ctx context.Context, columns []Column, s []Row) (n i
for _, column := range columns {
columnIndex, ok := pr.columnLookup[column]
if !ok {
return n, fmt.Errorf("column %v is not owned by basicReader", column)
return n, fmt.Errorf("column %v is not owned by basicRowReader", column)
}
r := pr.readers[columnIndex]
@ -252,7 +252,7 @@ func (pr *basicReader) fill(ctx context.Context, columns []Column, s []Row) (n i
// Seeking to an offset before the start of the column set is an error. Seeking
// to beyond the end of the column set will cause the next Read or ReadColumns
// to return io.EOF.
func (pr *basicReader) Seek(offset int64, whence int) (int64, error) {
func (pr *basicRowReader) Seek(offset int64, whence int) (int64, error) {
switch whence {
case io.SeekStart:
if offset < 0 {
@ -282,7 +282,7 @@ func (pr *basicReader) Seek(offset int64, whence int) (int64, error) {
// maxRows returns the total number of rows across the column set, determined
// by the column with the most rows.
func (pr *basicReader) maxRows() int {
func (pr *basicRowReader) maxRows() int {
var rows int
for _, c := range pr.columns {
rows = max(rows, c.ColumnDesc().RowsCount)
@ -290,9 +290,9 @@ func (pr *basicReader) maxRows() int {
return rows
}
// Reset resets the basicReader to read from the start of the provided columns.
// This permits reusing a basicReader rather than allocating a new one.
func (pr *basicReader) Reset(columns []Column) {
// Reset resets the basicRowReader to read from the start of the provided columns.
// This permits reusing a basicRowReader rather than allocating a new one.
func (pr *basicRowReader) Reset(columns []Column) {
if pr.columnLookup == nil {
pr.columnLookup = make(map[Column]int, len(columns))
} else {
@ -331,9 +331,9 @@ func closeAndClear(r []*columnReader) {
clear(r)
}
// Close closes the basicReader. Closed basicReaders can be reused by calling
// [basicReader.Reset].
func (pr *basicReader) Close() error {
// Close closes the basicRowReader. Closed basicRowReaders can be reused by calling
// [basicRowReader.Reset].
func (pr *basicRowReader) Close() error {
for _, r := range pr.readers {
if err := r.Close(); err != nil {
return err

@ -36,10 +36,10 @@ func Test_basicReader_ReadAll(t *testing.T) {
columns := buildTestColumns(t)
require.Len(t, columns, 4)
br := newBasicReader(columns)
br := newBasicRowReader(columns)
defer br.Close()
actualRows, err := readBasicReader(br, 3)
actualRows, err := readBasicRowReader(br, 3)
require.NoError(t, err)
require.Equal(t, basicReaderTestData, convertToTestPersons(actualRows))
}
@ -48,14 +48,14 @@ func Test_basicReader_ReadFromOffset(t *testing.T) {
columns := buildTestColumns(t)
require.Len(t, columns, 4)
br := newBasicReader(columns)
br := newBasicRowReader(columns)
defer br.Close()
// Seek to row 4
_, err := br.Seek(4, io.SeekStart)
require.NoError(t, err)
actualRows, err := readBasicReader(br, 3)
actualRows, err := readBasicRowReader(br, 3)
require.NoError(t, err)
require.Equal(t, basicReaderTestData[4:], convertToTestPersons(actualRows))
}
@ -64,18 +64,18 @@ func Test_basicReader_SeekToStart(t *testing.T) {
columns := buildTestColumns(t)
require.Len(t, columns, 4)
br := newBasicReader(columns)
br := newBasicRowReader(columns)
defer br.Close()
// First read everything
_, err := readBasicReader(br, 3)
_, err := readBasicRowReader(br, 3)
require.NoError(t, err)
// Seek back to start and read again
_, err = br.Seek(0, io.SeekStart)
require.NoError(t, err)
actualRows, err := readBasicReader(br, 3)
actualRows, err := readBasicRowReader(br, 3)
require.NoError(t, err)
require.Equal(t, basicReaderTestData, convertToTestPersons(actualRows))
}
@ -84,7 +84,7 @@ func Test_basicReader_ReadColumns(t *testing.T) {
columns := buildTestColumns(t)
require.Len(t, columns, 4)
br := newBasicReader(columns)
br := newBasicRowReader(columns)
defer br.Close()
// Read only birth_year and middle_name columns (indices 3 and 1)
@ -133,7 +133,7 @@ func Test_basicReader_Fill(t *testing.T) {
columns := buildTestColumns(t)
require.Len(t, columns, 4)
br := newBasicReader(columns)
br := newBasicRowReader(columns)
defer br.Close()
// Create rows with specific indices we want to fill
@ -218,17 +218,17 @@ func Test_basicReader_Reset(t *testing.T) {
columns := buildTestColumns(t)
require.Len(t, columns, 4)
br := newBasicReader(columns)
br := newBasicRowReader(columns)
defer br.Close()
// First read everything
_, err := readBasicReader(br, 3)
_, err := readBasicRowReader(br, 3)
require.NoError(t, err)
// Reset and read again
br.Reset(columns)
actualRows, err := readBasicReader(br, 3)
actualRows, err := readBasicRowReader(br, 3)
require.NoError(t, err)
require.Equal(t, basicReaderTestData, convertToTestPersons(actualRows))
}
@ -307,8 +307,8 @@ func buildInt64Column(t *testing.T, name string) *ColumnBuilder {
return builder
}
// readBasicReader reads all rows from a basicReader using the given batch size.
func readBasicReader(br *basicReader, batchSize int) ([]Row, error) {
// readBasicRowReader reads all rows from a basicRowReader using the given batch size.
func readBasicRowReader(br *basicRowReader, batchSize int) ([]Row, error) {
var (
all []Row
@ -321,8 +321,8 @@ func readBasicReader(br *basicReader, batchSize int) ([]Row, error) {
//
// This requires any Row/Value provided by br.Read is owned by the caller
// and is not retained by the reader; if a test fails and appears to have
// memory reuse, it's likely because code in basicReader changed and broke
// ownership semantics.
// memory reuse, it's likely because code in basicRowReader changed and
// broke ownership semantics.
clear(batch)
n, err := br.Read(context.Background(), batch)

@ -8,7 +8,7 @@ import (
"github.com/grafana/loki/v3/pkg/xcap"
)
// readerDownloader is a utility for downloading pages in bulk from a
// rowReaderDownloader is a utility for downloading pages in bulk from a
// [Dataset]. It works by caching page data from an inner dataset, and
// downloading pages in bulk any time an uncached page is requested.
//
@ -16,7 +16,7 @@ import (
//
// Downloading pages in bulk is important to minimize round trips to the
// backend storage. The proper behavior of bulk downloads is tied to
// [Reader.Read] operating in two phases:
// [RowReader.Read] operating in two phases:
//
// 1. Rows from primary columns are read and filtered by a predicate
// 2. Rows from secondary columns are read into the filtered rows
@ -28,8 +28,8 @@ import (
// of page, assigning other pages a priority level:
//
// - P1: Other pages of the same phase that overlap with the current read
// range from [readerDownloader.SetReadRange] and are not masked by
// [readerDownloader.SetMask].
// range from [rowReaderDownloader.SetReadRange] and are not masked by
// [rowReaderDownloader.SetMask].
//
// - P2: Secondary pages that overlap with the current read range and are not
// masked.
@ -41,9 +41,9 @@ import (
// - P3: All pages that include rows after the end of the read range.
//
// This excludes any page that is outside of the dataset ranges passed to
// [newReaderDownloader] and [readerDownloader.Reset].
// [newReaderDownloader] and [rowReaderDownloader.Reset].
//
// The readerDownloader targets a configurable batch size, which is the target
// The rowReaderDownloader targets a configurable batch size, which is the target
// size of pages to cache in memory at once.
//
// Batches of pages to download are built in four steps:
@ -68,13 +68,13 @@ import (
// - The minimum number of pages needed to download an entire dataset is one,
// if every page in that dataset is less than the target size.
//
// - The minimum number of pages needed to download a single [Reader.Read] call
// - The minimum number of pages needed to download a single [RowReader.Read] call
// is zero, if all pages have been downloaded in a previous call.
//
// - The maximum number of pages needed to download a single [Reader.Read] call
// - The maximum number of pages needed to download a single [RowReader.Read] call
// is two: one for the primary phase, and another for the secondary phase.
//
// - The separation of phases allows for the [Reader] to mask additional ranges
// - The separation of phases allows for the [RowReader] to mask additional ranges
// before the secondary phase. This helps reduce the number of P1 pages
// that are downloaded during the secondary phase.
//
@ -83,7 +83,7 @@ import (
//
// Cached pages before the read range are cleared when a new uncached page is
// requested.
type readerDownloader struct {
type rowReaderDownloader struct {
inner Dataset
origColumns, origPrimary, origSecondary []Column
@ -99,42 +99,42 @@ type readerDownloader struct {
// Dataset. The resulting Dataset only wraps around the provided columns.
//
// All uncached pages that have not been pruned by
// [readerDownloader.SetDatasetRanges] will be downloaded in bulk when an
// [rowReaderDownloader.SetDatasetRanges] will be downloaded in bulk when an
// uncached page is requested.
//
// # Initialization
//
// After a readerDownloader is created, it must be initialized by calling:
// After a rowReaderDownloader is created, it must be initialized by calling:
//
// 1. [readerDownloader.AddColumn] with each column that will be read, and
// 2. [readerDownloader.SetDatasetRanges] to define the valid ranges acrsos
// 1. [rowReaderDownloader.AddColumn] with each column that will be read, and
// 2. [rowReaderDownloader.SetDatasetRanges] to define the valid ranges acrsos
// the entire dataset.
//
// # Usage
//
// Use [readerDownloader.AllColumns], [readerDownloader.PrimaryColumns], and
// [readerDownloader.SecondaryColumns] to enable page batching; any pages
// Use [rowReaderDownloader.AllColumns], [rowReaderDownloader.PrimaryColumns], and
// [rowReaderDownloader.SecondaryColumns] to enable page batching; any pages
// loaded from these columns will trigger a bulk download.
//
// Before each usage of the columns, users should call
// [readerDownloader.SetReadRange] to define the range of rows that will be
// [rowReaderDownloader.SetReadRange] to define the range of rows that will be
// read next.
//
// If applicable, users should additionally call [readerDownloader.Mask] to
// If applicable, users should additionally call [rowReaderDownloader.Mask] to
// exclude any ranges of rows that should not be read; pages that are entirely
// within the mask will not be downloaded.
func newReaderDownloader(dset Dataset) *readerDownloader {
var rd readerDownloader
func newRowReaderDownloader(dset Dataset) *rowReaderDownloader {
var rd rowReaderDownloader
rd.Reset(dset)
return &rd
}
// AddColumn adds a column to the readerDownloader. This should be called
// AddColumn adds a column to the rowReaderDownloader. This should be called
// before the downloader is used.
//
// AddColumn must be called matching the order of columns in
// [ReaderOptions.Columns].
func (dl *readerDownloader) AddColumn(col Column, primary bool) {
func (dl *rowReaderDownloader) AddColumn(col Column, primary bool) {
wrappedCol := newReaderColumn(dl, col, primary)
dl.origColumns = append(dl.origColumns, col)
@ -151,7 +151,7 @@ func (dl *readerDownloader) AddColumn(col Column, primary bool) {
// SetDatasetRanges sets the valid ranges of rows that will be read. Pages
// which do not overlap with these ranges will never be downloaded.
func (dl *readerDownloader) SetDatasetRanges(r rowRanges) {
func (dl *rowReaderDownloader) SetDatasetRanges(r rowRanges) {
dl.dsetRanges = r
}
@ -160,7 +160,7 @@ func (dl *readerDownloader) SetDatasetRanges(r rowRanges) {
// range are never included in a batch.
//
// This method clears any previously set mask.
func (dl *readerDownloader) SetReadRange(r rowRange) {
func (dl *rowReaderDownloader) SetReadRange(r rowRange) {
dl.readRange = r
dl.rangeMask = sliceclear.Clear(dl.rangeMask)
}
@ -168,36 +168,36 @@ func (dl *readerDownloader) SetReadRange(r rowRange) {
// Mask marks a subset of the current read range as excluded. Mask may be
// called multiple times to exclude multiple ranges. Any page that is entirely
// within the combined mask will not be downloaded.
func (dl *readerDownloader) Mask(r rowRange) {
func (dl *rowReaderDownloader) Mask(r rowRange) {
dl.rangeMask.Add(r)
}
// OrigColumns returns the original columns of the readerDownloader in the order
// OrigColumns returns the original columns of the rowReaderDownloader in the order
// they were added.
func (dl *readerDownloader) OrigColumns() []Column { return dl.origColumns }
func (dl *rowReaderDownloader) OrigColumns() []Column { return dl.origColumns }
// OrigPrimaryColumns returns the original primary columns of the
// readerDownloader in the order they were added.
func (dl *readerDownloader) OrigPrimaryColumns() []Column { return dl.origPrimary }
// rowReaderDownloader in the order they were added.
func (dl *rowReaderDownloader) OrigPrimaryColumns() []Column { return dl.origPrimary }
// OrigSecondaryColumns returns the original secondary columns of the
// readerDownloader in the order they were added.
func (dl *readerDownloader) OrigSecondaryColumns() []Column { return dl.origSecondary }
// rowReaderDownloader in the order they were added.
func (dl *rowReaderDownloader) OrigSecondaryColumns() []Column { return dl.origSecondary }
// AllColumns returns the wrapped columns of the readerDownloader in the order
// AllColumns returns the wrapped columns of the rowReaderDownloader in the order
// they were added.
func (dl *readerDownloader) AllColumns() []Column { return dl.allColumns }
func (dl *rowReaderDownloader) AllColumns() []Column { return dl.allColumns }
// PrimaryColumns returns the wrapped primary columns of the readerDownloader
// PrimaryColumns returns the wrapped primary columns of the rowReaderDownloader
// in the order they were added.
func (dl *readerDownloader) PrimaryColumns() []Column { return dl.primary }
func (dl *rowReaderDownloader) PrimaryColumns() []Column { return dl.primary }
// SecondaryColumns returns the wrapped secondary columns of the
// readerDownloader in the order they were added.
func (dl *readerDownloader) SecondaryColumns() []Column { return dl.secondary }
// rowReaderDownloader in the order they were added.
func (dl *rowReaderDownloader) SecondaryColumns() []Column { return dl.secondary }
// initColumnPages populates the pages of all columns in the downloader.
func (dl *readerDownloader) initColumnPages(ctx context.Context) error {
func (dl *rowReaderDownloader) initColumnPages(ctx context.Context) error {
columns := dl.allColumns
var idx int
@ -216,7 +216,7 @@ func (dl *readerDownloader) initColumnPages(ctx context.Context) error {
}
// downloadBatch downloads a batch of pages from the inner dataset.
func (dl *readerDownloader) downloadBatch(ctx context.Context, requestor *readerPage) error {
func (dl *rowReaderDownloader) downloadBatch(ctx context.Context, requestor *readerPage) error {
for _, col := range dl.allColumns {
// Garbage collect any unused pages; this prevents them from being included
// in the batchSize calculation and also allows them to be freed by the GC.
@ -265,7 +265,7 @@ func (dl *readerDownloader) downloadBatch(ctx context.Context, requestor *reader
return nil
}
func (dl *readerDownloader) buildDownloadBatch(ctx context.Context, requestor *readerPage) ([]*readerPage, error) {
func (dl *rowReaderDownloader) buildDownloadBatch(ctx context.Context, requestor *readerPage) ([]*readerPage, error) {
var pageBatch []*readerPage
// Figure out how large our batch already is based on cache pages.
@ -339,7 +339,7 @@ func (dl *readerDownloader) buildDownloadBatch(ctx context.Context, requestor *r
// iterP1Pages returns an iterator over P1 pages in round-robin column order,
// with one page per column.
func (dl *readerDownloader) iterP1Pages(ctx context.Context, primary bool) result.Seq[*readerPage] {
func (dl *rowReaderDownloader) iterP1Pages(ctx context.Context, primary bool) result.Seq[*readerPage] {
return result.Iter(func(yield func(*readerPage) bool) error {
for result := range dl.iterColumnPages(ctx, primary) {
page, err := result.Value()
@ -372,7 +372,7 @@ func (dl *readerDownloader) iterP1Pages(ctx context.Context, primary bool) resul
// iterColumnPages returns an iterator over pages in columns in round-robin
// order across all columns (first page from each column, then second page from
// each column, etc.).
func (dl *readerDownloader) iterColumnPages(ctx context.Context, primary bool) result.Seq[*readerPage] {
func (dl *rowReaderDownloader) iterColumnPages(ctx context.Context, primary bool) result.Seq[*readerPage] {
phaseColumns := dl.primary
if !primary {
phaseColumns = dl.secondary
@ -411,7 +411,7 @@ func (dl *readerDownloader) iterColumnPages(ctx context.Context, primary bool) r
// iterP2Pages returns an iterator over P2 pages in round-robin column order,
// with one page per column.
func (dl *readerDownloader) iterP2Pages(ctx context.Context, primary bool) result.Seq[*readerPage] {
func (dl *rowReaderDownloader) iterP2Pages(ctx context.Context, primary bool) result.Seq[*readerPage] {
// For the primary phase, P2 pages are pages that would be P1 for the
// secondary phase. This means we can express it as iterP1Pages(ctx, !primary).
//
@ -427,7 +427,7 @@ func (dl *readerDownloader) iterP2Pages(ctx context.Context, primary bool) resul
// iterP3Pages returns an iterator over P3 pages in round-robin column order,
// with one page per column.
func (dl *readerDownloader) iterP3Pages(ctx context.Context, primary bool) result.Seq[*readerPage] {
func (dl *rowReaderDownloader) iterP3Pages(ctx context.Context, primary bool) result.Seq[*readerPage] {
return result.Iter(func(yield func(*readerPage) bool) error {
for result := range dl.iterColumnPages(ctx, primary) {
page, err := result.Value()
@ -457,7 +457,7 @@ func (dl *readerDownloader) iterP3Pages(ctx context.Context, primary bool) resul
})
}
func (dl *readerDownloader) Reset(dset Dataset) {
func (dl *rowReaderDownloader) Reset(dset Dataset) {
dl.inner = dset
dl.readRange = rowRange{}
@ -478,7 +478,7 @@ func (dl *readerDownloader) Reset(dset Dataset) {
}
type readerColumn struct {
dl *readerDownloader
dl *rowReaderDownloader
inner Column
primary bool // Whether this column is a primary column.
@ -487,7 +487,7 @@ type readerColumn struct {
var _ Column = (*readerColumn)(nil)
func newReaderColumn(dl *readerDownloader, col Column, primary bool) *readerColumn {
func newReaderColumn(dl *rowReaderDownloader, col Column, primary bool) *readerColumn {
return &readerColumn{
dl: dl,
inner: col,
@ -537,7 +537,7 @@ func (col *readerColumn) processPages(pages Pages) {
// GC garbage collects cached data from pages which will no longer be read: any
// page which ends before the read row range of the downloader.
//
// Using the minimum read row range permits failed calls to [Reader.Read] to be
// Using the minimum read row range permits failed calls to [RowReader.Read] to be
// retried without needing to redownload the pages involved in that call.
func (col *readerColumn) GC() {
for _, page := range col.pages {

@ -21,7 +21,7 @@ import (
func Test_Reader_ReadAll(t *testing.T) {
dset, columns := buildTestDataset(t)
r := NewReader(ReaderOptions{Dataset: dset, Columns: columns})
r := NewRowReader(ReaderOptions{Dataset: dset, Columns: columns})
defer r.Close()
actualRows, err := readDataset(r, 3)
@ -33,7 +33,7 @@ func Test_Reader_ReadWithPredicate(t *testing.T) {
dset, columns := buildTestDataset(t)
// Create a predicate that only returns people born after 1985
r := NewReader(ReaderOptions{
r := NewRowReader(ReaderOptions{
Dataset: dset,
Columns: columns,
Predicates: []Predicate{
@ -58,12 +58,12 @@ func Test_Reader_ReadWithPredicate(t *testing.T) {
require.Equal(t, expected, convertToTestPersons(actualRows))
}
// Test_Reader_ReadWithPageFiltering tests that a Reader can filter rows based
// TestRowReader_ReadWithPageFiltering tests that a RowReader can filter rows based
// on a predicate that has filtered pages out.
func Test_Reader_ReadWithPageFiltering(t *testing.T) {
func TestRowReader_ReadWithPageFiltering(t *testing.T) {
dset, columns := buildTestDataset(t)
r := NewReader(ReaderOptions{
r := NewRowReader(ReaderOptions{
Dataset: dset,
Columns: columns,
@ -94,9 +94,9 @@ func Test_Reader_ReadWithPageFiltering(t *testing.T) {
require.Equal(t, expected, convertToTestPersons(actualRows))
}
// Test_Reader_ReadWithPageFilteringOnEmptyPredicate tests that a Reader filters rows with empty predicate values.
// TestRowReader_ReadWithPageFilteringOnEmptyPredicate tests that a RowReader filters rows with empty predicate values.
// Filtering for an explicitly empty value also includes Null values and these rows should not be excluded by page skipping.
func Test_Reader_ReadWithPageFilteringOnEmptyPredicate(t *testing.T) {
func TestRowReader_ReadWithPageFilteringOnEmptyPredicate(t *testing.T) {
// Create builders for each column
firstNameBuilder := buildStringColumn(t, "first_name")
lastNameBuilder := buildStringColumn(t, "middle_name")
@ -118,7 +118,7 @@ func Test_Reader_ReadWithPageFilteringOnEmptyPredicate(t *testing.T) {
cols, err := result.Collect(dset.ListColumns(context.Background()))
require.NoError(t, err)
r := NewReader(ReaderOptions{
r := NewRowReader(ReaderOptions{
Dataset: dset,
Columns: cols,
@ -160,7 +160,7 @@ func Test_Reader_ReadWithPredicate_NoSecondary(t *testing.T) {
dset, columns := buildTestDataset(t)
// Create a predicate that only returns people born after 1985
r := NewReader(ReaderOptions{
r := NewRowReader(ReaderOptions{
Dataset: dset,
Columns: []Column{columns[3]},
Predicates: []Predicate{
@ -192,7 +192,7 @@ func Test_Reader_ReadWithPredicate_NoSecondary(t *testing.T) {
func Test_Reader_Reset(t *testing.T) {
dset, columns := buildTestDataset(t)
r := NewReader(ReaderOptions{Dataset: dset, Columns: columns})
r := NewRowReader(ReaderOptions{Dataset: dset, Columns: columns})
defer r.Close()
// First read everything
@ -282,13 +282,13 @@ func mergeRows(rows ...[]Row) []Row {
return res
}
// readDataset reads all rows from a Reader using the given batch size.
func readDataset(br *Reader, batchSize int) ([]Row, error) {
// readDataset reads all rows from a RowReader using the given batch size.
func readDataset(br *RowReader, batchSize int) ([]Row, error) {
return readDatasetWithContext(context.Background(), br, batchSize)
}
// readDatasetWithContext reads all rows from a Reader using the given batch size and context.
func readDatasetWithContext(ctx context.Context, br *Reader, batchSize int) ([]Row, error) {
// readDatasetWithContext reads all rows from a RowReader using the given batch size and context.
func readDatasetWithContext(ctx context.Context, br *RowReader, batchSize int) ([]Row, error) {
var (
all []Row
@ -391,7 +391,7 @@ func Test_BuildPredicateRanges(t *testing.T) {
ctx := context.Background()
for _, tc := range tt {
t.Run(tc.name, func(t *testing.T) {
r := NewReader(ReaderOptions{
r := NewRowReader(ReaderOptions{
Dataset: ds,
Columns: cols,
Predicates: []Predicate{tc.predicate},
@ -552,7 +552,7 @@ func BenchmarkReader(b *testing.B) {
batch := make([]Row, rp.batchSize)
for b.Loop() {
reader := NewReader(opts)
reader := NewRowReader(opts)
var rowsRead int
for {
n, err := reader.Read(context.Background(), batch)
@ -606,7 +606,7 @@ func BenchmarkPredicateExecution(b *testing.B) {
currentPos := 0
batch := make([]Row, 1000)
// read the dataset once to pick a random row for predicate generation
reader := NewReader(ReaderOptions{
reader := NewRowReader(ReaderOptions{
Dataset: ds,
Columns: cols,
})
@ -685,7 +685,7 @@ func BenchmarkPredicateExecution(b *testing.B) {
b.ReportAllocs()
for b.Loop() {
reader := NewReader(ReaderOptions{
reader := NewRowReader(ReaderOptions{
Dataset: ds,
Columns: cols,
Predicates: pp.predicates,
@ -889,7 +889,7 @@ func Test_DatasetGenerator(t *testing.T) {
func Test_Reader_Stats(t *testing.T) {
dset, columns := buildTestDataset(t)
r := NewReader(ReaderOptions{
r := NewRowReader(ReaderOptions{
Dataset: dset,
Columns: columns,
Predicates: []Predicate{

@ -50,7 +50,7 @@ func IterSection(ctx context.Context, section *Section) result.Seq[IndexPointer]
return err
}
r := dataset.NewReader(dataset.ReaderOptions{
r := dataset.NewRowReader(dataset.ReaderOptions{
Dataset: dset,
Columns: columns,
Prefetch: true,

@ -21,7 +21,7 @@ type RowReader struct {
buf []dataset.Row
reader *dataset.Reader
reader *dataset.RowReader
columns []dataset.Column
symbols *symbolizer.Symbolizer
@ -104,7 +104,7 @@ func (r *RowReader) initReader() error {
}
if r.reader == nil {
r.reader = dataset.NewReader(readerOpts)
r.reader = dataset.NewRowReader(readerOpts)
} else {
r.reader.Reset(readerOpts)
}

@ -96,7 +96,7 @@ func readDataset(t *testing.T, obj *dataobj.Object) []dataset.Row {
dset, err := columnar.MakeDataset(sec, sec.Columns())
require.NoError(t, err)
reader := dataset.NewReader(dataset.ReaderOptions{
reader := dataset.NewRowReader(dataset.ReaderOptions{
Dataset: dset,
Columns: dset.Columns(),
})

@ -15,7 +15,7 @@ import (
// [columnar.RecordBatch] values from a reader that only supports reads through
// a slice of [dataset.Row].
type ReaderAdapter struct {
inner *dataset.Reader
inner *dataset.RowReader
colTypes []datasetmd.PhysicalType
buf []dataset.Row
@ -23,7 +23,7 @@ type ReaderAdapter struct {
// NewReaderAdapter creates a ReaderAdapter with the provided dataset reader options.
func NewReaderAdapter(innerOpts dataset.ReaderOptions) *ReaderAdapter {
r := &ReaderAdapter{inner: dataset.NewReader(innerOpts)}
r := &ReaderAdapter{inner: dataset.NewRowReader(innerOpts)}
r.Reset(innerOpts)
return r
}

@ -51,7 +51,7 @@ func IterSection(ctx context.Context, section *Section) result.Seq[Record] {
return err
}
r := dataset.NewReader(dataset.ReaderOptions{
r := dataset.NewRowReader(dataset.ReaderOptions{
Dataset: dset,
Columns: columns,
Prefetch: true,

@ -27,7 +27,7 @@ type RowReader struct {
buf []dataset.Row
reader *dataset.Reader
reader *dataset.RowReader
columns []dataset.Column
symbols *symbolizer.Symbolizer
@ -149,7 +149,7 @@ func (r *RowReader) initReader() error {
}
if r.reader == nil {
r.reader = dataset.NewReader(readerOpts)
r.reader = dataset.NewRowReader(readerOpts)
} else {
r.reader.Reset(readerOpts)
}

@ -69,7 +69,7 @@ func mergeTables(buf *tableBuffer, pageSize, pageRowCount int, compressionOpts *
return nil, err
}
r := dataset.NewReader(dataset.ReaderOptions{
r := dataset.NewRowReader(dataset.ReaderOptions{
Dataset: t,
Columns: dsetColumns,
@ -150,7 +150,7 @@ var _ loser.Sequence = (*tableSequence)(nil)
func tableSequenceAt(seq *tableSequence) result.Result[dataset.Row] { return seq.At() }
func tableSequenceClose(seq *tableSequence) { seq.Close() }
func NewDatasetSequence(r *dataset.Reader, bufferSize int) DatasetSequence {
func NewDatasetSequence(r *dataset.RowReader, bufferSize int) DatasetSequence {
return DatasetSequence{
r: r,
buf: make([]dataset.Row, bufferSize),
@ -160,7 +160,7 @@ func NewDatasetSequence(r *dataset.Reader, bufferSize int) DatasetSequence {
type DatasetSequence struct {
curValue result.Result[dataset.Row]
r *dataset.Reader
r *dataset.RowReader
buf []dataset.Row
off int // Offset into buf

@ -109,7 +109,7 @@ func Test_mergeTables(t *testing.T) {
var actual []string
r := dataset.NewReader(dataset.ReaderOptions{
r := dataset.NewRowReader(dataset.ReaderOptions{
Dataset: mergedTable,
Columns: mergedColumns,
})
@ -155,7 +155,7 @@ func Test_table_backfillMetadata(t *testing.T) {
columns, err := result.Collect(table.ListColumns(context.Background()))
require.NoError(t, err)
r := dataset.NewReader(dataset.ReaderOptions{
r := dataset.NewRowReader(dataset.ReaderOptions{
Dataset: table,
Columns: columns,
})

@ -51,7 +51,7 @@ func IterSection(ctx context.Context, section *Section) result.Seq[SectionPointe
return err
}
r := dataset.NewReader(dataset.ReaderOptions{
r := dataset.NewRowReader(dataset.ReaderOptions{
Dataset: dset,
Columns: columns,
Prefetch: true,

@ -28,7 +28,7 @@ type RowReader struct {
buf []dataset.Row
reader *dataset.Reader
reader *dataset.RowReader
columns []dataset.Column
symbols *symbolizer.Symbolizer
@ -137,7 +137,7 @@ func (r *RowReader) initReader() error {
}
if r.reader == nil {
r.reader = dataset.NewReader(readerOpts)
r.reader = dataset.NewRowReader(readerOpts)
} else {
r.reader.Reset(readerOpts)
}

@ -51,7 +51,7 @@ func IterSection(ctx context.Context, section *Section) result.Seq[Stream] {
return err
}
r := dataset.NewReader(dataset.ReaderOptions{
r := dataset.NewRowReader(dataset.ReaderOptions{
Dataset: dset,
Columns: columns,
Prefetch: true,

@ -23,7 +23,7 @@ type RowReader struct {
buf []dataset.Row
reader *dataset.Reader
reader *dataset.RowReader
columns []dataset.Column
symbols *symbolizer.Symbolizer
@ -107,7 +107,7 @@ func (r *RowReader) initReader() error {
}
if r.reader == nil {
r.reader = dataset.NewReader(readerOpts)
r.reader = dataset.NewRowReader(readerOpts)
} else {
r.reader.Reset(readerOpts)
}

Loading…
Cancel
Save