diff --git a/pkg/dataobj/internal/encoding/dataset_streams.go b/pkg/dataobj/internal/encoding/dataset_streams.go new file mode 100644 index 0000000000000..5843cc4c2cd47 --- /dev/null +++ b/pkg/dataobj/internal/encoding/dataset_streams.go @@ -0,0 +1,152 @@ +package encoding + +import ( + "context" + "fmt" + + "github.com/grafana/loki/v3/pkg/dataobj/internal/dataset" + "github.com/grafana/loki/v3/pkg/dataobj/internal/metadata/filemd" + "github.com/grafana/loki/v3/pkg/dataobj/internal/metadata/streamsmd" + "github.com/grafana/loki/v3/pkg/dataobj/internal/result" +) + +// StreamsDataset implements returns a [dataset.Dataset] from a +// [StreamsDecoder] for the given section. +func StreamsDataset(dec StreamsDecoder, sec *filemd.SectionInfo) dataset.Dataset { + return &streamsDataset{dec: dec, sec: sec} +} + +type streamsDataset struct { + dec StreamsDecoder + sec *filemd.SectionInfo +} + +func (ds *streamsDataset) ListColumns(ctx context.Context) result.Seq[dataset.Column] { + return result.Iter(func(yield func(dataset.Column) bool) error { + columns, err := ds.dec.Columns(ctx, ds.sec) + if err != nil { + return err + } + + for _, column := range columns { + if !yield(&streamsDatasetColumn{dec: ds.dec, desc: column}) { + return nil + } + } + + return err + }) + +} + +func (ds *streamsDataset) ListPages(ctx context.Context, columns []dataset.Column) result.Seq[dataset.Pages] { + // TODO(rfratto): Switch to batch retrieval instead of iterating over each column. + return result.Iter(func(yield func(dataset.Pages) bool) error { + for _, column := range columns { + pages, err := result.Collect(column.ListPages(ctx)) + if err != nil { + return err + } else if !yield(pages) { + return nil + } + } + + return nil + }) +} + +func (ds *streamsDataset) ReadPages(ctx context.Context, pages []dataset.Page) result.Seq[dataset.PageData] { + // TODO(rfratto): Switch to batch retrieval instead of iterating over each page. + return result.Iter(func(yield func(dataset.PageData) bool) error { + for _, page := range pages { + data, err := page.ReadPage(ctx) + if err != nil { + return err + } else if !yield(data) { + return nil + } + } + + return nil + }) +} + +type streamsDatasetColumn struct { + dec StreamsDecoder + desc *streamsmd.ColumnDesc + + info *dataset.ColumnInfo +} + +func (col *streamsDatasetColumn) ColumnInfo() *dataset.ColumnInfo { + if col.info != nil { + return col.info + } + + col.info = &dataset.ColumnInfo{ + Name: col.desc.Info.Name, + Type: col.desc.Info.ValueType, + Compression: col.desc.Info.Compression, + + RowsCount: int(col.desc.Info.RowsCount), + CompressedSize: int(col.desc.Info.CompressedSize), + UncompressedSize: int(col.desc.Info.UncompressedSize), + + Statistics: col.desc.Info.Statistics, + } + return col.info +} + +func (col *streamsDatasetColumn) ListPages(ctx context.Context) result.Seq[dataset.Page] { + return result.Iter(func(yield func(dataset.Page) bool) error { + pageSets, err := result.Collect(col.dec.Pages(ctx, []*streamsmd.ColumnDesc{col.desc})) + if err != nil { + return err + } else if len(pageSets) != 1 { + return fmt.Errorf("unexpected number of page sets: got=%d want=1", len(pageSets)) + } + + for _, page := range pageSets[0] { + if !yield(&streamsDatasetPage{dec: col.dec, desc: page}) { + return nil + } + } + + return nil + }) +} + +type streamsDatasetPage struct { + dec StreamsDecoder + desc *streamsmd.PageDesc + + info *dataset.PageInfo +} + +func (p *streamsDatasetPage) PageInfo() *dataset.PageInfo { + if p.info != nil { + return p.info + } + + p.info = &dataset.PageInfo{ + UncompressedSize: int(p.desc.Info.UncompressedSize), + CompressedSize: int(p.desc.Info.CompressedSize), + CRC32: p.desc.Info.Crc32, + RowCount: int(p.desc.Info.RowsCount), + + Encoding: p.desc.Info.Encoding, + Stats: p.desc.Info.Statistics, + } + return p.info +} + +func (p *streamsDatasetPage) ReadPage(ctx context.Context) (dataset.PageData, error) { + pages, err := result.Collect(p.dec.ReadPages(ctx, []*streamsmd.PageDesc{p.desc})) + if err != nil { + return nil, err + } else if len(pages) != 1 { + return nil, fmt.Errorf("unexpected number of pages: got=%d want=1", len(pages)) + } + + return pages[0], nil +} diff --git a/pkg/dataobj/internal/encoding/decoder.go b/pkg/dataobj/internal/encoding/decoder.go new file mode 100644 index 0000000000000..d6ed5a87e4dfb --- /dev/null +++ b/pkg/dataobj/internal/encoding/decoder.go @@ -0,0 +1,41 @@ +package encoding + +import ( + "context" + + "github.com/grafana/loki/v3/pkg/dataobj/internal/dataset" + "github.com/grafana/loki/v3/pkg/dataobj/internal/metadata/filemd" + "github.com/grafana/loki/v3/pkg/dataobj/internal/metadata/streamsmd" + "github.com/grafana/loki/v3/pkg/dataobj/internal/result" +) + +// Decoders. To cleanly separate the APIs per section, section-specific +// Decoders should be created and returned by the top-level [Decoder] +// interface. +type ( + // A Decoder decodes a data object. + Decoder interface { + // Sections returns the list of sections within a data object. + Sections(ctx context.Context) ([]*filemd.SectionInfo, error) + + // StreamsDecoder returns a decoder for streams sections. + StreamsDecoder() StreamsDecoder + } + + // StreamsDecoder supports decoding data within a streams section. + StreamsDecoder interface { + // Columns describes the set of columns in the provided section. + Columns(ctx context.Context, section *filemd.SectionInfo) ([]*streamsmd.ColumnDesc, error) + + // Pages retrieves the set of pages for the provided columns. The order of + // page lists emitted by the sequence matches the order of columns + // provided: the first page list corresponds to the first column, and so + // on. + Pages(ctx context.Context, columns []*streamsmd.ColumnDesc) result.Seq[[]*streamsmd.PageDesc] + + // ReadPages reads the provided set of pages, iterating over their data + // matching the argument order. If an error is encountered while retrieving + // pages, an error is emitted and iteration stops. + ReadPages(ctx context.Context, pages []*streamsmd.PageDesc) result.Seq[dataset.PageData] + } +) diff --git a/pkg/dataobj/internal/encoding/decoder_metadata.go b/pkg/dataobj/internal/encoding/decoder_metadata.go new file mode 100644 index 0000000000000..c1c3011ac6612 --- /dev/null +++ b/pkg/dataobj/internal/encoding/decoder_metadata.go @@ -0,0 +1,83 @@ +package encoding + +import ( + "bytes" + "encoding/binary" + "fmt" + "io" + + "github.com/gogo/protobuf/proto" + + "github.com/grafana/loki/v3/pkg/dataobj/internal/metadata/filemd" + "github.com/grafana/loki/v3/pkg/dataobj/internal/metadata/streamsmd" + "github.com/grafana/loki/v3/pkg/dataobj/internal/streamio" +) + +// decode* methods for metadata shared by Decoder implementations. + +// decodeFileMetadata decodes file metadata from r. +func decodeFileMetadata(r streamio.Reader) (*filemd.Metadata, error) { + gotVersion, err := streamio.ReadUvarint(r) + if err != nil { + return nil, fmt.Errorf("read file format version: %w", err) + } else if gotVersion != fileFormatVersion { + return nil, fmt.Errorf("unexpected file format version: got=%d want=%d", gotVersion, fileFormatVersion) + } + + var md filemd.Metadata + if err := decodeProto(r, &md); err != nil { + return nil, fmt.Errorf("file metadata: %w", err) + } + return &md, nil +} + +// decodeStreamsMetadata decodes stream section metadta from r. +func decodeStreamsMetadata(r streamio.Reader) (*streamsmd.Metadata, error) { + gotVersion, err := streamio.ReadUvarint(r) + if err != nil { + return nil, fmt.Errorf("read streams section format version: %w", err) + } else if gotVersion != streamsFormatVersion { + return nil, fmt.Errorf("unexpected streams section format version: got=%d want=%d", gotVersion, streamsFormatVersion) + } + + var md streamsmd.Metadata + if err := decodeProto(r, &md); err != nil { + return nil, fmt.Errorf("streams section metadata: %w", err) + } + return &md, nil +} + +// decodeStreamsColumnMetadata decodes stream column metadata from r. +func decodeStreamsColumnMetadata(r streamio.Reader) (*streamsmd.ColumnMetadata, error) { + var metadata streamsmd.ColumnMetadata + if err := decodeProto(r, &metadata); err != nil { + return nil, fmt.Errorf("streams column metadata: %w", err) + } + return &metadata, nil +} + +// decodeProto decodes a proto message from r and stores it in pb. Proto +// messages are expected to be encoded with their size, followed by the proto +// bytes. +func decodeProto(r streamio.Reader, pb proto.Message) error { + size, err := binary.ReadUvarint(r) + if err != nil { + return fmt.Errorf("read proto message size: %w", err) + } + + buf := bytesBufferPool.Get().(*bytes.Buffer) + buf.Reset() + defer bytesBufferPool.Put(buf) + + n, err := io.Copy(buf, io.LimitReader(r, int64(size))) + if err != nil { + return fmt.Errorf("read proto message: %w", err) + } else if uint64(n) != size { + return fmt.Errorf("read proto message: got=%d want=%d", n, size) + } + + if err := proto.Unmarshal(buf.Bytes(), pb); err != nil { + return fmt.Errorf("unmarshal proto message: %w", err) + } + return nil +} diff --git a/pkg/dataobj/internal/encoding/decoder_readseeker.go b/pkg/dataobj/internal/encoding/decoder_readseeker.go new file mode 100644 index 0000000000000..725273603eb63 --- /dev/null +++ b/pkg/dataobj/internal/encoding/decoder_readseeker.go @@ -0,0 +1,123 @@ +package encoding + +import ( + "bufio" + "context" + "encoding/binary" + "fmt" + "io" + + "github.com/grafana/loki/v3/pkg/dataobj/internal/dataset" + "github.com/grafana/loki/v3/pkg/dataobj/internal/metadata/filemd" + "github.com/grafana/loki/v3/pkg/dataobj/internal/metadata/streamsmd" + "github.com/grafana/loki/v3/pkg/dataobj/internal/result" +) + +type readSeekerDecoder struct { + rs io.ReadSeeker +} + +// ReadSeekerDecoder decodes a data object from the provided [io.ReadSeeker]. +func ReadSeekerDecoder(rs io.ReadSeeker) Decoder { + return &readSeekerDecoder{rs: rs} +} + +func (dec *readSeekerDecoder) Sections(_ context.Context) ([]*filemd.SectionInfo, error) { + var metadataSize uint32 + if _, err := dec.rs.Seek(-8, io.SeekEnd); err != nil { + return nil, fmt.Errorf("seek to file metadata size: %w", err) + } else if err := binary.Read(dec.rs, binary.LittleEndian, &metadataSize); err != nil { + return nil, fmt.Errorf("reading file metadata size: %w", err) + } + + if _, err := dec.rs.Seek(-int64(metadataSize)-8, io.SeekEnd); err != nil { + return nil, fmt.Errorf("seek to file metadata: %w", err) + } + + r := bufio.NewReader(io.LimitReader(dec.rs, int64(metadataSize))) + + md, err := decodeFileMetadata(r) + if err != nil { + return nil, err + } + return md.Sections, nil +} + +func (dec *readSeekerDecoder) StreamsDecoder() StreamsDecoder { + return &readSeekerStreamsDecoder{rs: dec.rs} +} + +type readSeekerStreamsDecoder struct { + rs io.ReadSeeker +} + +func (dec *readSeekerStreamsDecoder) Columns(_ context.Context, section *filemd.SectionInfo) ([]*streamsmd.ColumnDesc, error) { + if section.Type != filemd.SECTION_TYPE_STREAMS { + return nil, fmt.Errorf("unexpected section type: got=%d want=%d", section.Type, filemd.SECTION_TYPE_STREAMS) + } + + if _, err := dec.rs.Seek(int64(section.MetadataOffset), io.SeekStart); err != nil { + return nil, fmt.Errorf("seek to streams metadata: %w", err) + } + r := bufio.NewReader(io.LimitReader(dec.rs, int64(section.MetadataSize))) + + md, err := decodeStreamsMetadata(r) + if err != nil { + return nil, err + } + return md.Columns, nil +} + +func (dec *readSeekerStreamsDecoder) Pages(ctx context.Context, columns []*streamsmd.ColumnDesc) result.Seq[[]*streamsmd.PageDesc] { + getPages := func(_ context.Context, column *streamsmd.ColumnDesc) ([]*streamsmd.PageDesc, error) { + if _, err := dec.rs.Seek(int64(column.Info.MetadataOffset), io.SeekStart); err != nil { + return nil, fmt.Errorf("seek to column metadata: %w", err) + } + r := bufio.NewReader(io.LimitReader(dec.rs, int64(column.Info.MetadataSize))) + + md, err := decodeStreamsColumnMetadata(r) + if err != nil { + return nil, err + } + return md.Pages, nil + } + + return result.Iter(func(yield func([]*streamsmd.PageDesc) bool) error { + for _, column := range columns { + pages, err := getPages(ctx, column) + if err != nil { + return err + } else if !yield(pages) { + return nil + } + } + + return nil + }) +} + +func (dec *readSeekerStreamsDecoder) ReadPages(ctx context.Context, pages []*streamsmd.PageDesc) result.Seq[dataset.PageData] { + getPageData := func(_ context.Context, page *streamsmd.PageDesc) (dataset.PageData, error) { + if _, err := dec.rs.Seek(int64(page.Info.DataOffset), io.SeekStart); err != nil { + return nil, err + } + data := make([]byte, page.Info.DataSize) + if _, err := io.ReadFull(dec.rs, data); err != nil { + return nil, fmt.Errorf("read page data: %w", err) + } + return dataset.PageData(data), nil + } + + return result.Iter(func(yield func(dataset.PageData) bool) error { + for _, page := range pages { + data, err := getPageData(ctx, page) + if err != nil { + return err + } else if !yield(data) { + return nil + } + } + + return nil + }) +} diff --git a/pkg/dataobj/internal/encoding/encoder.go b/pkg/dataobj/internal/encoding/encoder.go new file mode 100644 index 0000000000000..28a64524e9735 --- /dev/null +++ b/pkg/dataobj/internal/encoding/encoder.go @@ -0,0 +1,152 @@ +package encoding + +import ( + "bytes" + "encoding/binary" + "fmt" + "math" + + "github.com/gogo/protobuf/proto" + + "github.com/grafana/loki/v3/pkg/dataobj/internal/metadata/filemd" + "github.com/grafana/loki/v3/pkg/dataobj/internal/streamio" +) + +// Encoder encodes a data object. Data objects are hierarchical, split into +// distinct sections that contain their own hierarchy. +// +// To support hierarchical encoding, a set of Open* methods are provided to +// open a child element. Only one child element may be open at a given time; +// call Commit or Discard on a child element to close it. +type Encoder struct { + w streamio.Writer + + startOffset int // Byte offset in the file where data starts after the header. + + sections []*filemd.SectionInfo + curSection *filemd.SectionInfo + + data *bytes.Buffer +} + +// NewEncoder creates a new Encoder which writes a data object to the provided +// writer. +func NewEncoder(w streamio.Writer) *Encoder { + buf := bytesBufferPool.Get().(*bytes.Buffer) + buf.Reset() + + return &Encoder{ + w: w, + + startOffset: len(magic), + + data: buf, + } +} + +// OpenStreams opens a [StreamsEncoder]. OpenSterams fails if there is another +// open section. +func (enc *Encoder) OpenStreams() (*StreamsEncoder, error) { + if enc.curSection != nil { + return nil, ErrElementExist + } + + // MetadtaOffset and MetadataSize aren't available until the section is + // closed. We temporarily set these fields to the maximum values so they're + // accounted for in the MetadataSize estimate. + enc.curSection = &filemd.SectionInfo{ + Type: filemd.SECTION_TYPE_STREAMS, + MetadataOffset: math.MaxUint32, + MetadataSize: math.MaxUint32, + } + + return newStreamsEncoder( + enc, + enc.startOffset+enc.data.Len(), + ), nil +} + +// MetadataSize returns an estimate of the current size of the metadata for the +// data object. MetadataSize does not include the size of data appended. The +// estimate includes the currently open element. +func (enc *Encoder) MetadataSize() int { return elementMetadataSize(enc) } + +func (enc *Encoder) metadata() proto.Message { + sections := enc.sections[:len(enc.sections):cap(enc.sections)] + if enc.curSection != nil { + sections = append(sections, enc.curSection) + } + return &filemd.Metadata{Sections: sections} +} + +// Flush flushes any buffered data to the underlying writer. After flushing, +// enc is reset. Flush fails if there is currently an open section. +func (enc *Encoder) Flush() error { + if enc.curSection != nil { + return ErrElementExist + } + + metadataBuffer := bytesBufferPool.Get().(*bytes.Buffer) + metadataBuffer.Reset() + defer bytesBufferPool.Put(metadataBuffer) + + // The file metadata should start with the version. + if err := streamio.WriteUvarint(metadataBuffer, fileFormatVersion); err != nil { + return err + } else if err := elementMetadataWrite(enc, metadataBuffer); err != nil { + return err + } + + // The overall structure is: + // + // header: + // [magic] + // body: + // [data] + // [metadata] + // tailer: + // [file metadata size (32 bits)] + // [magic] + // + // The file metadata size *must not* be varint since we need the last 8 bytes + // of the file to consistently retrieve the tailer. + + if _, err := enc.w.Write(magic); err != nil { + return fmt.Errorf("writing magic header: %w", err) + } else if _, err := enc.w.Write(enc.data.Bytes()); err != nil { + return fmt.Errorf("writing data: %w", err) + } else if _, err := enc.w.Write(metadataBuffer.Bytes()); err != nil { + return fmt.Errorf("writing metadata: %w", err) + } else if err := binary.Write(enc.w, binary.LittleEndian, uint32(metadataBuffer.Len())); err != nil { + return fmt.Errorf("writing metadata size: %w", err) + } else if _, err := enc.w.Write(magic); err != nil { + return fmt.Errorf("writing magic tailer: %w", err) + } + + enc.sections = nil + enc.data.Reset() + return nil +} + +func (enc *Encoder) append(data, metadata []byte) error { + if enc.curSection == nil { + return errElementNoExist + } + + if len(data) == 0 && len(metadata) == 0 { + // Section was discarded. + enc.curSection = nil + return nil + } + + enc.curSection.MetadataOffset = uint32(enc.startOffset + enc.data.Len() + len(data)) + enc.curSection.MetadataSize = uint32(len(metadata)) + + // bytes.Buffer.Write never fails. + _, _ = enc.data.Write(data) + _, _ = enc.data.Write(metadata) + + enc.sections = append(enc.sections, enc.curSection) + enc.curSection = nil + return nil +} diff --git a/pkg/dataobj/internal/encoding/encoder_helpers.go b/pkg/dataobj/internal/encoding/encoder_helpers.go new file mode 100644 index 0000000000000..87504eec811fa --- /dev/null +++ b/pkg/dataobj/internal/encoding/encoder_helpers.go @@ -0,0 +1,42 @@ +package encoding + +import ( + "io" + + "github.com/gogo/protobuf/proto" + + "github.com/grafana/loki/v3/pkg/dataobj/internal/streamio" +) + +type element interface { + metadata() proto.Message +} + +func elementMetadataSize(e element) int { + return proto.Size(e.metadata()) +} + +func elementMetadataWrite(e element, w streamio.Writer) error { + buf := protoBufferPool.Get().(*proto.Buffer) + buf.Reset() + defer protoBufferPool.Put(buf) + + if err := buf.Marshal(e.metadata()); err != nil { + return err + } + + // Every protobuf message is always prepended with its size as a uvarint. + messageSize := len(buf.Bytes()) + if err := streamio.WriteUvarint(w, uint64(messageSize)); err != nil { + return err + } + + sz, err := w.Write(buf.Bytes()) + if err != nil { + return err + } else if sz != messageSize { + return io.ErrShortWrite + } + + return nil +} diff --git a/pkg/dataobj/internal/encoding/encoder_streams.go b/pkg/dataobj/internal/encoding/encoder_streams.go new file mode 100644 index 0000000000000..b02e5a90bd714 --- /dev/null +++ b/pkg/dataobj/internal/encoding/encoder_streams.go @@ -0,0 +1,259 @@ +package encoding + +import ( + "bytes" + "math" + + "github.com/gogo/protobuf/proto" + + "github.com/grafana/loki/v3/pkg/dataobj/internal/dataset" + "github.com/grafana/loki/v3/pkg/dataobj/internal/metadata/datasetmd" + "github.com/grafana/loki/v3/pkg/dataobj/internal/metadata/streamsmd" + "github.com/grafana/loki/v3/pkg/dataobj/internal/streamio" +) + +// StreamsEncoder encodes an individual streams section in a data object. +// StreamsEncoder are created by [Encoder]s. +type StreamsEncoder struct { + parent *Encoder + + startOffset int // Byte offset in the file where the column starts. + closed bool // true if StreamsEncoder has been closed. + + data *bytes.Buffer + columns []*streamsmd.ColumnDesc // closed columns. + curColumn *streamsmd.ColumnDesc // curColumn is the currently open column. +} + +func newStreamsEncoder(parent *Encoder, offset int) *StreamsEncoder { + buf := bytesBufferPool.Get().(*bytes.Buffer) + buf.Reset() + + return &StreamsEncoder{ + parent: parent, + startOffset: offset, + + data: buf, + } +} + +// OpenColumn opens a new column in the streams section. OpenColumn fails if +// there is another open column or if the StreamsEncoder has been closed. +func (enc *StreamsEncoder) OpenColumn(columnType streamsmd.ColumnType, info *dataset.ColumnInfo) (*StreamsColumnEncoder, error) { + if enc.curColumn != nil { + return nil, ErrElementExist + } else if enc.closed { + return nil, ErrClosed + } + + // MetadataOffset and MetadataSize aren't available until the column is + // closed. We temporarily set these fields to the maximum values so they're + // accounted for in the MetadataSize estimate. + enc.curColumn = &streamsmd.ColumnDesc{ + Type: columnType, + Info: &datasetmd.ColumnInfo{ + Name: info.Name, + ValueType: info.Type, + RowsCount: uint32(info.RowsCount), + Compression: info.Compression, + UncompressedSize: uint32(info.UncompressedSize), + CompressedSize: uint32(info.CompressedSize), + Statistics: info.Statistics, + + MetadataOffset: math.MaxUint32, + MetadataSize: math.MaxUint32, + }, + } + + return newStreamsColumnEncoder( + enc, + enc.startOffset+enc.data.Len(), + ), nil +} + +// MetadataSize returns an estimate of the current size of the metadata for the +// stream. MetadataSize includes an estimate for the currently open element. +func (enc *StreamsEncoder) MetadataSize() int { return elementMetadataSize(enc) } + +func (enc *StreamsEncoder) metadata() proto.Message { + columns := enc.columns[:len(enc.columns):cap(enc.columns)] + if enc.curColumn != nil { + columns = append(columns, enc.curColumn) + } + return &streamsmd.Metadata{Columns: columns} +} + +// Commit closes the section, flushing all data to the parent element. After +// Commit is called, the StreamsEncoder can no longer be modified. +// +// Commit fails if there is an open column. +func (enc *StreamsEncoder) Commit() error { + if enc.closed { + return ErrClosed + } else if enc.curColumn != nil { + return ErrElementExist + } + + defer bytesBufferPool.Put(enc.data) + + if len(enc.columns) == 0 { + // No data was written; discard. + return enc.parent.append(nil, nil) + } + + metadataBuffer := bytesBufferPool.Get().(*bytes.Buffer) + metadataBuffer.Reset() + defer bytesBufferPool.Put(metadataBuffer) + + // The section metadata should start with its version. + if err := streamio.WriteUvarint(metadataBuffer, streamsFormatVersion); err != nil { + return err + } else if err := elementMetadataWrite(enc, metadataBuffer); err != nil { + return err + } + return enc.parent.append(enc.data.Bytes(), metadataBuffer.Bytes()) +} + +// Discard discards the section, discarding any data written to it. After +// Discard is called, the StreamsEncoder can no longer be modified. +// +// Discard fails if there is an open column. +func (enc *StreamsEncoder) Discard() error { + if enc.closed { + return ErrClosed + } else if enc.curColumn != nil { + return ErrElementExist + } + enc.closed = true + + defer bytesBufferPool.Put(enc.data) + + return enc.parent.append(nil, nil) +} + +// append adds data and metadata to enc. append must only be called from child +// elements on Close and Discard. Discard calls must pass nil for both data and +// metadata to denote a discard. +func (enc *StreamsEncoder) append(data, metadata []byte) error { + if enc.closed { + return ErrClosed + } else if enc.curColumn == nil { + return errElementNoExist + } + + if len(data) == 0 && len(metadata) == 0 { + // Column was discarded. + enc.curColumn = nil + return nil + } + + enc.curColumn.Info.MetadataOffset = uint32(enc.startOffset + enc.data.Len() + len(data)) + enc.curColumn.Info.MetadataSize = uint32(len(metadata)) + + // bytes.Buffer.Write never fails. + _, _ = enc.data.Write(data) + _, _ = enc.data.Write(metadata) + + enc.columns = append(enc.columns, enc.curColumn) + enc.curColumn = nil + return nil +} + +// StreamsColumnEncoder encodes an individual column in a streams section. +// StreamsColumnEncoder are created by [StreamsEncoder]. +type StreamsColumnEncoder struct { + parent *StreamsEncoder + + startOffset int // Byte offset in the file where the column starts. + closed bool // true if StreamsColumnEncoder has been closed. + + data *bytes.Buffer // All page data. + pages []*streamsmd.PageDesc +} + +func newStreamsColumnEncoder(parent *StreamsEncoder, offset int) *StreamsColumnEncoder { + buf := bytesBufferPool.Get().(*bytes.Buffer) + buf.Reset() + + return &StreamsColumnEncoder{ + parent: parent, + startOffset: offset, + + data: buf, + } +} + +// AppendPage appens a new [dataset.MemPage] to the column. AppendPage fails if +// the column has been closed. +func (enc *StreamsColumnEncoder) AppendPage(page *dataset.MemPage) error { + if enc.closed { + return ErrClosed + } + + // It's possible the caller can pass an incorrect value for UncompressedSize + // and CompressedSize, but those fields are purely for stats so we don't + // check it. + enc.pages = append(enc.pages, &streamsmd.PageDesc{ + Info: &datasetmd.PageInfo{ + UncompressedSize: uint32(page.Info.UncompressedSize), + CompressedSize: uint32(page.Info.CompressedSize), + Crc32: page.Info.CRC32, + RowsCount: uint32(page.Info.RowCount), + Encoding: page.Info.Encoding, + + DataOffset: uint32(enc.startOffset + enc.data.Len()), + DataSize: uint32(len(page.Data)), + + Statistics: page.Info.Stats, + }, + }) + + _, _ = enc.data.Write(page.Data) // bytes.Buffer.Write never fails. + return nil +} + +// MetadataSize returns an estimate of the current size of the metadata for the +// column. MetadataSize does not include the size of data appended. +func (enc *StreamsColumnEncoder) MetadataSize() int { return elementMetadataSize(enc) } + +func (enc *StreamsColumnEncoder) metadata() proto.Message { + return &streamsmd.ColumnMetadata{Pages: enc.pages} +} + +// Commit closes the column, flushing all data to the parent element. After +// Commit is called, the StreamsColumnEncoder can no longer be modified. +func (enc *StreamsColumnEncoder) Commit() error { + if enc.closed { + return ErrClosed + } + enc.closed = true + + defer bytesBufferPool.Put(enc.data) + + if len(enc.pages) == 0 { + // No data was written; discard. + return enc.parent.append(nil, nil) + } + + metadataBuffer := bytesBufferPool.Get().(*bytes.Buffer) + metadataBuffer.Reset() + defer bytesBufferPool.Put(metadataBuffer) + + if err := elementMetadataWrite(enc, metadataBuffer); err != nil { + return err + } + return enc.parent.append(enc.data.Bytes(), metadataBuffer.Bytes()) +} + +// Discard discards the column, discarding any data written to it. After +// Discard is called, the StreamsColumnEncoder can no longer be modified. +func (enc *StreamsColumnEncoder) Discard() error { + if enc.closed { + return ErrClosed + } + enc.closed = true + + defer bytesBufferPool.Put(enc.data) + + return enc.parent.append(nil, nil) // Notify parent of discard. +} diff --git a/pkg/dataobj/internal/encoding/encoding.go b/pkg/dataobj/internal/encoding/encoding.go new file mode 100644 index 0000000000000..8b229970a8aea --- /dev/null +++ b/pkg/dataobj/internal/encoding/encoding.go @@ -0,0 +1,23 @@ +// Package encoding provides utilities for encoding and decoding data objects. +package encoding + +import "errors" + +var ( + magic = []byte("THOR") +) + +const ( + fileFormatVersion = 0x1 + streamsFormatVersion = 0x1 +) + +var ( + ErrElementExist = errors.New("open element already exists") + ErrClosed = errors.New("element is closed") + + // errElementNoExist is used when a child element tries to notify its parent + // of it closing but the parent doesn't have a child open. This would + // indicate a bug in the encoder so it's not exposed to callers. + errElementNoExist = errors.New("open element does not exist") +) diff --git a/pkg/dataobj/internal/encoding/encoding_test.go b/pkg/dataobj/internal/encoding/encoding_test.go new file mode 100644 index 0000000000000..f5f7400e986b9 --- /dev/null +++ b/pkg/dataobj/internal/encoding/encoding_test.go @@ -0,0 +1,109 @@ +package encoding_test + +import ( + "bytes" + "context" + "testing" + + "github.com/stretchr/testify/require" + + "github.com/grafana/loki/v3/pkg/dataobj/internal/dataset" + "github.com/grafana/loki/v3/pkg/dataobj/internal/encoding" + "github.com/grafana/loki/v3/pkg/dataobj/internal/metadata/datasetmd" + "github.com/grafana/loki/v3/pkg/dataobj/internal/metadata/filemd" + "github.com/grafana/loki/v3/pkg/dataobj/internal/metadata/streamsmd" + "github.com/grafana/loki/v3/pkg/dataobj/internal/result" +) + +func Test(t *testing.T) { + type Country struct { + Name string + Capital string + } + + countries := []Country{ + {"India", "New Delhi"}, + {"USA", "Washington D.C."}, + {"UK", "London"}, + {"France", "Paris"}, + {"Germany", "Berlin"}, + } + + var buf bytes.Buffer + + t.Run("Encode", func(t *testing.T) { + nameBuilder, err := dataset.NewColumnBuilder("name", dataset.BuilderOptions{ + Value: datasetmd.VALUE_TYPE_STRING, + Encoding: datasetmd.ENCODING_TYPE_PLAIN, + Compression: datasetmd.COMPRESSION_TYPE_NONE, + }) + require.NoError(t, err) + + capitalBuilder, err := dataset.NewColumnBuilder("capital", dataset.BuilderOptions{ + Value: datasetmd.VALUE_TYPE_STRING, + Encoding: datasetmd.ENCODING_TYPE_PLAIN, + Compression: datasetmd.COMPRESSION_TYPE_NONE, + }) + require.NoError(t, err) + + for i, c := range countries { + require.NoError(t, nameBuilder.Append(i, dataset.StringValue(c.Name))) + require.NoError(t, capitalBuilder.Append(i, dataset.StringValue(c.Capital))) + } + + nameColumn, err := nameBuilder.Flush() + require.NoError(t, err) + capitalColumn, err := capitalBuilder.Flush() + require.NoError(t, err) + + enc := encoding.NewEncoder(&buf) + streamsEnc, err := enc.OpenStreams() + require.NoError(t, err) + + colEnc, err := streamsEnc.OpenColumn(streamsmd.COLUMN_TYPE_LABEL, &nameColumn.Info) + require.NoError(t, err) + for _, page := range nameColumn.Pages { + require.NoError(t, colEnc.AppendPage(page)) + } + require.NoError(t, colEnc.Commit()) + + colEnc, err = streamsEnc.OpenColumn(streamsmd.COLUMN_TYPE_LABEL, &capitalColumn.Info) + require.NoError(t, err) + for _, page := range capitalColumn.Pages { + require.NoError(t, colEnc.AppendPage(page)) + } + require.NoError(t, colEnc.Commit()) + + require.NoError(t, streamsEnc.Commit()) + require.NoError(t, enc.Flush()) + }) + + t.Run("Decode", func(t *testing.T) { + dec := encoding.ReadSeekerDecoder(bytes.NewReader(buf.Bytes())) + sections, err := dec.Sections(context.TODO()) + require.NoError(t, err) + require.Len(t, sections, 1) + require.Equal(t, filemd.SECTION_TYPE_STREAMS, sections[0].Type) + + dset := encoding.StreamsDataset(dec.StreamsDecoder(), sections[0]) + + columns, err := result.Collect(dset.ListColumns(context.Background())) + require.NoError(t, err) + + var actual []Country + + for result := range dataset.Iter(context.Background(), columns) { + row, err := result.Value() + require.NoError(t, err) + require.Len(t, row.Values, 2) + require.Equal(t, len(actual), row.Index) + + actual = append(actual, Country{ + Name: row.Values[0].String(), + Capital: row.Values[1].String(), + }) + } + + require.Equal(t, countries, actual) + }) +} diff --git a/pkg/dataobj/internal/encoding/pools.go b/pkg/dataobj/internal/encoding/pools.go new file mode 100644 index 0000000000000..5edf16eb6acdc --- /dev/null +++ b/pkg/dataobj/internal/encoding/pools.go @@ -0,0 +1,20 @@ +package encoding + +import ( + "bytes" + "sync" + + "github.com/gogo/protobuf/proto" +) + +var bytesBufferPool = sync.Pool{ + New: func() any { + return new(bytes.Buffer) + }, +} + +var protoBufferPool = sync.Pool{ + New: func() any { + return new(proto.Buffer) + }, +} diff --git a/pkg/dataobj/internal/metadata/datasetmd/datasetmd.pb.go b/pkg/dataobj/internal/metadata/datasetmd/datasetmd.pb.go index 4121ca928c598..fcbbefbebef32 100644 --- a/pkg/dataobj/internal/metadata/datasetmd/datasetmd.pb.go +++ b/pkg/dataobj/internal/metadata/datasetmd/datasetmd.pb.go @@ -58,6 +58,39 @@ func (ValueType) EnumDescriptor() ([]byte, []int) { return fileDescriptor_7ab9d5b21b743868, []int{0} } +// CompressionType represents the valid compression types that can be used for +// compressing values in a page. +type CompressionType int32 + +const ( + // Invalid compression type. + COMPRESSION_TYPE_UNSPECIFIED CompressionType = 0 + // No compression. + COMPRESSION_TYPE_NONE CompressionType = 1 + // Snappy compression. + COMPRESSION_TYPE_SNAPPY CompressionType = 2 + // Zstd compression. + COMPRESSION_TYPE_ZSTD CompressionType = 3 +) + +var CompressionType_name = map[int32]string{ + 0: "COMPRESSION_TYPE_UNSPECIFIED", + 1: "COMPRESSION_TYPE_NONE", + 2: "COMPRESSION_TYPE_SNAPPY", + 3: "COMPRESSION_TYPE_ZSTD", +} + +var CompressionType_value = map[string]int32{ + "COMPRESSION_TYPE_UNSPECIFIED": 0, + "COMPRESSION_TYPE_NONE": 1, + "COMPRESSION_TYPE_SNAPPY": 2, + "COMPRESSION_TYPE_ZSTD": 3, +} + +func (CompressionType) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_7ab9d5b21b743868, []int{1} +} + // EncodingType represents the valid types that a sequence of values which a // column can be encoded with. type EncodingType int32 @@ -90,40 +123,125 @@ var EncodingType_value = map[string]int32{ } func (EncodingType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_7ab9d5b21b743868, []int{1} + return fileDescriptor_7ab9d5b21b743868, []int{2} } -// CompressionType represents the valid compression types that can be used for -// compressing values in a page. -type CompressionType int32 +// ColumnInfo describes an individual column within a data set. +type ColumnInfo struct { + // Optional name of the column. + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + // Type of values stored within the column. + ValueType ValueType `protobuf:"varint,2,opt,name=value_type,json=valueType,proto3,enum=dataobj.metadata.dataset.v1.ValueType" json:"value_type,omitempty"` + // Total number of rows in the entire column. + RowsCount uint32 `protobuf:"varint,3,opt,name=rows_count,json=rowsCount,proto3" json:"rows_count,omitempty"` + // Compression type used for all pages. + Compression CompressionType `protobuf:"varint,4,opt,name=compression,proto3,enum=dataobj.metadata.dataset.v1.CompressionType" json:"compression,omitempty"` + // Total uncompressed size of all pages in the column. + UncompressedSize uint32 `protobuf:"varint,5,opt,name=uncompressed_size,json=uncompressedSize,proto3" json:"uncompressed_size,omitempty"` + // Total compressed size of all pages in the column. Compressed size may + // match uncompressed size if no compression is used. + CompressedSize uint32 `protobuf:"varint,6,opt,name=compressed_size,json=compressedSize,proto3" json:"compressed_size,omitempty"` + // Byte offset from the start of the data object to the column's metadata. + MetadataOffset uint32 `protobuf:"varint,7,opt,name=metadata_offset,json=metadataOffset,proto3" json:"metadata_offset,omitempty"` + // Size of the column's metadata in bytes. + MetadataSize uint32 `protobuf:"varint,8,opt,name=metadata_size,json=metadataSize,proto3" json:"metadata_size,omitempty"` + // Statistics for the column. + Statistics *Statistics `protobuf:"bytes,9,opt,name=statistics,proto3" json:"statistics,omitempty"` +} -const ( - // Invalid compression type. - COMPRESSION_TYPE_UNSPECIFIED CompressionType = 0 - // No compression. - COMPRESSION_TYPE_NONE CompressionType = 1 - // Snappy compression. - COMPRESSION_TYPE_SNAPPY CompressionType = 2 - // Zstd compression. - COMPRESSION_TYPE_ZSTD CompressionType = 3 -) +func (m *ColumnInfo) Reset() { *m = ColumnInfo{} } +func (*ColumnInfo) ProtoMessage() {} +func (*ColumnInfo) Descriptor() ([]byte, []int) { + return fileDescriptor_7ab9d5b21b743868, []int{0} +} +func (m *ColumnInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ColumnInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ColumnInfo.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *ColumnInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_ColumnInfo.Merge(m, src) +} +func (m *ColumnInfo) XXX_Size() int { + return m.Size() +} +func (m *ColumnInfo) XXX_DiscardUnknown() { + xxx_messageInfo_ColumnInfo.DiscardUnknown(m) +} -var CompressionType_name = map[int32]string{ - 0: "COMPRESSION_TYPE_UNSPECIFIED", - 1: "COMPRESSION_TYPE_NONE", - 2: "COMPRESSION_TYPE_SNAPPY", - 3: "COMPRESSION_TYPE_ZSTD", +var xxx_messageInfo_ColumnInfo proto.InternalMessageInfo + +func (m *ColumnInfo) GetName() string { + if m != nil { + return m.Name + } + return "" } -var CompressionType_value = map[string]int32{ - "COMPRESSION_TYPE_UNSPECIFIED": 0, - "COMPRESSION_TYPE_NONE": 1, - "COMPRESSION_TYPE_SNAPPY": 2, - "COMPRESSION_TYPE_ZSTD": 3, +func (m *ColumnInfo) GetValueType() ValueType { + if m != nil { + return m.ValueType + } + return VALUE_TYPE_UNSPECIFIED } -func (CompressionType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_7ab9d5b21b743868, []int{2} +func (m *ColumnInfo) GetRowsCount() uint32 { + if m != nil { + return m.RowsCount + } + return 0 +} + +func (m *ColumnInfo) GetCompression() CompressionType { + if m != nil { + return m.Compression + } + return COMPRESSION_TYPE_UNSPECIFIED +} + +func (m *ColumnInfo) GetUncompressedSize() uint32 { + if m != nil { + return m.UncompressedSize + } + return 0 +} + +func (m *ColumnInfo) GetCompressedSize() uint32 { + if m != nil { + return m.CompressedSize + } + return 0 +} + +func (m *ColumnInfo) GetMetadataOffset() uint32 { + if m != nil { + return m.MetadataOffset + } + return 0 +} + +func (m *ColumnInfo) GetMetadataSize() uint32 { + if m != nil { + return m.MetadataSize + } + return 0 +} + +func (m *ColumnInfo) GetStatistics() *Statistics { + if m != nil { + return m.Statistics + } + return nil } // Statistics about a column or a page. All statistics are optional and are @@ -138,7 +256,7 @@ type Statistics struct { func (m *Statistics) Reset() { *m = Statistics{} } func (*Statistics) ProtoMessage() {} func (*Statistics) Descriptor() ([]byte, []int) { - return fileDescriptor_7ab9d5b21b743868, []int{0} + return fileDescriptor_7ab9d5b21b743868, []int{1} } func (m *Statistics) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -181,11 +299,122 @@ func (m *Statistics) GetMaxValue() []byte { return nil } +// Page describes an individual page within a column. +type PageInfo struct { + // Uncompressed size of the page within the data object. + UncompressedSize uint32 `protobuf:"varint,1,opt,name=uncompressed_size,json=uncompressedSize,proto3" json:"uncompressed_size,omitempty"` + // Compressed size of the page within the data object. Compression size + // will match uncompressed size if no compression is used. + CompressedSize uint32 `protobuf:"varint,2,opt,name=compressed_size,json=compressedSize,proto3" json:"compressed_size,omitempty"` + // CRC32 checksum of the page data. + Crc32 uint32 `protobuf:"varint,3,opt,name=crc32,proto3" json:"crc32,omitempty"` + // Number of rows in the page. + RowsCount uint32 `protobuf:"varint,4,opt,name=rows_count,json=rowsCount,proto3" json:"rows_count,omitempty"` + // Encoding type used for the page. + Encoding EncodingType `protobuf:"varint,5,opt,name=encoding,proto3,enum=dataobj.metadata.dataset.v1.EncodingType" json:"encoding,omitempty"` + // Byte offset from the start of the data object to the page's data. + DataOffset uint32 `protobuf:"varint,6,opt,name=data_offset,json=dataOffset,proto3" json:"data_offset,omitempty"` + // Size of the page's data in bytes. + DataSize uint32 `protobuf:"varint,7,opt,name=data_size,json=dataSize,proto3" json:"data_size,omitempty"` + // Optional statistics for the page. + Statistics *Statistics `protobuf:"bytes,8,opt,name=statistics,proto3" json:"statistics,omitempty"` +} + +func (m *PageInfo) Reset() { *m = PageInfo{} } +func (*PageInfo) ProtoMessage() {} +func (*PageInfo) Descriptor() ([]byte, []int) { + return fileDescriptor_7ab9d5b21b743868, []int{2} +} +func (m *PageInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *PageInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_PageInfo.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *PageInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_PageInfo.Merge(m, src) +} +func (m *PageInfo) XXX_Size() int { + return m.Size() +} +func (m *PageInfo) XXX_DiscardUnknown() { + xxx_messageInfo_PageInfo.DiscardUnknown(m) +} + +var xxx_messageInfo_PageInfo proto.InternalMessageInfo + +func (m *PageInfo) GetUncompressedSize() uint32 { + if m != nil { + return m.UncompressedSize + } + return 0 +} + +func (m *PageInfo) GetCompressedSize() uint32 { + if m != nil { + return m.CompressedSize + } + return 0 +} + +func (m *PageInfo) GetCrc32() uint32 { + if m != nil { + return m.Crc32 + } + return 0 +} + +func (m *PageInfo) GetRowsCount() uint32 { + if m != nil { + return m.RowsCount + } + return 0 +} + +func (m *PageInfo) GetEncoding() EncodingType { + if m != nil { + return m.Encoding + } + return ENCODING_TYPE_UNSPECIFIED +} + +func (m *PageInfo) GetDataOffset() uint32 { + if m != nil { + return m.DataOffset + } + return 0 +} + +func (m *PageInfo) GetDataSize() uint32 { + if m != nil { + return m.DataSize + } + return 0 +} + +func (m *PageInfo) GetStatistics() *Statistics { + if m != nil { + return m.Statistics + } + return nil +} + func init() { proto.RegisterEnum("dataobj.metadata.dataset.v1.ValueType", ValueType_name, ValueType_value) - proto.RegisterEnum("dataobj.metadata.dataset.v1.EncodingType", EncodingType_name, EncodingType_value) proto.RegisterEnum("dataobj.metadata.dataset.v1.CompressionType", CompressionType_name, CompressionType_value) + proto.RegisterEnum("dataobj.metadata.dataset.v1.EncodingType", EncodingType_name, EncodingType_value) + proto.RegisterType((*ColumnInfo)(nil), "dataobj.metadata.dataset.v1.ColumnInfo") proto.RegisterType((*Statistics)(nil), "dataobj.metadata.dataset.v1.Statistics") + proto.RegisterType((*PageInfo)(nil), "dataobj.metadata.dataset.v1.PageInfo") } func init() { @@ -193,34 +422,50 @@ func init() { } var fileDescriptor_7ab9d5b21b743868 = []byte{ - // 420 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x92, 0x31, 0x6f, 0xd3, 0x40, - 0x1c, 0xc5, 0x7d, 0x89, 0x84, 0xe8, 0x5f, 0x95, 0x38, 0x4c, 0x4b, 0x5b, 0x02, 0xa7, 0x8a, 0x09, - 0x65, 0xb0, 0x85, 0x8a, 0x60, 0x76, 0x92, 0x6b, 0x75, 0x52, 0x7a, 0xb1, 0x62, 0xb7, 0x52, 0xbb, - 0x44, 0x97, 0xc4, 0x84, 0x23, 0xf1, 0x9d, 0x65, 0x5f, 0xa3, 0x74, 0x63, 0x62, 0xe6, 0x63, 0xf0, - 0x51, 0x18, 0x33, 0x76, 0x24, 0xce, 0xc2, 0xd8, 0x8f, 0x80, 0x6a, 0x8c, 0x68, 0x48, 0x86, 0x6e, - 0xff, 0x7b, 0xbf, 0xf7, 0xf4, 0x6e, 0x78, 0xf0, 0x21, 0x19, 0x8f, 0xdc, 0xa1, 0x30, 0x42, 0xf7, - 0x3f, 0xbb, 0x52, 0x99, 0x28, 0x55, 0x62, 0xe2, 0xc6, 0x91, 0x11, 0x77, 0x62, 0x41, 0xb2, 0xc8, - 0xc4, 0xc3, 0x7f, 0x97, 0x93, 0xa4, 0xda, 0x68, 0xbb, 0x56, 0x86, 0x9c, 0xbf, 0x5e, 0xa7, 0x74, - 0x38, 0xd3, 0xb7, 0xaf, 0x8f, 0x01, 0x02, 0x23, 0x8c, 0xcc, 0x8c, 0x1c, 0x64, 0x76, 0x0d, 0xb6, - 0x62, 0xa9, 0x7a, 0x53, 0x31, 0xb9, 0x8a, 0xf6, 0xd1, 0x21, 0x7a, 0xb3, 0xdd, 0x7d, 0x1c, 0x4b, - 0x75, 0x7e, 0xf7, 0x2e, 0xa0, 0x98, 0x95, 0xb0, 0x52, 0x42, 0x31, 0x2b, 0x60, 0x7d, 0x0c, 0x5b, - 0xc5, 0x11, 0x5e, 0x27, 0x91, 0xfd, 0x02, 0x9e, 0x9f, 0x7b, 0xed, 0x33, 0xda, 0x0b, 0x2f, 0x7c, - 0xda, 0x3b, 0xe3, 0x81, 0x4f, 0x9b, 0xec, 0x98, 0xd1, 0x16, 0xb6, 0xec, 0x1d, 0xc0, 0xf7, 0x18, - 0xe3, 0xe1, 0xfb, 0x77, 0x18, 0xd9, 0xbb, 0xf0, 0xf4, 0x7e, 0xe2, 0x8f, 0x5c, 0xf9, 0x4f, 0x0e, - 0xc2, 0x2e, 0xe3, 0x27, 0xb8, 0x5a, 0xbf, 0x86, 0x6d, 0xaa, 0x06, 0x7a, 0x28, 0xd5, 0xa8, 0xe8, - 0x7b, 0x05, 0x07, 0x94, 0x37, 0x3b, 0x2d, 0xc6, 0x4f, 0x36, 0x55, 0xee, 0xc1, 0xb3, 0x55, 0xec, - 0xb7, 0x3d, 0xc6, 0x31, 0x5a, 0x07, 0x2d, 0xda, 0x0e, 0x3d, 0x5c, 0xb1, 0xf7, 0x61, 0x67, 0x15, - 0x34, 0x58, 0x78, 0xea, 0xf9, 0xb8, 0x5a, 0xff, 0x8a, 0xe0, 0x49, 0x53, 0xc7, 0x49, 0x1a, 0x65, - 0x99, 0xd4, 0xaa, 0xa8, 0x3f, 0x84, 0x97, 0xcd, 0xce, 0xa9, 0xdf, 0xa5, 0x41, 0xc0, 0x3a, 0x7c, - 0xd3, 0x0f, 0x0e, 0x60, 0x77, 0xcd, 0xc1, 0x3b, 0x9c, 0x62, 0x64, 0xd7, 0x60, 0x6f, 0x0d, 0x05, - 0xdc, 0xf3, 0xfd, 0x0b, 0x5c, 0xd9, 0x98, 0xbb, 0x0c, 0xc2, 0x16, 0xae, 0x36, 0x66, 0xf3, 0x05, - 0xb1, 0x6e, 0x16, 0xc4, 0xba, 0x5d, 0x10, 0xf4, 0x25, 0x27, 0xe8, 0x7b, 0x4e, 0xd0, 0x8f, 0x9c, - 0xa0, 0x79, 0x4e, 0xd0, 0xcf, 0x9c, 0xa0, 0x5f, 0x39, 0xb1, 0x6e, 0x73, 0x82, 0xbe, 0x2d, 0x89, - 0x35, 0x5f, 0x12, 0xeb, 0x66, 0x49, 0xac, 0xcb, 0xc6, 0x48, 0x9a, 0x4f, 0x57, 0x7d, 0x67, 0xa0, - 0x63, 0x77, 0x94, 0x8a, 0x8f, 0x42, 0x09, 0x77, 0xa2, 0xc7, 0xd2, 0x9d, 0x1e, 0xb9, 0x0f, 0xdc, - 0x57, 0xff, 0x51, 0x31, 0xab, 0xa3, 0xdf, 0x01, 0x00, 0x00, 0xff, 0xff, 0x79, 0x22, 0x11, 0xce, - 0x91, 0x02, 0x00, 0x00, + // 679 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x54, 0x4f, 0x4f, 0xdb, 0x4e, + 0x10, 0xcd, 0x26, 0xc0, 0x2f, 0x19, 0xfe, 0x99, 0xfd, 0x41, 0x31, 0x4d, 0x71, 0x23, 0x2a, 0x95, + 0x94, 0x56, 0xb6, 0x1a, 0xaa, 0xf6, 0x1c, 0x12, 0x83, 0x2c, 0x81, 0x63, 0xd9, 0x06, 0x09, 0x2e, + 0x96, 0x71, 0x36, 0xa9, 0x4b, 0x6c, 0x47, 0xb1, 0x93, 0x02, 0xa7, 0x9e, 0x7a, 0xee, 0xc7, 0xe8, + 0xf7, 0xe8, 0xa5, 0x47, 0x8e, 0x1c, 0x8b, 0x73, 0xe9, 0x91, 0x8f, 0x50, 0x65, 0xf3, 0xcf, 0x24, + 0x69, 0x84, 0x7a, 0x1b, 0xbf, 0xf7, 0x66, 0x66, 0x3d, 0x6f, 0x76, 0xe1, 0x43, 0xfd, 0xa2, 0x2a, + 0x94, 0xcd, 0xc0, 0xf4, 0xce, 0x3f, 0x09, 0xb6, 0x1b, 0x90, 0x86, 0x6b, 0xd6, 0x04, 0x87, 0x04, + 0x66, 0x07, 0xa4, 0x8c, 0x4f, 0x02, 0xa7, 0x3c, 0x8c, 0xf8, 0x7a, 0xc3, 0x0b, 0x3c, 0x9c, 0xee, + 0x25, 0xf1, 0x7d, 0x2d, 0xdf, 0x53, 0xf0, 0xad, 0xb7, 0x5b, 0x3f, 0x12, 0x00, 0x05, 0xaf, 0xd6, + 0x74, 0x5c, 0xc9, 0xad, 0x78, 0x18, 0xc3, 0x8c, 0x6b, 0x3a, 0x84, 0x45, 0x19, 0x94, 0x4d, 0xa9, + 0x34, 0xc6, 0x22, 0x40, 0xcb, 0xac, 0x35, 0x89, 0x11, 0x5c, 0xd5, 0x09, 0x1b, 0xcf, 0xa0, 0xec, + 0x52, 0xee, 0x25, 0x3f, 0xa5, 0x28, 0x7f, 0xd2, 0x91, 0xeb, 0x57, 0x75, 0xa2, 0xa6, 0x5a, 0xfd, + 0x10, 0x6f, 0x02, 0x34, 0xbc, 0xcf, 0xbe, 0x61, 0x79, 0x4d, 0x37, 0x60, 0x13, 0x19, 0x94, 0x5d, + 0x54, 0x53, 0x1d, 0xa4, 0xd0, 0x01, 0xb0, 0x0c, 0xf3, 0x96, 0xe7, 0xd4, 0x1b, 0xc4, 0xf7, 0x6d, + 0xcf, 0x65, 0x67, 0x68, 0x9b, 0x37, 0x53, 0xdb, 0x14, 0x86, 0x7a, 0xda, 0x2c, 0x5a, 0x00, 0xbf, + 0x86, 0x95, 0xa6, 0xdb, 0x07, 0x48, 0xd9, 0xf0, 0xed, 0x6b, 0xc2, 0xce, 0xd2, 0xae, 0x4c, 0x94, + 0xd0, 0xec, 0x6b, 0x82, 0xb7, 0x61, 0x79, 0x54, 0x3a, 0x47, 0xa5, 0x4b, 0xe3, 0xc2, 0xfe, 0x49, + 0x0c, 0xaf, 0x52, 0xf1, 0x49, 0xc0, 0xfe, 0xd7, 0x15, 0xf6, 0xe1, 0x12, 0x45, 0xf1, 0x0b, 0x58, + 0x1c, 0x08, 0x69, 0xbd, 0x24, 0x95, 0x2d, 0xf4, 0x41, 0x5a, 0xed, 0x00, 0xc0, 0x0f, 0xcc, 0xc0, + 0xf6, 0x03, 0xdb, 0xf2, 0xd9, 0x54, 0x06, 0x65, 0xe7, 0x73, 0xdb, 0x53, 0x7f, 0x59, 0x1b, 0xc8, + 0xd5, 0x48, 0xea, 0xd6, 0x3e, 0xc0, 0x90, 0xc1, 0x69, 0x48, 0x39, 0xb6, 0x6b, 0xd0, 0xd1, 0x53, + 0x27, 0x17, 0xd4, 0xa4, 0x63, 0xbb, 0xd4, 0x15, 0x4a, 0x9a, 0x97, 0x3d, 0x32, 0xde, 0x23, 0xcd, + 0x4b, 0x4a, 0x6e, 0xb5, 0xe3, 0x90, 0x54, 0xcc, 0x2a, 0xa1, 0xbb, 0x30, 0x71, 0x82, 0xe8, 0xf1, + 0x13, 0x8c, 0x4f, 0x9c, 0xe0, 0x2a, 0xcc, 0x5a, 0x0d, 0x6b, 0x37, 0xd7, 0xdb, 0x80, 0xee, 0xc7, + 0xc8, 0x72, 0xcc, 0x8c, 0x2e, 0x87, 0x08, 0x49, 0xe2, 0x5a, 0x5e, 0xd9, 0x76, 0xab, 0xd4, 0xc3, + 0xa5, 0xdc, 0xab, 0xa9, 0x63, 0x12, 0x7b, 0x62, 0xba, 0x16, 0x83, 0x54, 0xfc, 0x1c, 0xe6, 0xa3, + 0xce, 0x75, 0x2d, 0x86, 0x88, 0x6b, 0x69, 0x48, 0x0d, 0x1d, 0xeb, 0x1a, 0x9b, 0xfc, 0x8b, 0x5b, + 0xc9, 0x7f, 0x76, 0x6b, 0xe7, 0x02, 0x52, 0x83, 0x1b, 0x82, 0x9f, 0xc2, 0x93, 0x93, 0xfc, 0xe1, + 0xb1, 0x68, 0xe8, 0xa7, 0x8a, 0x68, 0x1c, 0xcb, 0x9a, 0x22, 0x16, 0xa4, 0x7d, 0x49, 0x2c, 0x32, + 0x31, 0xbc, 0x0a, 0x4c, 0x84, 0x93, 0x64, 0xfd, 0xfd, 0x3b, 0x06, 0xe1, 0x35, 0x58, 0x89, 0x66, + 0x74, 0xe1, 0xf8, 0x08, 0xac, 0xe9, 0xaa, 0x24, 0x1f, 0x30, 0x89, 0x9d, 0xaf, 0x08, 0x96, 0x47, + 0x2e, 0x0a, 0xce, 0xc0, 0xb3, 0x42, 0xe9, 0x48, 0x51, 0x45, 0x4d, 0x93, 0x4a, 0xf2, 0xa4, 0xce, + 0x1b, 0xb0, 0x36, 0xa6, 0x90, 0x4b, 0xb2, 0xc8, 0x20, 0x9c, 0x86, 0xf5, 0x31, 0x4a, 0x93, 0xf3, + 0x8a, 0x72, 0xca, 0xc4, 0x27, 0xe6, 0x9d, 0x69, 0x7a, 0x91, 0x49, 0xec, 0x5c, 0xc1, 0x42, 0xd4, + 0x16, 0xbc, 0x09, 0x1b, 0xa2, 0x5c, 0x28, 0x15, 0x25, 0xf9, 0x60, 0xd2, 0x09, 0xd6, 0xe1, 0xff, + 0x87, 0xb4, 0x72, 0x98, 0x97, 0x64, 0x06, 0x8d, 0x13, 0x45, 0xf1, 0x50, 0xcf, 0x33, 0x71, 0xcc, + 0xc2, 0xea, 0x43, 0x62, 0x4f, 0xd2, 0x8f, 0xf2, 0x0a, 0x93, 0xd8, 0xbb, 0xbc, 0xb9, 0xe3, 0x62, + 0xb7, 0x77, 0x5c, 0xec, 0xfe, 0x8e, 0x43, 0x5f, 0x42, 0x0e, 0x7d, 0x0f, 0x39, 0xf4, 0x33, 0xe4, + 0xd0, 0x4d, 0xc8, 0xa1, 0x5f, 0x21, 0x87, 0x7e, 0x87, 0x5c, 0xec, 0x3e, 0xe4, 0xd0, 0xb7, 0x36, + 0x17, 0xbb, 0x69, 0x73, 0xb1, 0xdb, 0x36, 0x17, 0x3b, 0xdb, 0xab, 0xda, 0xc1, 0xc7, 0xe6, 0x39, + 0x6f, 0x79, 0x8e, 0x50, 0x6d, 0x98, 0x15, 0xd3, 0x35, 0x85, 0x9a, 0x77, 0x61, 0x0b, 0xad, 0x5d, + 0xe1, 0x91, 0x6f, 0xf1, 0xf9, 0x1c, 0x7d, 0x82, 0x77, 0xff, 0x04, 0x00, 0x00, 0xff, 0xff, 0x5f, + 0xec, 0x7c, 0x9f, 0xbd, 0x05, 0x00, 0x00, } func (x ValueType) String() string { @@ -230,20 +475,68 @@ func (x ValueType) String() string { } return strconv.Itoa(int(x)) } -func (x EncodingType) String() string { - s, ok := EncodingType_name[int32(x)] +func (x CompressionType) String() string { + s, ok := CompressionType_name[int32(x)] if ok { return s } return strconv.Itoa(int(x)) } -func (x CompressionType) String() string { - s, ok := CompressionType_name[int32(x)] +func (x EncodingType) String() string { + s, ok := EncodingType_name[int32(x)] if ok { return s } return strconv.Itoa(int(x)) } +func (this *ColumnInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*ColumnInfo) + if !ok { + that2, ok := that.(ColumnInfo) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.Name != that1.Name { + return false + } + if this.ValueType != that1.ValueType { + return false + } + if this.RowsCount != that1.RowsCount { + return false + } + if this.Compression != that1.Compression { + return false + } + if this.UncompressedSize != that1.UncompressedSize { + return false + } + if this.CompressedSize != that1.CompressedSize { + return false + } + if this.MetadataOffset != that1.MetadataOffset { + return false + } + if this.MetadataSize != that1.MetadataSize { + return false + } + if !this.Statistics.Equal(that1.Statistics) { + return false + } + return true +} func (this *Statistics) Equal(that interface{}) bool { if that == nil { return this == nil @@ -271,6 +564,71 @@ func (this *Statistics) Equal(that interface{}) bool { } return true } +func (this *PageInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*PageInfo) + if !ok { + that2, ok := that.(PageInfo) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.UncompressedSize != that1.UncompressedSize { + return false + } + if this.CompressedSize != that1.CompressedSize { + return false + } + if this.Crc32 != that1.Crc32 { + return false + } + if this.RowsCount != that1.RowsCount { + return false + } + if this.Encoding != that1.Encoding { + return false + } + if this.DataOffset != that1.DataOffset { + return false + } + if this.DataSize != that1.DataSize { + return false + } + if !this.Statistics.Equal(that1.Statistics) { + return false + } + return true +} +func (this *ColumnInfo) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 13) + s = append(s, "&datasetmd.ColumnInfo{") + s = append(s, "Name: "+fmt.Sprintf("%#v", this.Name)+",\n") + s = append(s, "ValueType: "+fmt.Sprintf("%#v", this.ValueType)+",\n") + s = append(s, "RowsCount: "+fmt.Sprintf("%#v", this.RowsCount)+",\n") + s = append(s, "Compression: "+fmt.Sprintf("%#v", this.Compression)+",\n") + s = append(s, "UncompressedSize: "+fmt.Sprintf("%#v", this.UncompressedSize)+",\n") + s = append(s, "CompressedSize: "+fmt.Sprintf("%#v", this.CompressedSize)+",\n") + s = append(s, "MetadataOffset: "+fmt.Sprintf("%#v", this.MetadataOffset)+",\n") + s = append(s, "MetadataSize: "+fmt.Sprintf("%#v", this.MetadataSize)+",\n") + if this.Statistics != nil { + s = append(s, "Statistics: "+fmt.Sprintf("%#v", this.Statistics)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} func (this *Statistics) GoString() string { if this == nil { return "nil" @@ -282,7 +640,26 @@ func (this *Statistics) GoString() string { s = append(s, "}") return strings.Join(s, "") } -func valueToGoStringDatasetmd(v interface{}, typ string) string { +func (this *PageInfo) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 12) + s = append(s, "&datasetmd.PageInfo{") + s = append(s, "UncompressedSize: "+fmt.Sprintf("%#v", this.UncompressedSize)+",\n") + s = append(s, "CompressedSize: "+fmt.Sprintf("%#v", this.CompressedSize)+",\n") + s = append(s, "Crc32: "+fmt.Sprintf("%#v", this.Crc32)+",\n") + s = append(s, "RowsCount: "+fmt.Sprintf("%#v", this.RowsCount)+",\n") + s = append(s, "Encoding: "+fmt.Sprintf("%#v", this.Encoding)+",\n") + s = append(s, "DataOffset: "+fmt.Sprintf("%#v", this.DataOffset)+",\n") + s = append(s, "DataSize: "+fmt.Sprintf("%#v", this.DataSize)+",\n") + if this.Statistics != nil { + s = append(s, "Statistics: "+fmt.Sprintf("%#v", this.Statistics)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func valueToGoStringDatasetmd(v interface{}, typ string) string { rv := reflect.ValueOf(v) if rv.IsNil() { return "nil" @@ -290,6 +667,83 @@ func valueToGoStringDatasetmd(v interface{}, typ string) string { pv := reflect.Indirect(rv).Interface() return fmt.Sprintf("func(v %v) *%v { return &v } ( %#v )", typ, typ, pv) } +func (m *ColumnInfo) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ColumnInfo) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ColumnInfo) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Statistics != nil { + { + size, err := m.Statistics.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintDatasetmd(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x4a + } + if m.MetadataSize != 0 { + i = encodeVarintDatasetmd(dAtA, i, uint64(m.MetadataSize)) + i-- + dAtA[i] = 0x40 + } + if m.MetadataOffset != 0 { + i = encodeVarintDatasetmd(dAtA, i, uint64(m.MetadataOffset)) + i-- + dAtA[i] = 0x38 + } + if m.CompressedSize != 0 { + i = encodeVarintDatasetmd(dAtA, i, uint64(m.CompressedSize)) + i-- + dAtA[i] = 0x30 + } + if m.UncompressedSize != 0 { + i = encodeVarintDatasetmd(dAtA, i, uint64(m.UncompressedSize)) + i-- + dAtA[i] = 0x28 + } + if m.Compression != 0 { + i = encodeVarintDatasetmd(dAtA, i, uint64(m.Compression)) + i-- + dAtA[i] = 0x20 + } + if m.RowsCount != 0 { + i = encodeVarintDatasetmd(dAtA, i, uint64(m.RowsCount)) + i-- + dAtA[i] = 0x18 + } + if m.ValueType != 0 { + i = encodeVarintDatasetmd(dAtA, i, uint64(m.ValueType)) + i-- + dAtA[i] = 0x10 + } + if len(m.Name) > 0 { + i -= len(m.Name) + copy(dAtA[i:], m.Name) + i = encodeVarintDatasetmd(dAtA, i, uint64(len(m.Name))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + func (m *Statistics) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -327,6 +781,76 @@ func (m *Statistics) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *PageInfo) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *PageInfo) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *PageInfo) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Statistics != nil { + { + size, err := m.Statistics.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintDatasetmd(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x42 + } + if m.DataSize != 0 { + i = encodeVarintDatasetmd(dAtA, i, uint64(m.DataSize)) + i-- + dAtA[i] = 0x38 + } + if m.DataOffset != 0 { + i = encodeVarintDatasetmd(dAtA, i, uint64(m.DataOffset)) + i-- + dAtA[i] = 0x30 + } + if m.Encoding != 0 { + i = encodeVarintDatasetmd(dAtA, i, uint64(m.Encoding)) + i-- + dAtA[i] = 0x28 + } + if m.RowsCount != 0 { + i = encodeVarintDatasetmd(dAtA, i, uint64(m.RowsCount)) + i-- + dAtA[i] = 0x20 + } + if m.Crc32 != 0 { + i = encodeVarintDatasetmd(dAtA, i, uint64(m.Crc32)) + i-- + dAtA[i] = 0x18 + } + if m.CompressedSize != 0 { + i = encodeVarintDatasetmd(dAtA, i, uint64(m.CompressedSize)) + i-- + dAtA[i] = 0x10 + } + if m.UncompressedSize != 0 { + i = encodeVarintDatasetmd(dAtA, i, uint64(m.UncompressedSize)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + func encodeVarintDatasetmd(dAtA []byte, offset int, v uint64) int { offset -= sovDatasetmd(v) base := offset @@ -338,6 +862,44 @@ func encodeVarintDatasetmd(dAtA []byte, offset int, v uint64) int { dAtA[offset] = uint8(v) return base } +func (m *ColumnInfo) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Name) + if l > 0 { + n += 1 + l + sovDatasetmd(uint64(l)) + } + if m.ValueType != 0 { + n += 1 + sovDatasetmd(uint64(m.ValueType)) + } + if m.RowsCount != 0 { + n += 1 + sovDatasetmd(uint64(m.RowsCount)) + } + if m.Compression != 0 { + n += 1 + sovDatasetmd(uint64(m.Compression)) + } + if m.UncompressedSize != 0 { + n += 1 + sovDatasetmd(uint64(m.UncompressedSize)) + } + if m.CompressedSize != 0 { + n += 1 + sovDatasetmd(uint64(m.CompressedSize)) + } + if m.MetadataOffset != 0 { + n += 1 + sovDatasetmd(uint64(m.MetadataOffset)) + } + if m.MetadataSize != 0 { + n += 1 + sovDatasetmd(uint64(m.MetadataSize)) + } + if m.Statistics != nil { + l = m.Statistics.Size() + n += 1 + l + sovDatasetmd(uint64(l)) + } + return n +} + func (m *Statistics) Size() (n int) { if m == nil { return 0 @@ -355,12 +917,64 @@ func (m *Statistics) Size() (n int) { return n } +func (m *PageInfo) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.UncompressedSize != 0 { + n += 1 + sovDatasetmd(uint64(m.UncompressedSize)) + } + if m.CompressedSize != 0 { + n += 1 + sovDatasetmd(uint64(m.CompressedSize)) + } + if m.Crc32 != 0 { + n += 1 + sovDatasetmd(uint64(m.Crc32)) + } + if m.RowsCount != 0 { + n += 1 + sovDatasetmd(uint64(m.RowsCount)) + } + if m.Encoding != 0 { + n += 1 + sovDatasetmd(uint64(m.Encoding)) + } + if m.DataOffset != 0 { + n += 1 + sovDatasetmd(uint64(m.DataOffset)) + } + if m.DataSize != 0 { + n += 1 + sovDatasetmd(uint64(m.DataSize)) + } + if m.Statistics != nil { + l = m.Statistics.Size() + n += 1 + l + sovDatasetmd(uint64(l)) + } + return n +} + func sovDatasetmd(x uint64) (n int) { return (math_bits.Len64(x|1) + 6) / 7 } func sozDatasetmd(x uint64) (n int) { return sovDatasetmd(uint64((x << 1) ^ uint64((int64(x) >> 63)))) } +func (this *ColumnInfo) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&ColumnInfo{`, + `Name:` + fmt.Sprintf("%v", this.Name) + `,`, + `ValueType:` + fmt.Sprintf("%v", this.ValueType) + `,`, + `RowsCount:` + fmt.Sprintf("%v", this.RowsCount) + `,`, + `Compression:` + fmt.Sprintf("%v", this.Compression) + `,`, + `UncompressedSize:` + fmt.Sprintf("%v", this.UncompressedSize) + `,`, + `CompressedSize:` + fmt.Sprintf("%v", this.CompressedSize) + `,`, + `MetadataOffset:` + fmt.Sprintf("%v", this.MetadataOffset) + `,`, + `MetadataSize:` + fmt.Sprintf("%v", this.MetadataSize) + `,`, + `Statistics:` + strings.Replace(this.Statistics.String(), "Statistics", "Statistics", 1) + `,`, + `}`, + }, "") + return s +} func (this *Statistics) String() string { if this == nil { return "nil" @@ -372,6 +986,23 @@ func (this *Statistics) String() string { }, "") return s } +func (this *PageInfo) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&PageInfo{`, + `UncompressedSize:` + fmt.Sprintf("%v", this.UncompressedSize) + `,`, + `CompressedSize:` + fmt.Sprintf("%v", this.CompressedSize) + `,`, + `Crc32:` + fmt.Sprintf("%v", this.Crc32) + `,`, + `RowsCount:` + fmt.Sprintf("%v", this.RowsCount) + `,`, + `Encoding:` + fmt.Sprintf("%v", this.Encoding) + `,`, + `DataOffset:` + fmt.Sprintf("%v", this.DataOffset) + `,`, + `DataSize:` + fmt.Sprintf("%v", this.DataSize) + `,`, + `Statistics:` + strings.Replace(this.Statistics.String(), "Statistics", "Statistics", 1) + `,`, + `}`, + }, "") + return s +} func valueToStringDatasetmd(v interface{}) string { rv := reflect.ValueOf(v) if rv.IsNil() { @@ -380,7 +1011,7 @@ func valueToStringDatasetmd(v interface{}) string { pv := reflect.Indirect(rv).Interface() return fmt.Sprintf("*%v", pv) } -func (m *Statistics) Unmarshal(dAtA []byte) error { +func (m *ColumnInfo) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -403,17 +1034,17 @@ func (m *Statistics) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: Statistics: wiretype end group for non-group") + return fmt.Errorf("proto: ColumnInfo: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: Statistics: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ColumnInfo: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field MinValue", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) } - var byteLen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowDatasetmd @@ -423,31 +1054,29 @@ func (m *Statistics) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - byteLen |= int(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - if byteLen < 0 { + intStringLen := int(stringLen) + if intStringLen < 0 { return ErrInvalidLengthDatasetmd } - postIndex := iNdEx + byteLen + postIndex := iNdEx + intStringLen if postIndex < 0 { return ErrInvalidLengthDatasetmd } if postIndex > l { return io.ErrUnexpectedEOF } - m.MinValue = append(m.MinValue[:0], dAtA[iNdEx:postIndex]...) - if m.MinValue == nil { - m.MinValue = []byte{} - } + m.Name = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field MaxValue", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ValueType", wireType) } - var byteLen int + m.ValueType = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowDatasetmd @@ -457,24 +1086,502 @@ func (m *Statistics) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - byteLen |= int(b&0x7F) << shift + m.ValueType |= ValueType(b&0x7F) << shift if b < 0x80 { break } } - if byteLen < 0 { - return ErrInvalidLengthDatasetmd + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RowsCount", wireType) } - postIndex := iNdEx + byteLen - if postIndex < 0 { - return ErrInvalidLengthDatasetmd + m.RowsCount = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDatasetmd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.RowsCount |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } } - if postIndex > l { - return io.ErrUnexpectedEOF + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Compression", wireType) } - m.MaxValue = append(m.MaxValue[:0], dAtA[iNdEx:postIndex]...) - if m.MaxValue == nil { - m.MaxValue = []byte{} + m.Compression = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDatasetmd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Compression |= CompressionType(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field UncompressedSize", wireType) + } + m.UncompressedSize = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDatasetmd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.UncompressedSize |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field CompressedSize", wireType) + } + m.CompressedSize = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDatasetmd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.CompressedSize |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MetadataOffset", wireType) + } + m.MetadataOffset = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDatasetmd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MetadataOffset |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 8: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MetadataSize", wireType) + } + m.MetadataSize = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDatasetmd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MetadataSize |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Statistics", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDatasetmd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthDatasetmd + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDatasetmd + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Statistics == nil { + m.Statistics = &Statistics{} + } + if err := m.Statistics.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipDatasetmd(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthDatasetmd + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDatasetmd + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Statistics) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDatasetmd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Statistics: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Statistics: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MinValue", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDatasetmd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthDatasetmd + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthDatasetmd + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.MinValue = append(m.MinValue[:0], dAtA[iNdEx:postIndex]...) + if m.MinValue == nil { + m.MinValue = []byte{} + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MaxValue", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDatasetmd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthDatasetmd + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthDatasetmd + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.MaxValue = append(m.MaxValue[:0], dAtA[iNdEx:postIndex]...) + if m.MaxValue == nil { + m.MaxValue = []byte{} + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipDatasetmd(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthDatasetmd + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthDatasetmd + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *PageInfo) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDatasetmd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: PageInfo: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: PageInfo: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field UncompressedSize", wireType) + } + m.UncompressedSize = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDatasetmd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.UncompressedSize |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field CompressedSize", wireType) + } + m.CompressedSize = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDatasetmd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.CompressedSize |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Crc32", wireType) + } + m.Crc32 = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDatasetmd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Crc32 |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RowsCount", wireType) + } + m.RowsCount = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDatasetmd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.RowsCount |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Encoding", wireType) + } + m.Encoding = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDatasetmd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Encoding |= EncodingType(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field DataOffset", wireType) + } + m.DataOffset = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDatasetmd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.DataOffset |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field DataSize", wireType) + } + m.DataSize = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDatasetmd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.DataSize |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Statistics", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDatasetmd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthDatasetmd + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthDatasetmd + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Statistics == nil { + m.Statistics = &Statistics{} + } + if err := m.Statistics.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err } iNdEx = postIndex default: diff --git a/pkg/dataobj/internal/metadata/datasetmd/datasetmd.proto b/pkg/dataobj/internal/metadata/datasetmd/datasetmd.proto index e396f9a627f49..f6b827838eb03 100644 --- a/pkg/dataobj/internal/metadata/datasetmd/datasetmd.proto +++ b/pkg/dataobj/internal/metadata/datasetmd/datasetmd.proto @@ -5,6 +5,37 @@ package dataobj.metadata.dataset.v1; option go_package = "github.com/grafana/loki/v3/pkg/dataobj/internal/metadata/datasetmd"; +// ColumnInfo describes an individual column within a data set. +message ColumnInfo { + // Optional name of the column. + string name = 1; + + // Type of values stored within the column. + ValueType value_type = 2; + + // Total number of rows in the entire column. + uint32 rows_count = 3; + + // Compression type used for all pages. + CompressionType compression = 4; + + // Total uncompressed size of all pages in the column. + uint32 uncompressed_size = 5; + + // Total compressed size of all pages in the column. Compressed size may + // match uncompressed size if no compression is used. + uint32 compressed_size = 6; + + // Byte offset from the start of the data object to the column's metadata. + uint32 metadata_offset = 7; + + // Size of the column's metadata in bytes. + uint32 metadata_size = 8; + + // Statistics for the column. + Statistics statistics = 9; +} + // ValueType represents the valid types that values within a column can have. enum ValueType { // Invalid value type. @@ -20,24 +51,6 @@ enum ValueType { VALUE_TYPE_STRING = 3; } -// EncodingType represents the valid types that a sequence of values which a -// column can be encoded with. -enum EncodingType { - // Invalid encoding type. - ENCODING_TYPE_UNSPECIFIED = 0; - - // Plain encoding; data is stored as-is. - ENCODING_TYPE_PLAIN = 1; - - // Delta encoding. The first value within the page is stored as-is, and - // subsequent values are stored as the delta from the previous value. - ENCODING_TYPE_DELTA = 2; - - // Bitmap encoding. Bitmaps effiently store repeating sequences of unsigned - // integers using a combination of run-length encoding and bitpacking. - ENCODING_TYPE_BITMAP = 3; -} - // CompressionType represents the valid compression types that can be used for // compressing values in a page. enum CompressionType { @@ -63,3 +76,49 @@ message Statistics { // Maximum value. bytes max_value = 2; } + +// Page describes an individual page within a column. +message PageInfo { + // Uncompressed size of the page within the data object. + uint32 uncompressed_size = 1; + + // Compressed size of the page within the data object. Compression size + // will match uncompressed size if no compression is used. + uint32 compressed_size = 2; + + // CRC32 checksum of the page data. + uint32 crc32 = 3; + + // Number of rows in the page. + uint32 rows_count = 4; + + // Encoding type used for the page. + EncodingType encoding = 5; + + // Byte offset from the start of the data object to the page's data. + uint32 data_offset = 6; + + // Size of the page's data in bytes. + uint32 data_size = 7; + + // Optional statistics for the page. + Statistics statistics = 8; +} + +// EncodingType represents the valid types that a sequence of values which a +// column can be encoded with. +enum EncodingType { + // Invalid encoding type. + ENCODING_TYPE_UNSPECIFIED = 0; + + // Plain encoding; data is stored as-is. + ENCODING_TYPE_PLAIN = 1; + + // Delta encoding. The first value within the page is stored as-is, and + // subsequent values are stored as the delta from the previous value. + ENCODING_TYPE_DELTA = 2; + + // Bitmap encoding. Bitmaps effiently store repeating sequences of unsigned + // integers using a combination of run-length encoding and bitpacking. + ENCODING_TYPE_BITMAP = 3; +} diff --git a/pkg/dataobj/internal/metadata/filemd/filemd.pb.go b/pkg/dataobj/internal/metadata/filemd/filemd.pb.go new file mode 100644 index 0000000000000..b3b65c6563e8e --- /dev/null +++ b/pkg/dataobj/internal/metadata/filemd/filemd.pb.go @@ -0,0 +1,761 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: pkg/dataobj/internal/metadata/filemd/filemd.proto + +package filemd + +import ( + fmt "fmt" + proto "github.com/gogo/protobuf/proto" + io "io" + math "math" + math_bits "math/bits" + reflect "reflect" + strconv "strconv" + strings "strings" +) + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package + +type SectionType int32 + +const ( + // SECTION_TYPE_UNSPECIFIED is an invalid section type. + SECTION_TYPE_UNSPECIFIED SectionType = 0 + // SECTION_TYPE_STREAMS is a section containing references to streams that + // exist within the data object. SECTION_TYPE_STREAMS does not contain any + // actual log data. + SECTION_TYPE_STREAMS SectionType = 1 +) + +var SectionType_name = map[int32]string{ + 0: "SECTION_TYPE_UNSPECIFIED", + 1: "SECTION_TYPE_STREAMS", +} + +var SectionType_value = map[string]int32{ + "SECTION_TYPE_UNSPECIFIED": 0, + "SECTION_TYPE_STREAMS": 1, +} + +func (SectionType) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_be80f52d1e05bad9, []int{0} +} + +// Metadata for the overall data object. +type Metadata struct { + // Sections within the data object. + Sections []*SectionInfo `protobuf:"bytes,1,rep,name=sections,proto3" json:"sections,omitempty"` +} + +func (m *Metadata) Reset() { *m = Metadata{} } +func (*Metadata) ProtoMessage() {} +func (*Metadata) Descriptor() ([]byte, []int) { + return fileDescriptor_be80f52d1e05bad9, []int{0} +} +func (m *Metadata) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Metadata) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Metadata.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Metadata) XXX_Merge(src proto.Message) { + xxx_messageInfo_Metadata.Merge(m, src) +} +func (m *Metadata) XXX_Size() int { + return m.Size() +} +func (m *Metadata) XXX_DiscardUnknown() { + xxx_messageInfo_Metadata.DiscardUnknown(m) +} + +var xxx_messageInfo_Metadata proto.InternalMessageInfo + +func (m *Metadata) GetSections() []*SectionInfo { + if m != nil { + return m.Sections + } + return nil +} + +// SectionInfo describes a section within the data object. +type SectionInfo struct { + // Type of the section within the data object. + Type SectionType `protobuf:"varint,1,opt,name=type,proto3,enum=dataobj.metadata.file.v1.SectionType" json:"type,omitempty"` + // Byte offset of the section's metadata from the start of the data object. + MetadataOffset uint32 `protobuf:"varint,2,opt,name=metadata_offset,json=metadataOffset,proto3" json:"metadata_offset,omitempty"` + // Size of the section's metadata in bytes. + MetadataSize uint32 `protobuf:"varint,3,opt,name=metadata_size,json=metadataSize,proto3" json:"metadata_size,omitempty"` +} + +func (m *SectionInfo) Reset() { *m = SectionInfo{} } +func (*SectionInfo) ProtoMessage() {} +func (*SectionInfo) Descriptor() ([]byte, []int) { + return fileDescriptor_be80f52d1e05bad9, []int{1} +} +func (m *SectionInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SectionInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SectionInfo.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *SectionInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_SectionInfo.Merge(m, src) +} +func (m *SectionInfo) XXX_Size() int { + return m.Size() +} +func (m *SectionInfo) XXX_DiscardUnknown() { + xxx_messageInfo_SectionInfo.DiscardUnknown(m) +} + +var xxx_messageInfo_SectionInfo proto.InternalMessageInfo + +func (m *SectionInfo) GetType() SectionType { + if m != nil { + return m.Type + } + return SECTION_TYPE_UNSPECIFIED +} + +func (m *SectionInfo) GetMetadataOffset() uint32 { + if m != nil { + return m.MetadataOffset + } + return 0 +} + +func (m *SectionInfo) GetMetadataSize() uint32 { + if m != nil { + return m.MetadataSize + } + return 0 +} + +func init() { + proto.RegisterEnum("dataobj.metadata.file.v1.SectionType", SectionType_name, SectionType_value) + proto.RegisterType((*Metadata)(nil), "dataobj.metadata.file.v1.Metadata") + proto.RegisterType((*SectionInfo)(nil), "dataobj.metadata.file.v1.SectionInfo") +} + +func init() { + proto.RegisterFile("pkg/dataobj/internal/metadata/filemd/filemd.proto", fileDescriptor_be80f52d1e05bad9) +} + +var fileDescriptor_be80f52d1e05bad9 = []byte{ + // 344 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x32, 0x2c, 0xc8, 0x4e, 0xd7, + 0x4f, 0x49, 0x2c, 0x49, 0xcc, 0x4f, 0xca, 0xd2, 0xcf, 0xcc, 0x2b, 0x49, 0x2d, 0xca, 0x4b, 0xcc, + 0xd1, 0xcf, 0x4d, 0x2d, 0x49, 0x04, 0x09, 0xea, 0xa7, 0x65, 0xe6, 0xa4, 0xe6, 0xa6, 0x40, 0x29, + 0xbd, 0x82, 0xa2, 0xfc, 0x92, 0x7c, 0x21, 0x09, 0xa8, 0x72, 0x3d, 0x98, 0x2a, 0x3d, 0x90, 0xb4, + 0x5e, 0x99, 0xa1, 0x92, 0x2f, 0x17, 0x87, 0x2f, 0x54, 0x4c, 0xc8, 0x91, 0x8b, 0xa3, 0x38, 0x35, + 0xb9, 0x24, 0x33, 0x3f, 0xaf, 0x58, 0x82, 0x51, 0x81, 0x59, 0x83, 0xdb, 0x48, 0x55, 0x0f, 0x97, + 0x46, 0xbd, 0x60, 0x88, 0x4a, 0xcf, 0xbc, 0xb4, 0xfc, 0x20, 0xb8, 0x36, 0xa5, 0x69, 0x8c, 0x5c, + 0xdc, 0x48, 0x32, 0x42, 0x96, 0x5c, 0x2c, 0x25, 0x95, 0x05, 0xa9, 0x12, 0x8c, 0x0a, 0x8c, 0x1a, + 0x7c, 0x44, 0x18, 0x17, 0x52, 0x59, 0x90, 0x1a, 0x04, 0xd6, 0x22, 0xa4, 0xce, 0xc5, 0x0f, 0x53, + 0x15, 0x9f, 0x9f, 0x96, 0x56, 0x9c, 0x5a, 0x22, 0xc1, 0xa4, 0xc0, 0xa8, 0xc1, 0x1b, 0xc4, 0x07, + 0x13, 0xf6, 0x07, 0x8b, 0x0a, 0x29, 0x73, 0xf1, 0xc2, 0x15, 0x16, 0x67, 0x56, 0xa5, 0x4a, 0x30, + 0x83, 0x95, 0xf1, 0xc0, 0x04, 0x83, 0x33, 0xab, 0x52, 0xb5, 0x5c, 0xe1, 0xee, 0x02, 0x59, 0x21, + 0x24, 0xc3, 0x25, 0x11, 0xec, 0xea, 0x1c, 0xe2, 0xe9, 0xef, 0x17, 0x1f, 0x12, 0x19, 0xe0, 0x1a, + 0x1f, 0xea, 0x17, 0x1c, 0xe0, 0xea, 0xec, 0xe9, 0xe6, 0xe9, 0xea, 0x22, 0xc0, 0x20, 0x24, 0xc1, + 0x25, 0x82, 0x22, 0x1b, 0x1c, 0x12, 0xe4, 0xea, 0xe8, 0x1b, 0x2c, 0xc0, 0xe8, 0x54, 0x7a, 0xe1, + 0xa1, 0x1c, 0xc3, 0x8d, 0x87, 0x72, 0x0c, 0x1f, 0x1e, 0xca, 0x31, 0x36, 0x3c, 0x92, 0x63, 0x5c, + 0xf1, 0x48, 0x8e, 0xf1, 0xc4, 0x23, 0x39, 0xc6, 0x0b, 0x8f, 0xe4, 0x18, 0x1f, 0x3c, 0x92, 0x63, + 0x7c, 0xf1, 0x48, 0x8e, 0xe1, 0xc3, 0x23, 0x39, 0xc6, 0x09, 0x8f, 0xe5, 0x18, 0x2e, 0x3c, 0x96, + 0x63, 0xb8, 0xf1, 0x58, 0x8e, 0x21, 0xca, 0x3e, 0x3d, 0xb3, 0x24, 0xa3, 0x34, 0x49, 0x2f, 0x39, + 0x3f, 0x57, 0x3f, 0xbd, 0x28, 0x31, 0x2d, 0x31, 0x2f, 0x51, 0x3f, 0x27, 0x3f, 0x3b, 0x53, 0xbf, + 0xcc, 0x58, 0x9f, 0x98, 0xc8, 0x4c, 0x62, 0x03, 0x47, 0xa3, 0x31, 0x20, 0x00, 0x00, 0xff, 0xff, + 0xc3, 0x39, 0x5a, 0x9d, 0xfb, 0x01, 0x00, 0x00, +} + +func (x SectionType) String() string { + s, ok := SectionType_name[int32(x)] + if ok { + return s + } + return strconv.Itoa(int(x)) +} +func (this *Metadata) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*Metadata) + if !ok { + that2, ok := that.(Metadata) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.Sections) != len(that1.Sections) { + return false + } + for i := range this.Sections { + if !this.Sections[i].Equal(that1.Sections[i]) { + return false + } + } + return true +} +func (this *SectionInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*SectionInfo) + if !ok { + that2, ok := that.(SectionInfo) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.Type != that1.Type { + return false + } + if this.MetadataOffset != that1.MetadataOffset { + return false + } + if this.MetadataSize != that1.MetadataSize { + return false + } + return true +} +func (this *Metadata) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&filemd.Metadata{") + if this.Sections != nil { + s = append(s, "Sections: "+fmt.Sprintf("%#v", this.Sections)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *SectionInfo) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 7) + s = append(s, "&filemd.SectionInfo{") + s = append(s, "Type: "+fmt.Sprintf("%#v", this.Type)+",\n") + s = append(s, "MetadataOffset: "+fmt.Sprintf("%#v", this.MetadataOffset)+",\n") + s = append(s, "MetadataSize: "+fmt.Sprintf("%#v", this.MetadataSize)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func valueToGoStringFilemd(v interface{}, typ string) string { + rv := reflect.ValueOf(v) + if rv.IsNil() { + return "nil" + } + pv := reflect.Indirect(rv).Interface() + return fmt.Sprintf("func(v %v) *%v { return &v } ( %#v )", typ, typ, pv) +} +func (m *Metadata) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Metadata) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Metadata) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Sections) > 0 { + for iNdEx := len(m.Sections) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Sections[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintFilemd(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *SectionInfo) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SectionInfo) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SectionInfo) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.MetadataSize != 0 { + i = encodeVarintFilemd(dAtA, i, uint64(m.MetadataSize)) + i-- + dAtA[i] = 0x18 + } + if m.MetadataOffset != 0 { + i = encodeVarintFilemd(dAtA, i, uint64(m.MetadataOffset)) + i-- + dAtA[i] = 0x10 + } + if m.Type != 0 { + i = encodeVarintFilemd(dAtA, i, uint64(m.Type)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func encodeVarintFilemd(dAtA []byte, offset int, v uint64) int { + offset -= sovFilemd(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *Metadata) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Sections) > 0 { + for _, e := range m.Sections { + l = e.Size() + n += 1 + l + sovFilemd(uint64(l)) + } + } + return n +} + +func (m *SectionInfo) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Type != 0 { + n += 1 + sovFilemd(uint64(m.Type)) + } + if m.MetadataOffset != 0 { + n += 1 + sovFilemd(uint64(m.MetadataOffset)) + } + if m.MetadataSize != 0 { + n += 1 + sovFilemd(uint64(m.MetadataSize)) + } + return n +} + +func sovFilemd(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} +func sozFilemd(x uint64) (n int) { + return sovFilemd(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (this *Metadata) String() string { + if this == nil { + return "nil" + } + repeatedStringForSections := "[]*SectionInfo{" + for _, f := range this.Sections { + repeatedStringForSections += strings.Replace(f.String(), "SectionInfo", "SectionInfo", 1) + "," + } + repeatedStringForSections += "}" + s := strings.Join([]string{`&Metadata{`, + `Sections:` + repeatedStringForSections + `,`, + `}`, + }, "") + return s +} +func (this *SectionInfo) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&SectionInfo{`, + `Type:` + fmt.Sprintf("%v", this.Type) + `,`, + `MetadataOffset:` + fmt.Sprintf("%v", this.MetadataOffset) + `,`, + `MetadataSize:` + fmt.Sprintf("%v", this.MetadataSize) + `,`, + `}`, + }, "") + return s +} +func valueToStringFilemd(v interface{}) string { + rv := reflect.ValueOf(v) + if rv.IsNil() { + return "nil" + } + pv := reflect.Indirect(rv).Interface() + return fmt.Sprintf("*%v", pv) +} +func (m *Metadata) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowFilemd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Metadata: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Metadata: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Sections", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowFilemd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthFilemd + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthFilemd + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Sections = append(m.Sections, &SectionInfo{}) + if err := m.Sections[len(m.Sections)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipFilemd(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthFilemd + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthFilemd + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SectionInfo) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowFilemd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SectionInfo: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SectionInfo: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + } + m.Type = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowFilemd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Type |= SectionType(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MetadataOffset", wireType) + } + m.MetadataOffset = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowFilemd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MetadataOffset |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MetadataSize", wireType) + } + m.MetadataSize = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowFilemd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MetadataSize |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipFilemd(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthFilemd + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthFilemd + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipFilemd(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowFilemd + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowFilemd + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + return iNdEx, nil + case 1: + iNdEx += 8 + return iNdEx, nil + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowFilemd + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if length < 0 { + return 0, ErrInvalidLengthFilemd + } + iNdEx += length + if iNdEx < 0 { + return 0, ErrInvalidLengthFilemd + } + return iNdEx, nil + case 3: + for { + var innerWire uint64 + var start int = iNdEx + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowFilemd + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + innerWire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + innerWireType := int(innerWire & 0x7) + if innerWireType == 4 { + break + } + next, err := skipFilemd(dAtA[start:]) + if err != nil { + return 0, err + } + iNdEx = start + next + if iNdEx < 0 { + return 0, ErrInvalidLengthFilemd + } + } + return iNdEx, nil + case 4: + return iNdEx, nil + case 5: + iNdEx += 4 + return iNdEx, nil + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + } + panic("unreachable") +} + +var ( + ErrInvalidLengthFilemd = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowFilemd = fmt.Errorf("proto: integer overflow") +) diff --git a/pkg/dataobj/internal/metadata/filemd/filemd.proto b/pkg/dataobj/internal/metadata/filemd/filemd.proto new file mode 100644 index 0000000000000..e32159e30cfb9 --- /dev/null +++ b/pkg/dataobj/internal/metadata/filemd/filemd.proto @@ -0,0 +1,39 @@ +// filemd.proto holds file-level metadata for a data object. Data objects are +// split into multiple distinct "sections," each of which contains independent +// data related to storing logs. +// +// Each section has its own metadata; the file-level metadata points to the +// metadata of each section stored within the data object. +syntax = "proto3"; + +package dataobj.metadata.file.v1; + +option go_package = "github.com/grafana/loki/v3/pkg/dataobj/internal/metadata/filemd"; + +// Metadata for the overall data object. +message Metadata { + // Sections within the data object. + repeated SectionInfo sections = 1; +} + +// SectionInfo describes a section within the data object. +message SectionInfo { + // Type of the section within the data object. + SectionType type = 1; + + // Byte offset of the section's metadata from the start of the data object. + uint32 metadata_offset = 2; + + // Size of the section's metadata in bytes. + uint32 metadata_size = 3; +} + +enum SectionType { + // SECTION_TYPE_UNSPECIFIED is an invalid section type. + SECTION_TYPE_UNSPECIFIED = 0; + + // SECTION_TYPE_STREAMS is a section containing references to streams that + // exist within the data object. SECTION_TYPE_STREAMS does not contain any + // actual log data. + SECTION_TYPE_STREAMS = 1; +} diff --git a/pkg/dataobj/internal/metadata/streamsmd/streamsmd.pb.go b/pkg/dataobj/internal/metadata/streamsmd/streamsmd.pb.go new file mode 100644 index 0000000000000..0415d2e93ca0e --- /dev/null +++ b/pkg/dataobj/internal/metadata/streamsmd/streamsmd.pb.go @@ -0,0 +1,1242 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: pkg/dataobj/internal/metadata/streamsmd/streamsmd.proto + +package streamsmd + +import ( + fmt "fmt" + proto "github.com/gogo/protobuf/proto" + datasetmd "github.com/grafana/loki/v3/pkg/dataobj/internal/metadata/datasetmd" + io "io" + math "math" + math_bits "math/bits" + reflect "reflect" + strconv "strconv" + strings "strings" +) + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package + +// ColumnType represents the valid types that a stream's column can have. +type ColumnType int32 + +const ( + // Invalid column type. + COLUMN_TYPE_UNSPECIFIED ColumnType = 0 + // COLUMN_TYPE_STREAM_ID is a column containing the stream ID. Stream IDs are + // unique across the entire data object. + COLUMN_TYPE_STREAM_ID ColumnType = 1 + // COLUMN_TYPE_MIN_TIMESTAMP is a column containing the minimum timestamp of + // a stream. + COLUMN_TYPE_MIN_TIMESTAMP ColumnType = 2 + // COLUMN_TYPE_MAX_TIMESTAMP is a column containing the maximum timestamp of + // a stream. + COLUMN_TYPE_MAX_TIMESTAMP ColumnType = 3 + // COLUMN_TYPE_LABEL is a column containing a label. + COLUMN_TYPE_LABEL ColumnType = 4 + // COLUMN_TYPE_ROWS is a column indicating the number of rows for a stream. + COLUMN_TYPE_ROWS ColumnType = 5 +) + +var ColumnType_name = map[int32]string{ + 0: "COLUMN_TYPE_UNSPECIFIED", + 1: "COLUMN_TYPE_STREAM_ID", + 2: "COLUMN_TYPE_MIN_TIMESTAMP", + 3: "COLUMN_TYPE_MAX_TIMESTAMP", + 4: "COLUMN_TYPE_LABEL", + 5: "COLUMN_TYPE_ROWS", +} + +var ColumnType_value = map[string]int32{ + "COLUMN_TYPE_UNSPECIFIED": 0, + "COLUMN_TYPE_STREAM_ID": 1, + "COLUMN_TYPE_MIN_TIMESTAMP": 2, + "COLUMN_TYPE_MAX_TIMESTAMP": 3, + "COLUMN_TYPE_LABEL": 4, + "COLUMN_TYPE_ROWS": 5, +} + +func (ColumnType) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_7b94842ca2f0bf8d, []int{0} +} + +// Metadata describes the metadata for the streams section. +type Metadata struct { + // Columns within the stream. + Columns []*ColumnDesc `protobuf:"bytes,1,rep,name=columns,proto3" json:"columns,omitempty"` +} + +func (m *Metadata) Reset() { *m = Metadata{} } +func (*Metadata) ProtoMessage() {} +func (*Metadata) Descriptor() ([]byte, []int) { + return fileDescriptor_7b94842ca2f0bf8d, []int{0} +} +func (m *Metadata) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Metadata) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Metadata.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Metadata) XXX_Merge(src proto.Message) { + xxx_messageInfo_Metadata.Merge(m, src) +} +func (m *Metadata) XXX_Size() int { + return m.Size() +} +func (m *Metadata) XXX_DiscardUnknown() { + xxx_messageInfo_Metadata.DiscardUnknown(m) +} + +var xxx_messageInfo_Metadata proto.InternalMessageInfo + +func (m *Metadata) GetColumns() []*ColumnDesc { + if m != nil { + return m.Columns + } + return nil +} + +// ColumnDesc describes an individual column within the streams table. +type ColumnDesc struct { + // Information about the column. + Info *datasetmd.ColumnInfo `protobuf:"bytes,1,opt,name=info,proto3" json:"info,omitempty"` + // Column type. + Type ColumnType `protobuf:"varint,2,opt,name=type,proto3,enum=dataobj.metadata.streams.v1.ColumnType" json:"type,omitempty"` +} + +func (m *ColumnDesc) Reset() { *m = ColumnDesc{} } +func (*ColumnDesc) ProtoMessage() {} +func (*ColumnDesc) Descriptor() ([]byte, []int) { + return fileDescriptor_7b94842ca2f0bf8d, []int{1} +} +func (m *ColumnDesc) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ColumnDesc) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ColumnDesc.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *ColumnDesc) XXX_Merge(src proto.Message) { + xxx_messageInfo_ColumnDesc.Merge(m, src) +} +func (m *ColumnDesc) XXX_Size() int { + return m.Size() +} +func (m *ColumnDesc) XXX_DiscardUnknown() { + xxx_messageInfo_ColumnDesc.DiscardUnknown(m) +} + +var xxx_messageInfo_ColumnDesc proto.InternalMessageInfo + +func (m *ColumnDesc) GetInfo() *datasetmd.ColumnInfo { + if m != nil { + return m.Info + } + return nil +} + +func (m *ColumnDesc) GetType() ColumnType { + if m != nil { + return m.Type + } + return COLUMN_TYPE_UNSPECIFIED +} + +// ColumnMetadata describes the metadata for a column. +type ColumnMetadata struct { + // Pages within the column. + Pages []*PageDesc `protobuf:"bytes,1,rep,name=pages,proto3" json:"pages,omitempty"` +} + +func (m *ColumnMetadata) Reset() { *m = ColumnMetadata{} } +func (*ColumnMetadata) ProtoMessage() {} +func (*ColumnMetadata) Descriptor() ([]byte, []int) { + return fileDescriptor_7b94842ca2f0bf8d, []int{2} +} +func (m *ColumnMetadata) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ColumnMetadata) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ColumnMetadata.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *ColumnMetadata) XXX_Merge(src proto.Message) { + xxx_messageInfo_ColumnMetadata.Merge(m, src) +} +func (m *ColumnMetadata) XXX_Size() int { + return m.Size() +} +func (m *ColumnMetadata) XXX_DiscardUnknown() { + xxx_messageInfo_ColumnMetadata.DiscardUnknown(m) +} + +var xxx_messageInfo_ColumnMetadata proto.InternalMessageInfo + +func (m *ColumnMetadata) GetPages() []*PageDesc { + if m != nil { + return m.Pages + } + return nil +} + +// PageDesc describes an individual page within a column. +type PageDesc struct { + // Information about the page. + Info *datasetmd.PageInfo `protobuf:"bytes,1,opt,name=info,proto3" json:"info,omitempty"` +} + +func (m *PageDesc) Reset() { *m = PageDesc{} } +func (*PageDesc) ProtoMessage() {} +func (*PageDesc) Descriptor() ([]byte, []int) { + return fileDescriptor_7b94842ca2f0bf8d, []int{3} +} +func (m *PageDesc) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *PageDesc) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_PageDesc.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *PageDesc) XXX_Merge(src proto.Message) { + xxx_messageInfo_PageDesc.Merge(m, src) +} +func (m *PageDesc) XXX_Size() int { + return m.Size() +} +func (m *PageDesc) XXX_DiscardUnknown() { + xxx_messageInfo_PageDesc.DiscardUnknown(m) +} + +var xxx_messageInfo_PageDesc proto.InternalMessageInfo + +func (m *PageDesc) GetInfo() *datasetmd.PageInfo { + if m != nil { + return m.Info + } + return nil +} + +func init() { + proto.RegisterEnum("dataobj.metadata.streams.v1.ColumnType", ColumnType_name, ColumnType_value) + proto.RegisterType((*Metadata)(nil), "dataobj.metadata.streams.v1.Metadata") + proto.RegisterType((*ColumnDesc)(nil), "dataobj.metadata.streams.v1.ColumnDesc") + proto.RegisterType((*ColumnMetadata)(nil), "dataobj.metadata.streams.v1.ColumnMetadata") + proto.RegisterType((*PageDesc)(nil), "dataobj.metadata.streams.v1.PageDesc") +} + +func init() { + proto.RegisterFile("pkg/dataobj/internal/metadata/streamsmd/streamsmd.proto", fileDescriptor_7b94842ca2f0bf8d) +} + +var fileDescriptor_7b94842ca2f0bf8d = []byte{ + // 439 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x52, 0x3d, 0x6f, 0xd3, 0x40, + 0x18, 0xf6, 0xf5, 0x03, 0xaa, 0x43, 0xaa, 0xcc, 0x89, 0x8a, 0x94, 0x8a, 0x53, 0x14, 0xa9, 0xa2, + 0x62, 0xf0, 0x09, 0x3a, 0x20, 0xd4, 0xc9, 0x49, 0x8c, 0x64, 0x29, 0x97, 0x46, 0xb6, 0x2b, 0x3e, + 0x16, 0xeb, 0x92, 0x5c, 0x8c, 0x69, 0xec, 0xb3, 0xec, 0x6b, 0x45, 0x37, 0x26, 0x66, 0x7e, 0x06, + 0x1b, 0x7f, 0x83, 0x31, 0x63, 0x47, 0xe2, 0x2c, 0x8c, 0xfd, 0x09, 0xc8, 0x8e, 0x5d, 0x1b, 0x81, + 0xd2, 0x2c, 0xd6, 0xab, 0xe7, 0xcb, 0xef, 0x73, 0x7a, 0xe1, 0xab, 0xe8, 0xdc, 0x23, 0x63, 0x26, + 0x99, 0x18, 0x7e, 0x22, 0x7e, 0x28, 0x79, 0x1c, 0xb2, 0x29, 0x09, 0xb8, 0x64, 0x19, 0x48, 0x12, + 0x19, 0x73, 0x16, 0x24, 0xc1, 0xb8, 0x9a, 0xb4, 0x28, 0x16, 0x52, 0xa0, 0x83, 0xc2, 0xa4, 0x95, + 0x5a, 0xad, 0x50, 0x68, 0x97, 0x2f, 0x9e, 0xdc, 0x91, 0x9a, 0x7d, 0x12, 0x2e, 0x83, 0x71, 0x35, + 0x2d, 0x53, 0x5b, 0x14, 0xee, 0xd0, 0x42, 0x85, 0x74, 0x78, 0x7f, 0x24, 0xa6, 0x17, 0x41, 0x98, + 0x34, 0x40, 0x73, 0xf3, 0xe8, 0xc1, 0xcb, 0x67, 0xda, 0x8a, 0x7f, 0x6a, 0x9d, 0x5c, 0xdb, 0xe5, + 0xc9, 0xc8, 0x2a, 0x7d, 0xad, 0xaf, 0x00, 0xc2, 0x0a, 0x47, 0x27, 0x70, 0xcb, 0x0f, 0x27, 0xa2, + 0x01, 0x9a, 0xe0, 0xff, 0x71, 0xc5, 0x3a, 0x55, 0x9c, 0x19, 0x4e, 0x84, 0x95, 0x9b, 0x32, 0xb3, + 0xbc, 0x8a, 0x78, 0x63, 0xa3, 0x09, 0x8e, 0x76, 0xd7, 0xda, 0xc5, 0xb9, 0x8a, 0xb8, 0x95, 0x9b, + 0x5a, 0x14, 0xee, 0x2e, 0xb1, 0xdb, 0x76, 0x27, 0x70, 0x3b, 0x62, 0x1e, 0x2f, 0xbb, 0x1d, 0xae, + 0xcc, 0x1b, 0x30, 0x8f, 0xe7, 0xcd, 0x96, 0x9e, 0x96, 0x01, 0x77, 0x4a, 0x08, 0xbd, 0xfe, 0xab, + 0xd4, 0xe1, 0xca, 0x52, 0x99, 0xa9, 0xaa, 0xf4, 0xfc, 0xc7, 0xed, 0xf3, 0x64, 0xab, 0xa2, 0x03, + 0xf8, 0xb8, 0x73, 0xda, 0x3b, 0xa3, 0x7d, 0xd7, 0x79, 0x3f, 0x30, 0xdc, 0xb3, 0xbe, 0x3d, 0x30, + 0x3a, 0xe6, 0x1b, 0xd3, 0xe8, 0xaa, 0x0a, 0xda, 0x87, 0x7b, 0x75, 0xd2, 0x76, 0x2c, 0x43, 0xa7, + 0xae, 0xd9, 0x55, 0x01, 0x7a, 0x0a, 0xf7, 0xeb, 0x14, 0x35, 0xfb, 0xae, 0x63, 0x52, 0xc3, 0x76, + 0x74, 0x3a, 0x50, 0x37, 0xfe, 0xa1, 0xf5, 0x77, 0x35, 0x7a, 0x13, 0xed, 0xc1, 0x87, 0x75, 0xba, + 0xa7, 0xb7, 0x8d, 0x9e, 0xba, 0x85, 0x1e, 0x41, 0xb5, 0x0e, 0x5b, 0xa7, 0x6f, 0x6d, 0x75, 0xbb, + 0xfd, 0x79, 0x36, 0xc7, 0xca, 0xf5, 0x1c, 0x2b, 0x37, 0x73, 0x0c, 0xbe, 0xa4, 0x18, 0x7c, 0x4f, + 0x31, 0xf8, 0x99, 0x62, 0x30, 0x4b, 0x31, 0xf8, 0x95, 0x62, 0xf0, 0x3b, 0xc5, 0xca, 0x4d, 0x8a, + 0xc1, 0xb7, 0x05, 0x56, 0x66, 0x0b, 0xac, 0x5c, 0x2f, 0xb0, 0xf2, 0xa1, 0xed, 0xf9, 0xf2, 0xe3, + 0xc5, 0x50, 0x1b, 0x89, 0x80, 0x78, 0x31, 0x9b, 0xb0, 0x90, 0x91, 0xa9, 0x38, 0xf7, 0xc9, 0xe5, + 0x31, 0x59, 0xf3, 0xfe, 0x87, 0xf7, 0xf2, 0x03, 0x3d, 0xfe, 0x13, 0x00, 0x00, 0xff, 0xff, 0x66, + 0x7a, 0xe2, 0x3c, 0x31, 0x03, 0x00, 0x00, +} + +func (x ColumnType) String() string { + s, ok := ColumnType_name[int32(x)] + if ok { + return s + } + return strconv.Itoa(int(x)) +} +func (this *Metadata) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*Metadata) + if !ok { + that2, ok := that.(Metadata) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.Columns) != len(that1.Columns) { + return false + } + for i := range this.Columns { + if !this.Columns[i].Equal(that1.Columns[i]) { + return false + } + } + return true +} +func (this *ColumnDesc) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*ColumnDesc) + if !ok { + that2, ok := that.(ColumnDesc) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if !this.Info.Equal(that1.Info) { + return false + } + if this.Type != that1.Type { + return false + } + return true +} +func (this *ColumnMetadata) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*ColumnMetadata) + if !ok { + that2, ok := that.(ColumnMetadata) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.Pages) != len(that1.Pages) { + return false + } + for i := range this.Pages { + if !this.Pages[i].Equal(that1.Pages[i]) { + return false + } + } + return true +} +func (this *PageDesc) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*PageDesc) + if !ok { + that2, ok := that.(PageDesc) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if !this.Info.Equal(that1.Info) { + return false + } + return true +} +func (this *Metadata) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&streamsmd.Metadata{") + if this.Columns != nil { + s = append(s, "Columns: "+fmt.Sprintf("%#v", this.Columns)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *ColumnDesc) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&streamsmd.ColumnDesc{") + if this.Info != nil { + s = append(s, "Info: "+fmt.Sprintf("%#v", this.Info)+",\n") + } + s = append(s, "Type: "+fmt.Sprintf("%#v", this.Type)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *ColumnMetadata) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&streamsmd.ColumnMetadata{") + if this.Pages != nil { + s = append(s, "Pages: "+fmt.Sprintf("%#v", this.Pages)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *PageDesc) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&streamsmd.PageDesc{") + if this.Info != nil { + s = append(s, "Info: "+fmt.Sprintf("%#v", this.Info)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func valueToGoStringStreamsmd(v interface{}, typ string) string { + rv := reflect.ValueOf(v) + if rv.IsNil() { + return "nil" + } + pv := reflect.Indirect(rv).Interface() + return fmt.Sprintf("func(v %v) *%v { return &v } ( %#v )", typ, typ, pv) +} +func (m *Metadata) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Metadata) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Metadata) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Columns) > 0 { + for iNdEx := len(m.Columns) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Columns[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintStreamsmd(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *ColumnDesc) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ColumnDesc) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ColumnDesc) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Type != 0 { + i = encodeVarintStreamsmd(dAtA, i, uint64(m.Type)) + i-- + dAtA[i] = 0x10 + } + if m.Info != nil { + { + size, err := m.Info.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintStreamsmd(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ColumnMetadata) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ColumnMetadata) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ColumnMetadata) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Pages) > 0 { + for iNdEx := len(m.Pages) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Pages[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintStreamsmd(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *PageDesc) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *PageDesc) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *PageDesc) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Info != nil { + { + size, err := m.Info.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintStreamsmd(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func encodeVarintStreamsmd(dAtA []byte, offset int, v uint64) int { + offset -= sovStreamsmd(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *Metadata) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Columns) > 0 { + for _, e := range m.Columns { + l = e.Size() + n += 1 + l + sovStreamsmd(uint64(l)) + } + } + return n +} + +func (m *ColumnDesc) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Info != nil { + l = m.Info.Size() + n += 1 + l + sovStreamsmd(uint64(l)) + } + if m.Type != 0 { + n += 1 + sovStreamsmd(uint64(m.Type)) + } + return n +} + +func (m *ColumnMetadata) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Pages) > 0 { + for _, e := range m.Pages { + l = e.Size() + n += 1 + l + sovStreamsmd(uint64(l)) + } + } + return n +} + +func (m *PageDesc) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Info != nil { + l = m.Info.Size() + n += 1 + l + sovStreamsmd(uint64(l)) + } + return n +} + +func sovStreamsmd(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} +func sozStreamsmd(x uint64) (n int) { + return sovStreamsmd(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (this *Metadata) String() string { + if this == nil { + return "nil" + } + repeatedStringForColumns := "[]*ColumnDesc{" + for _, f := range this.Columns { + repeatedStringForColumns += strings.Replace(f.String(), "ColumnDesc", "ColumnDesc", 1) + "," + } + repeatedStringForColumns += "}" + s := strings.Join([]string{`&Metadata{`, + `Columns:` + repeatedStringForColumns + `,`, + `}`, + }, "") + return s +} +func (this *ColumnDesc) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&ColumnDesc{`, + `Info:` + strings.Replace(fmt.Sprintf("%v", this.Info), "ColumnInfo", "datasetmd.ColumnInfo", 1) + `,`, + `Type:` + fmt.Sprintf("%v", this.Type) + `,`, + `}`, + }, "") + return s +} +func (this *ColumnMetadata) String() string { + if this == nil { + return "nil" + } + repeatedStringForPages := "[]*PageDesc{" + for _, f := range this.Pages { + repeatedStringForPages += strings.Replace(f.String(), "PageDesc", "PageDesc", 1) + "," + } + repeatedStringForPages += "}" + s := strings.Join([]string{`&ColumnMetadata{`, + `Pages:` + repeatedStringForPages + `,`, + `}`, + }, "") + return s +} +func (this *PageDesc) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&PageDesc{`, + `Info:` + strings.Replace(fmt.Sprintf("%v", this.Info), "PageInfo", "datasetmd.PageInfo", 1) + `,`, + `}`, + }, "") + return s +} +func valueToStringStreamsmd(v interface{}) string { + rv := reflect.ValueOf(v) + if rv.IsNil() { + return "nil" + } + pv := reflect.Indirect(rv).Interface() + return fmt.Sprintf("*%v", pv) +} +func (m *Metadata) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStreamsmd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Metadata: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Metadata: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Columns", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStreamsmd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthStreamsmd + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthStreamsmd + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Columns = append(m.Columns, &ColumnDesc{}) + if err := m.Columns[len(m.Columns)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipStreamsmd(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthStreamsmd + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthStreamsmd + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ColumnDesc) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStreamsmd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ColumnDesc: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ColumnDesc: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Info", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStreamsmd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthStreamsmd + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthStreamsmd + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Info == nil { + m.Info = &datasetmd.ColumnInfo{} + } + if err := m.Info.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + } + m.Type = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStreamsmd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Type |= ColumnType(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipStreamsmd(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthStreamsmd + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthStreamsmd + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ColumnMetadata) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStreamsmd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ColumnMetadata: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ColumnMetadata: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Pages", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStreamsmd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthStreamsmd + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthStreamsmd + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Pages = append(m.Pages, &PageDesc{}) + if err := m.Pages[len(m.Pages)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipStreamsmd(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthStreamsmd + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthStreamsmd + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *PageDesc) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStreamsmd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: PageDesc: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: PageDesc: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Info", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStreamsmd + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthStreamsmd + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthStreamsmd + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Info == nil { + m.Info = &datasetmd.PageInfo{} + } + if err := m.Info.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipStreamsmd(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthStreamsmd + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthStreamsmd + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipStreamsmd(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowStreamsmd + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowStreamsmd + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + return iNdEx, nil + case 1: + iNdEx += 8 + return iNdEx, nil + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowStreamsmd + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if length < 0 { + return 0, ErrInvalidLengthStreamsmd + } + iNdEx += length + if iNdEx < 0 { + return 0, ErrInvalidLengthStreamsmd + } + return iNdEx, nil + case 3: + for { + var innerWire uint64 + var start int = iNdEx + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowStreamsmd + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + innerWire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + innerWireType := int(innerWire & 0x7) + if innerWireType == 4 { + break + } + next, err := skipStreamsmd(dAtA[start:]) + if err != nil { + return 0, err + } + iNdEx = start + next + if iNdEx < 0 { + return 0, ErrInvalidLengthStreamsmd + } + } + return iNdEx, nil + case 4: + return iNdEx, nil + case 5: + iNdEx += 4 + return iNdEx, nil + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + } + panic("unreachable") +} + +var ( + ErrInvalidLengthStreamsmd = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowStreamsmd = fmt.Errorf("proto: integer overflow") +) diff --git a/pkg/dataobj/internal/metadata/streamsmd/streamsmd.proto b/pkg/dataobj/internal/metadata/streamsmd/streamsmd.proto new file mode 100644 index 0000000000000..5f58faa91aa33 --- /dev/null +++ b/pkg/dataobj/internal/metadata/streamsmd/streamsmd.proto @@ -0,0 +1,60 @@ +// streamsmd.proto holds metadata for the streams section of a data object. The +// streams section conains references to log streams within the data object. +syntax = "proto3"; + +package dataobj.metadata.streams.v1; + +import "pkg/dataobj/internal/metadata/datasetmd/datasetmd.proto"; + +option go_package = "github.com/grafana/loki/v3/pkg/dataobj/internal/metadata/streamsmd"; + +// Metadata describes the metadata for the streams section. +message Metadata { + // Columns within the stream. + repeated ColumnDesc columns = 1; +} + +// ColumnDesc describes an individual column within the streams table. +message ColumnDesc { + // Information about the column. + dataobj.metadata.dataset.v1.ColumnInfo info = 1; + + // Column type. + ColumnType type = 2; +} + +// ColumnType represents the valid types that a stream's column can have. +enum ColumnType { + // Invalid column type. + COLUMN_TYPE_UNSPECIFIED = 0; + + // COLUMN_TYPE_STREAM_ID is a column containing the stream ID. Stream IDs are + // unique across the entire data object. + COLUMN_TYPE_STREAM_ID = 1; + + // COLUMN_TYPE_MIN_TIMESTAMP is a column containing the minimum timestamp of + // a stream. + COLUMN_TYPE_MIN_TIMESTAMP = 2; + + // COLUMN_TYPE_MAX_TIMESTAMP is a column containing the maximum timestamp of + // a stream. + COLUMN_TYPE_MAX_TIMESTAMP = 3; + + // COLUMN_TYPE_LABEL is a column containing a label. + COLUMN_TYPE_LABEL = 4; + + // COLUMN_TYPE_ROWS is a column indicating the number of rows for a stream. + COLUMN_TYPE_ROWS = 5; +} + +// ColumnMetadata describes the metadata for a column. +message ColumnMetadata { + // Pages within the column. + repeated PageDesc pages = 1; +} + +// PageDesc describes an individual page within a column. +message PageDesc { + // Information about the page. + dataobj.metadata.dataset.v1.PageInfo info = 1; +}