diff --git a/bloom.go b/bloom.go index 8918e542..f84abf20 100644 --- a/bloom.go +++ b/bloom.go @@ -5,8 +5,8 @@ import ( "github.com/segmentio/parquet-go/bloom" "github.com/segmentio/parquet-go/bloom/xxhash" + "github.com/segmentio/parquet-go/deprecated" "github.com/segmentio/parquet-go/encoding" - "github.com/segmentio/parquet-go/encoding/plain" "github.com/segmentio/parquet-go/format" "github.com/segmentio/parquet-go/internal/unsafecast" ) @@ -143,54 +143,58 @@ type splitBlockEncoding struct { encoding.NotSupported } -func (splitBlockEncoding) EncodeBoolean(dst, src []byte) ([]byte, error) { +func (splitBlockEncoding) EncodeBoolean(dst []byte, src []byte) ([]byte, error) { splitBlockEncodeUint8(bloom.MakeSplitBlockFilter(dst), src) return dst, nil } -func (splitBlockEncoding) EncodeInt32(dst, src []byte) ([]byte, error) { - splitBlockEncodeUint32(bloom.MakeSplitBlockFilter(dst), unsafecast.BytesToUint32(src)) +func (splitBlockEncoding) EncodeInt32(dst []byte, src []int32) ([]byte, error) { + splitBlockEncodeUint32(bloom.MakeSplitBlockFilter(dst), unsafecast.Int32ToUint32(src)) return dst, nil } -func (splitBlockEncoding) EncodeInt64(dst, src []byte) ([]byte, error) { - splitBlockEncodeUint64(bloom.MakeSplitBlockFilter(dst), unsafecast.BytesToUint64(src)) +func (splitBlockEncoding) EncodeInt64(dst []byte, src []int64) ([]byte, error) { + splitBlockEncodeUint64(bloom.MakeSplitBlockFilter(dst), unsafecast.Int64ToUint64(src)) return dst, nil } -func (e splitBlockEncoding) EncodeInt96(dst, src []byte) ([]byte, error) { - splitBlockEncodeFixedLenByteArray(bloom.MakeSplitBlockFilter(dst), src, 12) +func (e splitBlockEncoding) EncodeInt96(dst []byte, src []deprecated.Int96) ([]byte, error) { + splitBlockEncodeFixedLenByteArray(bloom.MakeSplitBlockFilter(dst), deprecated.Int96ToBytes(src), 12) return dst, nil } -func (splitBlockEncoding) EncodeFloat(dst, src []byte) ([]byte, error) { - splitBlockEncodeUint32(bloom.MakeSplitBlockFilter(dst), unsafecast.BytesToUint32(src)) +func (splitBlockEncoding) EncodeFloat(dst []byte, src []float32) ([]byte, error) { + splitBlockEncodeUint32(bloom.MakeSplitBlockFilter(dst), unsafecast.Float32ToUint32(src)) return dst, nil } -func (splitBlockEncoding) EncodeDouble(dst, src []byte) ([]byte, error) { - splitBlockEncodeUint64(bloom.MakeSplitBlockFilter(dst), unsafecast.BytesToUint64(src)) +func (splitBlockEncoding) EncodeDouble(dst []byte, src []float64) ([]byte, error) { + splitBlockEncodeUint64(bloom.MakeSplitBlockFilter(dst), unsafecast.Float64ToUint64(src)) return dst, nil } -func (splitBlockEncoding) EncodeByteArray(dst, src []byte) ([]byte, error) { +func (splitBlockEncoding) EncodeByteArray(dst []byte, src []byte, offsets []uint32) ([]byte, error) { filter := bloom.MakeSplitBlockFilter(dst) buffer := make([]uint64, 0, filterEncodeBufferSize) + baseOffset := offsets[0] + + for _, endOffset := range offsets[1:] { + value := src[baseOffset:endOffset:endOffset] + baseOffset = endOffset - err := plain.RangeByteArray(src, func(value []byte) error { if len(buffer) == cap(buffer) { filter.InsertBulk(buffer) buffer = buffer[:0] } + buffer = append(buffer, xxhash.Sum64(value)) - return nil - }) + } filter.InsertBulk(buffer) - return dst, err + return dst, nil } -func (splitBlockEncoding) EncodeFixedLenByteArray(dst, src []byte, size int) ([]byte, error) { +func (splitBlockEncoding) EncodeFixedLenByteArray(dst []byte, src []byte, size int) ([]byte, error) { filter := bloom.MakeSplitBlockFilter(dst) if size == 16 { splitBlockEncodeUint128(filter, unsafecast.BytesToUint128(src)) diff --git a/bloom_test.go b/bloom_test.go index cdbfc91a..978e68c7 100644 --- a/bloom_test.go +++ b/bloom_test.go @@ -6,7 +6,6 @@ import ( "github.com/segmentio/parquet-go/bloom" "github.com/segmentio/parquet-go/deprecated" - "github.com/segmentio/parquet-go/encoding/plain" "github.com/segmentio/parquet-go/internal/quick" "github.com/segmentio/parquet-go/internal/unsafecast" ) @@ -16,7 +15,7 @@ func TestSplitBlockFilter(t *testing.T) { return make(bloom.SplitBlockFilter, bloom.NumSplitBlocksOf(int64(numValues), 11)) } - encoding := SplitBlockFilter("$").Encoding() + enc := SplitBlockFilter("$").Encoding() check := func(filter bloom.SplitBlockFilter, value Value) bool { return filter.Check(value.hash(&bloom.XXH64{})) @@ -30,7 +29,7 @@ func TestSplitBlockFilter(t *testing.T) { scenario: "BOOLEAN", function: func(values []bool) bool { filter := newFilter(len(values)) - encoding.EncodeBoolean(filter.Bytes(), unsafecast.BoolToBytes(values)) + enc.EncodeBoolean(filter.Bytes(), unsafecast.BoolToBytes(values)) for _, v := range values { if !check(filter, ValueOf(v)) { return false @@ -44,7 +43,7 @@ func TestSplitBlockFilter(t *testing.T) { scenario: "INT32", function: func(values []int32) bool { filter := newFilter(len(values)) - encoding.EncodeInt32(filter.Bytes(), unsafecast.Int32ToBytes(values)) + enc.EncodeInt32(filter.Bytes(), values) for _, v := range values { if !check(filter, ValueOf(v)) { return false @@ -58,7 +57,7 @@ func TestSplitBlockFilter(t *testing.T) { scenario: "INT64", function: func(values []int64) bool { filter := newFilter(len(values)) - encoding.EncodeInt64(filter.Bytes(), unsafecast.Int64ToBytes(values)) + enc.EncodeInt64(filter.Bytes(), values) for _, v := range values { if !check(filter, ValueOf(v)) { return false @@ -72,7 +71,7 @@ func TestSplitBlockFilter(t *testing.T) { scenario: "INT96", function: func(values []deprecated.Int96) bool { filter := newFilter(len(values)) - encoding.EncodeInt96(filter.Bytes(), deprecated.Int96ToBytes(values)) + enc.EncodeInt96(filter.Bytes(), values) for _, v := range values { if !check(filter, ValueOf(v)) { return false @@ -86,7 +85,7 @@ func TestSplitBlockFilter(t *testing.T) { scenario: "FLOAT", function: func(values []float32) bool { filter := newFilter(len(values)) - encoding.EncodeFloat(filter.Bytes(), unsafecast.Float32ToBytes(values)) + enc.EncodeFloat(filter.Bytes(), values) for _, v := range values { if !check(filter, ValueOf(v)) { return false @@ -100,7 +99,7 @@ func TestSplitBlockFilter(t *testing.T) { scenario: "DOUBLE", function: func(values []float64) bool { filter := newFilter(len(values)) - encoding.EncodeDouble(filter.Bytes(), unsafecast.Float64ToBytes(values)) + enc.EncodeDouble(filter.Bytes(), values) for _, v := range values { if !check(filter, ValueOf(v)) { return false @@ -113,12 +112,15 @@ func TestSplitBlockFilter(t *testing.T) { { scenario: "BYTE_ARRAY", function: func(values [][]byte) bool { - byteArrays := make([]byte, 0) + content := make([]byte, 0, 512) + offsets := make([]uint32, len(values)) for _, value := range values { - byteArrays = plain.AppendByteArray(byteArrays, value) + offsets = append(offsets, uint32(len(content))) + content = append(content, value...) } + offsets = append(offsets, uint32(len(content))) filter := newFilter(len(values)) - encoding.EncodeByteArray(filter.Bytes(), byteArrays) + enc.EncodeByteArray(filter.Bytes(), content, offsets) for _, v := range values { if !check(filter, ValueOf(v)) { return false @@ -132,7 +134,7 @@ func TestSplitBlockFilter(t *testing.T) { scenario: "FIXED_LEN_BYTE_ARRAY", function: func(values []byte) bool { filter := newFilter(len(values)) - encoding.EncodeFixedLenByteArray(filter.Bytes(), values, 1) + enc.EncodeFixedLenByteArray(filter.Bytes(), values, 1) for _, v := range values { if !check(filter, ValueOf([1]byte{v})) { return false @@ -163,9 +165,8 @@ func BenchmarkSplitBlockFilter(b *testing.B) { v[i] = r.Int63() } - v64 := unsafecast.Int64ToBytes(v) for i := 0; i < b.N; i++ { - e.EncodeInt64(f, v64) + e.EncodeInt64(f, v) } b.SetBytes(8 * N) diff --git a/buffer.go b/buffer.go index 1251a835..f4004150 100644 --- a/buffer.go +++ b/buffer.go @@ -67,7 +67,11 @@ func (buf *Buffer) configure(schema *Schema) { encoding := encodingOf(leaf.node) if isDictionaryEncoding(encoding) { - dictBuffer := make([]byte, 0, columnType.EstimateSize(bufferCap)) + estimatedDictBufferSize := columnType.EstimateSize(bufferCap) + dictBuffer := columnType.NewValues( + make([]byte, 0, estimatedDictBufferSize), + nil, + ) dictionary = columnType.NewDictionary(columnIndex, 0, dictBuffer) columnType = dictionary.Type() } diff --git a/buffer_test.go b/buffer_test.go index 8e674bf7..03c100ec 100644 --- a/buffer_test.go +++ b/buffer_test.go @@ -178,7 +178,7 @@ func TestBuffer(t *testing.T) { typ parquet.Type }{ {scenario: "plain", typ: test.typ}, - {scenario: "indexed", typ: test.typ.NewDictionary(0, 0, nil).Type()}, + {scenario: "indexed", typ: test.typ.NewDictionary(0, 0, test.typ.NewValues(nil, nil)).Type()}, } { t.Run(config.scenario, func(t *testing.T) { for _, mod := range [...]struct { diff --git a/column.go b/column.go index b3a2b3d7..1efd835d 100644 --- a/column.go +++ b/column.go @@ -487,11 +487,13 @@ func schemaRepetitionTypeOf(s *format.SchemaElement) format.FieldRepetitionType } type dictPage struct { - values []byte + values []byte + offsets []uint32 } func (p *dictPage) reset() { p.values = p.values[:0] + p.offsets = p.offsets[:0] } type dataPage struct { @@ -499,6 +501,7 @@ type dataPage struct { definitionLevels []byte data []byte values []byte + offsets []uint32 dictionary Dictionary } @@ -507,6 +510,7 @@ func (p *dataPage) reset() { p.definitionLevels = p.definitionLevels[:0] p.data = p.data[:0] p.values = p.values[:0] + p.offsets = p.offsets[:0] p.dictionary = nil } @@ -602,25 +606,26 @@ func (c *Column) decodeDataPageV2(header DataPageHeaderV2, page *dataPage) (Page } func (c *Column) decodeDataPage(header DataPageHeader, numValues int64, page *dataPage, data []byte) (Page, error) { - encoding := LookupEncoding(header.Encoding()) + pageEncoding := LookupEncoding(header.Encoding()) pageType := c.Type() - if isDictionaryEncoding(encoding) { + if isDictionaryEncoding(pageEncoding) { // In some legacy configurations, the PLAIN_DICTIONARY encoding is used // on data page headers to indicate that the page contains indexes into // the dictionary page, but the page is still encoded using the RLE // encoding in this case, so we convert it to RLE_DICTIONARY. - encoding = &RLEDictionary + pageEncoding = &RLEDictionary pageType = indexedPageType{newIndexedType(pageType, page.dictionary)} } - var err error - page.values, err = pageType.Decode(page.values, data, encoding) + values := pageType.NewValues(page.values, page.offsets) + values, err := pageType.Decode(values, data, pageEncoding) if err != nil { return nil, err } + page.values, page.offsets = values.Data() - newPage := pageType.NewPage(c.Index(), int(numValues), page.values) + newPage := pageType.NewPage(c.Index(), int(numValues), values) switch { case c.maxRepetitionLevel > 0: newPage = newRepeatedPage(newPage.Buffer(), c.maxRepetitionLevel, c.maxDefinitionLevel, page.repetitionLevels, page.definitionLevels) @@ -681,19 +686,19 @@ func (c *Column) decodeDictionary(header DictionaryPageHeader, page *dataPage, d } pageType := c.Type() - encoding := header.Encoding() - if encoding == format.PlainDictionary { - encoding = format.Plain + pageEncoding := header.Encoding() + if pageEncoding == format.PlainDictionary { + pageEncoding = format.Plain } - var err error - page.values, err = pageType.Decode(page.values, page.data, LookupEncoding(encoding)) + values := pageType.NewValues(dict.values, dict.offsets) + values, err := pageType.Decode(values, page.data, LookupEncoding(pageEncoding)) if err != nil { return nil, err } - - dict.values = append(dict.values[:0], page.values...) - return pageType.NewDictionary(int(c.index), int(header.NumValues()), dict.values), nil + dict.values, dict.offsets = values.Data() + values = pageType.NewValues(dict.values, dict.offsets) + return pageType.NewDictionary(int(c.index), int(header.NumValues()), values), nil } var ( diff --git a/column_buffer.go b/column_buffer.go index af07e5ca..69c8fffd 100644 --- a/column_buffer.go +++ b/column_buffer.go @@ -423,7 +423,7 @@ type repeatedColumnBuffer struct { reordered bool maxRepetitionLevel byte maxDefinitionLevel byte - rows []region + rows []offsetMapping repetitionLevels []byte definitionLevels []byte buffer []Value @@ -431,23 +431,21 @@ type repeatedColumnBuffer struct { nullOrdering nullOrdering } -// The region type maps the logical offset of rows within the repetition and -// definition levels, to the base offsets in the underlying column buffers +// The offsetMapping type maps the logical offset of rows within the repetition +// and definition levels, to the base offsets in the underlying column buffers // where the non-null values have been written. -type region struct { +type offsetMapping struct { offset uint32 baseOffset uint32 } -func sizeOfRegion(regions []region) int64 { return 8 * int64(len(regions)) } - func newRepeatedColumnBuffer(base ColumnBuffer, maxRepetitionLevel, maxDefinitionLevel byte, nullOrdering nullOrdering) *repeatedColumnBuffer { n := base.Cap() return &repeatedColumnBuffer{ base: base, maxRepetitionLevel: maxRepetitionLevel, maxDefinitionLevel: maxDefinitionLevel, - rows: make([]region, 0, n/8), + rows: make([]offsetMapping, 0, n/8), repetitionLevels: make([]byte, 0, n), definitionLevels: make([]byte, 0, n), nullOrdering: nullOrdering, @@ -460,7 +458,7 @@ func (col *repeatedColumnBuffer) Clone() ColumnBuffer { reordered: col.reordered, maxRepetitionLevel: col.maxRepetitionLevel, maxDefinitionLevel: col.maxDefinitionLevel, - rows: append([]region{}, col.rows...), + rows: append([]offsetMapping{}, col.rows...), repetitionLevels: append([]byte{}, col.repetitionLevels...), definitionLevels: append([]byte{}, col.definitionLevels...), nullOrdering: col.nullOrdering, @@ -538,7 +536,7 @@ func (col *repeatedColumnBuffer) Page() BufferedPage { } } - column.rows = append(column.rows, region{ + column.rows = append(column.rows, offsetMapping{ offset: uint32(len(column.repetitionLevels)), baseOffset: uint32(baseOffset), }) @@ -576,7 +574,7 @@ func (col *repeatedColumnBuffer) Reset() { } func (col *repeatedColumnBuffer) Size() int64 { - return sizeOfRegion(col.rows) + int64(len(col.repetitionLevels)) + int64(len(col.definitionLevels)) + col.base.Size() + return int64(8*len(col.rows)+len(col.repetitionLevels)+len(col.definitionLevels)) + col.base.Size() } func (col *repeatedColumnBuffer) Cap() int { return cap(col.rows) } @@ -666,7 +664,7 @@ func (col *repeatedColumnBuffer) writeRow(row []Value) error { } if row[0].repetitionLevel == 0 { - col.rows = append(col.rows, region{ + col.rows = append(col.rows, offsetMapping{ offset: uint32(len(col.repetitionLevels)), baseOffset: uint32(baseOffset), }) @@ -682,7 +680,7 @@ func (col *repeatedColumnBuffer) writeRow(row []Value) error { func (col *repeatedColumnBuffer) writeValues(row sparse.Array, levels columnLevels) { if levels.repetitionLevel == 0 { - col.rows = append(col.rows, region{ + col.rows = append(col.rows, offsetMapping{ offset: uint32(len(col.repetitionLevels)), baseOffset: uint32(col.base.NumValues()), }) @@ -1355,7 +1353,8 @@ func (col *doubleColumnBuffer) ReadValuesAt(values []Value, offset int64) (n int type byteArrayColumnBuffer struct { byteArrayPage - offsets []uint32 + lengths []uint32 + scratch []byte } func newByteArrayColumnBuffer(typ Type, columnIndex int16, numValues int32) *byteArrayColumnBuffer { @@ -1363,30 +1362,31 @@ func newByteArrayColumnBuffer(typ Type, columnIndex int16, numValues int32) *byt byteArrayPage: byteArrayPage{ typ: typ, values: make([]byte, 0, typ.EstimateSize(int(numValues))), + offsets: make([]uint32, 0, numValues+1), columnIndex: ^columnIndex, }, - offsets: make([]uint32, 0, numValues), + lengths: make([]uint32, 0, numValues), } } -func (col *byteArrayColumnBuffer) cloneOffsets() []uint32 { - offsets := make([]uint32, len(col.offsets)) - copy(offsets, col.offsets) - return offsets -} - func (col *byteArrayColumnBuffer) Clone() ColumnBuffer { return &byteArrayColumnBuffer{ byteArrayPage: byteArrayPage{ typ: col.typ, values: col.cloneValues(), - numValues: col.numValues, + offsets: col.cloneOffsets(), columnIndex: col.columnIndex, }, - offsets: col.cloneOffsets(), + lengths: col.cloneLengths(), } } +func (col *byteArrayColumnBuffer) cloneLengths() []uint32 { + lengths := make([]uint32, len(col.lengths)) + copy(lengths, col.lengths) + return lengths +} + func (col *byteArrayColumnBuffer) ColumnIndex() ColumnIndex { return byteArrayColumnIndex{&col.byteArrayPage} } @@ -1402,44 +1402,49 @@ func (col *byteArrayColumnBuffer) Dictionary() Dictionary { return nil } func (col *byteArrayColumnBuffer) Pages() Pages { return onePage(col.Page()) } func (col *byteArrayColumnBuffer) Page() BufferedPage { - if len(col.offsets) > 0 && orderOfUint32(col.offsets) < 1 { // unordered? - values := make([]byte, 0, len(col.values)) // TODO: pool this buffer? - - for _, offset := range col.offsets { - values = plain.AppendByteArray(values, col.valueAt(offset)) + if len(col.lengths) > 0 && orderOfUint32(col.offsets) < 1 { // unordered? + if cap(col.scratch) < len(col.values) { + col.scratch = make([]byte, 0, cap(col.values)) + } else { + col.scratch = col.scratch[:0] } - col.values = values - col.offsets = col.offsets[:0] - - for i := 0; i < len(col.values); { - n := plain.ByteArrayLength(col.values[i:]) - col.offsets = append(col.offsets, uint32(i)) - i += plain.ByteArrayLengthSize - i += n + for i := range col.lengths { + n := len(col.scratch) + col.scratch = append(col.scratch, col.index(i)...) + col.offsets[i] = uint32(n) } + + col.values, col.scratch = col.scratch, col.values } + // The offsets have the total length as the last item. Since we are about to + // expose the column buffer's internal state as a Page value we ensure that + // the last offset is the total length of all values. + col.offsets = append(col.offsets[:len(col.lengths)], uint32(len(col.values))) return &col.byteArrayPage } func (col *byteArrayColumnBuffer) Reset() { col.values = col.values[:0] col.offsets = col.offsets[:0] - col.numValues = 0 + col.lengths = col.lengths[:0] } -func (col *byteArrayColumnBuffer) Cap() int { return cap(col.offsets) } +func (col *byteArrayColumnBuffer) NumRows() int64 { return int64(col.Len()) } + +func (col *byteArrayColumnBuffer) NumValues() int64 { return int64(col.Len()) } + +func (col *byteArrayColumnBuffer) Cap() int { return cap(col.lengths) } -func (col *byteArrayColumnBuffer) Len() int { return len(col.offsets) } +func (col *byteArrayColumnBuffer) Len() int { return len(col.lengths) } func (col *byteArrayColumnBuffer) Less(i, j int) bool { - a := col.valueAt(col.offsets[i]) - b := col.valueAt(col.offsets[j]) - return bytes.Compare(a, b) < 0 + return bytes.Compare(col.index(i), col.index(j)) < 0 } func (col *byteArrayColumnBuffer) Swap(i, j int) { col.offsets[i], col.offsets[j] = col.offsets[j], col.offsets[i] + col.lengths[i], col.lengths[j] = col.lengths[j], col.lengths[i] } func (col *byteArrayColumnBuffer) Write(b []byte) (int, error) { @@ -1453,14 +1458,17 @@ func (col *byteArrayColumnBuffer) WriteByteArrays(values []byte) (int, error) { } func (col *byteArrayColumnBuffer) writeByteArrays(values []byte) (count, bytes int, err error) { - baseCount, baseBytes := len(col.offsets), len(col.values) + baseCount := len(col.lengths) + baseBytes := len(col.values) + (plain.ByteArrayLengthSize * len(col.lengths)) err = plain.RangeByteArray(values, func(value []byte) error { col.append(unsafecast.BytesToString(value)) return nil }) - return len(col.offsets) - baseCount, len(col.values) - baseBytes, err + count = len(col.lengths) - baseCount + bytes = (len(col.values) - baseBytes) + (plain.ByteArrayLengthSize * count) + return count, bytes, err } func (col *byteArrayColumnBuffer) WriteValues(values []Value) (int, error) { @@ -1480,12 +1488,12 @@ func (col *byteArrayColumnBuffer) ReadValuesAt(values []Value, offset int64) (n i := int(offset) switch { case i < 0: - return 0, errRowIndexOutOfBounds(offset, int64(len(col.offsets))) - case i >= len(col.offsets): + return 0, errRowIndexOutOfBounds(offset, int64(len(col.lengths))) + case i >= len(col.lengths): return 0, io.EOF default: - for n < len(values) && i < len(col.offsets) { - values[n] = col.makeValueBytes(col.valueAt(col.offsets[i])) + for n < len(values) && i < len(col.lengths) { + values[n] = col.makeValueBytes(col.index(i)) n++ i++ } @@ -1498,8 +1506,15 @@ func (col *byteArrayColumnBuffer) ReadValuesAt(values []Value, offset int64) (n func (col *byteArrayColumnBuffer) append(value string) { col.offsets = append(col.offsets, uint32(len(col.values))) - col.values = plain.AppendByteArrayString(col.values, value) - col.numValues++ + col.lengths = append(col.lengths, uint32(len(value))) + col.values = append(col.values, value...) +} + +func (col *byteArrayColumnBuffer) index(i int) []byte { + offset := col.offsets[i] + length := col.lengths[i] + end := offset + length + return col.values[offset:end:end] } type fixedLenByteArrayColumnBuffer struct { diff --git a/dictionary.go b/dictionary.go index 7021613f..36436444 100644 --- a/dictionary.go +++ b/dictionary.go @@ -105,8 +105,8 @@ type booleanDictionary struct { table [2]int32 } -func newBooleanDictionary(typ Type, columnIndex int16, numValues int32, values []byte) *booleanDictionary { - indexOfFalse, indexOfTrue := int32(-1), int32(-1) +func newBooleanDictionary(typ Type, columnIndex int16, numValues int32, data encoding.Values) *booleanDictionary { + indexOfFalse, indexOfTrue, values := int32(-1), int32(-1), data.Boolean() for i := int32(0); i < numValues && indexOfFalse < 0 && indexOfTrue < 0; i += 8 { v := values[i] @@ -220,11 +220,11 @@ type int32Dictionary struct { table *hashprobe.Int32Table } -func newInt32Dictionary(typ Type, columnIndex int16, numValues int32, values []byte) *int32Dictionary { +func newInt32Dictionary(typ Type, columnIndex int16, numValues int32, data encoding.Values) *int32Dictionary { return &int32Dictionary{ int32Page: int32Page{ typ: typ, - values: unsafecast.BytesToInt32(values)[:numValues], + values: data.Int32()[:numValues], columnIndex: ^columnIndex, }, } @@ -320,11 +320,11 @@ type int64Dictionary struct { table *hashprobe.Int64Table } -func newInt64Dictionary(typ Type, columnIndex int16, numValues int32, values []byte) *int64Dictionary { +func newInt64Dictionary(typ Type, columnIndex int16, numValues int32, data encoding.Values) *int64Dictionary { return &int64Dictionary{ int64Page: int64Page{ typ: typ, - values: unsafecast.BytesToInt64(values)[:numValues], + values: data.Int64()[:numValues], columnIndex: ^columnIndex, }, } @@ -407,11 +407,11 @@ type int96Dictionary struct { hashmap map[deprecated.Int96]int32 } -func newInt96Dictionary(typ Type, columnIndex int16, numValues int32, values []byte) *int96Dictionary { +func newInt96Dictionary(typ Type, columnIndex int16, numValues int32, data encoding.Values) *int96Dictionary { return &int96Dictionary{ int96Page: int96Page{ typ: typ, - values: deprecated.BytesToInt96(values)[:numValues], + values: data.Int96()[:numValues], columnIndex: ^columnIndex, }, } @@ -502,11 +502,11 @@ type floatDictionary struct { table *hashprobe.Float32Table } -func newFloatDictionary(typ Type, columnIndex int16, numValues int32, values []byte) *floatDictionary { +func newFloatDictionary(typ Type, columnIndex int16, numValues int32, data encoding.Values) *floatDictionary { return &floatDictionary{ floatPage: floatPage{ typ: typ, - values: unsafecast.BytesToFloat32(values)[:numValues], + values: data.Float()[:numValues], columnIndex: ^columnIndex, }, } @@ -589,11 +589,11 @@ type doubleDictionary struct { table *hashprobe.Float64Table } -func newDoubleDictionary(typ Type, columnIndex int16, numValues int32, values []byte) *doubleDictionary { +func newDoubleDictionary(typ Type, columnIndex int16, numValues int32, data encoding.Values) *doubleDictionary { return &doubleDictionary{ doublePage: doublePage{ typ: typ, - values: unsafecast.BytesToFloat64(values)[:numValues], + values: data.Double()[:numValues], columnIndex: ^columnIndex, }, } @@ -673,38 +673,37 @@ func (d *doubleDictionary) Page() BufferedPage { type byteArrayDictionary struct { byteArrayPage - table map[string]int32 - offsets []uint32 + table map[string]int32 } -func newByteArrayDictionary(typ Type, columnIndex int16, numValues int32, values []byte) *byteArrayDictionary { - d := &byteArrayDictionary{ +func newByteArrayDictionary(typ Type, columnIndex int16, numValues int32, data encoding.Values) *byteArrayDictionary { + values, offsets := data.ByteArray() + // The first offset must always be zero, and the last offset is the length + // of the values in bytes. + // + // As an optimization we make the assumption that the backing array of the + // offsets slice belongs to the dictionary. + switch { + case cap(offsets) == 0: + offsets = make([]uint32, 1, 8) + case len(offsets) == 0: + offsets = append(offsets[:0], 0) + } + return &byteArrayDictionary{ byteArrayPage: byteArrayPage{ typ: typ, values: values, + offsets: offsets, columnIndex: ^columnIndex, }, - offsets: make([]uint32, 0, numValues), } - - for i := 0; i < len(values); { - d.offsets = append(d.offsets, uint32(i)) - n := plain.ByteArrayLength(values[i:]) - i += plain.ByteArrayLengthSize - i += n - } - - d.numValues = int32(len(d.offsets)) - return d } func (d *byteArrayDictionary) Type() Type { return newIndexedType(d.typ, d) } -func (d *byteArrayDictionary) Len() int { return len(d.offsets) } +func (d *byteArrayDictionary) Len() int { return d.len() } -func (d *byteArrayDictionary) Index(i int32) Value { return d.makeValueBytes(d.index(i)) } - -func (d *byteArrayDictionary) index(i int32) []byte { return d.valueAt(d.offsets[i]) } +func (d *byteArrayDictionary) Index(i int32) Value { return d.makeValueBytes(d.index(int(i))) } func (d *byteArrayDictionary) Insert(indexes []int32, values []Value) { model := Value{} @@ -712,14 +711,11 @@ func (d *byteArrayDictionary) Insert(indexes []int32, values []Value) { } func (d *byteArrayDictionary) init() { - d.table = make(map[string]int32, d.numValues) + numValues := d.len() + d.table = make(map[string]int32, numValues) - for i := 0; i < len(d.values); { - n := plain.ByteArrayLength(d.values[i:]) - i += plain.ByteArrayLengthSize - s := d.values[i : i+n] - i += n - d.table[string(s)] = int32(len(d.table)) + for i := 0; i < numValues; i++ { + d.table[string(d.index(i))] = int32(len(d.table)) } } @@ -736,11 +732,10 @@ func (d *byteArrayDictionary) insert(indexes []int32, rows sparse.Array) { index, exists := d.table[value] if !exists { value = cloneString(value) - index = d.numValues - d.numValues++ + index = int32(len(d.table)) d.table[value] = index + d.values = append(d.values, value...) d.offsets = append(d.offsets, uint32(len(d.values))) - d.values = plain.AppendByteArrayString(d.values, value) } indexes[i] = index @@ -761,7 +756,7 @@ func (d *byteArrayDictionary) Lookup(indexes []int32, values []Value) { func (d *byteArrayDictionary) Bounds(indexes []int32) (min, max Value) { if len(indexes) > 0 { - base := d.index(indexes[0]) + base := d.index(int(indexes[0])) minValue := unsafecast.BytesToString(base) maxValue := minValue values := [64]string{} @@ -791,9 +786,8 @@ func (d *byteArrayDictionary) Bounds(indexes []int32) (min, max Value) { } func (d *byteArrayDictionary) Reset() { - d.offsets = d.offsets[:0] + d.offsets = d.offsets[:1] d.values = d.values[:0] - d.numValues = 0 for k := range d.table { delete(d.table, k) @@ -809,8 +803,8 @@ type fixedLenByteArrayDictionary struct { hashmap map[string]int32 } -func newFixedLenByteArrayDictionary(typ Type, columnIndex int16, numValues int32, data []byte) *fixedLenByteArrayDictionary { - size := typ.Length() +func newFixedLenByteArrayDictionary(typ Type, columnIndex int16, numValues int32, values encoding.Values) *fixedLenByteArrayDictionary { + data, size := values.FixedLenByteArray() return &fixedLenByteArrayDictionary{ fixedLenByteArrayPage: fixedLenByteArrayPage{ typ: typ, @@ -924,11 +918,11 @@ type uint32Dictionary struct { table *hashprobe.Uint32Table } -func newUint32Dictionary(typ Type, columnIndex int16, numValues int32, data []byte) *uint32Dictionary { +func newUint32Dictionary(typ Type, columnIndex int16, numValues int32, data encoding.Values) *uint32Dictionary { return &uint32Dictionary{ uint32Page: uint32Page{ typ: typ, - values: unsafecast.BytesToUint32(data)[:numValues], + values: data.Uint32()[:numValues], columnIndex: ^columnIndex, }, } @@ -1011,11 +1005,11 @@ type uint64Dictionary struct { table *hashprobe.Uint64Table } -func newUint64Dictionary(typ Type, columnIndex int16, numValues int32, data []byte) *uint64Dictionary { +func newUint64Dictionary(typ Type, columnIndex int16, numValues int32, data encoding.Values) *uint64Dictionary { return &uint64Dictionary{ uint64Page: uint64Page{ typ: typ, - values: unsafecast.BytesToUint64(data), + values: data.Uint64()[:numValues], columnIndex: ^columnIndex, }, } @@ -1098,11 +1092,11 @@ type be128Dictionary struct { table *hashprobe.Uint128Table } -func newBE128Dictionary(typ Type, columnIndex int16, numValues int32, data []byte) *be128Dictionary { +func newBE128Dictionary(typ Type, columnIndex int16, numValues int32, data encoding.Values) *be128Dictionary { return &be128Dictionary{ be128Page: be128Page{ typ: typ, - values: unsafecast.BytesToUint128(data), + values: data.Uint128()[:numValues], columnIndex: ^columnIndex, }, } @@ -1227,10 +1221,14 @@ func (t *indexedType) NewColumnBuffer(columnIndex, numValues int) ColumnBuffer { return newIndexedColumnBuffer(t, makeColumnIndex(columnIndex), makeNumValues(numValues)) } -func (t *indexedType) NewPage(columnIndex, numValues int, data []byte) Page { +func (t *indexedType) NewPage(columnIndex, numValues int, data encoding.Values) Page { return newIndexedPage(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } +func (t *indexedType) NewValues(values []byte, _ []uint32) encoding.Values { + return encoding.Int32ValuesFromBytes(values) +} + // indexedPage is an implementation of the BufferedPage interface which stores // indexes instead of plain value. The indexes reference the values in a // dictionary that the page was created for. @@ -1240,16 +1238,17 @@ type indexedPage struct { columnIndex int16 } -func newIndexedPage(typ *indexedType, columnIndex int16, numValues int32, values []byte) *indexedPage { +func newIndexedPage(typ *indexedType, columnIndex int16, numValues int32, data encoding.Values) *indexedPage { // RLE encoded values that contain dictionary indexes in data pages are // sometimes truncated when they contain only zeros. We account for this // special case here and extend the values buffer if it is shorter than // needed to hold `numValues`. - size := 4 * int(numValues) + size := int(numValues) + values := data.Int32() if len(values) < size { if cap(values) < size { - tmp := make([]byte, size) + tmp := make([]int32, size) copy(tmp, values) values = tmp } else { @@ -1262,7 +1261,7 @@ func newIndexedPage(typ *indexedType, columnIndex int16, numValues int32, values return &indexedPage{ typ: typ, - values: unsafecast.BytesToInt32(values[:size]), + values: values[:size], columnIndex: ^columnIndex, } } @@ -1285,7 +1284,7 @@ func (page *indexedPage) RepetitionLevels() []byte { return nil } func (page *indexedPage) DefinitionLevels() []byte { return nil } -func (page *indexedPage) Data() []byte { return unsafecast.Int32ToBytes(page.values) } +func (page *indexedPage) Data() encoding.Values { return encoding.Int32Values(page.values) } func (page *indexedPage) Values() ValueReader { return &indexedPageValues{page: page} } @@ -1322,12 +1321,12 @@ func (page *indexedPage) Slice(i, j int64) BufferedPage { // its dictionary instead of plain values. type indexedPageType struct{ *indexedType } -func (t indexedPageType) Encode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.EncodeInt32(dst, src) +func (t indexedPageType) Encode(dst []byte, src encoding.Values, enc encoding.Encoding) ([]byte, error) { + return encoding.EncodeInt32(dst, src, enc) } -func (t indexedPageType) Decode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.DecodeInt32(dst, src) +func (t indexedPageType) Decode(dst encoding.Values, src []byte, enc encoding.Encoding) (encoding.Values, error) { + return encoding.DecodeInt32(dst, src, enc) } type indexedPageValues struct { diff --git a/dictionary_amd64.s b/dictionary_amd64.s index 8aa4663c..dd9e2596 100644 --- a/dictionary_amd64.s +++ b/dictionary_amd64.s @@ -797,6 +797,7 @@ indexOutOfBounds: TEXT ·dictionaryLookupByteArrayString(SB), NOSPLIT, $0-104 MOVQ dict_base+0(FP), AX MOVQ dict_len+8(FP), BX + DECQ BX // the offsets have the total length as last element MOVQ page+24(FP), CX @@ -816,16 +817,17 @@ loop: CMPL DI, BX JAE indexOutOfBounds - // Load the offset within the dictionary page where the value is stored. + // Load the offsets within the dictionary page where the value is stored. // We trust the offsets to be correct since they are generated internally by // the dictionary code, there is no need to check that they are within the // bounds of the dictionary page. - MOVL (AX)(DI*4), DI + MOVL 0(AX)(DI*4), DX + MOVL 4(AX)(DI*4), DI - // Load the value from the dictionary page. The page uses the PLAIN encoding - // where each byte array is prefixed with a 4 bytes little endian length. - LEAQ 4(CX)(DI*1), DX - MOVL (CX)(DI*1), DI + // Compute the length of the value (the difference between two consecutive + // offsets), and the pointer to the first byte of the string value. + SUBL DX, DI + LEAQ (CX)(DX*1), DX // Store the length and pointer to the value into the output location. // The memory layout is expected to hold a pointer and length, which are diff --git a/dictionary_purego.go b/dictionary_purego.go index 23688869..8a5f691b 100644 --- a/dictionary_purego.go +++ b/dictionary_purego.go @@ -39,7 +39,7 @@ func (d *doubleDictionary) lookup(indexes []int32, rows sparse.Array) { func (d *byteArrayDictionary) lookupString(indexes []int32, rows sparse.Array) { checkLookupIndexBounds(indexes, rows) for i, j := range indexes { - v := d.index(j) + v := d.index(int(j)) *(*string)(rows.Index(i)) = *(*string)(unsafe.Pointer(&v)) } } diff --git a/dictionary_test.go b/dictionary_test.go index b2ffb26a..ccf5b386 100644 --- a/dictionary_test.go +++ b/dictionary_test.go @@ -36,7 +36,7 @@ func TestDictionary(t *testing.T) { func testDictionary(t *testing.T, typ parquet.Type, numValues int) { const columnIndex = 1 - dict := typ.NewDictionary(columnIndex, 0, nil) + dict := typ.NewDictionary(columnIndex, 0, typ.NewValues(nil, nil)) values := make([]parquet.Value, numValues) indexes := make([]int32, numValues) lookups := make([]parquet.Value, numValues) @@ -167,7 +167,8 @@ func BenchmarkDictionary(b *testing.B) { b.Run(test.scenario, func(b *testing.B) { for j, typ := range dictionaryTypes { for _, numValues := range []int{1e2, 1e3, 1e4, 1e5, 1e6} { - dict := typ.NewDictionary(0, 0, make([]byte, 0, 4*numValues)) + buf := typ.NewValues(make([]byte, 0, 4*numValues), nil) + dict := typ.NewDictionary(0, 0, buf) values := make([]parquet.Value, numValues) f := randValueFuncOf(typ) diff --git a/encoding/bitpacked/bitpacked.go b/encoding/bitpacked/bitpacked.go index 4a3ff471..f49cdb8d 100644 --- a/encoding/bitpacked/bitpacked.go +++ b/encoding/bitpacked/bitpacked.go @@ -18,12 +18,12 @@ func (e *Encoding) Encoding() format.Encoding { return format.BitPacked } -func (e *Encoding) EncodeLevels(dst, src []byte) ([]byte, error) { +func (e *Encoding) EncodeLevels(dst []byte, src []uint8) ([]byte, error) { dst, err := encodeLevels(dst[:0], src, uint(e.BitWidth)) return dst, e.wrap(err) } -func (e *Encoding) DecodeLevels(dst, src []byte) ([]byte, error) { +func (e *Encoding) DecodeLevels(dst []uint8, src []byte) ([]uint8, error) { dst, err := decodeLevels(dst[:0], src, uint(e.BitWidth)) return dst, e.wrap(err) } diff --git a/encoding/bytestreamsplit/bytestreamsplit.go b/encoding/bytestreamsplit/bytestreamsplit.go index bc4e2a54..5f7899ca 100644 --- a/encoding/bytestreamsplit/bytestreamsplit.go +++ b/encoding/bytestreamsplit/bytestreamsplit.go @@ -3,6 +3,7 @@ package bytestreamsplit import ( "github.com/segmentio/parquet-go/encoding" "github.com/segmentio/parquet-go/format" + "github.com/segmentio/parquet-go/internal/unsafecast" ) // This encoder implements a version of the Byte Stream Split encoding as described @@ -19,40 +20,34 @@ func (e *Encoding) Encoding() format.Encoding { return format.ByteStreamSplit } -func (e *Encoding) EncodeFloat(dst, src []byte) ([]byte, error) { - if (len(src) % 4) != 0 { - return dst[:0], encoding.ErrEncodeInvalidInputSize(e, "FLOAT", len(src)) - } - dst = resize(dst, len(src)) - encodeFloat(dst, src) +func (e *Encoding) EncodeFloat(dst []byte, src []float32) ([]byte, error) { + dst = resize(dst, 4*len(src)) + encodeFloat(dst, unsafecast.Float32ToBytes(src)) return dst, nil } -func (e *Encoding) EncodeDouble(dst, src []byte) ([]byte, error) { - if (len(src) % 8) != 0 { - return dst[:0], encoding.ErrEncodeInvalidInputSize(e, "DOUBLE", len(src)) - } - dst = resize(dst, len(src)) - encodeDouble(dst, src) +func (e *Encoding) EncodeDouble(dst []byte, src []float64) ([]byte, error) { + dst = resize(dst, 8*len(src)) + encodeDouble(dst, unsafecast.Float64ToBytes(src)) return dst, nil } -func (e *Encoding) DecodeFloat(dst, src []byte) ([]byte, error) { +func (e *Encoding) DecodeFloat(dst []float32, src []byte) ([]float32, error) { if (len(src) % 4) != 0 { - return dst[:0], encoding.ErrDecodeInvalidInputSize(e, "FLOAT", len(src)) + return dst, encoding.ErrDecodeInvalidInputSize(e, "FLOAT", len(src)) } - dst = resize(dst, len(src)) - decodeFloat(dst, src) - return dst, nil + buf := resize(unsafecast.Float32ToBytes(dst), len(src)) + decodeFloat(buf, src) + return unsafecast.BytesToFloat32(buf), nil } -func (e *Encoding) DecodeDouble(dst, src []byte) ([]byte, error) { +func (e *Encoding) DecodeDouble(dst []float64, src []byte) ([]float64, error) { if (len(src) % 8) != 0 { - return dst[:0], encoding.ErrDecodeInvalidInputSize(e, "DOUBLE", len(src)) + return dst, encoding.ErrDecodeInvalidInputSize(e, "DOUBLE", len(src)) } - dst = resize(dst, len(src)) - decodeDouble(dst, src) - return dst, nil + buf := resize(unsafecast.Float64ToBytes(dst), len(src)) + decodeDouble(buf, src) + return unsafecast.BytesToFloat64(buf), nil } func resize(buf []byte, size int) []byte { diff --git a/encoding/bytestreamsplit/bytestreamsplit_test.go b/encoding/bytestreamsplit/bytestreamsplit_test.go index 389b823e..d6cab66a 100644 --- a/encoding/bytestreamsplit/bytestreamsplit_test.go +++ b/encoding/bytestreamsplit/bytestreamsplit_test.go @@ -12,7 +12,7 @@ import ( ) func FuzzEncodeFloat(f *testing.F) { - fuzz.EncodeDouble(f, new(bytestreamsplit.Encoding)) + fuzz.EncodeFloat(f, new(bytestreamsplit.Encoding)) } func FuzzEncodeDouble(f *testing.F) { diff --git a/encoding/delta/binary_packed.go b/encoding/delta/binary_packed.go index 3228d57c..7b0ac1a7 100644 --- a/encoding/delta/binary_packed.go +++ b/encoding/delta/binary_packed.go @@ -25,28 +25,24 @@ func (e *BinaryPackedEncoding) Encoding() format.Encoding { return format.DeltaBinaryPacked } -func (e *BinaryPackedEncoding) EncodeInt32(dst, src []byte) ([]byte, error) { - if (len(src) % 4) != 0 { - return dst[:0], encoding.ErrEncodeInvalidInputSize(e, "INT64", len(src)) - } - return encodeInt32(dst[:0], bytesToInt32(src)), nil +func (e *BinaryPackedEncoding) EncodeInt32(dst []byte, src []int32) ([]byte, error) { + return encodeInt32(dst[:0], src), nil } -func (e *BinaryPackedEncoding) EncodeInt64(dst, src []byte) ([]byte, error) { - if (len(src) % 8) != 0 { - return dst[:0], encoding.ErrEncodeInvalidInputSize(e, "INT64", len(src)) - } - return encodeInt64(dst[:0], bytesToInt64(src)), nil +func (e *BinaryPackedEncoding) EncodeInt64(dst []byte, src []int64) ([]byte, error) { + return encodeInt64(dst[:0], src), nil } -func (e *BinaryPackedEncoding) DecodeInt32(dst, src []byte) ([]byte, error) { - dst, _, err := decodeInt32(dst[:0], src) - return dst, e.wrap(err) +func (e *BinaryPackedEncoding) DecodeInt32(dst []int32, src []byte) ([]int32, error) { + buf := unsafecast.Int32ToBytes(dst) + buf, _, err := decodeInt32(buf[:0], src) + return unsafecast.BytesToInt32(buf), e.wrap(err) } -func (e *BinaryPackedEncoding) DecodeInt64(dst, src []byte) ([]byte, error) { - dst, _, err := decodeInt64(dst[:0], src) - return dst, e.wrap(err) +func (e *BinaryPackedEncoding) DecodeInt64(dst []int64, src []byte) ([]int64, error) { + buf := unsafecast.Int64ToBytes(dst) + buf, _, err := decodeInt64(buf[:0], src) + return unsafecast.BytesToInt64(buf), e.wrap(err) } func (e *BinaryPackedEncoding) wrap(err error) error { diff --git a/encoding/delta/byte_array.go b/encoding/delta/byte_array.go index 38ba7ad2..d8cee612 100644 --- a/encoding/delta/byte_array.go +++ b/encoding/delta/byte_array.go @@ -2,11 +2,9 @@ package delta import ( "bytes" - "fmt" "sort" "github.com/segmentio/parquet-go/encoding" - "github.com/segmentio/parquet-go/encoding/plain" "github.com/segmentio/parquet-go/format" ) @@ -26,7 +24,7 @@ func (e *ByteArrayEncoding) Encoding() format.Encoding { return format.DeltaByteArray } -func (e *ByteArrayEncoding) EncodeByteArray(dst, src []byte) ([]byte, error) { +func (e *ByteArrayEncoding) EncodeByteArray(dst []byte, src []byte, offsets []uint32) ([]byte, error) { prefix := getInt32Buffer() defer putInt32Buffer(prefix) @@ -34,36 +32,27 @@ func (e *ByteArrayEncoding) EncodeByteArray(dst, src []byte) ([]byte, error) { defer putInt32Buffer(length) totalSize := 0 - lastValue := ([]byte)(nil) - - for i := 0; i < len(src); { - r := len(src) - i - if r < plain.ByteArrayLengthSize { - return dst[:0], plain.ErrTooShort(r) - } - n := plain.ByteArrayLength(src[i:]) - i += plain.ByteArrayLengthSize - r -= plain.ByteArrayLengthSize - if n > r { - return dst[:0], plain.ErrTooShort(n) - } - if n > plain.MaxByteArrayLength { - return dst[:0], plain.ErrTooLarge(n) - } - v := src[i : i+n : i+n] - p := 0 - - if len(v) <= maxLinearSearchPrefixLength { - p = linearSearchPrefixLength(lastValue, v) - } else { - p = binarySearchPrefixLength(lastValue, v) + if len(offsets) > 0 { + lastValue := ([]byte)(nil) + baseOffset := offsets[0] + + for _, endOffset := range offsets[1:] { + v := src[baseOffset:endOffset:endOffset] + n := int(endOffset - baseOffset) + p := 0 + baseOffset = endOffset + + if len(v) <= maxLinearSearchPrefixLength { + p = linearSearchPrefixLength(lastValue, v) + } else { + p = binarySearchPrefixLength(lastValue, v) + } + + prefix.values = append(prefix.values, int32(p)) + length.values = append(length.values, int32(n-p)) + lastValue = v + totalSize += n - p } - - prefix.values = append(prefix.values, int32(p)) - length.values = append(length.values, int32(n-p)) - lastValue = v - totalSize += n - p - i += n } dst = dst[:0] @@ -71,23 +60,24 @@ func (e *ByteArrayEncoding) EncodeByteArray(dst, src []byte) ([]byte, error) { dst = encodeInt32(dst, length.values) dst = resize(dst, len(dst)+totalSize) - b := dst[len(dst)-totalSize:] - i := plain.ByteArrayLengthSize - j := 0 + if len(offsets) > 0 { + b := dst[len(dst)-totalSize:] + i := int(offsets[0]) + j := 0 - _ = length.values[:len(prefix.values)] + _ = length.values[:len(prefix.values)] - for k, p := range prefix.values { - n := p + length.values[k] - j += copy(b[j:], src[i+int(p):i+int(n)]) - i += plain.ByteArrayLengthSize - i += int(n) + for k, p := range prefix.values { + n := p + length.values[k] + j += copy(b[j:], src[i+int(p):i+int(n)]) + i += int(n) + } } return dst, nil } -func (e *ByteArrayEncoding) EncodeFixedLenByteArray(dst, src []byte, size int) ([]byte, error) { +func (e *ByteArrayEncoding) EncodeFixedLenByteArray(dst []byte, src []byte, size int) ([]byte, error) { // The parquet specs say that this encoding is only supported for BYTE_ARRAY // values, but the reference Java implementation appears to support // FIXED_LEN_BYTE_ARRAY as well: @@ -135,8 +125,8 @@ func (e *ByteArrayEncoding) EncodeFixedLenByteArray(dst, src []byte, size int) ( return dst, nil } -func (e *ByteArrayEncoding) DecodeByteArray(dst, src []byte) ([]byte, error) { - dst = dst[:0] +func (e *ByteArrayEncoding) DecodeByteArray(dst []byte, src []byte, offsets []uint32) ([]byte, []uint32, error) { + dst, offsets = dst[:0], offsets[:0] prefix := getInt32Buffer() defer putInt32Buffer(prefix) @@ -147,23 +137,23 @@ func (e *ByteArrayEncoding) DecodeByteArray(dst, src []byte) ([]byte, error) { var err error src, err = prefix.decode(src) if err != nil { - return dst, encoding.Errorf(e, "decoding prefix lengths: %w", err) + return dst, offsets, e.wrapf("decoding prefix lengths: %w", err) } src, err = suffix.decode(src) if err != nil { - return dst, encoding.Errorf(e, "decoding suffix lengths: %w", err) + return dst, offsets, e.wrapf("decoding suffix lengths: %w", err) } if len(prefix.values) != len(suffix.values) { - return dst, encoding.Error(e, errPrefixAndSuffixLengthMismatch(len(prefix.values), len(suffix.values))) + return dst, offsets, e.wrap(errPrefixAndSuffixLengthMismatch(len(prefix.values), len(suffix.values))) } - return decodeByteArray(dst, src, prefix.values, suffix.values) + return decodeByteArray(dst, src, prefix.values, suffix.values, offsets) } -func (e *ByteArrayEncoding) DecodeFixedLenByteArray(dst, src []byte, size int) ([]byte, error) { +func (e *ByteArrayEncoding) DecodeFixedLenByteArray(dst []byte, src []byte, size int) ([]byte, error) { dst = dst[:0] if size < 0 || size > encoding.MaxFixedLenByteArraySize { - return dst, encoding.Error(e, encoding.ErrInvalidArgument) + return dst, e.wrap(encoding.ErrInvalidArgument) } prefix := getInt32Buffer() @@ -175,16 +165,27 @@ func (e *ByteArrayEncoding) DecodeFixedLenByteArray(dst, src []byte, size int) ( var err error src, err = prefix.decode(src) if err != nil { - return dst, fmt.Errorf("decoding prefix lengths: %w", err) + return dst, e.wrapf("decoding prefix lengths: %w", err) } src, err = suffix.decode(src) if err != nil { - return dst, fmt.Errorf("decoding suffix lengths: %w", err) + return dst, e.wrapf("decoding suffix lengths: %w", err) } if len(prefix.values) != len(suffix.values) { - return dst, errPrefixAndSuffixLengthMismatch(len(prefix.values), len(suffix.values)) + return dst, e.wrap(errPrefixAndSuffixLengthMismatch(len(prefix.values), len(suffix.values))) } - return decodeFixedLenByteArray(dst, src, size, prefix.values, suffix.values) + return decodeFixedLenByteArray(dst[:0], src, size, prefix.values, suffix.values) +} + +func (e *ByteArrayEncoding) wrap(err error) error { + if err != nil { + err = encoding.Error(e, err) + } + return err +} + +func (e *ByteArrayEncoding) wrapf(msg string, args ...interface{}) error { + return encoding.Errorf(e, msg, args...) } func linearSearchPrefixLength(base, data []byte) (n int) { diff --git a/encoding/delta/byte_array_amd64.go b/encoding/delta/byte_array_amd64.go index f05b02c9..71a077ee 100644 --- a/encoding/delta/byte_array_amd64.go +++ b/encoding/delta/byte_array_amd64.go @@ -3,7 +3,6 @@ package delta import ( - "github.com/segmentio/parquet-go/encoding/plain" "golang.org/x/sys/cpu" ) @@ -48,20 +47,30 @@ func validatePrefixAndSuffixLengthValues(prefix, suffix []int32, maxLength int) return totalPrefixLength, totalSuffixLength, nil } +//go:noescape +func decodeByteArrayOffsets(offsets []uint32, prefix, suffix []int32) + //go:noescape func decodeByteArrayAVX2(dst, src []byte, prefix, suffix []int32) int -func decodeByteArray(dst, src []byte, prefix, suffix []int32) ([]byte, error) { +func decodeByteArray(dst, src []byte, prefix, suffix []int32, offsets []uint32) ([]byte, []uint32, error) { totalPrefixLength, totalSuffixLength, err := validatePrefixAndSuffixLengthValues(prefix, suffix, len(src)) if err != nil { - return dst, err + return dst, offsets, err } - totalLength := plain.ByteArrayLengthSize*len(prefix) + totalPrefixLength + totalSuffixLength + totalLength := totalPrefixLength + totalSuffixLength dst = resizeNoMemclr(dst, totalLength+padding) + if size := len(prefix) + 1; cap(offsets) < size { + offsets = make([]uint32, size) + } else { + offsets = offsets[:size] + } + _ = prefix[:len(suffix)] _ = suffix[:len(prefix)] + decodeByteArrayOffsets(offsets, prefix, suffix) var lastValue []byte var i int @@ -88,23 +97,18 @@ func decodeByteArray(dst, src []byte, prefix, suffix []int32) ([]byte, error) { for k := range prefix { p := int(prefix[k]) n := int(suffix[k]) - plain.PutByteArrayLength(dst[i:], p+n) - i += plain.ByteArrayLengthSize - k := i + lastValueOffset := i i += copy(dst[i:], lastValue[:p]) i += copy(dst[i:], src[j:j+n]) j += n - lastValue = dst[k:] + lastValue = dst[lastValueOffset:] } - return dst[:totalLength], nil + return dst[:totalLength], offsets, nil } //go:noescape -func decodeFixedLenByteArrayAVX2(dst, src []byte, prefix, suffix []int32) int - -//go:noescape -func decodeFixedLenByteArrayAVX2x128bits(dst, src []byte, prefix, suffix []int32) int +func decodeByteArrayAVX2x128bits(dst, src []byte, prefix, suffix []int32) int func decodeFixedLenByteArray(dst, src []byte, size int, prefix, suffix []int32) ([]byte, error) { totalPrefixLength, totalSuffixLength, err := validatePrefixAndSuffixLengthValues(prefix, suffix, len(src)) @@ -133,9 +137,9 @@ func decodeFixedLenByteArray(dst, src []byte, size int, prefix, suffix []int32) if k > 0 && n >= padding { if size == 16 { - i = decodeFixedLenByteArrayAVX2x128bits(dst, src, prefix[:k], suffix[:k]) + i = decodeByteArrayAVX2x128bits(dst, src, prefix[:k], suffix[:k]) } else { - i = decodeFixedLenByteArrayAVX2(dst, src, prefix[:k], suffix[:k]) + i = decodeByteArrayAVX2(dst, src, prefix[:k], suffix[:k]) } j = len(src) - n prefix = prefix[k:] diff --git a/encoding/delta/byte_array_amd64.s b/encoding/delta/byte_array_amd64.s index dec5f0d2..b8b70983 100644 --- a/encoding/delta/byte_array_amd64.s +++ b/encoding/delta/byte_array_amd64.s @@ -122,73 +122,31 @@ done: MOVB R12, ok+72(FP) RET -// func decodeByteArrayAVX2(dst, src []byte, prefix, suffix []int32) int -TEXT ·decodeByteArrayAVX2(SB), NOSPLIT, $0-104 - MOVQ dst_base+0(FP), AX - MOVQ src_base+24(FP), BX - MOVQ prefix_base+48(FP), CX - MOVQ suffix_base+72(FP), DX - MOVQ suffix_len+80(FP), DI +// func decodeByteArrayOffsets(offsets []uint32, prefix, suffix []int32) +TEXT ·decodeByteArrayOffsets(SB), NOSPLIT, $0-72 + MOVQ offsets_base+0(FP), AX + MOVQ prefix_base+24(FP), BX + MOVQ suffix_base+48(FP), CX + MOVQ suffix_len+56(FP), DX - ADDQ $4, AX XORQ SI, SI - XORQ R8, R8 - XORQ R9, R9 - MOVQ AX, R10 // last value - + XORQ R10, R10 JMP test loop: - MOVLQZX (CX)(SI*4), R8 // prefix length - MOVLQZX (DX)(SI*4), R9 // suffix length - MOVQ R8, R11 - ADDQ R9, R11 - MOVL R11, -4(AX) -prefix: - VMOVDQU (R10), X0 - VMOVDQU X0, (AX) - CMPQ R8, $16 - JA copyPrefix -suffix: - VMOVDQU (BX), X1 - VMOVDQU X1, (AX)(R8*1) - CMPQ R9, $16 - JA copySuffix -next: - MOVQ AX, R10 - LEAQ 4(AX)(R11*1), AX - LEAQ 0(BX)(R9*1), BX + MOVL (BX)(SI*4), R8 + MOVL (CX)(SI*4), R9 + MOVL R10, (AX)(SI*4) + ADDL R8, R10 + ADDL R9, R10 INCQ SI test: - CMPQ SI, DI + CMPQ SI, DX JNE loop - MOVQ dst_base+0(FP), BX - SUBQ BX, AX - SUBQ $4, AX - MOVQ AX, ret+96(FP) - VZEROUPPER + MOVL R10, (AX)(SI*4) RET -copyPrefix: - MOVQ $16, R12 -copyPrefixLoop: - VMOVDQU (R10)(R12*1), Y0 - VMOVDQU Y0, (AX)(R12*1) - ADDQ $32, R12 - CMPQ R12, R8 - JB copyPrefixLoop - JMP suffix -copySuffix: - MOVQ $16, R12 - LEAQ (AX)(R8*1), R13 -copySuffixLoop: - VMOVDQU (BX)(R12*1), Y1 - VMOVDQU Y1, (R13)(R12*1) - ADDQ $32, R12 - CMPQ R12, R9 - JB copySuffixLoop - JMP next -// func decodeFixedLenByteArrayAVX2(dst, src []byte, prefix, suffix []int32) int -TEXT ·decodeFixedLenByteArrayAVX2(SB), NOSPLIT, $0-104 +// func decodeByteArrayAVX2(dst, src []byte, prefix, suffix []int32) int +TEXT ·decodeByteArrayAVX2(SB), NOSPLIT, $0-104 MOVQ dst_base+0(FP), AX MOVQ src_base+24(FP), BX MOVQ prefix_base+48(FP), CX @@ -248,8 +206,8 @@ copySuffixLoop: JB copySuffixLoop JMP next -// func decodeFixedLenByteArrayAVX2x128bits(dst, src []byte, prefix, suffix []int32) int -TEXT ·decodeFixedLenByteArrayAVX2x128bits(SB), NOSPLIT, $0-104 +// func decodeByteArrayAVX2x128bits(dst, src []byte, prefix, suffix []int32) int +TEXT ·decodeByteArrayAVX2x128bits(SB), NOSPLIT, $0-104 MOVQ dst_base+0(FP), AX MOVQ src_base+24(FP), BX MOVQ prefix_base+48(FP), CX diff --git a/encoding/delta/byte_array_purego.go b/encoding/delta/byte_array_purego.go index d9751817..972c1fec 100644 --- a/encoding/delta/byte_array_purego.go +++ b/encoding/delta/byte_array_purego.go @@ -2,11 +2,7 @@ package delta -import ( - "github.com/segmentio/parquet-go/encoding/plain" -) - -func decodeByteArray(dst, src []byte, prefix, suffix []int32) ([]byte, error) { +func decodeByteArray(dst, src []byte, prefix, suffix []int32, offsets []uint32) ([]byte, []uint32, error) { _ = prefix[:len(suffix)] _ = suffix[:len(prefix)] @@ -15,25 +11,26 @@ func decodeByteArray(dst, src []byte, prefix, suffix []int32) ([]byte, error) { n := int(suffix[i]) p := int(prefix[i]) if n < 0 { - return dst, errInvalidNegativeValueLength(n) + return dst, offsets, errInvalidNegativeValueLength(n) } if n > len(src) { - return dst, errValueLengthOutOfBounds(n, len(src)) + return dst, offsets, errValueLengthOutOfBounds(n, len(src)) } if p < 0 { - return dst, errInvalidNegativePrefixLength(p) + return dst, offsets, errInvalidNegativePrefixLength(p) } if p > len(lastValue) { - return dst, errPrefixLengthOutOfBounds(p, len(lastValue)) + return dst, offsets, errPrefixLengthOutOfBounds(p, len(lastValue)) } - dst = plain.AppendByteArrayLength(dst, p+n) j := len(dst) + offsets = append(offsets, uint32(j)) dst = append(dst, lastValue[:p]...) dst = append(dst, src[:n]...) lastValue = dst[j:] src = src[n:] } - return dst, nil + + return dst, append(offsets, uint32(len(dst))), nil } func decodeFixedLenByteArray(dst, src []byte, size int, prefix, suffix []int32) ([]byte, error) { diff --git a/encoding/delta/delta.go b/encoding/delta/delta.go index 6d27f0f9..f05f0875 100644 --- a/encoding/delta/delta.go +++ b/encoding/delta/delta.go @@ -3,7 +3,6 @@ package delta import ( "fmt" "sync" - "unsafe" "github.com/segmentio/parquet-go/internal/unsafecast" ) @@ -12,6 +11,14 @@ type int32Buffer struct { values []int32 } +func (buf *int32Buffer) resize(size int) { + if cap(buf.values) < size { + buf.values = make([]int32, size, 2*size) + } else { + buf.values = buf.values[:size] + } +} + func (buf *int32Buffer) decode(src []byte) ([]byte, error) { values, remain, err := decodeInt32(unsafecast.Int32ToBytes(buf.values[:0]), src) buf.values = unsafecast.BytesToInt32(values) @@ -38,14 +45,6 @@ func putInt32Buffer(b *int32Buffer) { int32BufferPool.Put(b) } -func bytesToInt32(b []byte) []int32 { - return unsafe.Slice(*(**int32)(unsafe.Pointer(&b)), len(b)/4) -} - -func bytesToInt64(b []byte) []int64 { - return unsafe.Slice(*(**int64)(unsafe.Pointer(&b)), len(b)/8) -} - func resizeNoMemclr(buf []byte, size int) []byte { if cap(buf) < size { return grow(buf, size) diff --git a/encoding/delta/length_byte_array.go b/encoding/delta/length_byte_array.go index 639e28fd..ad83c5db 100644 --- a/encoding/delta/length_byte_array.go +++ b/encoding/delta/length_byte_array.go @@ -2,7 +2,6 @@ package delta import ( "github.com/segmentio/parquet-go/encoding" - "github.com/segmentio/parquet-go/encoding/plain" "github.com/segmentio/parquet-go/format" ) @@ -18,63 +17,54 @@ func (e *LengthByteArrayEncoding) Encoding() format.Encoding { return format.DeltaLengthByteArray } -func (e *LengthByteArrayEncoding) EncodeByteArray(dst, src []byte) ([]byte, error) { - dst = dst[:0] +func (e *LengthByteArrayEncoding) EncodeByteArray(dst []byte, src []byte, offsets []uint32) ([]byte, error) { + if len(offsets) == 0 { + return dst[:0], nil + } length := getInt32Buffer() defer putInt32Buffer(length) - totalSize := 0 - - for i := 0; i < len(src); { - r := len(src) - i - if r < plain.ByteArrayLengthSize { - return dst, encoding.Error(e, plain.ErrTooShort(r)) - } - n := plain.ByteArrayLength(src[i:]) - i += plain.ByteArrayLengthSize - r -= plain.ByteArrayLengthSize - if n > r { - return dst, encoding.Error(e, plain.ErrTooShort(n)) - } - if n > plain.MaxByteArrayLength { - return dst, encoding.Error(e, plain.ErrTooLarge(n)) - } - length.values = append(length.values, int32(n)) - totalSize += n - i += n - } + length.resize(len(offsets) - 1) + encodeByteArrayLengths(length.values, offsets) + dst = dst[:0] dst = encodeInt32(dst, length.values) - dst = resize(dst, len(dst)+totalSize) - - b := dst[len(dst)-totalSize:] - i := plain.ByteArrayLengthSize - j := 0 - - for _, n := range length.values { - j += copy(b[j:], src[i:i+int(n)]) - i += plain.ByteArrayLengthSize - i += int(n) - } - + dst = append(dst, src...) return dst, nil } -func (e *LengthByteArrayEncoding) DecodeByteArray(dst, src []byte) ([]byte, error) { - dst = dst[:0] +func (e *LengthByteArrayEncoding) DecodeByteArray(dst []byte, src []byte, offsets []uint32) ([]byte, []uint32, error) { + dst, offsets = dst[:0], offsets[:0] length := getInt32Buffer() defer putInt32Buffer(length) src, err := length.decode(src) if err != nil { - return dst, encoding.Error(e, err) + return dst, offsets, e.wrap(err) + } + + if size := len(length.values) + 1; cap(offsets) < size { + offsets = make([]uint32, size) + } else { + offsets = offsets[:size] + } + + lastOffset, invalidLength := decodeByteArrayLengths(offsets, length.values) + if invalidLength != 0 { + return dst, offsets, e.wrap(errInvalidNegativeValueLength(int(invalidLength))) + } + if int(lastOffset) > len(src) { + return dst, offsets, e.wrap(errValueLengthOutOfBounds(int(lastOffset), len(src))) } - dst, err = decodeLengthByteArray(dst, src, length.values) + return append(dst, src[:lastOffset]...), offsets, nil +} + +func (e *LengthByteArrayEncoding) wrap(err error) error { if err != nil { err = encoding.Error(e, err) } - return dst, err + return err } diff --git a/encoding/delta/length_byte_array_amd64.go b/encoding/delta/length_byte_array_amd64.go index 13601530..905e8516 100644 --- a/encoding/delta/length_byte_array_amd64.go +++ b/encoding/delta/length_byte_array_amd64.go @@ -2,83 +2,8 @@ package delta -import ( - "github.com/segmentio/parquet-go/encoding/plain" - "golang.org/x/sys/cpu" -) - //go:noescape -func validateLengthValuesAVX2(lengths []int32) (totalLength int, ok bool) - -func validateLengthValues(lengths []int32, maxLength int) (totalLength int, err error) { - if cpu.X86.HasAVX2 { - totalLength, ok := validateLengthValuesAVX2(lengths) - if ok { - return totalLength, nil - } - } - - for i := range lengths { - n := int(lengths[i]) - if n < 0 { - return 0, errInvalidNegativeValueLength(n) - } - if n > maxLength { - return 0, errValueLengthOutOfBounds(n, maxLength) - } - totalLength += n - } - - if totalLength > maxLength { - err = errValueLengthOutOfBounds(totalLength, maxLength) - } - return totalLength, err -} +func encodeByteArrayLengths(lengths []int32, offsets []uint32) //go:noescape -func decodeLengthByteArrayAVX2(dst, src []byte, lengths []int32) int - -func decodeLengthByteArray(dst, src []byte, lengths []int32) ([]byte, error) { - totalLength, err := validateLengthValues(lengths, len(src)) - if err != nil { - return dst, err - } - - size := plain.ByteArrayLengthSize * len(lengths) - size += totalLength - src = src[:totalLength] - dst = resizeNoMemclr(dst, size+padding) - - i := 0 - j := 0 - k := 0 - n := 0 - - // To leverage the SEE optimized implementation of the function we must - // create enough padding at the end to prevent the opportunistic reads - // and writes from overflowing past the buffer's limits. - if cpu.X86.HasAVX2 && len(src) > padding { - k = len(lengths) - - for k > 0 && n < padding { - k-- - n += int(lengths[k]) - } - - if k > 0 && n >= padding { - i = decodeLengthByteArrayAVX2(dst, src, lengths[:k]) - j = len(src) - n - } else { - k = 0 - } - } - - for _, n := range lengths[k:] { - plain.PutByteArrayLength(dst[i:], int(n)) - i += plain.ByteArrayLengthSize - i += copy(dst[i:], src[j:j+int(n)]) - j += int(n) - } - - return dst[:size], nil -} +func decodeByteArrayLengths(offsets []uint32, lengths []int32) (lastOffset uint32, invalidLength int32) diff --git a/encoding/delta/length_byte_array_amd64.s b/encoding/delta/length_byte_array_amd64.s index 7cc43d0f..bc6292e2 100644 --- a/encoding/delta/length_byte_array_amd64.s +++ b/encoding/delta/length_byte_array_amd64.s @@ -1,139 +1,122 @@ //go:build !purego -#include "funcdata.h" #include "textflag.h" -// func validateLengthValuesAVX2(lengths []int32) (totalLength int, ok bool) -TEXT ·validateLengthValuesAVX2(SB), NOSPLIT, $0-33 +// func encodeByteArrayLengths(lengths []int32, offsets []uint32) +TEXT ·encodeByteArrayLengths(SB), NOSPLIT, $0-48 MOVQ lengths_base+0(FP), AX MOVQ lengths_len+8(FP), CX - - XORQ BX, BX // totalLength - XORQ DX, DX // err + MOVQ offsets_base+24(FP), BX XORQ SI, SI - XORQ DI, DI - XORQ R8, R8 - CMPQ CX, $16 + CMPQ CX, $4 JB test - MOVQ CX, DI - SHRQ $4, DI - SHLQ $4, DI - - VPXOR X0, X0, X0 // totalLengths - VPXOR X1, X1, X1 // negative test -loopAVX2: - VMOVDQU (AX)(SI*4), Y2 - VMOVDQU 32(AX)(SI*4), Y3 - VPADDD Y2, Y0, Y0 - VPADDD Y3, Y0, Y0 - VPOR Y2, Y1, Y1 - VPOR Y3, Y1, Y1 - ADDQ $16, SI - CMPQ SI, DI - JNE loopAVX2 - - // If any of the 32 bit words has its most significant bit set to 1, - // then at least one of the values was negative, which must be reported as - // an error. - VMOVMSKPS Y1, R8 - CMPQ R8, $0 - JNE done - - VPSRLDQ $4, Y0, Y1 - VPSRLDQ $8, Y0, Y2 - VPSRLDQ $12, Y0, Y3 - - VPADDD Y1, Y0, Y0 - VPADDD Y3, Y2, Y2 - VPADDD Y2, Y0, Y0 - - VPERM2I128 $1, Y0, Y0, Y1 - VPADDD Y1, Y0, Y0 - VZEROUPPER - MOVQ X0, BX - ANDQ $0x7FFFFFFF, BX + MOVQ CX, DX + SHRQ $2, DX + SHLQ $2, DX +loopSSE2: + MOVOU 0(BX)(SI*4), X0 + MOVOU 4(BX)(SI*4), X1 + PSUBL X0, X1 + MOVOU X1, (AX)(SI*4) + ADDQ $4, SI + CMPQ SI, DX + JNE loopSSE2 JMP test loop: - MOVL (AX)(SI*4), DI - ADDL DI, BX - ORL DI, R8 + MOVL 0(BX)(SI*4), R8 + MOVL 4(BX)(SI*4), R9 + SUBL R8, R9 + MOVL R9, (AX)(SI*4) INCQ SI test: CMPQ SI, CX JNE loop - CMPL R8, $0 - JL done - MOVB $1, DX -done: - MOVQ BX, totalLength+24(FP) - MOVB DX, ok+32(FP) RET -// This function is an optimization of the decodeLengthByteArray using AVX2 -// instructions to implement an opportunistic copy strategy which improves -// throughput compared to using runtime.memmove (via Go's copy). -// -// Parquet columns of type BYTE_ARRAY will often hold short strings, rarely -// exceeding a couple hundred bytes in size. Making a function call to -// runtime.memmove for each value results in spending most of the CPU time -// on branching rather than actually copying bytes to the output buffer. -// -// This function works by always assuming it can copy 16 bytes of data between -// the input and outputs, even in the event where a value is shorter than this. -// -// The pointers to the current positions for input and output pointers are -// always adjusted by the right number of bytes so that the next writes -// overwrite any extra bytes that were written in the previous iteration of the -// copy loop. -// -// The throughput of this function is not as good as runtime.memmove for large -// buffers, but it ends up being close to an order of magnitude higher for the -// common case of working with short strings. -// -// func decodeLengthByteArrayAVX2(dst, src []byte, lengths []int32) int -TEXT ·decodeLengthByteArrayAVX2(SB), NOSPLIT, $0-80 - MOVQ dst_base+0(FP), AX - MOVQ src_base+24(FP), BX - MOVQ lengths_base+48(FP), DX - MOVQ lengths_len+56(FP), DI - - LEAQ (DX)(DI*4), DI - LEAQ 4(AX), AX - XORQ CX, CX +// func decodeByteArrayLengths(offsets []uint32, length []int32) (lastOffset uint32, invalidLength int32) +TEXT ·decodeByteArrayLengths(SB), NOSPLIT, $0-56 + MOVQ offsets_base+0(FP), AX + MOVQ lengths_base+24(FP), BX + MOVQ lengths_len+32(FP), CX + + XORQ DX, DX // lastOffset + XORQ DI, DI // invalidLength + XORQ SI, SI + + CMPQ CX, $4 + JL test + + MOVQ CX, R8 + SHRQ $2, R8 + SHLQ $2, R8 + + MOVL $0, (AX) + PXOR X0, X0 + PXOR X3, X3 + // This loop computes the prefix sum of the lengths array in order to + // generate values of the offsets array. + // + // We stick to SSE2 to keep the code simple (the Go compiler appears to + // assume that SSE2 must be supported on AMD64) which already yields most + // of the performance that we could get on this subroutine if we were using + // AVX2. + // + // The X3 register also accumulates a mask of all length values, which is + // checked after the loop to determine whether any of the lengths were + // negative. + // + // The following article contains a description of the prefix sum algorithm + // used in this function: https://en.algorithmica.org/hpc/algorithms/prefix/ +loopSSE2: + MOVOU (BX)(SI*4), X1 + POR X1, X3 + + MOVOA X1, X2 + PSLLDQ $4, X2 + PADDD X2, X1 + + MOVOA X1, X2 + PSLLDQ $8, X2 + PADDD X2, X1 + + PADDD X1, X0 + MOVOU X0, 4(AX)(SI*4) + + PSHUFD $0b11111111, X0, X0 + + ADDQ $4, SI + CMPQ SI, R8 + JNE loopSSE2 + + // If any of the most significant bits of double words in the X3 register + // are set to 1, it indicates that one of the lengths was negative and + // therefore the prefix sum is invalid. + // + // TODO: we report the invalid length as -1, effectively losing the original + // value due to the aggregation within X3. This is something that we might + // want to address in the future to provide better error reporting. + MOVMSKPS X3, R8 + MOVL $-1, R9 + CMPL R8, $0 + CMOVLNE R9, DI + + MOVQ X0, DX JMP test loop: - MOVL (DX), CX - MOVL CX, -4(AX) - // First pass moves 16 bytes, this makes it a very fast path for short - // strings. - VMOVDQU (BX), X0 - VMOVDQU X0, (AX) - CMPQ CX, $16 - JA copy -next: - LEAQ 4(AX)(CX*1), AX - LEAQ 0(BX)(CX*1), BX - LEAQ 4(DX), DX + MOVL (BX)(SI*4), R8 + MOVL DX, (AX)(SI*4) + ADDL R8, DX + CMPL R8, $0 + CMOVLLT R8, DI + INCQ SI test: - CMPQ DX, DI + CMPQ SI, CX JNE loop - MOVQ dst_base+0(FP), BX - SUBQ BX, AX - SUBQ $4, AX - MOVQ AX, ret+72(FP) - VZEROUPPER + + MOVL DX, (AX)(SI*4) + MOVL DX, lastOffset+48(FP) + MOVL DI, invalidLength+52(FP) RET -copy: - // Values longer than 16 bytes enter this loop and move 32 byte chunks - // which helps improve throughput on larger chunks. - MOVQ $16, SI -copyLoop32: - VMOVDQU (BX)(SI*1), Y0 - VMOVDQU Y0, (AX)(SI*1) - ADDQ $32, SI - CMPQ SI, CX - JAE next - JMP copyLoop32 diff --git a/encoding/delta/length_byte_array_purego.go b/encoding/delta/length_byte_array_purego.go index 91d6efa9..0c0fb6ba 100644 --- a/encoding/delta/length_byte_array_purego.go +++ b/encoding/delta/length_byte_array_purego.go @@ -2,21 +2,23 @@ package delta -import ( - "github.com/segmentio/parquet-go/encoding/plain" -) - -func decodeLengthByteArray(dst, src []byte, lengths []int32) ([]byte, error) { +func encodeByteArrayLengths(lengths []int32, offsets []uint32) { for i := range lengths { - n := int(lengths[i]) + lengths[i] = int32(offsets[i+1] - offsets[i]) + } +} + +func decodeByteArrayLengths(offsets []uint32, lengths []int32) (uint32, int32) { + lastOffset := uint32(0) + + for i, n := range lengths { if n < 0 { - return dst, errInvalidNegativeValueLength(n) - } - if n > len(src) { - return dst, errValueLengthOutOfBounds(n, len(src)) + return lastOffset, n } - dst = plain.AppendByteArray(dst, src[:n]) - src = src[n:] + offsets[i] = lastOffset + lastOffset += uint32(n) } - return dst, nil + + offsets[len(lengths)] = lastOffset + return lastOffset, 0 } diff --git a/encoding/delta/length_byte_array_test.go b/encoding/delta/length_byte_array_test.go index 32f7755f..81a60913 100644 --- a/encoding/delta/length_byte_array_test.go +++ b/encoding/delta/length_byte_array_test.go @@ -1,73 +1,34 @@ package delta -import ( - "fmt" - "math/rand" - "testing" +import "testing" - "github.com/segmentio/parquet-go/encoding/plain" -) +func TestDecodeByteArrayLengths(t *testing.T) { + lengths := make([]int32, 999) + offsets := make([]uint32, len(lengths)+1) -func TestDecodeLengthByteArray(t *testing.T) { - const characters = "1234567890qwertyuiopasdfghjklzxcvbnm" - const numValues = 1000 - - src := []byte{} - dst := []byte{} - lengths := []int32{} - - for i := 0; i < numValues; i++ { - n := i % len(characters) - src = append(src, characters[:n]...) - lengths = append(lengths, int32(n)) + totalLength := uint32(0) + for i := range lengths { + lengths[i] = int32(i) + totalLength += uint32(i) } - dst, err := decodeLengthByteArray(dst, src, lengths) - if err != nil { - t.Fatal(err) + lastOffset, invalidLength := decodeByteArrayLengths(offsets, lengths) + if invalidLength != 0 { + t.Fatal("wrong invalid length:", invalidLength) + } + if lastOffset != totalLength { + t.Fatalf("wrong last offset: want=%d got=%d", lastOffset, totalLength) } - index := 0 - err = plain.RangeByteArray(dst, func(got []byte) error { - want := characters[:index%len(characters)] - - if want != string(got) { - return fmt.Errorf("wrong value at index %d: want=%q got=%q", index, want, got) + expectOffset := uint32(0) + for i, offset := range offsets[:len(lengths)] { + if offset != expectOffset { + t.Fatalf("wrong offset at index %d: want=%d got=%d", i, expectOffset, offset) } - - index++ - return nil - }) - if err != nil { - t.Fatal(err) + expectOffset += uint32(lengths[i]) } -} - -func BenchmarkDecodeLengthByteArray(b *testing.B) { - const padding = 64 - - for _, maxLen := range []int{0, 10, 20, 100, 1000} { - b.Run(fmt.Sprintf("maxLen=%d", maxLen), func(b *testing.B) { - lengths := make([]int32, 1000) - totalLength := 0 - prng := rand.New(rand.NewSource(int64(maxLen))) - - if maxLen > 0 { - for i := range lengths { - lengths[i] = prng.Int31n(int32(maxLen)) + 1 - totalLength += int(lengths[i]) - } - } - - size := plain.ByteArrayLengthSize*len(lengths) + totalLength - dst := make([]byte, size+padding) - src := make([]byte, totalLength) - b.SetBytes(int64(size)) - b.ResetTimer() - for i := 0; i < b.N; i++ { - dst, _ = decodeLengthByteArray(dst[:0], src, lengths) - } - }) + if offsets[len(lengths)] != lastOffset { + t.Fatalf("wrong last offset: want=%d got=%d", lastOffset, offsets[len(lengths)]) } } diff --git a/encoding/encoding.go b/encoding/encoding.go index 10329971..03bb2520 100644 --- a/encoding/encoding.go +++ b/encoding/encoding.go @@ -5,6 +5,7 @@ package encoding import ( "math" + "github.com/segmentio/parquet-go/deprecated" "github.com/segmentio/parquet-go/format" ) @@ -27,29 +28,30 @@ type Encoding interface { // destination buffer, potentially reallocating it if it was too short to // contain the output. // - // The source are expected to be encoded using the PLAIN encoding, and - // therefore the methods act as conversions into the target encoding. - EncodeLevels(dst, src []byte) ([]byte, error) - EncodeBoolean(dst, src []byte) ([]byte, error) - EncodeInt32(dst, src []byte) ([]byte, error) - EncodeInt64(dst, src []byte) ([]byte, error) - EncodeInt96(dst, src []byte) ([]byte, error) - EncodeFloat(dst, src []byte) ([]byte, error) - EncodeDouble(dst, src []byte) ([]byte, error) - EncodeByteArray(dst, src []byte) ([]byte, error) - EncodeFixedLenByteArray(dst, src []byte, size int) ([]byte, error) + // The methods panic if the type of src values differ from the type of + // values being encoded. + EncodeLevels(dst []byte, src []uint8) ([]byte, error) + EncodeBoolean(dst []byte, src []byte) ([]byte, error) + EncodeInt32(dst []byte, src []int32) ([]byte, error) + EncodeInt64(dst []byte, src []int64) ([]byte, error) + EncodeInt96(dst []byte, src []deprecated.Int96) ([]byte, error) + EncodeFloat(dst []byte, src []float32) ([]byte, error) + EncodeDouble(dst []byte, src []float64) ([]byte, error) + EncodeByteArray(dst []byte, src []byte, offsets []uint32) ([]byte, error) + EncodeFixedLenByteArray(dst []byte, src []byte, size int) ([]byte, error) // Decode methods deserialize from the source buffer into the destination // slice, potentially growing it if it was too short to contain the result. // - // Values are written in the destination buffer in the PLAIN encoding. - DecodeLevels(dst, src []byte) ([]byte, error) - DecodeBoolean(dst, src []byte) ([]byte, error) - DecodeInt32(dst, src []byte) ([]byte, error) - DecodeInt64(dst, src []byte) ([]byte, error) - DecodeInt96(dst, src []byte) ([]byte, error) - DecodeFloat(dst, src []byte) ([]byte, error) - DecodeDouble(dst, src []byte) ([]byte, error) - DecodeByteArray(dst, src []byte) ([]byte, error) - DecodeFixedLenByteArray(dst, src []byte, size int) ([]byte, error) + // The methods panic if the type of dst values differ from the type of + // values being decoded. + DecodeLevels(dst []uint8, src []byte) ([]uint8, error) + DecodeBoolean(dst []byte, src []byte) ([]byte, error) + DecodeInt32(dst []int32, src []byte) ([]int32, error) + DecodeInt64(dst []int64, src []byte) ([]int64, error) + DecodeInt96(dst []deprecated.Int96, src []byte) ([]deprecated.Int96, error) + DecodeFloat(dst []float32, src []byte) ([]float32, error) + DecodeDouble(dst []float64, src []byte) ([]float64, error) + DecodeByteArray(dst []byte, src []byte, offsets []uint32) ([]byte, []uint32, error) + DecodeFixedLenByteArray(dst []byte, src []byte, size int) ([]byte, error) } diff --git a/encoding/encoding_test.go b/encoding/encoding_test.go index 8c5425d7..05dd9d3f 100644 --- a/encoding/encoding_test.go +++ b/encoding/encoding_test.go @@ -2,7 +2,6 @@ package encoding_test import ( "bytes" - "encoding/binary" "io" "math" "math/bits" @@ -297,7 +296,7 @@ func testBooleanEncoding(t *testing.T, e encoding.Encoding) { assertNoError(t, err) values, err = e.DecodeBoolean(values, buffer) assertNoError(t, err) - assertBytesEqual(t, input, values) + assertEqualBytes(t, input, values) }) } } @@ -307,16 +306,16 @@ func testLevelsEncoding(t *testing.T, e encoding.Encoding) { buffer := []byte{} values := []byte{} - for _, test := range levelsTests { - setBitWidth(e, maxLenInt8(unsafecast.BytesToInt8(test))) + for _, input := range levelsTests { + setBitWidth(e, maxLenInt8(unsafecast.BytesToInt8(input))) t.Run("", func(t *testing.T) { var err error - buffer, err = e.EncodeLevels(buffer, test) + buffer, err = e.EncodeLevels(buffer, input) assertNoError(t, err) values, err = e.DecodeLevels(values, buffer) assertNoError(t, err) - assertBytesEqual(t, test, values[:len(test)]) + assertEqualBytes(t, input, values[:len(input)]) }) } } @@ -324,19 +323,18 @@ func testLevelsEncoding(t *testing.T, e encoding.Encoding) { func testInt32Encoding(t *testing.T, e encoding.Encoding) { testCanEncodeInt32(t, e) buffer := []byte{} - values := []byte{} + values := []int32{} - for _, test := range int32Tests { - setBitWidth(e, maxLenInt32(test)) + for _, input := range int32Tests { + setBitWidth(e, maxLenInt32(input)) t.Run("", func(t *testing.T) { var err error - var input = unsafecast.Int32ToBytes(test) buffer, err = e.EncodeInt32(buffer, input) assertNoError(t, err) values, err = e.DecodeInt32(values, buffer) assertNoError(t, err) - assertBytesEqual(t, input, values) + assertEqualInt32(t, input, values) }) } } @@ -344,19 +342,18 @@ func testInt32Encoding(t *testing.T, e encoding.Encoding) { func testInt64Encoding(t *testing.T, e encoding.Encoding) { testCanEncodeInt64(t, e) buffer := []byte{} - values := []byte{} + values := []int64{} - for _, test := range int64Tests { - setBitWidth(e, maxLenInt64(test)) + for _, input := range int64Tests { + setBitWidth(e, maxLenInt64(input)) t.Run("", func(t *testing.T) { var err error - var input = unsafecast.Int64ToBytes(test) buffer, err = e.EncodeInt64(buffer, input) assertNoError(t, err) values, err = e.DecodeInt64(values, buffer) assertNoError(t, err) - assertBytesEqual(t, input, values) + assertEqualInt64(t, input, values) }) } } @@ -364,17 +361,16 @@ func testInt64Encoding(t *testing.T, e encoding.Encoding) { func testInt96Encoding(t *testing.T, e encoding.Encoding) { testCanEncodeInt96(t, e) buffer := []byte{} - values := []byte{} + values := []deprecated.Int96{} - for _, test := range int96Tests { + for _, input := range int96Tests { t.Run("", func(t *testing.T) { var err error - var input = deprecated.Int96ToBytes(test) buffer, err = e.EncodeInt96(buffer, input) assertNoError(t, err) values, err = e.DecodeInt96(values, buffer) assertNoError(t, err) - assertBytesEqual(t, input, values) + assertEqualInt96(t, input, values) }) } } @@ -382,17 +378,16 @@ func testInt96Encoding(t *testing.T, e encoding.Encoding) { func testFloatEncoding(t *testing.T, e encoding.Encoding) { testCanEncodeFloat(t, e) buffer := []byte{} - values := []byte{} + values := []float32{} - for _, test := range floatTests { + for _, input := range floatTests { t.Run("", func(t *testing.T) { var err error - var input = unsafecast.Float32ToBytes(test) buffer, err = e.EncodeFloat(buffer, input) assertNoError(t, err) values, err = e.DecodeFloat(values, buffer) assertNoError(t, err) - assertBytesEqual(t, input, values) + assertEqualFloat32(t, input, values) }) } } @@ -400,41 +395,46 @@ func testFloatEncoding(t *testing.T, e encoding.Encoding) { func testDoubleEncoding(t *testing.T, e encoding.Encoding) { testCanEncodeDouble(t, e) buffer := []byte{} - values := []byte{} + values := []float64{} - for _, test := range doubleTests { + for _, input := range doubleTests { t.Run("", func(t *testing.T) { var err error - var input = unsafecast.Float64ToBytes(test) buffer, err = e.EncodeDouble(buffer, input) assertNoError(t, err) values, err = e.DecodeDouble(values, buffer) assertNoError(t, err) - assertBytesEqual(t, input, values) + assertEqualFloat64(t, input, values) }) } } func testByteArrayEncoding(t *testing.T, e encoding.Encoding) { testCanEncodeByteArray(t, e) + input := []byte{} buffer := []byte{} values := []byte{} - byteArrays := []byte{} + offsets := []uint32{} for _, test := range byteArrayTests { - byteArrays = byteArrays[:0] + offsets, input = offsets[:0], input[:0] + lastOffset := uint32(0) for _, value := range test { - byteArrays = plain.AppendByteArray(byteArrays, value) + offsets = append(offsets, lastOffset) + input = append(input, value...) + lastOffset += uint32(len(value)) } + offsets = append(offsets, lastOffset) + t.Run("", func(t *testing.T) { var err error - buffer, err = e.EncodeByteArray(buffer, byteArrays) + buffer, err = e.EncodeByteArray(buffer, input, offsets) assertNoError(t, err) - values, err = e.DecodeByteArray(values, buffer) + values, _, err = e.DecodeByteArray(values, buffer, offsets) assertNoError(t, err) - assertBytesEqual(t, byteArrays, values) + assertEqualBytes(t, input, values) }) } } @@ -451,7 +451,7 @@ func testFixedLenByteArrayEncoding(t *testing.T, e encoding.Encoding) { assertNoError(t, err) values, err = e.DecodeFixedLenByteArray(values, buffer, test.size) assertNoError(t, err) - assertBytesEqual(t, test.data, values) + assertEqualBytes(t, test.data, values) }) } } @@ -505,13 +505,38 @@ func assertNoError(t *testing.T, err error) { } } -func assertBytesEqual(t *testing.T, want, got []byte) { +func assertEqualBytes(t *testing.T, want, got []byte) { t.Helper() if !bytes.Equal(want, got) { t.Fatalf("values mismatch:\nwant = %q\ngot = %q", want, got) } } +func assertEqualInt32(t *testing.T, want, got []int32) { + t.Helper() + assertEqualBytes(t, unsafecast.Int32ToBytes(want), unsafecast.Int32ToBytes(got)) +} + +func assertEqualInt64(t *testing.T, want, got []int64) { + t.Helper() + assertEqualBytes(t, unsafecast.Int64ToBytes(want), unsafecast.Int64ToBytes(got)) +} + +func assertEqualInt96(t *testing.T, want, got []deprecated.Int96) { + t.Helper() + assertEqualBytes(t, deprecated.Int96ToBytes(want), deprecated.Int96ToBytes(got)) +} + +func assertEqualFloat32(t *testing.T, want, got []float32) { + t.Helper() + assertEqualBytes(t, unsafecast.Float32ToBytes(want), unsafecast.Float32ToBytes(got)) +} + +func assertEqualFloat64(t *testing.T, want, got []float64) { + t.Helper() + assertEqualBytes(t, unsafecast.Float64ToBytes(want), unsafecast.Float64ToBytes(got)) +} + const ( benchmarkNumValues = 10e3 ) @@ -598,9 +623,9 @@ func benchmarkEncodeInt32(b *testing.B, e encoding.Encoding) { testCanEncodeInt32(b, e) buffer := make([]byte, 0) values := generateInt32Values(benchmarkNumValues, newRand()) - setBitWidth(e, maxLenInt32(unsafecast.BytesToInt32(values))) + setBitWidth(e, maxLenInt32(values)) - reportThroughput(b, benchmarkNumValues, len(values), func() { + reportThroughput(b, benchmarkNumValues, 4*len(values), func() { benchmarkZeroAllocsPerRun(b, func() { buffer, _ = e.EncodeInt32(buffer, values) }) @@ -611,9 +636,9 @@ func benchmarkEncodeInt64(b *testing.B, e encoding.Encoding) { testCanEncodeInt64(b, e) buffer := make([]byte, 0) values := generateInt64Values(benchmarkNumValues, newRand()) - setBitWidth(e, maxLenInt64(unsafecast.BytesToInt64(values))) + setBitWidth(e, maxLenInt64(values)) - reportThroughput(b, benchmarkNumValues, len(values), func() { + reportThroughput(b, benchmarkNumValues, 8*len(values), func() { benchmarkZeroAllocsPerRun(b, func() { buffer, _ = e.EncodeInt64(buffer, values) }) @@ -625,7 +650,7 @@ func benchmarkEncodeFloat(b *testing.B, e encoding.Encoding) { buffer := make([]byte, 0) values := generateFloatValues(benchmarkNumValues, newRand()) - reportThroughput(b, benchmarkNumValues, len(values), func() { + reportThroughput(b, benchmarkNumValues, 4*len(values), func() { benchmarkZeroAllocsPerRun(b, func() { buffer, _ = e.EncodeFloat(buffer, values) }) @@ -637,7 +662,7 @@ func benchmarkEncodeDouble(b *testing.B, e encoding.Encoding) { buffer := make([]byte, 0) values := generateDoubleValues(benchmarkNumValues, newRand()) - reportThroughput(b, benchmarkNumValues, len(values), func() { + reportThroughput(b, benchmarkNumValues, 8*len(values), func() { benchmarkZeroAllocsPerRun(b, func() { buffer, _ = e.EncodeDouble(buffer, values) }) @@ -647,11 +672,12 @@ func benchmarkEncodeDouble(b *testing.B, e encoding.Encoding) { func benchmarkEncodeByteArray(b *testing.B, e encoding.Encoding) { testCanEncodeByteArray(b, e) buffer := make([]byte, 0) - values := generateByteArrayValues(benchmarkNumValues, newRand()) + values, offsets := generateByteArrayValues(benchmarkNumValues, newRand()) - reportThroughput(b, benchmarkNumValues, len(values), func() { + numBytes := len(values) + 4*len(offsets) + reportThroughput(b, benchmarkNumValues, numBytes, func() { benchmarkZeroAllocsPerRun(b, func() { - buffer, _ = e.EncodeByteArray(buffer, values) + buffer, _ = e.EncodeByteArray(buffer, values, offsets) }) }) } @@ -720,13 +746,12 @@ func benchmarkDecode(b *testing.B, e encoding.Encoding) { func benchmarkDecodeBoolean(b *testing.B, e encoding.Encoding) { testCanEncodeBoolean(b, e) values := generateBooleanValues(benchmarkNumValues, newRand()) - output := make([]byte, 0) setBitWidth(e, 1) buffer, _ := e.EncodeBoolean(nil, values) reportThroughput(b, benchmarkNumValues, len(values), func() { benchmarkZeroAllocsPerRun(b, func() { - output, _ = e.DecodeBoolean(output, buffer) + values, _ = e.DecodeBoolean(values, buffer) }) }) } @@ -734,13 +759,12 @@ func benchmarkDecodeBoolean(b *testing.B, e encoding.Encoding) { func benchmarkDecodeLevels(b *testing.B, e encoding.Encoding) { testCanEncodeLevels(b, e) values := generateLevelValues(benchmarkNumValues, newRand()) - output := make([]byte, 0) setBitWidth(e, maxLenInt8(unsafecast.BytesToInt8(values))) buffer, _ := e.EncodeLevels(nil, values) reportThroughput(b, benchmarkNumValues, len(values), func() { benchmarkZeroAllocsPerRun(b, func() { - output, _ = e.DecodeLevels(output, buffer) + values, _ = e.DecodeLevels(values, buffer) }) }) } @@ -748,13 +772,12 @@ func benchmarkDecodeLevels(b *testing.B, e encoding.Encoding) { func benchmarkDecodeInt32(b *testing.B, e encoding.Encoding) { testCanEncodeInt32(b, e) values := generateInt32Values(benchmarkNumValues, newRand()) - output := make([]byte, 0) - setBitWidth(e, maxLenInt32(unsafecast.BytesToInt32(values))) + setBitWidth(e, maxLenInt32(values)) buffer, _ := e.EncodeInt32(nil, values) - reportThroughput(b, benchmarkNumValues, len(values), func() { + reportThroughput(b, benchmarkNumValues, 4*len(values), func() { benchmarkZeroAllocsPerRun(b, func() { - output, _ = e.DecodeInt32(output, buffer) + values, _ = e.DecodeInt32(values, buffer) }) }) } @@ -762,13 +785,12 @@ func benchmarkDecodeInt32(b *testing.B, e encoding.Encoding) { func benchmarkDecodeInt64(b *testing.B, e encoding.Encoding) { testCanEncodeInt64(b, e) values := generateInt64Values(benchmarkNumValues, newRand()) - output := make([]byte, 0) - setBitWidth(e, maxLenInt64(unsafecast.BytesToInt64(values))) + setBitWidth(e, maxLenInt64(values)) buffer, _ := e.EncodeInt64(nil, values) - reportThroughput(b, benchmarkNumValues, len(values), func() { + reportThroughput(b, benchmarkNumValues, 8*len(values), func() { benchmarkZeroAllocsPerRun(b, func() { - output, _ = e.DecodeInt64(output, buffer) + values, _ = e.DecodeInt64(values, buffer) }) }) } @@ -776,12 +798,11 @@ func benchmarkDecodeInt64(b *testing.B, e encoding.Encoding) { func benchmarkDecodeFloat(b *testing.B, e encoding.Encoding) { testCanEncodeFloat(b, e) values := generateFloatValues(benchmarkNumValues, newRand()) - output := make([]byte, 0) buffer, _ := e.EncodeFloat(nil, values) - reportThroughput(b, benchmarkNumValues, len(values), func() { + reportThroughput(b, benchmarkNumValues, 4*len(values), func() { benchmarkZeroAllocsPerRun(b, func() { - output, _ = e.DecodeFloat(output, buffer) + values, _ = e.DecodeFloat(values, buffer) }) }) } @@ -789,25 +810,24 @@ func benchmarkDecodeFloat(b *testing.B, e encoding.Encoding) { func benchmarkDecodeDouble(b *testing.B, e encoding.Encoding) { testCanEncodeDouble(b, e) values := generateDoubleValues(benchmarkNumValues, newRand()) - output := make([]byte, 0) buffer, _ := e.EncodeDouble(nil, values) - reportThroughput(b, benchmarkNumValues, len(values), func() { + reportThroughput(b, benchmarkNumValues, 8*len(values), func() { benchmarkZeroAllocsPerRun(b, func() { - output, _ = e.DecodeDouble(output, buffer) + values, _ = e.DecodeDouble(values, buffer) }) }) } func benchmarkDecodeByteArray(b *testing.B, e encoding.Encoding) { testCanEncodeByteArray(b, e) - values := generateByteArrayValues(benchmarkNumValues, newRand()) - output := make([]byte, 0) - buffer, _ := e.EncodeByteArray(nil, values) + values, offsets := generateByteArrayValues(benchmarkNumValues, newRand()) + buffer, _ := e.EncodeByteArray(nil, values, offsets) - reportThroughput(b, benchmarkNumValues, len(values), func() { + numBytes := len(values) + 4*len(offsets) + reportThroughput(b, benchmarkNumValues, numBytes, func() { benchmarkZeroAllocsPerRun(b, func() { - output, _ = e.DecodeByteArray(output, buffer) + values, offsets, _ = e.DecodeByteArray(values, buffer, offsets) }) }) } @@ -816,12 +836,11 @@ func benchmarkDecodeFixedLenByteArray(b *testing.B, e encoding.Encoding) { testCanEncodeFixedLenByteArray(b, e) const size = 16 values := generateFixedLenByteArrayValues(benchmarkNumValues, newRand(), size) - output := make([]byte, 0) buffer, _ := e.EncodeFixedLenByteArray(nil, values, size) reportThroughput(b, benchmarkNumValues, len(values), func() { benchmarkZeroAllocsPerRun(b, func() { - output, _ = e.DecodeFixedLenByteArray(output, buffer, size) + values, _ = e.DecodeFixedLenByteArray(values, buffer, size) }) }) } @@ -840,66 +859,67 @@ func reportThroughput(b *testing.B, numValues, numBytes int, do func()) { b.ReportMetric(float64(b.N*numValues)/seconds, "value/s") } +func generateLevelValues(n int, r *rand.Rand) []uint8 { + values := make([]uint8, n) + for i := range values { + values[i] = uint8(r.Intn(6)) + } + return values +} + func generateBooleanValues(n int, r *rand.Rand) []byte { values := make([]byte, n/8+1) io.ReadFull(r, values) return values } -func generateLevelValues(n int, r *rand.Rand) []byte { - values := make([]byte, n) +func generateInt32Values(n int, r *rand.Rand) []int32 { + values := make([]int32, n) for i := range values { - values[i] = byte(r.Intn(6)) - } - return values -} - -func generateInt32Values(n int, r *rand.Rand) []byte { - values := make([]byte, 4*n) - for i := 0; i < n; i++ { - binary.LittleEndian.PutUint32(values[i*4:], uint32(r.Int31n(100))) + values[i] = r.Int31n(100) } return values } -func generateInt64Values(n int, r *rand.Rand) []byte { - values := make([]byte, 8*n) - for i := 0; i < n; i++ { - binary.LittleEndian.PutUint64(values[i*8:], uint64(r.Int63n(100))) +func generateInt64Values(n int, r *rand.Rand) []int64 { + values := make([]int64, n) + for i := range values { + values[i] = r.Int63n(100) } return values } -func generateFloatValues(n int, r *rand.Rand) []byte { - values := make([]byte, 4*n) - for i := 0; i < n; i++ { - binary.LittleEndian.PutUint32(values[i*4:], math.Float32bits(r.Float32())) +func generateFloatValues(n int, r *rand.Rand) []float32 { + values := make([]float32, n) + for i := range values { + values[i] = r.Float32() } return values } -func generateDoubleValues(n int, r *rand.Rand) []byte { - values := make([]byte, 8*n) - for i := 0; i < n; i++ { - binary.LittleEndian.PutUint64(values[i*8:], math.Float64bits(r.Float64())) +func generateDoubleValues(n int, r *rand.Rand) []float64 { + values := make([]float64, n) + for i := range values { + values[i] = r.Float64() } return values } -func generateByteArrayValues(n int, r *rand.Rand) []byte { +func generateByteArrayValues(n int, r *rand.Rand) ([]byte, []uint32) { const maxLen = 21 - values := make([]byte, plain.ByteArrayLengthSize*n+n*maxLen) + offsets := make([]uint32, n+1) + values := make([]byte, n*maxLen) length := 0 for i := 0; i < n; i++ { k := r.Intn(maxLen) + 1 - plain.PutByteArrayLength(values[length:], k) - length += plain.ByteArrayLengthSize io.ReadFull(r, values[length:length+k]) + offsets[i] = uint32(length) length += k } - return values[:length] + offsets[n] = uint32(length) + return values[:length], offsets } func generateFixedLenByteArrayValues(n int, r *rand.Rand, size int) []byte { diff --git a/encoding/fuzz/fuzz.go b/encoding/fuzz/fuzz.go index b3b1aa54..a6d5cb96 100644 --- a/encoding/fuzz/fuzz.go +++ b/encoding/fuzz/fuzz.go @@ -5,19 +5,19 @@ package fuzz import ( - "bytes" "math/rand" "testing" + "unsafe" "github.com/segmentio/parquet-go/encoding" - "github.com/segmentio/parquet-go/encoding/plain" + "github.com/segmentio/parquet-go/internal/unsafecast" ) func EncodeBoolean(f *testing.F, e encoding.Encoding) { encode(f, e, encoding.Encoding.EncodeBoolean, encoding.Encoding.DecodeBoolean, - generatePlainBooleanList, + generate[byte], ) } @@ -25,7 +25,7 @@ func EncodeLevels(f *testing.F, e encoding.Encoding) { encode(f, e, encoding.Encoding.EncodeLevels, encoding.Encoding.DecodeLevels, - generatePlainValueList(1), + generate[byte], ) } @@ -33,7 +33,7 @@ func EncodeInt32(f *testing.F, e encoding.Encoding) { encode(f, e, encoding.Encoding.EncodeInt32, encoding.Encoding.DecodeInt32, - generatePlainValueList(4), + generate[int32], ) } @@ -41,7 +41,7 @@ func EncodeInt64(f *testing.F, e encoding.Encoding) { encode(f, e, encoding.Encoding.EncodeInt64, encoding.Encoding.DecodeInt64, - generatePlainValueList(8), + generate[int64], ) } @@ -49,7 +49,7 @@ func EncodeFloat(f *testing.F, e encoding.Encoding) { encode(f, e, encoding.Encoding.EncodeFloat, encoding.Encoding.DecodeFloat, - generatePlainValueList(4), + generate[float32], ) } @@ -57,27 +57,80 @@ func EncodeDouble(f *testing.F, e encoding.Encoding) { encode(f, e, encoding.Encoding.EncodeDouble, encoding.Encoding.DecodeDouble, - generatePlainValueList(8), + generate[float64], ) } func EncodeByteArray(f *testing.F, e encoding.Encoding) { encode(f, e, - encoding.Encoding.EncodeByteArray, - encoding.Encoding.DecodeByteArray, - generatePlainByteArrayList, + func(enc encoding.Encoding, dst []byte, src []string) ([]byte, error) { + size := 0 + for _, s := range src { + size += len(s) + } + + offsets := make([]uint32, 0, len(src)+1) + values := make([]byte, 0, size) + + for _, s := range src { + offsets = append(offsets, uint32(len(values))) + values = append(values, s...) + } + + offsets = append(offsets, uint32(len(values))) + return enc.EncodeByteArray(dst, values, offsets) + }, + + func(enc encoding.Encoding, dst []string, src []byte) ([]string, error) { + dst = dst[:0] + + values, offsets, err := enc.DecodeByteArray(nil, src, nil) + if err != nil { + return dst, err + } + + if len(offsets) > 0 { + baseOffset := offsets[0] + + for _, endOffset := range offsets[1:] { + dst = append(dst, unsafecast.BytesToString(values[baseOffset:endOffset])) + baseOffset = endOffset + } + } + + return dst, nil + }, + + func(dst []string, src []byte, prng *rand.Rand) []string { + limit := len(src)/10 + 1 + + for i := 0; i < len(src); { + n := prng.Intn(limit) + 1 + r := len(src) - i + if n > r { + n = r + } + dst = append(dst, unsafecast.BytesToString(src[i:i+n])) + i += n + } + + return dst + }, ) } -type encodingFunc func(enc encoding.Encoding, dst, src []byte) ([]byte, error) +type encodingFunc[T comparable] func(encoding.Encoding, []byte, []T) ([]byte, error) -type generateFunc func(dst, src []byte, prng *rand.Rand) []byte +type decodingFunc[T comparable] func(encoding.Encoding, []T, []byte) ([]T, error) -func encode(f *testing.F, e encoding.Encoding, encode encodingFunc, decode encodingFunc, generate generateFunc) { +type generateFunc[T comparable] func(dst []T, src []byte, prng *rand.Rand) []T + +func encode[T comparable](f *testing.F, e encoding.Encoding, encode encodingFunc[T], decode decodingFunc[T], generate generateFunc[T]) { const bufferSize = 64 * 1024 + var zero T var err error - var buf = make([]byte, bufferSize) - var src = make([]byte, bufferSize) + var buf = make([]T, bufferSize/unsafe.Sizeof(zero)) + var src = make([]T, bufferSize/unsafe.Sizeof(zero)) var dst = make([]byte, bufferSize) var prng = rand.New(rand.NewSource(0)) @@ -97,39 +150,25 @@ func encode(f *testing.F, e encoding.Encoding, encode encodingFunc, decode encod return } - if !bytes.Equal(buf, src) { + if !equal(buf, src) { t.Error("decoded output does not match the original input") return } - - // Likely invalid inputs, look for panics. - buf, _ = decode(e, buf, input) }) } -func generatePlainBooleanList(dst, src []byte, _ *rand.Rand) []byte { - return append(dst, src...) -} - -func generatePlainByteArrayList(dst, src []byte, prng *rand.Rand) []byte { - limit := len(src)/10 + 1 - - for i := 0; i < len(src); { - n := prng.Intn(limit) + 1 - r := len(src) - i - if n > r { - n = r +func equal[T comparable](a, b []T) bool { + if len(a) != len(b) { + return false + } + for i := range a { + if a[i] != b[i] { + return false } - dst = plain.AppendByteArray(dst, src[i:i+n]) - i += n } - - return dst + return true } -func generatePlainValueList(size int) func(dst, src []byte, _ *rand.Rand) []byte { - return func(dst, src []byte, _ *rand.Rand) []byte { - n := (len(src) / size) * size - return append(dst, src[:n]...) - } +func generate[T comparable](dst []T, src []byte, prng *rand.Rand) []T { + return append(dst[:0], unsafecast.Slice[T](src)...) } diff --git a/encoding/notsupported.go b/encoding/notsupported.go index 284dc3f9..07712e0e 100644 --- a/encoding/notsupported.go +++ b/encoding/notsupported.go @@ -4,6 +4,7 @@ import ( "errors" "fmt" + "github.com/segmentio/parquet-go/deprecated" "github.com/segmentio/parquet-go/format" ) @@ -53,18 +54,18 @@ func errInvalidInputSize(e Encoding, op, typ string, size int) error { return Errorf(e, "cannot %s %s from input of size %d: %w", op, typ, size, ErrInvalidArgument) } -// CanEncodeBoolean reports whether e can encode BOOLEAN values. -func CanEncodeBoolean(e Encoding) bool { - _, err := e.EncodeBoolean(nil, nil) - return !errors.Is(err, ErrNotSupported) -} - // CanEncodeInt8 reports whether e can encode LEVELS values. func CanEncodeLevels(e Encoding) bool { _, err := e.EncodeLevels(nil, nil) return !errors.Is(err, ErrNotSupported) } +// CanEncodeBoolean reports whether e can encode BOOLEAN values. +func CanEncodeBoolean(e Encoding) bool { + _, err := e.EncodeBoolean(nil, nil) + return !errors.Is(err, ErrNotSupported) +} + // CanEncodeInt32 reports whether e can encode INT32 values. func CanEncodeInt32(e Encoding) bool { _, err := e.EncodeInt32(nil, nil) @@ -97,7 +98,7 @@ func CanEncodeDouble(e Encoding) bool { // CanEncodeByteArray reports whether e can encode BYTE_ARRAY values. func CanEncodeByteArray(e Encoding) bool { - _, err := e.EncodeByteArray(nil, nil) + _, err := e.EncodeByteArray(nil, nil, zeroOffsets[:]) return !errors.Is(err, ErrNotSupported) } @@ -108,6 +109,8 @@ func CanEncodeFixedLenByteArray(e Encoding) bool { return !errors.Is(err, ErrNotSupported) } +var zeroOffsets [1]uint32 + // NotSupported is a type satisfying the Encoding interface which does not // support encoding nor decoding any value types. type NotSupported struct { @@ -121,76 +124,76 @@ func (NotSupported) Encoding() format.Encoding { return -1 } -func (NotSupported) EncodeLevels(dst, src []byte) ([]byte, error) { +func (NotSupported) EncodeLevels(dst []byte, src []uint8) ([]byte, error) { return dst[:0], errNotSupported("LEVELS") } -func (NotSupported) EncodeBoolean(dst, src []byte) ([]byte, error) { +func (NotSupported) EncodeBoolean(dst []byte, src []byte) ([]byte, error) { return dst[:0], errNotSupported("BOOLEAN") } -func (NotSupported) EncodeInt32(dst, src []byte) ([]byte, error) { +func (NotSupported) EncodeInt32(dst []byte, src []int32) ([]byte, error) { return dst[:0], errNotSupported("INT32") } -func (NotSupported) EncodeInt64(dst, src []byte) ([]byte, error) { +func (NotSupported) EncodeInt64(dst []byte, src []int64) ([]byte, error) { return dst[:0], errNotSupported("INT64") } -func (NotSupported) EncodeInt96(dst, src []byte) ([]byte, error) { +func (NotSupported) EncodeInt96(dst []byte, src []deprecated.Int96) ([]byte, error) { return dst[:0], errNotSupported("INT96") } -func (NotSupported) EncodeFloat(dst, src []byte) ([]byte, error) { +func (NotSupported) EncodeFloat(dst []byte, src []float32) ([]byte, error) { return dst[:0], errNotSupported("FLOAT") } -func (NotSupported) EncodeDouble(dst, src []byte) ([]byte, error) { +func (NotSupported) EncodeDouble(dst []byte, src []float64) ([]byte, error) { return dst[:0], errNotSupported("DOUBLE") } -func (NotSupported) EncodeByteArray(dst, src []byte) ([]byte, error) { +func (NotSupported) EncodeByteArray(dst []byte, src []byte, offsets []uint32) ([]byte, error) { return dst[:0], errNotSupported("BYTE_ARRAY") } -func (NotSupported) EncodeFixedLenByteArray(dst, src []byte, size int) ([]byte, error) { +func (NotSupported) EncodeFixedLenByteArray(dst []byte, src []byte, size int) ([]byte, error) { return dst[:0], errNotSupported("FIXED_LEN_BYTE_ARRAY") } -func (NotSupported) DecodeLevels(dst, src []byte) ([]byte, error) { - return dst[:0], errNotSupported("LEVELS") +func (NotSupported) DecodeLevels(dst []uint8, src []byte) ([]uint8, error) { + return dst, errNotSupported("LEVELS") } -func (NotSupported) DecodeBoolean(dst, src []byte) ([]byte, error) { - return dst[:0], errNotSupported("BOOLEAN") +func (NotSupported) DecodeBoolean(dst []byte, src []byte) ([]byte, error) { + return dst, errNotSupported("BOOLEAN") } -func (NotSupported) DecodeInt32(dst, src []byte) ([]byte, error) { - return dst[:0], errNotSupported("INT32") +func (NotSupported) DecodeInt32(dst []int32, src []byte) ([]int32, error) { + return dst, errNotSupported("INT32") } -func (NotSupported) DecodeInt64(dst, src []byte) ([]byte, error) { - return dst[:0], errNotSupported("INT64") +func (NotSupported) DecodeInt64(dst []int64, src []byte) ([]int64, error) { + return dst, errNotSupported("INT64") } -func (NotSupported) DecodeInt96(dst, src []byte) ([]byte, error) { - return dst[:0], errNotSupported("INT96") +func (NotSupported) DecodeInt96(dst []deprecated.Int96, src []byte) ([]deprecated.Int96, error) { + return dst, errNotSupported("INT96") } -func (NotSupported) DecodeFloat(dst, src []byte) ([]byte, error) { - return dst[:0], errNotSupported("FLOAT") +func (NotSupported) DecodeFloat(dst []float32, src []byte) ([]float32, error) { + return dst, errNotSupported("FLOAT") } -func (NotSupported) DecodeDouble(dst, src []byte) ([]byte, error) { - return dst[:0], errNotSupported("DOUBLE") +func (NotSupported) DecodeDouble(dst []float64, src []byte) ([]float64, error) { + return dst, errNotSupported("DOUBLE") } -func (NotSupported) DecodeByteArray(dst, src []byte) ([]byte, error) { - return dst[:0], errNotSupported("BYTE_ARRAY") +func (NotSupported) DecodeByteArray(dst []byte, src []byte, offsets []uint32) ([]byte, []uint32, error) { + return dst, offsets, errNotSupported("BYTE_ARRAY") } -func (NotSupported) DecodeFixedLenByteArray(dst, src []byte, size int) ([]byte, error) { - return dst[:0], errNotSupported("FIXED_LEN_BYTE_ARRAY") +func (NotSupported) DecodeFixedLenByteArray(dst []byte, src []byte, size int) ([]byte, error) { + return dst, errNotSupported("FIXED_LEN_BYTE_ARRAY") } func errNotSupported(typ string) error { diff --git a/encoding/plain/dictionary.go b/encoding/plain/dictionary.go index 15672143..2c99f92f 100644 --- a/encoding/plain/dictionary.go +++ b/encoding/plain/dictionary.go @@ -18,10 +18,10 @@ func (e *DictionaryEncoding) Encoding() format.Encoding { return format.PlainDictionary } -func (e *DictionaryEncoding) EncodeInt32(dst, src []byte) ([]byte, error) { +func (e *DictionaryEncoding) EncodeInt32(dst []byte, src []int32) ([]byte, error) { return e.plain.EncodeInt32(dst, src) } -func (e *DictionaryEncoding) DecodeInt32(dst, src []byte) ([]byte, error) { +func (e *DictionaryEncoding) DecodeInt32(dst []int32, src []byte) ([]int32, error) { return e.plain.DecodeInt32(dst, src) } diff --git a/encoding/plain/plain.go b/encoding/plain/plain.go index cd4aeac9..7f1da9c5 100644 --- a/encoding/plain/plain.go +++ b/encoding/plain/plain.go @@ -12,6 +12,7 @@ import ( "github.com/segmentio/parquet-go/deprecated" "github.com/segmentio/parquet-go/encoding" "github.com/segmentio/parquet-go/format" + "github.com/segmentio/parquet-go/internal/unsafecast" ) const ( @@ -31,111 +32,117 @@ func (e *Encoding) Encoding() format.Encoding { return format.Plain } -func (e *Encoding) EncodeBoolean(dst, src []byte) ([]byte, error) { +func (e *Encoding) EncodeBoolean(dst []byte, src []byte) ([]byte, error) { return append(dst[:0], src...), nil } -func (e *Encoding) EncodeInt32(dst, src []byte) ([]byte, error) { - if (len(src) % 4) != 0 { - return dst[:0], encoding.ErrEncodeInvalidInputSize(e, "INT32", len(src)) - } - return append(dst[:0], src...), nil +func (e *Encoding) EncodeInt32(dst []byte, src []int32) ([]byte, error) { + return append(dst[:0], unsafecast.Int32ToBytes(src)...), nil } -func (e *Encoding) EncodeInt64(dst, src []byte) ([]byte, error) { - if (len(src) % 8) != 0 { - return dst[:0], encoding.ErrEncodeInvalidInputSize(e, "INT64", len(src)) - } - return append(dst[:0], src...), nil +func (e *Encoding) EncodeInt64(dst []byte, src []int64) ([]byte, error) { + return append(dst[:0], unsafecast.Int64ToBytes(src)...), nil } -func (e *Encoding) EncodeInt96(dst, src []byte) ([]byte, error) { - if (len(src) % 12) != 0 { - return dst[:0], encoding.ErrEncodeInvalidInputSize(e, "INT96", len(src)) - } - return append(dst[:0], src...), nil +func (e *Encoding) EncodeInt96(dst []byte, src []deprecated.Int96) ([]byte, error) { + return append(dst[:0], deprecated.Int96ToBytes(src)...), nil } -func (e *Encoding) EncodeFloat(dst, src []byte) ([]byte, error) { - if (len(src) % 4) != 0 { - return dst[:0], encoding.ErrEncodeInvalidInputSize(e, "FLOAT", len(src)) - } - return append(dst[:0], src...), nil +func (e *Encoding) EncodeFloat(dst []byte, src []float32) ([]byte, error) { + return append(dst[:0], unsafecast.Float32ToBytes(src)...), nil } -func (e *Encoding) EncodeDouble(dst, src []byte) ([]byte, error) { - if (len(src) % 8) != 0 { - return dst[:0], encoding.ErrEncodeInvalidInputSize(e, "DOUBLE", len(src)) - } - return append(dst[:0], src...), nil +func (e *Encoding) EncodeDouble(dst []byte, src []float64) ([]byte, error) { + return append(dst[:0], unsafecast.Float64ToBytes(src)...), nil } -func (e *Encoding) EncodeByteArray(dst []byte, src []byte) ([]byte, error) { - if err := ValidateByteArray(src); err != nil { - return dst[:0], encoding.Error(e, err) +func (e *Encoding) EncodeByteArray(dst []byte, src []byte, offsets []uint32) ([]byte, error) { + dst = dst[:0] + + if len(offsets) > 0 { + baseOffset := offsets[0] + + for _, endOffset := range offsets[1:] { + dst = AppendByteArray(dst, src[baseOffset:endOffset:endOffset]) + baseOffset = endOffset + } } - return append(dst[:0], src...), nil + + return dst, nil } -func (e *Encoding) EncodeFixedLenByteArray(dst, src []byte, size int) ([]byte, error) { +func (e *Encoding) EncodeFixedLenByteArray(dst []byte, src []byte, size int) ([]byte, error) { if size < 0 || size > encoding.MaxFixedLenByteArraySize { return dst[:0], encoding.Error(e, encoding.ErrInvalidArgument) } return append(dst[:0], src...), nil } -func (e *Encoding) DecodeBoolean(dst, src []byte) ([]byte, error) { +func (e *Encoding) DecodeBoolean(dst []byte, src []byte) ([]byte, error) { return append(dst[:0], src...), nil } -func (e *Encoding) DecodeInt32(dst, src []byte) ([]byte, error) { +func (e *Encoding) DecodeInt32(dst []int32, src []byte) ([]int32, error) { if (len(src) % 4) != 0 { - return dst[:0], encoding.ErrDecodeInvalidInputSize(e, "INT32", len(src)) + return dst, encoding.ErrDecodeInvalidInputSize(e, "INT32", len(src)) } - return append(dst[:0], src...), nil + return append(dst[:0], unsafecast.BytesToInt32(src)...), nil } -func (e *Encoding) DecodeInt64(dst, src []byte) ([]byte, error) { +func (e *Encoding) DecodeInt64(dst []int64, src []byte) ([]int64, error) { if (len(src) % 8) != 0 { - return dst[:0], encoding.ErrDecodeInvalidInputSize(e, "INT64", len(src)) + return dst, encoding.ErrDecodeInvalidInputSize(e, "INT64", len(src)) } - return append(dst[:0], src...), nil + return append(dst[:0], unsafecast.BytesToInt64(src)...), nil } -func (e *Encoding) DecodeInt96(dst, src []byte) ([]byte, error) { +func (e *Encoding) DecodeInt96(dst []deprecated.Int96, src []byte) ([]deprecated.Int96, error) { if (len(src) % 12) != 0 { - return dst[:0], encoding.ErrDecodeInvalidInputSize(e, "INT96", len(src)) + return dst, encoding.ErrDecodeInvalidInputSize(e, "INT96", len(src)) } - return append(dst[:0], src...), nil + return append(dst[:0], deprecated.BytesToInt96(src)...), nil } -func (e *Encoding) DecodeFloat(dst, src []byte) ([]byte, error) { +func (e *Encoding) DecodeFloat(dst []float32, src []byte) ([]float32, error) { if (len(src) % 4) != 0 { - return dst[:0], encoding.ErrDecodeInvalidInputSize(e, "FLOAT", len(src)) + return dst, encoding.ErrDecodeInvalidInputSize(e, "FLOAT", len(src)) } - return append(dst[:0], src...), nil + return append(dst[:0], unsafecast.BytesToFloat32(src)...), nil } -func (e *Encoding) DecodeDouble(dst, src []byte) ([]byte, error) { +func (e *Encoding) DecodeDouble(dst []float64, src []byte) ([]float64, error) { if (len(src) % 8) != 0 { - return dst[:0], encoding.ErrDecodeInvalidInputSize(e, "DOUBLE", len(src)) + return dst, encoding.ErrDecodeInvalidInputSize(e, "DOUBLE", len(src)) } - return append(dst[:0], src...), nil + return append(dst[:0], unsafecast.BytesToFloat64(src)...), nil } -func (e *Encoding) DecodeByteArray(dst, src []byte) ([]byte, error) { - if err := ValidateByteArray(src); err != nil { - return dst[:0], encoding.Error(e, err) +func (e *Encoding) DecodeByteArray(dst []byte, src []byte, offsets []uint32) ([]byte, []uint32, error) { + dst, offsets = dst[:0], offsets[:0] + + for i := 0; i < len(src); { + if (len(src) - i) < ByteArrayLengthSize { + return dst, offsets, ErrTooShort(len(src)) + } + n := ByteArrayLength(src[i:]) + if n > (len(src) - ByteArrayLengthSize) { + return dst, offsets, ErrTooShort(len(src)) + } + i += ByteArrayLengthSize + offsets = append(offsets, uint32(len(dst))) + dst = append(dst, src[i:i+n]...) + i += n } - return append(dst[:0], src...), nil + + return dst, append(offsets, uint32(len(dst))), nil } -func (e *Encoding) DecodeFixedLenByteArray(dst, src []byte, size int) ([]byte, error) { +func (e *Encoding) DecodeFixedLenByteArray(dst []byte, src []byte, size int) ([]byte, error) { if size < 0 || size > encoding.MaxFixedLenByteArraySize { - return dst[:0], encoding.Error(e, encoding.ErrInvalidArgument) + return dst, encoding.Error(e, encoding.ErrInvalidArgument) } if (len(src) % size) != 0 { - return dst[:0], encoding.ErrDecodeInvalidInputSize(e, "FIXED_LEN_BYTE_ARRAY", len(src)) + return dst, encoding.ErrDecodeInvalidInputSize(e, "FIXED_LEN_BYTE_ARRAY", len(src)) } return append(dst[:0], src...), nil } @@ -238,25 +245,6 @@ func PutByteArrayLength(b []byte, n int) { binary.LittleEndian.PutUint32(b, uint32(n)) } -type status int - -const ( - ok status = iota - errTooShort - errTooLarge -) - -func ValidateByteArray(b []byte) error { - switch validateByteArray(b) { - case errTooShort: - return ErrTooShort(len(b)) - case errTooLarge: - return ErrTooLarge(len(b)) - default: // ok - return nil - } -} - func RangeByteArray(b []byte, do func([]byte) error) (err error) { for len(b) > 0 { var v []byte diff --git a/encoding/plain/plain_amd64.go b/encoding/plain/plain_amd64.go deleted file mode 100644 index b537771a..00000000 --- a/encoding/plain/plain_amd64.go +++ /dev/null @@ -1,6 +0,0 @@ -//go:build !purego - -package plain - -//go:noescape -func validateByteArray([]byte) status diff --git a/encoding/plain/plain_amd64.s b/encoding/plain/plain_amd64.s deleted file mode 100644 index 5f29fd73..00000000 --- a/encoding/plain/plain_amd64.s +++ /dev/null @@ -1,41 +0,0 @@ -//go:build !purego - -#include "textflag.h" - -// validateByteArray is implemented in assembly because it is the bottleneck in -// many cases when encoding byte arrays. This version of the function yields -// ~50% better throughput than the code generated by the Go compiler. -// -// func validateByteArray([]byte) status -TEXT ·validateByteArray(SB), NOSPLIT, $0-32 - MOVQ arg_base+0(FP), AX - MOVQ arg_len+8(FP), BX - - CMPQ BX, $0 - JE done - - CMPQ BX, $4 - JB errTooShort - - ADDQ AX, BX // end -loop: - MOVL (AX), CX - - CMPL CX, $0x7FFFFFFF - JA errTooLarge - - LEAQ 4(AX)(CX*1), AX - CMPQ AX, BX - JA errTooShort - JB loop -done: - XORQ AX, AX -return: - MOVQ AX, ret+24(FP) - RET -errTooShort: - MOVQ $1, AX - JMP return -errTooLarge: - MOVQ $2, AX - JMP return diff --git a/encoding/plain/plain_purego.go b/encoding/plain/plain_purego.go deleted file mode 100644 index 5f9a50da..00000000 --- a/encoding/plain/plain_purego.go +++ /dev/null @@ -1,23 +0,0 @@ -//go:build purego || !amd64 - -package plain - -func validateByteArray(b []byte) status { - for i := 0; i < len(b); { - r := len(b) - i - if r < ByteArrayLengthSize { - return errTooShort - } - n := ByteArrayLength(b[i:]) - i += ByteArrayLengthSize - r -= ByteArrayLengthSize - if n > r { - return errTooShort - } - if n > MaxByteArrayLength { - return errTooLarge - } - i += n - } - return ok -} diff --git a/encoding/plain/plain_test.go b/encoding/plain/plain_test.go index 84eb7b31..b6f09cf5 100644 --- a/encoding/plain/plain_test.go +++ b/encoding/plain/plain_test.go @@ -32,27 +32,3 @@ func TestAppendBoolean(t *testing.T) { t.Errorf("%08b\n", values) } } - -func TestValidateByteArray(t *testing.T) { - t.Run("ok", func(t *testing.T) { - var b []byte - b = plain.AppendByteArrayString(b, "Hello") - b = plain.AppendByteArrayString(b, "World") - b = plain.AppendByteArrayString(b, "!") - - if err := plain.ValidateByteArray(b); err != nil { - t.Error(err) - } - }) - - t.Run("errTooShort", func(t *testing.T) { - var b []byte - b = plain.AppendByteArrayString(b, "Hello") - b = plain.AppendByteArrayString(b, "World") - b = plain.AppendByteArrayString(b, "!") - - if plain.ValidateByteArray(b[:len(b)-1]) == nil { - t.Error("expected non-nil error") - } - }) -} diff --git a/encoding/rle/dictionary.go b/encoding/rle/dictionary.go index 3b0dcdb0..6a9523f8 100644 --- a/encoding/rle/dictionary.go +++ b/encoding/rle/dictionary.go @@ -20,23 +20,20 @@ func (e *DictionaryEncoding) Encoding() format.Encoding { return format.RLEDictionary } -func (e *DictionaryEncoding) EncodeInt32(dst, src []byte) ([]byte, error) { - if (len(src) % 4) != 0 { - return dst[:0], encoding.ErrEncodeInvalidInputSize(e, "INT32", len(src)) - } - src32 := unsafecast.BytesToInt32(src) - bitWidth := maxLenInt32(src32) +func (e *DictionaryEncoding) EncodeInt32(dst []byte, src []int32) ([]byte, error) { + bitWidth := maxLenInt32(src) dst = append(dst[:0], byte(bitWidth)) - dst, err := encodeInt32(dst, src32, uint(bitWidth)) + dst, err := encodeInt32(dst, src, uint(bitWidth)) return dst, e.wrap(err) } -func (e *DictionaryEncoding) DecodeInt32(dst, src []byte) ([]byte, error) { +func (e *DictionaryEncoding) DecodeInt32(dst []int32, src []byte) ([]int32, error) { if len(src) == 0 { return dst[:0], nil } - dst, err := decodeInt32(dst[:0], src[1:], uint(src[0])) - return dst, e.wrap(err) + buf := unsafecast.Int32ToBytes(dst) + buf, err := decodeInt32(buf[:0], src[1:], uint(src[0])) + return unsafecast.BytesToInt32(buf), e.wrap(err) } func (e *DictionaryEncoding) wrap(err error) error { diff --git a/encoding/rle/rle.go b/encoding/rle/rle.go index 968c0ef8..2a45704c 100644 --- a/encoding/rle/rle.go +++ b/encoding/rle/rle.go @@ -41,12 +41,12 @@ func (e *Encoding) Encoding() format.Encoding { return format.RLE } -func (e *Encoding) EncodeLevels(dst, src []byte) ([]byte, error) { +func (e *Encoding) EncodeLevels(dst []byte, src []uint8) ([]byte, error) { dst, err := encodeBytes(dst[:0], src, uint(e.BitWidth)) return dst, e.wrap(err) } -func (e *Encoding) EncodeBoolean(dst, src []byte) ([]byte, error) { +func (e *Encoding) EncodeBoolean(dst []byte, src []byte) ([]byte, error) { // In the case of encoding a boolean values, the 4 bytes length of the // output is expected by the parquet format. We add the bytes as placeholder // before appending the encoded data. @@ -56,20 +56,17 @@ func (e *Encoding) EncodeBoolean(dst, src []byte) ([]byte, error) { return dst, e.wrap(err) } -func (e *Encoding) EncodeInt32(dst, src []byte) ([]byte, error) { - if (len(src) % 4) != 0 { - return dst[:0], encoding.ErrEncodeInvalidInputSize(e, "INT32", len(src)) - } - dst, err := encodeInt32(dst[:0], unsafecast.BytesToInt32(src), uint(e.BitWidth)) +func (e *Encoding) EncodeInt32(dst []byte, src []int32) ([]byte, error) { + dst, err := encodeInt32(dst[:0], src, uint(e.BitWidth)) return dst, e.wrap(err) } -func (e *Encoding) DecodeLevels(dst, src []byte) ([]byte, error) { +func (e *Encoding) DecodeLevels(dst []uint8, src []byte) ([]uint8, error) { dst, err := decodeBytes(dst[:0], src, uint(e.BitWidth)) return dst, e.wrap(err) } -func (e *Encoding) DecodeBoolean(dst, src []byte) ([]byte, error) { +func (e *Encoding) DecodeBoolean(dst []byte, src []byte) ([]byte, error) { if len(src) == 4 { return dst[:0], nil } @@ -85,9 +82,10 @@ func (e *Encoding) DecodeBoolean(dst, src []byte) ([]byte, error) { return dst, e.wrap(err) } -func (e *Encoding) DecodeInt32(dst, src []byte) ([]byte, error) { - dst, err := decodeInt32(dst[:0], src, uint(e.BitWidth)) - return dst, e.wrap(err) +func (e *Encoding) DecodeInt32(dst []int32, src []byte) ([]int32, error) { + buf := unsafecast.Int32ToBytes(dst) + buf, err := decodeInt32(buf[:0], src, uint(e.BitWidth)) + return unsafecast.BytesToInt32(buf), e.wrap(err) } func (e *Encoding) wrap(err error) error { diff --git a/encoding/test/test_go18.go b/encoding/test/test_go18.go index 187bef89..fe1458db 100644 --- a/encoding/test/test_go18.go +++ b/encoding/test/test_go18.go @@ -7,7 +7,6 @@ import ( "testing" "github.com/segmentio/parquet-go/encoding" - "github.com/segmentio/parquet-go/internal/unsafecast" ) func EncodeInt32(t *testing.T, enc encoding.Encoding, min, max int, bitWidth uint) { @@ -60,9 +59,11 @@ func EncodeDouble(t *testing.T, enc encoding.Encoding, min, max int) { ) } -type encodingMethod func(encoding.Encoding, []byte, []byte) ([]byte, error) +type encodingFunc[T comparable] func(encoding.Encoding, []byte, []T) ([]byte, error) -func encode[T comparable](t *testing.T, enc encoding.Encoding, min, max int, encode, decode encodingMethod, valueOf func(int) T) { +type decodingFunc[T comparable] func(encoding.Encoding, []T, []byte) ([]T, error) + +func encode[T comparable](t *testing.T, enc encoding.Encoding, min, max int, encode encodingFunc[T], decode decodingFunc[T], valueOf func(int) T) { t.Helper() for k := min; k <= max; k++ { @@ -72,7 +73,7 @@ func encode[T comparable](t *testing.T, enc encoding.Encoding, min, max int, enc src[i] = valueOf(i) } - buf, err := encode(enc, nil, unsafecast.Slice[byte](src)) + buf, err := encode(enc, nil, src) if err != nil { t.Fatalf("encoding %d values: %v", k, err) } @@ -82,7 +83,7 @@ func encode[T comparable](t *testing.T, enc encoding.Encoding, min, max int, enc t.Fatalf("decoding %d values: %v", k, err) } - if err := assertEqual(src, unsafecast.Slice[T](res)); err != nil { + if err := assertEqual(src, res); err != nil { t.Fatalf("testing %d values: %v", k, err) } }) diff --git a/encoding/values.go b/encoding/values.go new file mode 100644 index 00000000..974c89d9 --- /dev/null +++ b/encoding/values.go @@ -0,0 +1,313 @@ +package encoding + +import ( + "fmt" + + "github.com/segmentio/parquet-go/deprecated" + "github.com/segmentio/parquet-go/internal/unsafecast" +) + +type Kind int32 + +const ( + Undefined Kind = iota + Boolean + Int32 + Int64 + Int96 + Float + Double + ByteArray + FixedLenByteArray +) + +func (kind Kind) String() string { + switch kind { + case Boolean: + return "BOOLEAN" + case Int32: + return "INT32" + case Int64: + return "INT64" + case Int96: + return "INT96" + case Float: + return "FLOAT" + case Double: + return "DOUBLE" + case ByteArray: + return "BYTE_ARRAY" + case FixedLenByteArray: + return "FIXED_LEN_BYTE_ARRAY" + default: + return "UNDEFINED" + } +} + +type Values struct { + kind Kind + size int32 + data []byte + offsets []uint32 +} + +func (v *Values) assertKind(kind Kind) { + if kind != v.kind { + panic(fmt.Sprintf("cannot convert values of type %s to type %s", v.kind, kind)) + } +} + +func (v *Values) assertSize(size int) { + if size != int(v.size) { + panic(fmt.Sprintf("cannot convert values of size %d to size %d", v.size, size)) + } +} + +func (v *Values) Size() int64 { + return int64(len(v.data)) +} + +func (v *Values) Kind() Kind { + return v.kind +} + +func (v *Values) Data() (data []byte, offsets []uint32) { + return v.data, v.offsets +} + +func (v *Values) Boolean() []byte { + v.assertKind(Boolean) + return v.data +} + +func (v *Values) Int32() []int32 { + v.assertKind(Int32) + return unsafecast.BytesToInt32(v.data) +} + +func (v *Values) Int64() []int64 { + v.assertKind(Int64) + return unsafecast.BytesToInt64(v.data) +} + +func (v *Values) Int96() []deprecated.Int96 { + v.assertKind(Int96) + return deprecated.BytesToInt96(v.data) +} + +func (v *Values) Float() []float32 { + v.assertKind(Float) + return unsafecast.BytesToFloat32(v.data) +} + +func (v *Values) Double() []float64 { + v.assertKind(Double) + return unsafecast.BytesToFloat64(v.data) +} + +func (v *Values) ByteArray() (data []byte, offsets []uint32) { + v.assertKind(ByteArray) + return v.data, v.offsets +} + +func (v *Values) FixedLenByteArray() (data []byte, size int) { + v.assertKind(FixedLenByteArray) + return v.data, int(v.size) +} + +func (v *Values) Uint32() []uint32 { + v.assertKind(Int32) + return unsafecast.BytesToUint32(v.data) +} + +func (v *Values) Uint64() []uint64 { + v.assertKind(Int64) + return unsafecast.BytesToUint64(v.data) +} + +func (v *Values) Uint128() [][16]byte { + v.assertKind(FixedLenByteArray) + v.assertSize(16) + return unsafecast.BytesToUint128(v.data) +} + +func BooleanValues(values []byte) Values { + return Values{ + kind: Boolean, + data: values, + } +} + +func Int32Values(values []int32) Values { + return Values{ + kind: Int32, + data: unsafecast.Int32ToBytes(values), + } +} + +func Int64Values(values []int64) Values { + return Values{ + kind: Int64, + data: unsafecast.Int64ToBytes(values), + } +} + +func Int96Values(values []deprecated.Int96) Values { + return Values{ + kind: Int96, + data: deprecated.Int96ToBytes(values), + } +} + +func FloatValues(values []float32) Values { + return Values{ + kind: Float, + data: unsafecast.Float32ToBytes(values), + } +} + +func DoubleValues(values []float64) Values { + return Values{ + kind: Double, + data: unsafecast.Float64ToBytes(values), + } +} + +func ByteArrayValues(values []byte, offsets []uint32) Values { + return Values{ + kind: ByteArray, + data: values, + offsets: offsets, + } +} + +func FixedLenByteArrayValues(values []byte, size int) Values { + return Values{ + kind: FixedLenByteArray, + size: int32(size), + data: values, + } +} + +func Uint32Values(values []uint32) Values { + return Int32Values(unsafecast.Uint32ToInt32(values)) +} + +func Uint64Values(values []uint64) Values { + return Int64Values(unsafecast.Uint64ToInt64(values)) +} + +func Uint128Values(values [][16]byte) Values { + return FixedLenByteArrayValues(unsafecast.Uint128ToBytes(values), 16) +} + +func Int32ValuesFromBytes(values []byte) Values { + return Values{ + kind: Int32, + data: values, + } +} + +func Int64ValuesFromBytes(values []byte) Values { + return Values{ + kind: Int64, + data: values, + } +} + +func Int96ValuesFromBytes(values []byte) Values { + return Values{ + kind: Int96, + data: values, + } +} + +func FloatValuesFromBytes(values []byte) Values { + return Values{ + kind: Float, + data: values, + } +} + +func DoubleValuesFromBytes(values []byte) Values { + return Values{ + kind: Double, + data: values, + } +} + +func EncodeBoolean(dst []byte, src Values, enc Encoding) ([]byte, error) { + return enc.EncodeBoolean(dst, src.Boolean()) +} + +func EncodeInt32(dst []byte, src Values, enc Encoding) ([]byte, error) { + return enc.EncodeInt32(dst, src.Int32()) +} + +func EncodeInt64(dst []byte, src Values, enc Encoding) ([]byte, error) { + return enc.EncodeInt64(dst, src.Int64()) +} + +func EncodeInt96(dst []byte, src Values, enc Encoding) ([]byte, error) { + return enc.EncodeInt96(dst, src.Int96()) +} + +func EncodeFloat(dst []byte, src Values, enc Encoding) ([]byte, error) { + return enc.EncodeFloat(dst, src.Float()) +} + +func EncodeDouble(dst []byte, src Values, enc Encoding) ([]byte, error) { + return enc.EncodeDouble(dst, src.Double()) +} + +func EncodeByteArray(dst []byte, src Values, enc Encoding) ([]byte, error) { + values, offsets := src.ByteArray() + return enc.EncodeByteArray(dst, values, offsets) +} + +func EncodeFixedLenByteArray(dst []byte, src Values, enc Encoding) ([]byte, error) { + data, size := src.FixedLenByteArray() + return enc.EncodeFixedLenByteArray(dst, data, size) +} + +func DecodeBoolean(dst Values, src []byte, enc Encoding) (Values, error) { + values, err := enc.DecodeBoolean(dst.Boolean(), src) + return BooleanValues(values), err +} + +func DecodeInt32(dst Values, src []byte, enc Encoding) (Values, error) { + values, err := enc.DecodeInt32(dst.Int32(), src) + return Int32Values(values), err +} + +func DecodeInt64(dst Values, src []byte, enc Encoding) (Values, error) { + values, err := enc.DecodeInt64(dst.Int64(), src) + return Int64Values(values), err +} + +func DecodeInt96(dst Values, src []byte, enc Encoding) (Values, error) { + values, err := enc.DecodeInt96(dst.Int96(), src) + return Int96Values(values), err +} + +func DecodeFloat(dst Values, src []byte, enc Encoding) (Values, error) { + values, err := enc.DecodeFloat(dst.Float(), src) + return FloatValues(values), err +} + +func DecodeDouble(dst Values, src []byte, enc Encoding) (Values, error) { + values, err := enc.DecodeDouble(dst.Double(), src) + return DoubleValues(values), err +} + +func DecodeByteArray(dst Values, src []byte, enc Encoding) (Values, error) { + values, offsets := dst.ByteArray() + values, offsets, err := enc.DecodeByteArray(values, src, offsets) + return ByteArrayValues(values, offsets), err +} + +func DecodeFixedLenByteArray(dst Values, src []byte, enc Encoding) (Values, error) { + data, size := dst.FixedLenByteArray() + values, err := enc.DecodeFixedLenByteArray(data, src, size) + return FixedLenByteArrayValues(values, size), err +} diff --git a/encoding/values_test.go b/encoding/values_test.go new file mode 100644 index 00000000..e581ec21 --- /dev/null +++ b/encoding/values_test.go @@ -0,0 +1,12 @@ +package encoding_test + +import ( + "testing" + "unsafe" + + "github.com/segmentio/parquet-go/encoding" +) + +func TestValuesSize(t *testing.T) { + t.Log(unsafe.Sizeof(encoding.Values{})) +} diff --git a/page.go b/page.go index 44621169..e5862d7b 100644 --- a/page.go +++ b/page.go @@ -2,14 +2,12 @@ package parquet import ( "bytes" - "encoding/binary" "fmt" "io" "github.com/segmentio/parquet-go/deprecated" - "github.com/segmentio/parquet-go/encoding/plain" + "github.com/segmentio/parquet-go/encoding" "github.com/segmentio/parquet-go/internal/bitpack" - "github.com/segmentio/parquet-go/internal/unsafecast" ) // Page values represent sequences of parquet values. From the Parquet @@ -90,14 +88,13 @@ type BufferedPage interface { // Returns the in-memory buffer holding the page values. // - // The buffer has the page values serialized in the PLAIN encoding. - // // The intent is for the returned value to be used as input parameter when // calling the Encode method of the associated Type. // - // The returned slice may be the same across multiple calls to this method, - // applications must treat the content as immutable. - Data() []byte + // The slices referenced by the encoding.Values may be the same across + // multiple calls to this method, applications must treat the content as + // immutable. + Data() encoding.Values } // CompressedPage is an extension of the Page interface implemented by pages @@ -264,7 +261,7 @@ func (page *errorPage) Slice(i, j int64) BufferedPage { return page } func (page *errorPage) Size() int64 { return 1 } func (page *errorPage) RepetitionLevels() []byte { return nil } func (page *errorPage) DefinitionLevels() []byte { return nil } -func (page *errorPage) Data() []byte { return nil } +func (page *errorPage) Data() encoding.Values { return encoding.Values{} } func (page *errorPage) Values() ValueReader { return errorPageValues{page: page} } func (page *errorPage) Buffer() BufferedPage { return page } @@ -313,7 +310,7 @@ func (page *optionalPage) RepetitionLevels() []byte { return nil } func (page *optionalPage) DefinitionLevels() []byte { return page.definitionLevels } -func (page *optionalPage) Data() []byte { return page.base.Data() } +func (page *optionalPage) Data() encoding.Values { return page.base.Data() } func (page *optionalPage) Values() ValueReader { return &optionalPageValues{ @@ -384,7 +381,7 @@ func (page *repeatedPage) RepetitionLevels() []byte { return page.repetitionLeve func (page *repeatedPage) DefinitionLevels() []byte { return page.definitionLevels } -func (page *repeatedPage) Data() []byte { return page.base.Data() } +func (page *repeatedPage) Data() encoding.Values { return page.base.Data() } func (page *repeatedPage) Values() ValueReader { return &repeatedPageValues{ @@ -464,10 +461,10 @@ type booleanPage struct { columnIndex int16 } -func newBooleanPage(typ Type, columnIndex int16, numValues int32, values []byte) *booleanPage { +func newBooleanPage(typ Type, columnIndex int16, numValues int32, values encoding.Values) *booleanPage { return &booleanPage{ typ: typ, - bits: values[:bitpack.ByteCount(uint(numValues))], + bits: values.Boolean()[:bitpack.ByteCount(uint(numValues))], numValues: numValues, columnIndex: ^columnIndex, } @@ -491,7 +488,7 @@ func (page *booleanPage) RepetitionLevels() []byte { return nil } func (page *booleanPage) DefinitionLevels() []byte { return nil } -func (page *booleanPage) Data() []byte { return page.bits } +func (page *booleanPage) Data() encoding.Values { return encoding.BooleanValues(page.bits) } func (page *booleanPage) Values() ValueReader { return &booleanPageValues{page: page} } @@ -589,10 +586,10 @@ type int32Page struct { columnIndex int16 } -func newInt32Page(typ Type, columnIndex int16, numValues int32, values []byte) *int32Page { +func newInt32Page(typ Type, columnIndex int16, numValues int32, values encoding.Values) *int32Page { return &int32Page{ typ: typ, - values: unsafecast.BytesToInt32(values)[:numValues], + values: values.Int32()[:numValues], columnIndex: ^columnIndex, } } @@ -615,7 +612,7 @@ func (page *int32Page) RepetitionLevels() []byte { return nil } func (page *int32Page) DefinitionLevels() []byte { return nil } -func (page *int32Page) Data() []byte { return unsafecast.Int32ToBytes(page.values) } +func (page *int32Page) Data() encoding.Values { return encoding.Int32Values(page.values) } func (page *int32Page) Values() ValueReader { return &int32PageValues{page: page} } @@ -664,10 +661,10 @@ type int64Page struct { columnIndex int16 } -func newInt64Page(typ Type, columnIndex int16, numValues int32, values []byte) *int64Page { +func newInt64Page(typ Type, columnIndex int16, numValues int32, values encoding.Values) *int64Page { return &int64Page{ typ: typ, - values: unsafecast.BytesToInt64(values)[:numValues], + values: values.Int64()[:numValues], columnIndex: ^columnIndex, } } @@ -690,7 +687,7 @@ func (page *int64Page) RepetitionLevels() []byte { return nil } func (page *int64Page) DefinitionLevels() []byte { return nil } -func (page *int64Page) Data() []byte { return unsafecast.Int64ToBytes(page.values) } +func (page *int64Page) Data() encoding.Values { return encoding.Int64Values(page.values) } func (page *int64Page) Values() ValueReader { return &int64PageValues{page: page} } @@ -739,10 +736,10 @@ type int96Page struct { columnIndex int16 } -func newInt96Page(typ Type, columnIndex int16, numValues int32, values []byte) *int96Page { +func newInt96Page(typ Type, columnIndex int16, numValues int32, values encoding.Values) *int96Page { return &int96Page{ typ: typ, - values: deprecated.BytesToInt96(values)[:numValues], + values: values.Int96()[:numValues], columnIndex: ^columnIndex, } } @@ -765,7 +762,7 @@ func (page *int96Page) RepetitionLevels() []byte { return nil } func (page *int96Page) DefinitionLevels() []byte { return nil } -func (page *int96Page) Data() []byte { return deprecated.Int96ToBytes(page.values) } +func (page *int96Page) Data() encoding.Values { return encoding.Int96Values(page.values) } func (page *int96Page) Values() ValueReader { return &int96PageValues{page: page} } @@ -816,10 +813,10 @@ type floatPage struct { columnIndex int16 } -func newFloatPage(typ Type, columnIndex int16, numValues int32, values []byte) *floatPage { +func newFloatPage(typ Type, columnIndex int16, numValues int32, values encoding.Values) *floatPage { return &floatPage{ typ: typ, - values: unsafecast.BytesToFloat32(values)[:numValues], + values: values.Float()[:numValues], columnIndex: ^columnIndex, } } @@ -842,7 +839,7 @@ func (page *floatPage) RepetitionLevels() []byte { return nil } func (page *floatPage) DefinitionLevels() []byte { return nil } -func (page *floatPage) Data() []byte { return unsafecast.Float32ToBytes(page.values) } +func (page *floatPage) Data() encoding.Values { return encoding.FloatValues(page.values) } func (page *floatPage) Values() ValueReader { return &floatPageValues{page: page} } @@ -891,10 +888,10 @@ type doublePage struct { columnIndex int16 } -func newDoublePage(typ Type, columnIndex int16, numValues int32, values []byte) *doublePage { +func newDoublePage(typ Type, columnIndex int16, numValues int32, values encoding.Values) *doublePage { return &doublePage{ typ: typ, - values: unsafecast.BytesToFloat64(values)[:numValues], + values: values.Double()[:numValues], columnIndex: ^columnIndex, } } @@ -917,7 +914,7 @@ func (page *doublePage) RepetitionLevels() []byte { return nil } func (page *doublePage) DefinitionLevels() []byte { return nil } -func (page *doublePage) Data() []byte { return unsafecast.Float64ToBytes(page.values) } +func (page *doublePage) Data() encoding.Values { return encoding.DoubleValues(page.values) } func (page *doublePage) Values() ValueReader { return &doublePageValues{page: page} } @@ -963,15 +960,16 @@ func (page *doublePage) makeValue(v float64) Value { type byteArrayPage struct { typ Type values []byte - numValues int32 + offsets []uint32 columnIndex int16 } -func newByteArrayPage(typ Type, columnIndex int16, numValues int32, values []byte) *byteArrayPage { +func newByteArrayPage(typ Type, columnIndex int16, numValues int32, values encoding.Values) *byteArrayPage { + data, offsets := values.ByteArray() return &byteArrayPage{ typ: typ, - values: values, - numValues: numValues, + values: data, + offsets: offsets[:numValues+1], columnIndex: ^columnIndex, } } @@ -982,74 +980,71 @@ func (page *byteArrayPage) Column() int { return int(^page.columnIndex) } func (page *byteArrayPage) Dictionary() Dictionary { return nil } -func (page *byteArrayPage) NumRows() int64 { return int64(page.numValues) } +func (page *byteArrayPage) NumRows() int64 { return int64(page.len()) } -func (page *byteArrayPage) NumValues() int64 { return int64(page.numValues) } +func (page *byteArrayPage) NumValues() int64 { return int64(page.len()) } func (page *byteArrayPage) NumNulls() int64 { return 0 } -func (page *byteArrayPage) Size() int64 { return int64(len(page.values)) } +func (page *byteArrayPage) Size() int64 { return int64(len(page.values)) + 4*int64(len(page.offsets)) } func (page *byteArrayPage) RepetitionLevels() []byte { return nil } func (page *byteArrayPage) DefinitionLevels() []byte { return nil } -func (page *byteArrayPage) Data() []byte { return page.values } +func (page *byteArrayPage) Data() encoding.Values { + return encoding.ByteArrayValues(page.values, page.offsets) +} func (page *byteArrayPage) Values() ValueReader { return &byteArrayPageValues{page: page} } func (page *byteArrayPage) Buffer() BufferedPage { return page } -func (page *byteArrayPage) valueAt(offset uint32) []byte { - length := binary.LittleEndian.Uint32(page.values[offset:]) - i := 4 + offset - j := 4 + offset + length - return page.values[i:j:j] +func (page *byteArrayPage) len() int { return len(page.offsets) - 1 } + +func (page *byteArrayPage) index(i int) []byte { + j := page.offsets[i+0] + k := page.offsets[i+1] + return page.values[j:k:k] } func (page *byteArrayPage) min() (min []byte) { - if len(page.values) > 0 { - min = page.valueAt(0) + if n := page.len(); n > 0 { + min = page.index(0) - for i := 4 + len(min); i < len(page.values); { - v := page.valueAt(uint32(i)) + for i := 1; i < n; i++ { + v := page.index(i) if bytes.Compare(v, min) < 0 { min = v } - - i += 4 - i += len(v) } } return min } func (page *byteArrayPage) max() (max []byte) { - if len(page.values) > 0 { - max = page.valueAt(0) + if n := page.len(); n > 0 { + max = page.index(0) - for i := 4 + len(max); i < len(page.values); { - v := page.valueAt(uint32(i)) + for i := 1; i < n; i++ { + v := page.index(i) if bytes.Compare(v, max) > 0 { max = v } - - i += 4 - i += len(v) } } return max } func (page *byteArrayPage) bounds() (min, max []byte) { - if len(page.values) > 0 { - min = page.valueAt(0) + if n := page.len(); n > 0 { + min = page.index(0) max = min - for i := 4 + len(min); i < len(page.values); { - v := page.valueAt(uint32(i)) + for i := 1; i < n; i++ { + v := page.index(i) switch { case bytes.Compare(v, min) < 0: @@ -1057,16 +1052,13 @@ func (page *byteArrayPage) bounds() (min, max []byte) { case bytes.Compare(v, max) > 0: max = v } - - i += 4 - i += len(v) } } return min, max } func (page *byteArrayPage) Bounds() (min, max Value, ok bool) { - if ok = len(page.values) > 0; ok { + if ok = len(page.offsets) > 1; ok { minBytes, maxBytes := page.bounds() min = page.makeValueBytes(minBytes) max = page.makeValueBytes(maxBytes) @@ -1080,37 +1072,26 @@ func (page *byteArrayPage) cloneValues() []byte { return values } +func (page *byteArrayPage) cloneOffsets() []uint32 { + offsets := make([]uint32, len(page.offsets)) + copy(offsets, page.offsets) + return offsets +} + func (page *byteArrayPage) Clone() BufferedPage { return &byteArrayPage{ typ: page.typ, values: page.cloneValues(), - numValues: page.numValues, + offsets: page.cloneOffsets(), columnIndex: page.columnIndex, } } func (page *byteArrayPage) Slice(i, j int64) BufferedPage { - numValues := j - i - - off0 := uint32(0) - for i > 0 { - off0 += binary.LittleEndian.Uint32(page.values[off0:]) - off0 += plain.ByteArrayLengthSize - i-- - j-- - } - - off1 := off0 - for j > 0 { - off1 += binary.LittleEndian.Uint32(page.values[off1:]) - off1 += plain.ByteArrayLengthSize - j-- - } - return &byteArrayPage{ typ: page.typ, - values: page.values[off0:off1:off1], - numValues: int32(numValues), + values: page.values, + offsets: page.offsets[i : j+1], columnIndex: page.columnIndex, } } @@ -1134,17 +1115,11 @@ type fixedLenByteArrayPage struct { columnIndex int16 } -func newFixedLenByteArrayPage(typ Type, columnIndex int16, numValues int32, data []byte) *fixedLenByteArrayPage { - size := typ.Length() - if (len(data) % size) != 0 { - panic("cannot create fixed-length byte array page from input which is not a multiple of the type size") - } - if int(numValues) != len(data)/size { - panic(fmt.Errorf("number of values mismatch in numValues and data arguments: %d != %d", numValues, len(data)/size)) - } +func newFixedLenByteArrayPage(typ Type, columnIndex int16, numValues int32, values encoding.Values) *fixedLenByteArrayPage { + data, size := values.FixedLenByteArray() return &fixedLenByteArrayPage{ typ: typ, - data: data, + data: data[:int(numValues)*size], size: size, columnIndex: ^columnIndex, } @@ -1168,7 +1143,9 @@ func (page *fixedLenByteArrayPage) RepetitionLevels() []byte { return nil } func (page *fixedLenByteArrayPage) DefinitionLevels() []byte { return nil } -func (page *fixedLenByteArrayPage) Data() []byte { return page.data } +func (page *fixedLenByteArrayPage) Data() encoding.Values { + return encoding.FixedLenByteArrayValues(page.data, page.size) +} func (page *fixedLenByteArrayPage) Values() ValueReader { return &fixedLenByteArrayPageValues{page: page} @@ -1229,10 +1206,10 @@ type uint32Page struct { columnIndex int16 } -func newUint32Page(typ Type, columnIndex int16, numValues int32, values []byte) *uint32Page { +func newUint32Page(typ Type, columnIndex int16, numValues int32, values encoding.Values) *uint32Page { return &uint32Page{ typ: typ, - values: unsafecast.BytesToUint32(values)[:numValues], + values: values.Uint32()[:numValues], columnIndex: ^columnIndex, } } @@ -1255,7 +1232,7 @@ func (page *uint32Page) RepetitionLevels() []byte { return nil } func (page *uint32Page) DefinitionLevels() []byte { return nil } -func (page *uint32Page) Data() []byte { return unsafecast.Uint32ToBytes(page.values) } +func (page *uint32Page) Data() encoding.Values { return encoding.Uint32Values(page.values) } func (page *uint32Page) Values() ValueReader { return &uint32PageValues{page: page} } @@ -1304,10 +1281,10 @@ type uint64Page struct { columnIndex int16 } -func newUint64Page(typ Type, columnIndex int16, numValues int32, values []byte) *uint64Page { +func newUint64Page(typ Type, columnIndex int16, numValues int32, values encoding.Values) *uint64Page { return &uint64Page{ typ: typ, - values: unsafecast.BytesToUint64(values)[:numValues], + values: values.Uint64()[:numValues], columnIndex: ^columnIndex, } } @@ -1330,7 +1307,7 @@ func (page *uint64Page) RepetitionLevels() []byte { return nil } func (page *uint64Page) DefinitionLevels() []byte { return nil } -func (page *uint64Page) Data() []byte { return unsafecast.Uint64ToBytes(page.values) } +func (page *uint64Page) Data() encoding.Values { return encoding.Uint64Values(page.values) } func (page *uint64Page) Values() ValueReader { return &uint64PageValues{page: page} } @@ -1379,16 +1356,10 @@ type be128Page struct { columnIndex int16 } -func newBE128Page(typ Type, columnIndex int16, numValues int32, data []byte) *be128Page { - if (len(data) % 16) != 0 { - panic("cannot create fixed-length byte array page from input which is not a multiple of the type size") - } - if int(numValues) != len(data)/16 { - panic(fmt.Errorf("number of values mismatch in numValues and data arguments: %d != %d", numValues, len(data)/16)) - } +func newBE128Page(typ Type, columnIndex int16, numValues int32, values encoding.Values) *be128Page { return &be128Page{ typ: typ, - values: unsafecast.BytesToUint128(data), + values: values.Uint128()[:numValues], columnIndex: ^columnIndex, } } @@ -1411,7 +1382,7 @@ func (page *be128Page) RepetitionLevels() []byte { return nil } func (page *be128Page) DefinitionLevels() []byte { return nil } -func (page *be128Page) Data() []byte { return unsafecast.Uint128ToBytes(page.values) } +func (page *be128Page) Data() encoding.Values { return encoding.Uint128Values(page.values) } func (page *be128Page) Values() ValueReader { return &be128PageValues{page: page} } @@ -1496,4 +1467,4 @@ func (page *nullPage) Slice(i, j int64) BufferedPage { } func (page *nullPage) RepetitionLevels() []byte { return nil } func (page *nullPage) DefinitionLevels() []byte { return nil } -func (page *nullPage) Data() []byte { return nil } +func (page *nullPage) Data() encoding.Values { return encoding.Values{} } diff --git a/page_values.go b/page_values.go index 65b6026d..efcb7257 100644 --- a/page_values.go +++ b/page_values.go @@ -307,8 +307,9 @@ func (r *byteArrayPageValues) ReadByteArrays(values []byte) (int, error) { } func (r *byteArrayPageValues) readByteArrays(values []byte) (c, n int, err error) { - for r.offset < len(r.page.values) { - b := r.page.valueAt(uint32(r.offset)) + numValues := r.page.len() + for r.offset < numValues { + b := r.page.index(r.offset) k := plain.ByteArrayLengthSize + len(b) if k > (len(values) - n) { break @@ -316,11 +317,10 @@ func (r *byteArrayPageValues) readByteArrays(values []byte) (c, n int, err error plain.PutByteArrayLength(values[n:], len(b)) n += plain.ByteArrayLengthSize n += copy(values[n:], b) - r.offset += plain.ByteArrayLengthSize - r.offset += len(b) + r.offset++ c++ } - if r.offset == len(r.page.values) { + if r.offset == numValues { err = io.EOF } else if n == 0 && len(values) > 0 { err = io.ErrShortBuffer @@ -329,14 +329,14 @@ func (r *byteArrayPageValues) readByteArrays(values []byte) (c, n int, err error } func (r *byteArrayPageValues) ReadValues(values []Value) (n int, err error) { - for n < len(values) && r.offset < len(r.page.values) { - value := r.page.valueAt(uint32(r.offset)) + numValues := r.page.len() + for n < len(values) && r.offset < numValues { + value := r.page.index(r.offset) values[n] = r.page.makeValueBytes(copyBytes(value)) - r.offset += plain.ByteArrayLengthSize - r.offset += len(value) + r.offset++ n++ } - if r.offset == len(r.page.values) { + if r.offset == numValues { err = io.EOF } return n, err diff --git a/type.go b/type.go index d3923afe..312989f2 100644 --- a/type.go +++ b/type.go @@ -140,43 +140,50 @@ type Type interface { // Creates a dictionary holding values of this type. // - // If the length of data is not zero, it must contain PLAIN encoded values - // of the dictionary. - // // The dictionary retains the data buffer, it does not make a copy of it. // If the application needs to share ownership of the memory buffer, it must // ensure that it will not be modified while the page is in use, or it must // make a copy of it prior to creating the dictionary. // - // The method panics if it is called on a group type. - NewDictionary(columnIndex, numValues int, data []byte) Dictionary + // The method panics if the data type does not correspond to the parquet + // type it is called on. + NewDictionary(columnIndex, numValues int, data encoding.Values) Dictionary // Creates a page belonging to a column at the given index, backed by the // data buffer. // - // If the length of data is not zero, it must contain PLAIN encoded values - // of the page. - // // The page retains the data buffer, it does not make a copy of it. If the // application needs to share ownership of the memory buffer, it must ensure // that it will not be modified while the page is in use, or it must make a // copy of it prior to creating the page. // - // The method panics if the data is not a valid PLAIN encoded representation - // of the page values. - NewPage(columnIndex, numValues int, data []byte) Page + // The method panics if the data type does not correspond to the parquet + // type it is called on. + NewPage(columnIndex, numValues int, data encoding.Values) Page + + // Creates an encoding.Values instance backed by the given buffers. + // + // The offsets is only used by BYTE_ARRAY types, where it represents the + // positions of each variable length value in the values buffer. + // + // The following expression creates an empty instance for any type: + // + // values := typ.NewValues(nil, nil) + // + // The method panics if it is called on group types. + NewValues(values []byte, offsets []uint32) encoding.Values // Assuming the src buffer contains PLAIN encoded values of the type it is // called on, applies the given encoding and produces the output to the dst // buffer passed as first argument by dispatching the call to one of the // encoding methods. - Encode(dst, src []byte, enc encoding.Encoding) ([]byte, error) + Encode(dst []byte, src encoding.Values, enc encoding.Encoding) ([]byte, error) // Assuming the src buffer contains values encoding in the given encoding, - // decodes the input and produces the PLAIN encoded values into the dst - // output buffer passed as first argument by dispatching the call to one - // of the encoding methods. - Decode(dst, src []byte, enc encoding.Encoding) ([]byte, error) + // decodes the input and produces the encoded values into the dst output + // buffer passed as first argument by dispatching the call to one of the + // encoding methods. + Decode(dst encoding.Values, src []byte, enc encoding.Encoding) (encoding.Values, error) } var ( @@ -251,20 +258,24 @@ func (t booleanType) NewColumnBuffer(columnIndex, numValues int) ColumnBuffer { return newBooleanColumnBuffer(t, makeColumnIndex(columnIndex), makeNumValues(numValues)) } -func (t booleanType) NewDictionary(columnIndex, numValues int, data []byte) Dictionary { +func (t booleanType) NewDictionary(columnIndex, numValues int, data encoding.Values) Dictionary { return newBooleanDictionary(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } -func (t booleanType) NewPage(columnIndex, numValues int, data []byte) Page { +func (t booleanType) NewPage(columnIndex, numValues int, data encoding.Values) Page { return newBooleanPage(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } -func (t booleanType) Encode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.EncodeBoolean(dst, src) +func (t booleanType) NewValues(values []byte, _ []uint32) encoding.Values { + return encoding.BooleanValues(values) +} + +func (t booleanType) Encode(dst []byte, src encoding.Values, enc encoding.Encoding) ([]byte, error) { + return encoding.EncodeBoolean(dst, src, enc) } -func (t booleanType) Decode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.DecodeBoolean(dst, src) +func (t booleanType) Decode(dst encoding.Values, src []byte, enc encoding.Encoding) (encoding.Values, error) { + return encoding.DecodeBoolean(dst, src, enc) } type int32Type struct{} @@ -287,20 +298,24 @@ func (t int32Type) NewColumnBuffer(columnIndex, numValues int) ColumnBuffer { return newInt32ColumnBuffer(t, makeColumnIndex(columnIndex), makeNumValues(numValues)) } -func (t int32Type) NewDictionary(columnIndex, numValues int, data []byte) Dictionary { +func (t int32Type) NewDictionary(columnIndex, numValues int, data encoding.Values) Dictionary { return newInt32Dictionary(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } -func (t int32Type) NewPage(columnIndex, numValues int, data []byte) Page { +func (t int32Type) NewPage(columnIndex, numValues int, data encoding.Values) Page { return newInt32Page(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } -func (t int32Type) Encode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.EncodeInt32(dst, src) +func (t int32Type) NewValues(values []byte, _ []uint32) encoding.Values { + return encoding.Int32ValuesFromBytes(values) } -func (t int32Type) Decode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.DecodeInt32(dst, src) +func (t int32Type) Encode(dst []byte, src encoding.Values, enc encoding.Encoding) ([]byte, error) { + return encoding.EncodeInt32(dst, src, enc) +} + +func (t int32Type) Decode(dst encoding.Values, src []byte, enc encoding.Encoding) (encoding.Values, error) { + return encoding.DecodeInt32(dst, src, enc) } type int64Type struct{} @@ -323,20 +338,24 @@ func (t int64Type) NewColumnBuffer(columnIndex, numValues int) ColumnBuffer { return newInt64ColumnBuffer(t, makeColumnIndex(columnIndex), makeNumValues(numValues)) } -func (t int64Type) NewDictionary(columnIndex, numValues int, data []byte) Dictionary { +func (t int64Type) NewDictionary(columnIndex, numValues int, data encoding.Values) Dictionary { return newInt64Dictionary(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } -func (t int64Type) NewPage(columnIndex, numValues int, data []byte) Page { +func (t int64Type) NewPage(columnIndex, numValues int, data encoding.Values) Page { return newInt64Page(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } -func (t int64Type) Encode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.EncodeInt64(dst, src) +func (t int64Type) NewValues(values []byte, _ []uint32) encoding.Values { + return encoding.Int64ValuesFromBytes(values) +} + +func (t int64Type) Encode(dst []byte, src encoding.Values, enc encoding.Encoding) ([]byte, error) { + return encoding.EncodeInt64(dst, src, enc) } -func (t int64Type) Decode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.DecodeInt64(dst, src) +func (t int64Type) Decode(dst encoding.Values, src []byte, enc encoding.Encoding) (encoding.Values, error) { + return encoding.DecodeInt64(dst, src, enc) } type int96Type struct{} @@ -360,20 +379,24 @@ func (t int96Type) NewColumnBuffer(columnIndex, numValues int) ColumnBuffer { return newInt96ColumnBuffer(t, makeColumnIndex(columnIndex), makeNumValues(numValues)) } -func (t int96Type) NewDictionary(columnIndex, numValues int, data []byte) Dictionary { +func (t int96Type) NewDictionary(columnIndex, numValues int, data encoding.Values) Dictionary { return newInt96Dictionary(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } -func (t int96Type) NewPage(columnIndex, numValues int, data []byte) Page { +func (t int96Type) NewPage(columnIndex, numValues int, data encoding.Values) Page { return newInt96Page(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } -func (t int96Type) Encode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.EncodeInt96(dst, src) +func (t int96Type) NewValues(values []byte, _ []uint32) encoding.Values { + return encoding.Int96ValuesFromBytes(values) } -func (t int96Type) Decode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.DecodeInt96(dst, src) +func (t int96Type) Encode(dst []byte, src encoding.Values, enc encoding.Encoding) ([]byte, error) { + return encoding.EncodeInt96(dst, src, enc) +} + +func (t int96Type) Decode(dst encoding.Values, src []byte, enc encoding.Encoding) (encoding.Values, error) { + return encoding.DecodeInt96(dst, src, enc) } type floatType struct{} @@ -396,20 +419,24 @@ func (t floatType) NewColumnBuffer(columnIndex, numValues int) ColumnBuffer { return newFloatColumnBuffer(t, makeColumnIndex(columnIndex), makeNumValues(numValues)) } -func (t floatType) NewDictionary(columnIndex, numValues int, data []byte) Dictionary { +func (t floatType) NewDictionary(columnIndex, numValues int, data encoding.Values) Dictionary { return newFloatDictionary(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } -func (t floatType) NewPage(columnIndex, numValues int, data []byte) Page { +func (t floatType) NewPage(columnIndex, numValues int, data encoding.Values) Page { return newFloatPage(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } -func (t floatType) Encode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.EncodeFloat(dst, src) +func (t floatType) NewValues(values []byte, _ []uint32) encoding.Values { + return encoding.FloatValuesFromBytes(values) +} + +func (t floatType) Encode(dst []byte, src encoding.Values, enc encoding.Encoding) ([]byte, error) { + return encoding.EncodeFloat(dst, src, enc) } -func (t floatType) Decode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.DecodeFloat(dst, src) +func (t floatType) Decode(dst encoding.Values, src []byte, enc encoding.Encoding) (encoding.Values, error) { + return encoding.DecodeFloat(dst, src, enc) } type doubleType struct{} @@ -432,20 +459,24 @@ func (t doubleType) NewColumnBuffer(columnIndex, numValues int) ColumnBuffer { return newDoubleColumnBuffer(t, makeColumnIndex(columnIndex), makeNumValues(numValues)) } -func (t doubleType) NewDictionary(columnIndex, numValues int, data []byte) Dictionary { +func (t doubleType) NewDictionary(columnIndex, numValues int, data encoding.Values) Dictionary { return newDoubleDictionary(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } -func (t doubleType) NewPage(columnIndex, numValues int, data []byte) Page { +func (t doubleType) NewPage(columnIndex, numValues int, data encoding.Values) Page { return newDoublePage(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } -func (t doubleType) Encode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.EncodeDouble(dst, src) +func (t doubleType) NewValues(values []byte, _ []uint32) encoding.Values { + return encoding.DoubleValuesFromBytes(values) } -func (t doubleType) Decode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.DecodeDouble(dst, src) +func (t doubleType) Encode(dst []byte, src encoding.Values, enc encoding.Encoding) ([]byte, error) { + return encoding.EncodeDouble(dst, src, enc) +} + +func (t doubleType) Decode(dst encoding.Values, src []byte, enc encoding.Encoding) (encoding.Values, error) { + return encoding.DecodeDouble(dst, src, enc) } type byteArrayType struct{} @@ -468,20 +499,24 @@ func (t byteArrayType) NewColumnBuffer(columnIndex, numValues int) ColumnBuffer return newByteArrayColumnBuffer(t, makeColumnIndex(columnIndex), makeNumValues(numValues)) } -func (t byteArrayType) NewDictionary(columnIndex, numValues int, data []byte) Dictionary { +func (t byteArrayType) NewDictionary(columnIndex, numValues int, data encoding.Values) Dictionary { return newByteArrayDictionary(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } -func (t byteArrayType) NewPage(columnIndex, numValues int, data []byte) Page { +func (t byteArrayType) NewPage(columnIndex, numValues int, data encoding.Values) Page { return newByteArrayPage(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } -func (t byteArrayType) Encode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.EncodeByteArray(dst, src) +func (t byteArrayType) NewValues(values []byte, offsets []uint32) encoding.Values { + return encoding.ByteArrayValues(values, offsets) +} + +func (t byteArrayType) Encode(dst []byte, src encoding.Values, enc encoding.Encoding) ([]byte, error) { + return encoding.EncodeByteArray(dst, src, enc) } -func (t byteArrayType) Decode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.DecodeByteArray(dst, src) +func (t byteArrayType) Decode(dst encoding.Values, src []byte, enc encoding.Encoding) (encoding.Values, error) { + return encoding.DecodeByteArray(dst, src, enc) } type fixedLenByteArrayType struct{ length int } @@ -516,20 +551,24 @@ func (t fixedLenByteArrayType) NewColumnBuffer(columnIndex, numValues int) Colum return newFixedLenByteArrayColumnBuffer(t, makeColumnIndex(columnIndex), makeNumValues(numValues)) } -func (t fixedLenByteArrayType) NewDictionary(columnIndex, numValues int, data []byte) Dictionary { +func (t fixedLenByteArrayType) NewDictionary(columnIndex, numValues int, data encoding.Values) Dictionary { return newFixedLenByteArrayDictionary(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } -func (t fixedLenByteArrayType) NewPage(columnIndex, numValues int, data []byte) Page { +func (t fixedLenByteArrayType) NewPage(columnIndex, numValues int, data encoding.Values) Page { return newFixedLenByteArrayPage(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } -func (t fixedLenByteArrayType) Encode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.EncodeFixedLenByteArray(dst, src, t.length) +func (t fixedLenByteArrayType) NewValues(values []byte, _ []uint32) encoding.Values { + return encoding.FixedLenByteArrayValues(values, t.length) } -func (t fixedLenByteArrayType) Decode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.DecodeFixedLenByteArray(dst, src, t.length) +func (t fixedLenByteArrayType) Encode(dst []byte, src encoding.Values, enc encoding.Encoding) ([]byte, error) { + return encoding.EncodeFixedLenByteArray(dst, src, enc) +} + +func (t fixedLenByteArrayType) Decode(dst encoding.Values, src []byte, enc encoding.Encoding) (encoding.Values, error) { + return encoding.DecodeFixedLenByteArray(dst, src, enc) } // BE128 stands for "big-endian 128 bits". This type is used as a special case @@ -573,20 +612,24 @@ func (t be128Type) NewColumnBuffer(columnIndex, numValues int) ColumnBuffer { return newBE128ColumnBuffer(t, makeColumnIndex(columnIndex), makeNumValues(numValues)) } -func (t be128Type) NewDictionary(columnIndex, numValues int, data []byte) Dictionary { +func (t be128Type) NewDictionary(columnIndex, numValues int, data encoding.Values) Dictionary { return newBE128Dictionary(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } -func (t be128Type) NewPage(columnIndex, numValues int, data []byte) Page { +func (t be128Type) NewPage(columnIndex, numValues int, data encoding.Values) Page { return newBE128Page(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } -func (t be128Type) Encode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.EncodeFixedLenByteArray(dst, src, 16) +func (t be128Type) NewValues(values []byte, _ []uint32) encoding.Values { + return encoding.FixedLenByteArrayValues(values, 16) +} + +func (t be128Type) Encode(dst []byte, src encoding.Values, enc encoding.Encoding) ([]byte, error) { + return encoding.EncodeFixedLenByteArray(dst, src, enc) } -func (t be128Type) Decode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.DecodeFixedLenByteArray(dst, src, 16) +func (t be128Type) Decode(dst encoding.Values, src []byte, enc encoding.Encoding) (encoding.Values, error) { + return encoding.DecodeFixedLenByteArray(dst, src, enc) } // FixedLenByteArrayType constructs a type for fixed-length values of the given @@ -739,7 +782,7 @@ func (t *intType) NewColumnBuffer(columnIndex, numValues int) ColumnBuffer { } } -func (t *intType) NewDictionary(columnIndex, numValues int, data []byte) Dictionary { +func (t *intType) NewDictionary(columnIndex, numValues int, data encoding.Values) Dictionary { if t.IsSigned { if t.BitWidth == 64 { return newInt64Dictionary(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) @@ -755,7 +798,7 @@ func (t *intType) NewDictionary(columnIndex, numValues int, data []byte) Diction } } -func (t *intType) NewPage(columnIndex, numValues int, data []byte) Page { +func (t *intType) NewPage(columnIndex, numValues int, data encoding.Values) Page { if t.IsSigned { if t.BitWidth == 64 { return newInt64Page(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) @@ -771,19 +814,27 @@ func (t *intType) NewPage(columnIndex, numValues int, data []byte) Page { } } -func (t *intType) Encode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { +func (t *intType) NewValues(values []byte, _ []uint32) encoding.Values { if t.BitWidth == 64 { - return enc.EncodeInt64(dst, src) + return encoding.Int64ValuesFromBytes(values) } else { - return enc.EncodeInt32(dst, src) + return encoding.Int32ValuesFromBytes(values) } } -func (t *intType) Decode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { +func (t *intType) Encode(dst []byte, src encoding.Values, enc encoding.Encoding) ([]byte, error) { if t.BitWidth == 64 { - return enc.DecodeInt64(dst, src) + return encoding.EncodeInt64(dst, src, enc) } else { - return enc.DecodeInt32(dst, src) + return encoding.EncodeInt32(dst, src, enc) + } +} + +func (t *intType) Decode(dst encoding.Values, src []byte, enc encoding.Encoding) (encoding.Values, error) { + if t.BitWidth == 64 { + return encoding.DecodeInt64(dst, src, enc) + } else { + return encoding.DecodeInt32(dst, src, enc) } } @@ -860,7 +911,7 @@ func (t *stringType) NewColumnIndexer(sizeLimit int) ColumnIndexer { return newByteArrayColumnIndexer(sizeLimit) } -func (t *stringType) NewDictionary(columnIndex, numValues int, data []byte) Dictionary { +func (t *stringType) NewDictionary(columnIndex, numValues int, data encoding.Values) Dictionary { return newByteArrayDictionary(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } @@ -868,16 +919,20 @@ func (t *stringType) NewColumnBuffer(columnIndex, numValues int) ColumnBuffer { return newByteArrayColumnBuffer(t, makeColumnIndex(columnIndex), makeNumValues(numValues)) } -func (t *stringType) NewPage(columnIndex, numValues int, data []byte) Page { +func (t *stringType) NewPage(columnIndex, numValues int, data encoding.Values) Page { return newByteArrayPage(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } -func (t *stringType) Encode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.EncodeByteArray(dst, src) +func (t *stringType) NewValues(values []byte, offsets []uint32) encoding.Values { + return encoding.ByteArrayValues(values, offsets) +} + +func (t *stringType) Encode(dst []byte, src encoding.Values, enc encoding.Encoding) ([]byte, error) { + return encoding.EncodeByteArray(dst, src, enc) } -func (t *stringType) Decode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.DecodeByteArray(dst, src) +func (t *stringType) Decode(dst encoding.Values, src []byte, enc encoding.Encoding) (encoding.Values, error) { + return encoding.DecodeByteArray(dst, src, enc) } // UUID constructs a leaf node of UUID logical type. @@ -913,7 +968,7 @@ func (t *uuidType) NewColumnIndexer(sizeLimit int) ColumnIndexer { return newBE128ColumnIndexer() } -func (t *uuidType) NewDictionary(columnIndex, numValues int, data []byte) Dictionary { +func (t *uuidType) NewDictionary(columnIndex, numValues int, data encoding.Values) Dictionary { return newBE128Dictionary(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } @@ -921,16 +976,20 @@ func (t *uuidType) NewColumnBuffer(columnIndex, numValues int) ColumnBuffer { return newBE128ColumnBuffer(t, makeColumnIndex(columnIndex), makeNumValues(numValues)) } -func (t *uuidType) NewPage(columnIndex, numValues int, data []byte) Page { +func (t *uuidType) NewPage(columnIndex, numValues int, data encoding.Values) Page { return newBE128Page(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } -func (t *uuidType) Encode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.EncodeFixedLenByteArray(dst, src, 16) +func (t *uuidType) NewValues(values []byte, _ []uint32) encoding.Values { + return encoding.FixedLenByteArrayValues(values, 16) } -func (t *uuidType) Decode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.DecodeFixedLenByteArray(dst, src, 16) +func (t *uuidType) Encode(dst []byte, src encoding.Values, enc encoding.Encoding) ([]byte, error) { + return encoding.EncodeFixedLenByteArray(dst, src, enc) +} + +func (t *uuidType) Decode(dst encoding.Values, src []byte, enc encoding.Encoding) (encoding.Values, error) { + return encoding.DecodeFixedLenByteArray(dst, src, enc) } // Enum constructs a leaf node with a logical type representing enumerations. @@ -972,7 +1031,7 @@ func (t *enumType) NewColumnIndexer(sizeLimit int) ColumnIndexer { return newByteArrayColumnIndexer(sizeLimit) } -func (t *enumType) NewDictionary(columnIndex, numValues int, data []byte) Dictionary { +func (t *enumType) NewDictionary(columnIndex, numValues int, data encoding.Values) Dictionary { return newByteArrayDictionary(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } @@ -980,16 +1039,20 @@ func (t *enumType) NewColumnBuffer(columnIndex, numValues int) ColumnBuffer { return newByteArrayColumnBuffer(t, makeColumnIndex(columnIndex), makeNumValues(numValues)) } -func (t *enumType) NewPage(columnIndex, numValues int, data []byte) Page { +func (t *enumType) NewPage(columnIndex, numValues int, data encoding.Values) Page { return newByteArrayPage(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } -func (t *enumType) Encode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.EncodeByteArray(dst, src) +func (t *enumType) NewValues(values []byte, offsets []uint32) encoding.Values { + return encoding.ByteArrayValues(values, offsets) +} + +func (t *enumType) Encode(dst []byte, src encoding.Values, enc encoding.Encoding) ([]byte, error) { + return encoding.EncodeByteArray(dst, src, enc) } -func (t *enumType) Decode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.DecodeByteArray(dst, src) +func (t *enumType) Decode(dst encoding.Values, src []byte, enc encoding.Encoding) (encoding.Values, error) { + return encoding.DecodeByteArray(dst, src, enc) } // JSON constructs a leaf node of JSON logical type. @@ -1031,7 +1094,7 @@ func (t *jsonType) NewColumnIndexer(sizeLimit int) ColumnIndexer { return newByteArrayColumnIndexer(sizeLimit) } -func (t *jsonType) NewDictionary(columnIndex, numValues int, data []byte) Dictionary { +func (t *jsonType) NewDictionary(columnIndex, numValues int, data encoding.Values) Dictionary { return newByteArrayDictionary(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } @@ -1039,16 +1102,20 @@ func (t *jsonType) NewColumnBuffer(columnIndex, numValues int) ColumnBuffer { return newByteArrayColumnBuffer(t, makeColumnIndex(columnIndex), makeNumValues(numValues)) } -func (t *jsonType) NewPage(columnIndex, numValues int, data []byte) Page { +func (t *jsonType) NewPage(columnIndex, numValues int, data encoding.Values) Page { return newByteArrayPage(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } -func (t *jsonType) Encode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.EncodeByteArray(dst, src) +func (t *jsonType) NewValues(values []byte, offsets []uint32) encoding.Values { + return encoding.ByteArrayValues(values, offsets) } -func (t *jsonType) Decode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.DecodeByteArray(dst, src) +func (t *jsonType) Encode(dst []byte, src encoding.Values, enc encoding.Encoding) ([]byte, error) { + return encoding.EncodeByteArray(dst, src, enc) +} + +func (t *jsonType) Decode(dst encoding.Values, src []byte, enc encoding.Encoding) (encoding.Values, error) { + return encoding.DecodeByteArray(dst, src, enc) } // BSON constructs a leaf node of BSON logical type. @@ -1090,7 +1157,7 @@ func (t *bsonType) NewColumnIndexer(sizeLimit int) ColumnIndexer { return newByteArrayColumnIndexer(sizeLimit) } -func (t *bsonType) NewDictionary(columnIndex, numValues int, data []byte) Dictionary { +func (t *bsonType) NewDictionary(columnIndex, numValues int, data encoding.Values) Dictionary { return newByteArrayDictionary(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } @@ -1098,16 +1165,20 @@ func (t *bsonType) NewColumnBuffer(columnIndex, numValues int) ColumnBuffer { return newByteArrayColumnBuffer(t, makeColumnIndex(columnIndex), makeNumValues(numValues)) } -func (t *bsonType) NewPage(columnIndex, numValues int, data []byte) Page { +func (t *bsonType) NewPage(columnIndex, numValues int, data encoding.Values) Page { return newByteArrayPage(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } -func (t *bsonType) Encode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.EncodeByteArray(dst, src) +func (t *bsonType) NewValues(values []byte, offsets []uint32) encoding.Values { + return encoding.ByteArrayValues(values, offsets) +} + +func (t *bsonType) Encode(dst []byte, src encoding.Values, enc encoding.Encoding) ([]byte, error) { + return encoding.EncodeByteArray(dst, src, enc) } -func (t *bsonType) Decode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.DecodeByteArray(dst, src) +func (t *bsonType) Decode(dst encoding.Values, src []byte, enc encoding.Encoding) (encoding.Values, error) { + return encoding.DecodeByteArray(dst, src, enc) } // Date constructs a leaf node of DATE logical type. @@ -1149,20 +1220,24 @@ func (t *dateType) NewColumnBuffer(columnIndex, numValues int) ColumnBuffer { return newInt32ColumnBuffer(t, makeColumnIndex(columnIndex), makeNumValues(numValues)) } -func (t *dateType) NewDictionary(columnIndex, numValues int, data []byte) Dictionary { +func (t *dateType) NewDictionary(columnIndex, numValues int, data encoding.Values) Dictionary { return newInt32Dictionary(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } -func (t *dateType) NewPage(columnIndex, numValues int, data []byte) Page { +func (t *dateType) NewPage(columnIndex, numValues int, data encoding.Values) Page { return newInt32Page(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } -func (t *dateType) Encode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.EncodeInt32(dst, src) +func (t *dateType) NewValues(values []byte, _ []uint32) encoding.Values { + return encoding.Int32ValuesFromBytes(values) } -func (t *dateType) Decode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.DecodeInt32(dst, src) +func (t *dateType) Encode(dst []byte, src encoding.Values, enc encoding.Encoding) ([]byte, error) { + return encoding.EncodeInt32(dst, src, enc) +} + +func (t *dateType) Decode(dst encoding.Values, src []byte, enc encoding.Encoding) (encoding.Values, error) { + return encoding.DecodeInt32(dst, src, enc) } // TimeUnit represents units of time in the parquet type system. @@ -1297,7 +1372,7 @@ func (t *timeType) NewColumnBuffer(columnIndex, numValues int) ColumnBuffer { } } -func (t *timeType) NewDictionary(columnIndex, numValues int, data []byte) Dictionary { +func (t *timeType) NewDictionary(columnIndex, numValues int, data encoding.Values) Dictionary { if t.useInt32() { return newInt32Dictionary(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } else { @@ -1305,7 +1380,7 @@ func (t *timeType) NewDictionary(columnIndex, numValues int, data []byte) Dictio } } -func (t *timeType) NewPage(columnIndex, numValues int, data []byte) Page { +func (t *timeType) NewPage(columnIndex, numValues int, data encoding.Values) Page { if t.useInt32() { return newInt32Page(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } else { @@ -1313,19 +1388,27 @@ func (t *timeType) NewPage(columnIndex, numValues int, data []byte) Page { } } -func (t *timeType) Encode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { +func (t *timeType) NewValues(values []byte, _ []uint32) encoding.Values { + if t.useInt32() { + return encoding.Int32ValuesFromBytes(values) + } else { + return encoding.Int64ValuesFromBytes(values) + } +} + +func (t *timeType) Encode(dst []byte, src encoding.Values, enc encoding.Encoding) ([]byte, error) { if t.useInt32() { - return enc.EncodeInt32(dst, src) + return encoding.EncodeInt32(dst, src, enc) } else { - return enc.EncodeInt64(dst, src) + return encoding.EncodeInt64(dst, src, enc) } } -func (t *timeType) Decode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { +func (t *timeType) Decode(dst encoding.Values, src []byte, enc encoding.Encoding) (encoding.Values, error) { if t.useInt32() { - return enc.DecodeInt32(dst, src) + return encoding.DecodeInt32(dst, src, enc) } else { - return enc.DecodeInt64(dst, src) + return encoding.DecodeInt64(dst, src, enc) } } @@ -1375,20 +1458,24 @@ func (t *timestampType) NewColumnBuffer(columnIndex, numValues int) ColumnBuffer return newInt64ColumnBuffer(t, makeColumnIndex(columnIndex), makeNumValues(numValues)) } -func (t *timestampType) NewDictionary(columnIndex, numValues int, data []byte) Dictionary { +func (t *timestampType) NewDictionary(columnIndex, numValues int, data encoding.Values) Dictionary { return newInt64Dictionary(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } -func (t *timestampType) NewPage(columnIndex, numValues int, data []byte) Page { +func (t *timestampType) NewPage(columnIndex, numValues int, data encoding.Values) Page { return newInt64Page(t, makeColumnIndex(columnIndex), makeNumValues(numValues), data) } -func (t *timestampType) Encode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.EncodeInt64(dst, src) +func (t *timestampType) NewValues(values []byte, _ []uint32) encoding.Values { + return encoding.Int64ValuesFromBytes(values) } -func (t *timestampType) Decode(dst, src []byte, enc encoding.Encoding) ([]byte, error) { - return enc.DecodeInt64(dst, src) +func (t *timestampType) Encode(dst []byte, src encoding.Values, enc encoding.Encoding) ([]byte, error) { + return encoding.EncodeInt64(dst, src, enc) +} + +func (t *timestampType) Decode(dst encoding.Values, src []byte, enc encoding.Encoding) (encoding.Values, error) { + return encoding.DecodeInt64(dst, src, enc) } // List constructs a node of LIST logical type. @@ -1430,7 +1517,7 @@ func (t *listType) NewColumnIndexer(int) ColumnIndexer { panic("create create column indexer from parquet LIST type") } -func (t *listType) NewDictionary(int, int, []byte) Dictionary { +func (t *listType) NewDictionary(int, int, encoding.Values) Dictionary { panic("cannot create dictionary from parquet LIST type") } @@ -1438,15 +1525,19 @@ func (t *listType) NewColumnBuffer(int, int) ColumnBuffer { panic("cannot create column buffer from parquet LIST type") } -func (t *listType) NewPage(int, int, []byte) Page { +func (t *listType) NewPage(int, int, encoding.Values) Page { panic("cannot create page from parquet LIST type") } -func (t *listType) Encode(dst, _ []byte, _ encoding.Encoding) ([]byte, error) { +func (t *listType) NewValues(values []byte, _ []uint32) encoding.Values { + panic("cannot create values from parquet LIST type") +} + +func (t *listType) Encode(_ []byte, _ encoding.Values, _ encoding.Encoding) ([]byte, error) { panic("cannot encode parquet LIST type") } -func (t *listType) Decode(dst, _ []byte, _ encoding.Encoding) ([]byte, error) { +func (t *listType) Decode(_ encoding.Values, _ []byte, _ encoding.Encoding) (encoding.Values, error) { panic("cannot decode parquet LIST type") } @@ -1494,7 +1585,7 @@ func (t *mapType) NewColumnIndexer(int) ColumnIndexer { panic("create create column indexer from parquet MAP type") } -func (t *mapType) NewDictionary(int, int, []byte) Dictionary { +func (t *mapType) NewDictionary(int, int, encoding.Values) Dictionary { panic("cannot create dictionary from parquet MAP type") } @@ -1502,15 +1593,19 @@ func (t *mapType) NewColumnBuffer(int, int) ColumnBuffer { panic("cannot create column buffer from parquet MAP type") } -func (t *mapType) NewPage(int, int, []byte) Page { +func (t *mapType) NewPage(int, int, encoding.Values) Page { panic("cannot create page from parquet MAP type") } -func (t *mapType) Encode(dst, _ []byte, _ encoding.Encoding) ([]byte, error) { +func (t *mapType) NewValues(values []byte, _ []uint32) encoding.Values { + panic("cannot create values from parquet MAP type") +} + +func (t *mapType) Encode(_ []byte, _ encoding.Values, _ encoding.Encoding) ([]byte, error) { panic("cannot encode parquet MAP type") } -func (t *mapType) Decode(dst, _ []byte, _ encoding.Encoding) ([]byte, error) { +func (t *mapType) Decode(_ encoding.Values, _ []byte, _ encoding.Encoding) (encoding.Values, error) { panic("cannot decode parquet MAP type") } @@ -1540,7 +1635,7 @@ func (t *nullType) NewColumnIndexer(int) ColumnIndexer { panic("create create column indexer from parquet NULL type") } -func (t *nullType) NewDictionary(int, int, []byte) Dictionary { +func (t *nullType) NewDictionary(int, int, encoding.Values) Dictionary { panic("cannot create dictionary from parquet NULL type") } @@ -1548,18 +1643,22 @@ func (t *nullType) NewColumnBuffer(int, int) ColumnBuffer { panic("cannot create column buffer from parquet NULL type") } -func (t *nullType) NewPage(columnIndex, numValues int, _ []byte) Page { +func (t *nullType) NewPage(columnIndex, numValues int, _ encoding.Values) Page { return newNullPage(t, makeColumnIndex(columnIndex), makeNumValues(numValues)) } -func (t *nullType) Encode(dst, _ []byte, _ encoding.Encoding) ([]byte, error) { - return dst[:0], nil +func (t *nullType) NewValues(_ []byte, _ []uint32) encoding.Values { + return encoding.Values{} } -func (t *nullType) Decode(dst, _ []byte, _ encoding.Encoding) ([]byte, error) { +func (t *nullType) Encode(dst []byte, _ encoding.Values, _ encoding.Encoding) ([]byte, error) { return dst[:0], nil } +func (t *nullType) Decode(dst encoding.Values, _ []byte, _ encoding.Encoding) (encoding.Values, error) { + return dst, nil +} + type groupType struct{} func (groupType) String() string { return "group" } @@ -1576,7 +1675,7 @@ func (groupType) NewColumnIndexer(int) ColumnIndexer { panic("cannot create column indexer from parquet group") } -func (groupType) NewDictionary(int, int, []byte) Dictionary { +func (groupType) NewDictionary(int, int, encoding.Values) Dictionary { panic("cannot create dictionary from parquet group") } @@ -1584,15 +1683,19 @@ func (t groupType) NewColumnBuffer(int, int) ColumnBuffer { panic("cannot create column buffer from parquet group") } -func (t groupType) NewPage(int, int, []byte) Page { +func (t groupType) NewPage(int, int, encoding.Values) Page { panic("cannot create page from parquet group") } -func (groupType) Encode(_, _ []byte, _ encoding.Encoding) ([]byte, error) { +func (t groupType) NewValues(_ []byte, _ []uint32) encoding.Values { + panic("cannot create values from parquet group") +} + +func (groupType) Encode(_ []byte, _ encoding.Values, _ encoding.Encoding) ([]byte, error) { panic("cannot encode parquet group") } -func (groupType) Decode(_, _ []byte, _ encoding.Encoding) ([]byte, error) { +func (groupType) Decode(_ encoding.Values, _ []byte, _ encoding.Encoding) (encoding.Values, error) { panic("cannot decode parquet group") } diff --git a/writer.go b/writer.go index c6aab354..66c6363a 100644 --- a/writer.go +++ b/writer.go @@ -310,7 +310,11 @@ func newWriter(output io.Writer, config *WriterConfig) *writer { } if isDictionaryEncoding(encoding) { - dictionary = columnType.NewDictionary(columnIndex, 0, make([]byte, 0, defaultDictBufferSize)) + dictBuffer := columnType.NewValues( + make([]byte, 0, defaultDictBufferSize), + nil, + ) + dictionary = columnType.NewDictionary(columnIndex, 0, dictBuffer) columnType = dictionary.Type() } @@ -720,18 +724,19 @@ func (wb *writerBuffers) reset() { wb.page = wb.page[:0] } +func encodeLevels(dst, src []byte, maxLevel byte) ([]byte, error) { + bitWidth := bits.Len8(maxLevel) + return levelEncodingsRLE[bitWidth-1].EncodeLevels(dst, src) +} + func (wb *writerBuffers) encodeRepetitionLevels(page BufferedPage, maxRepetitionLevel byte) (err error) { - bitWidth := bits.Len8(maxRepetitionLevel) - encoding := &levelEncodingsRLE[bitWidth-1] - wb.repetitions, err = encoding.EncodeLevels(wb.repetitions[:0], page.RepetitionLevels()) - return err + wb.repetitions, err = encodeLevels(wb.repetitions, page.RepetitionLevels(), maxRepetitionLevel) + return } func (wb *writerBuffers) encodeDefinitionLevels(page BufferedPage, maxDefinitionLevel byte) (err error) { - bitWidth := bits.Len8(maxDefinitionLevel) - encoding := &levelEncodingsRLE[bitWidth-1] - wb.definitions, err = encoding.EncodeLevels(wb.definitions[:0], page.DefinitionLevels()) - return err + wb.definitions, err = encodeLevels(wb.definitions, page.DefinitionLevels(), maxDefinitionLevel) + return } func (wb *writerBuffers) prependLevelsToDataPageV1(maxRepetitionLevel, maxDefinitionLevel byte) {