You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2021/05/22 19:21:19 UTC

[GitHub] [arrow] zeroshade opened a new pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

zeroshade opened a new pull request #10379:
URL: https://github.com/apache/arrow/pull/10379


   Adding the implementation of encoding types of data for Parquet including Plain, RLE, Dictionary, Delta Byte Array, Delta Packing types. It also includes hashing implementation for more efficient hash tables than using go's std map implementation as shown in the benchmarks included in the test files which do benchmark comparisons between a go-map based implementation and the hash table implementation that I ported from the C++ 
   
   In addition, while adding some test cases I discovered that apparently the -force-vector-width=32 argument on the asm generation was causing segfaults on the encoding tests, so let's let LLVM make it's own choice about the vector width and interleaving.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r664579006



##########
File path: go/parquet/internal/encoding/delta_byte_array.go
##########
@@ -0,0 +1,206 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// DeltaByteArrayEncoder is an encoder for writing bytearrays which are delta encoded
+// this is also known as incremental encoding or front compression. For each element
+// in a sequence of strings, we store the prefix length of the previous entry plus the suffix
+// see https://en.wikipedia.org/wiki/Incremental_encoding for a longer description.
+//
+// This is stored as a sequence of delta-encoded prefix lengths followed by the suffixes
+// encoded as delta length byte arrays.
+type DeltaByteArrayEncoder struct {
+	encoder
+
+	prefixEncoder *DeltaBitPackInt32Encoder
+	suffixEncoder *DeltaLengthByteArrayEncoder
+
+	lastVal parquet.ByteArray
+}
+
+func (enc *DeltaByteArrayEncoder) initEncoders() {
+	enc.prefixEncoder = &DeltaBitPackInt32Encoder{
+		deltaBitPackEncoder: &deltaBitPackEncoder{encoder: newEncoderBase(enc.encoding, nil, enc.mem)}}
+	enc.suffixEncoder = &DeltaLengthByteArrayEncoder{
+		newEncoderBase(enc.encoding, nil, enc.mem),
+		&DeltaBitPackInt32Encoder{
+			deltaBitPackEncoder: &deltaBitPackEncoder{encoder: newEncoderBase(enc.encoding, nil, enc.mem)}}}
+}
+
+// Type returns the underlying physical type this operates on, in this case ByteArrays only
+func (DeltaByteArrayEncoder) Type() parquet.Type { return parquet.Types.ByteArray }
+
+// Put writes a slice of ByteArrays to the encoder
+func (enc *DeltaByteArrayEncoder) Put(in []parquet.ByteArray) {
+	if len(in) == 0 {
+		return
+	}
+
+	var suf parquet.ByteArray
+	if enc.prefixEncoder == nil { // initialize our encoders if we haven't yet
+		enc.initEncoders()
+		enc.prefixEncoder.Put([]int32{0})
+		suf = in[0]
+		enc.lastVal = append([]byte(nil), in[0]...)
+		enc.suffixEncoder.Put([]parquet.ByteArray{suf})
+		in = in[1:]
+	}
+
+	// for each value, figure out the common prefix with the previous value
+	// and then write the prefix length and the suffix.
+	for _, val := range in {
+		l1 := enc.lastVal.Len()
+		l2 := val.Len()
+		j := 0
+		for j < l1 && j < l2 {
+			if enc.lastVal[j] != val[j] {
+				break
+			}
+			j++
+		}
+		enc.prefixEncoder.Put([]int32{int32(j)})
+		suf = val[j:]
+		enc.suffixEncoder.Put([]parquet.ByteArray{suf})
+		enc.lastVal = append([]byte(nil), val...)
+	}
+}
+
+// PutSpaced is like Put, but assumes the data is already spaced for nulls and uses the bitmap provided and offset
+// to compress the data before writing it without the null slots.
+func (enc *DeltaByteArrayEncoder) PutSpaced(in []parquet.ByteArray, validBits []byte, validBitsOffset int64) {
+	if validBits != nil {
+		data := make([]parquet.ByteArray, len(in))
+		nvalid := spacedCompress(in, data, validBits, validBitsOffset)
+		enc.Put(data[:nvalid])
+	} else {
+		enc.Put(in)
+	}
+}
+
+// Flush flushes any remaining data out and returns the finished encoded buffer.
+func (enc *DeltaByteArrayEncoder) FlushValues() Buffer {
+	if enc.prefixEncoder == nil {
+		enc.initEncoders()
+	}
+	prefixBuf := enc.prefixEncoder.FlushValues()
+	defer prefixBuf.Release()
+
+	suffixBuf := enc.suffixEncoder.FlushValues()
+	defer suffixBuf.Release()
+
+	ret := bufferPool.Get().(*memory.Buffer)
+	ret.ResizeNoShrink(prefixBuf.Len() + suffixBuf.Len())
+	copy(ret.Bytes(), prefixBuf.Bytes())
+	copy(ret.Bytes()[prefixBuf.Len():], suffixBuf.Bytes())
+	return poolBuffer{ret}
+}
+
+// DeltaByteArrayDecoder is a decoder for a column of data encoded using incremental or prefix encoding.
+type DeltaByteArrayDecoder struct {
+	*DeltaLengthByteArrayDecoder
+
+	prefixLengths []int32
+	lastVal       parquet.ByteArray
+}
+
+// Type returns the underlying physical type this decoder operates on, in this case ByteArrays only
+func (DeltaByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+func (d *DeltaByteArrayDecoder) Allocator() memory.Allocator { return d.mem }
+
+// SetData expects the data passed in to be the prefix lengths, followed by the
+// blocks of suffix data in order to initialize the decoder.
+func (d *DeltaByteArrayDecoder) SetData(nvalues int, data []byte) error {
+	prefixLenDec := DeltaBitPackInt32Decoder{
+		deltaBitPackDecoder: &deltaBitPackDecoder{
+			decoder: newDecoderBase(d.encoding, d.descr),
+			mem:     d.mem}}
+
+	if err := prefixLenDec.SetData(nvalues, data); err != nil {
+		return err
+	}
+
+	d.prefixLengths = make([]int32, nvalues)
+	// decode all the prefix lengths first so we know how many bytes it took to get the
+	// prefix lengths for nvalues
+	prefixLenDec.Decode(d.prefixLengths)
+
+	// now that we know how many bytes we needed for the prefix lengths, the rest are the
+	// delta length byte array encoding.
+	return d.DeltaLengthByteArrayDecoder.SetData(nvalues, data[int(prefixLenDec.bytesRead()):])
+}
+
+// Decode decodes byte arrays into the slice provided and returns the number of values actually decoded
+func (d *DeltaByteArrayDecoder) Decode(out []parquet.ByteArray) (int, error) {
+	max := utils.MinInt(len(out), d.nvals)
+	if max == 0 {
+		return 0, nil
+	}
+	out = out[:max]
+
+	var err error
+	if d.lastVal == nil {
+		_, err = d.DeltaLengthByteArrayDecoder.Decode(out[:1])
+		if err != nil {
+			return 0, err
+		}
+		d.lastVal = out[0]
+		out = out[1:]
+		d.prefixLengths = d.prefixLengths[1:]

Review comment:
       zero is stored as the first length, look at line 63 in this file for the encoder. When initializing the encoder, we write 0 as the first length. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r664583968



##########
File path: go/parquet/internal/encoding/delta_byte_array.go
##########
@@ -0,0 +1,206 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// DeltaByteArrayEncoder is an encoder for writing bytearrays which are delta encoded
+// this is also known as incremental encoding or front compression. For each element
+// in a sequence of strings, we store the prefix length of the previous entry plus the suffix
+// see https://en.wikipedia.org/wiki/Incremental_encoding for a longer description.
+//
+// This is stored as a sequence of delta-encoded prefix lengths followed by the suffixes
+// encoded as delta length byte arrays.
+type DeltaByteArrayEncoder struct {
+	encoder
+
+	prefixEncoder *DeltaBitPackInt32Encoder
+	suffixEncoder *DeltaLengthByteArrayEncoder
+
+	lastVal parquet.ByteArray
+}
+
+func (enc *DeltaByteArrayEncoder) initEncoders() {
+	enc.prefixEncoder = &DeltaBitPackInt32Encoder{
+		deltaBitPackEncoder: &deltaBitPackEncoder{encoder: newEncoderBase(enc.encoding, nil, enc.mem)}}
+	enc.suffixEncoder = &DeltaLengthByteArrayEncoder{
+		newEncoderBase(enc.encoding, nil, enc.mem),
+		&DeltaBitPackInt32Encoder{
+			deltaBitPackEncoder: &deltaBitPackEncoder{encoder: newEncoderBase(enc.encoding, nil, enc.mem)}}}
+}
+
+// Type returns the underlying physical type this operates on, in this case ByteArrays only
+func (DeltaByteArrayEncoder) Type() parquet.Type { return parquet.Types.ByteArray }
+
+// Put writes a slice of ByteArrays to the encoder
+func (enc *DeltaByteArrayEncoder) Put(in []parquet.ByteArray) {
+	if len(in) == 0 {
+		return
+	}
+
+	var suf parquet.ByteArray
+	if enc.prefixEncoder == nil { // initialize our encoders if we haven't yet
+		enc.initEncoders()
+		enc.prefixEncoder.Put([]int32{0})
+		suf = in[0]
+		enc.lastVal = append([]byte(nil), in[0]...)
+		enc.suffixEncoder.Put([]parquet.ByteArray{suf})
+		in = in[1:]
+	}
+
+	// for each value, figure out the common prefix with the previous value
+	// and then write the prefix length and the suffix.
+	for _, val := range in {
+		l1 := enc.lastVal.Len()
+		l2 := val.Len()
+		j := 0
+		for j < l1 && j < l2 {
+			if enc.lastVal[j] != val[j] {
+				break
+			}
+			j++
+		}
+		enc.prefixEncoder.Put([]int32{int32(j)})
+		suf = val[j:]
+		enc.suffixEncoder.Put([]parquet.ByteArray{suf})
+		enc.lastVal = append([]byte(nil), val...)
+	}
+}
+
+// PutSpaced is like Put, but assumes the data is already spaced for nulls and uses the bitmap provided and offset
+// to compress the data before writing it without the null slots.
+func (enc *DeltaByteArrayEncoder) PutSpaced(in []parquet.ByteArray, validBits []byte, validBitsOffset int64) {
+	if validBits != nil {
+		data := make([]parquet.ByteArray, len(in))
+		nvalid := spacedCompress(in, data, validBits, validBitsOffset)
+		enc.Put(data[:nvalid])
+	} else {
+		enc.Put(in)
+	}
+}
+
+// Flush flushes any remaining data out and returns the finished encoded buffer.
+func (enc *DeltaByteArrayEncoder) FlushValues() Buffer {
+	if enc.prefixEncoder == nil {
+		enc.initEncoders()
+	}
+	prefixBuf := enc.prefixEncoder.FlushValues()
+	defer prefixBuf.Release()
+
+	suffixBuf := enc.suffixEncoder.FlushValues()
+	defer suffixBuf.Release()
+
+	ret := bufferPool.Get().(*memory.Buffer)
+	ret.ResizeNoShrink(prefixBuf.Len() + suffixBuf.Len())
+	copy(ret.Bytes(), prefixBuf.Bytes())
+	copy(ret.Bytes()[prefixBuf.Len():], suffixBuf.Bytes())
+	return poolBuffer{ret}
+}
+
+// DeltaByteArrayDecoder is a decoder for a column of data encoded using incremental or prefix encoding.
+type DeltaByteArrayDecoder struct {
+	*DeltaLengthByteArrayDecoder
+
+	prefixLengths []int32
+	lastVal       parquet.ByteArray
+}
+
+// Type returns the underlying physical type this decoder operates on, in this case ByteArrays only
+func (DeltaByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+func (d *DeltaByteArrayDecoder) Allocator() memory.Allocator { return d.mem }
+
+// SetData expects the data passed in to be the prefix lengths, followed by the
+// blocks of suffix data in order to initialize the decoder.
+func (d *DeltaByteArrayDecoder) SetData(nvalues int, data []byte) error {
+	prefixLenDec := DeltaBitPackInt32Decoder{
+		deltaBitPackDecoder: &deltaBitPackDecoder{
+			decoder: newDecoderBase(d.encoding, d.descr),
+			mem:     d.mem}}
+
+	if err := prefixLenDec.SetData(nvalues, data); err != nil {
+		return err
+	}
+
+	d.prefixLengths = make([]int32, nvalues)
+	// decode all the prefix lengths first so we know how many bytes it took to get the
+	// prefix lengths for nvalues
+	prefixLenDec.Decode(d.prefixLengths)
+
+	// now that we know how many bytes we needed for the prefix lengths, the rest are the
+	// delta length byte array encoding.
+	return d.DeltaLengthByteArrayDecoder.SetData(nvalues, data[int(prefixLenDec.bytesRead()):])
+}
+
+// Decode decodes byte arrays into the slice provided and returns the number of values actually decoded
+func (d *DeltaByteArrayDecoder) Decode(out []parquet.ByteArray) (int, error) {
+	max := utils.MinInt(len(out), d.nvals)
+	if max == 0 {
+		return 0, nil
+	}
+	out = out[:max]
+
+	var err error
+	if d.lastVal == nil {
+		_, err = d.DeltaLengthByteArrayDecoder.Decode(out[:1])
+		if err != nil {
+			return 0, err
+		}
+		d.lastVal = out[0]
+		out = out[1:]
+		d.prefixLengths = d.prefixLengths[1:]
+	}
+
+	var prefixLen int32
+	suffixHolder := make([]parquet.ByteArray, 1)
+	for len(out) > 0 {
+		prefixLen, d.prefixLengths = d.prefixLengths[0], d.prefixLengths[1:]
+
+		prefix := d.lastVal[:prefixLen]
+		_, err = d.DeltaLengthByteArrayDecoder.Decode(suffixHolder)
+		if err != nil {
+			return 0, err
+		}
+
+		d.lastVal = make([]byte, 0, int(prefixLen)+len(suffixHolder[0]))
+		d.lastVal = append([]byte{}, prefix...)
+		d.lastVal = append(d.lastVal, suffixHolder[0]...)
+		out[0], out = d.lastVal, out[1:]

Review comment:
       A slice is three values:
   
   1. A pointer to where the data starts
   2. An integer holding the length of the slice
   3. An integer holding the total allocated space for that slice (the capacity)
   
   Slicing is no more expensive than a C++ case of maintaining a pointer + length, explicitly designed to be extremely efficient, think of them as a view on an array.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#issuecomment-863620823


   > @emkornfield so because of the way things are developed and interdependent, the most i'm able to remove while still being compileable in this PR is the unit tests, the level encoder, the memo table code and the hashing stuff for those memo tables. All in all 13 files
   
   if you don't mind that would be great, thats ~1/3 of the PR


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r644929054



##########
File path: go/parquet/internal/encoding/byte_array_decoder.go
##########
@@ -0,0 +1,84 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// PlainByteArrayDecoder decodes a data chunk for bytearrays according to
+// the plain encoding. The byte arrays will use slices to reference the
+// data rather than copying it.
+type PlainByteArrayDecoder struct {
+	decoder
+}
+
+// Type returns parquet.Types.ByteArray for this decoder
+func (PlainByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+// Decode will populate the slice of bytearrays in full or until the number
+// of values is emptied.
+//
+// Returns the number of values that were decoded.
+func (pbad *PlainByteArrayDecoder) Decode(out []parquet.ByteArray) (int, error) {
+	max := utils.MinInt(len(out), pbad.nvals)
+
+	for i := 0; i < max; i++ {
+		// there should always be at least four bytes which is the length of the
+		// next value in the data.
+		if len(pbad.data) < 4 {
+			return i, xerrors.New("parquet: eof reading bytearray")
+		}
+
+		// the first 4 bytes are a little endian uint32 length

Review comment:
       it seems that you also check below whether it is less then zero which would be a good indication that it is signed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r648390799



##########
File path: go/parquet/internal/encoding/delta_bit_packing.go
##########
@@ -0,0 +1,514 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"bytes"
+	"math"
+	"math/bits"
+	"reflect"
+
+	"github.com/apache/arrow/go/arrow"
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// see the deltaBitPack encoder for a description of the encoding format that is
+// used for delta-bitpacking.
+type deltaBitPackDecoder struct {
+	decoder
+
+	mem memory.Allocator
+
+	usedFirst            bool
+	bitdecoder           *utils.BitReader
+	blockSize            uint64
+	currentBlockVals     uint64
+	miniBlocks           uint64
+	valsPerMini          uint64
+	currentMiniBlockVals uint64
+	minDelta             int64
+	miniBlockIdx         uint64
+
+	deltaBitWidths *memory.Buffer
+	deltaBitWidth  byte
+
+	lastVal int64
+}
+
+// returns the number of bytes read so far
+func (d *deltaBitPackDecoder) bytesRead() int64 {
+	return d.bitdecoder.CurOffset()
+}
+
+func (d *deltaBitPackDecoder) Allocator() memory.Allocator { return d.mem }
+
+// SetData sets in the data to be decoded and the expected number of values to decode

Review comment:
       reworded, let me know your opinion on the new wording.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r644087064



##########
File path: go/parquet/internal/encoding/boolean_decoder.go
##########
@@ -0,0 +1,98 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"github.com/apache/arrow/go/arrow/bitutil"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+)
+
+// PlainBooleanDecoder is for the Plain Encoding type, there is no
+// dictionary decoding for bools.
+type PlainBooleanDecoder struct {
+	decoder
+
+	bitOffset int
+}
+
+// Type for the PlainBooleanDecoder is parquet.Types.Boolean
+func (PlainBooleanDecoder) Type() parquet.Type {
+	return parquet.Types.Boolean
+}
+
+// Decode fills out with bools decoded from the data at the current point
+// or until we reach the end of the data.
+//
+// Returns the number of values decoded
+func (dec *PlainBooleanDecoder) Decode(out []bool) (int, error) {
+	max := utils.MinInt(len(out), dec.nvals)
+
+	// if we aren't at a byte boundary, then get bools until we hit
+	// a byte boundary with the bit offset.
+	i := 0
+	for dec.bitOffset != 0 && dec.bitOffset < 8 && i < max {
+		out[i] = (dec.data[0] & byte(1<<dec.bitOffset)) != 0
+		dec.bitOffset++
+		i++
+	}
+	if dec.bitOffset == 8 {
+		dec.bitOffset = 0
+	}
+
+	// determine the number of full bytes worth of bits we can decode
+	// given the number of values we want to decode.
+	bitsRemain := max - i
+	batch := bitsRemain / 8 * 8
+	if batch > 0 { // only go in here if there's at least one full byte to decode
+		if i > 0 { // skip our data forward if we decoded anything above
+			dec.data = dec.data[1:]
+			out = out[i:]
+		}
+		// determine the number of aligned bytes we can grab using SIMD optimized
+		// functions to improve performance.
+		alignedBytes := bitutil.BytesForBits(int64(batch))
+		utils.BytesToBools(dec.data[:alignedBytes], out)
+		dec.data = dec.data[alignedBytes:]
+		out = out[alignedBytes*8:]
+	}
+
+	// grab any trailing bits now that we've got our aligned bytes.
+	for ; dec.bitOffset < (bitsRemain - batch); dec.bitOffset++ {
+		out[dec.bitOffset] = (dec.data[0] & byte(1<<dec.bitOffset)) != 0

Review comment:
       might be nice factor out the unaligned bit extract?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r664710540



##########
File path: go/parquet/internal/encoding/plain_encoder_types.gen.go
##########
@@ -0,0 +1,553 @@
+// Code generated by plain_encoder_types.gen.go.tmpl. DO NOT EDIT.
+
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"math"
+
+	"github.com/apache/arrow/go/arrow"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// PlainInt32Encoder is an encoder for int32 values using Plain Encoding
+// which in general is just storing the values as raw bytes of the appropriate size
+type PlainInt32Encoder struct {
+	encoder
+
+	bitSetReader utils.SetBitRunReader
+}
+
+// Put encodes a slice of values into the underlying buffer
+func (enc *PlainInt32Encoder) Put(in []int32) {
+	enc.append(arrow.Int32Traits.CastToBytes(in))

Review comment:
       added `write{{Type}}LE` and `copyFrom{{Type}}LE` functions for numeric types so that the library properly handles BigEndian in addition to LittleEndian now.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r644132276



##########
File path: go/parquet/internal/encoding/boolean_encoder.go
##########
@@ -0,0 +1,112 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"github.com/apache/arrow/go/arrow/bitutil"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+)
+
+const boolBufSize = 1024
+
+// PlainBooleanEncoder encodes bools as a bitmap as per the Plain Encoding
+type PlainBooleanEncoder struct {
+	encoder
+	nbits      int
+	bitsBuffer []byte
+}
+
+// Type for the PlainBooleanEncoder is parquet.Types.Boolean
+func (PlainBooleanEncoder) Type() parquet.Type {
+	return parquet.Types.Boolean
+}
+
+// Put encodes the contents of in into the underlying data buffer.
+func (enc *PlainBooleanEncoder) Put(in []bool) {
+	if enc.bitsBuffer == nil {
+		enc.bitsBuffer = make([]byte, boolBufSize)
+	}
+
+	bitOffset := 0
+	// first check if we are in the middle of a byte due to previous
+	// encoding of data and finish out that byte's bits.
+	if enc.nbits > 0 {
+		bitsToWrite := utils.MinInt(enc.nbits, len(in))
+		beg := (boolBufSize * 8) - enc.nbits
+		for i, val := range in[:bitsToWrite] {
+			bitmask := uint8(1 << uint((beg+i)%8))
+			if val {
+				enc.bitsBuffer[(beg+i)/8] |= bitmask
+			} else {
+				enc.bitsBuffer[(beg+i)/8] &= bitmask ^ 0xFF

Review comment:
       huh, you're right, i didn't even realize this was just a bitwise not.
   
   So Go doesn't have a `~` operator, but you can use the XOR operator to do a unary bitwise NOT. So this could just be `^bitmask` i'll fix that.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r644961073



##########
File path: go/parquet/internal/encoding/byte_array_decoder.go
##########
@@ -0,0 +1,84 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// PlainByteArrayDecoder decodes a data chunk for bytearrays according to
+// the plain encoding. The byte arrays will use slices to reference the
+// data rather than copying it.
+type PlainByteArrayDecoder struct {
+	decoder
+}
+
+// Type returns parquet.Types.ByteArray for this decoder
+func (PlainByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+// Decode will populate the slice of bytearrays in full or until the number
+// of values is emptied.
+//
+// Returns the number of values that were decoded.
+func (pbad *PlainByteArrayDecoder) Decode(out []parquet.ByteArray) (int, error) {
+	max := utils.MinInt(len(out), pbad.nvals)
+
+	for i := 0; i < max; i++ {
+		// there should always be at least four bytes which is the length of the
+		// next value in the data.
+		if len(pbad.data) < 4 {
+			return i, xerrors.New("parquet: eof reading bytearray")
+		}
+
+		// the first 4 bytes are a little endian uint32 length

Review comment:
       So the parquet-format spec only says "length in 4 bytes little endian" and does not specify signed or unsigned and the C++ implementation uses an unsigned int32 which is why I used uint32. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r644345648



##########
File path: go/parquet/internal/encoding/boolean_encoder.go
##########
@@ -0,0 +1,112 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"github.com/apache/arrow/go/arrow/bitutil"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+)
+
+const boolBufSize = 1024
+
+// PlainBooleanEncoder encodes bools as a bitmap as per the Plain Encoding
+type PlainBooleanEncoder struct {
+	encoder
+	nbits      int
+	bitsBuffer []byte
+}
+
+// Type for the PlainBooleanEncoder is parquet.Types.Boolean
+func (PlainBooleanEncoder) Type() parquet.Type {
+	return parquet.Types.Boolean
+}
+
+// Put encodes the contents of in into the underlying data buffer.
+func (enc *PlainBooleanEncoder) Put(in []bool) {
+	if enc.bitsBuffer == nil {
+		enc.bitsBuffer = make([]byte, boolBufSize)
+	}
+
+	bitOffset := 0
+	// first check if we are in the middle of a byte due to previous
+	// encoding of data and finish out that byte's bits.
+	if enc.nbits > 0 {
+		bitsToWrite := utils.MinInt(enc.nbits, len(in))
+		beg := (boolBufSize * 8) - enc.nbits
+		for i, val := range in[:bitsToWrite] {
+			bitmask := uint8(1 << uint((beg+i)%8))
+			if val {
+				enc.bitsBuffer[(beg+i)/8] |= bitmask

Review comment:
       So the combination of the changes I made to how I handle the bitmapwriter and utilize an `AppendBools` function resulted in about a 40% improvement in speed of encoding bools! It also turns out that Go definitely does optimize the /8 and the *8 as far as i can tell since there was very little performance difference between using them or the shifts directly. 
   
   The primary benefits that resulted in the performance improvements came from switching to use `bitutil.SetBit` / `bitutil.ClearBit` since they used the constant slices with the bitmasks that we index into rather than constructing them on the fly as it converted a modulus+shift operation into a simple index lookup inside of a tight loop.
   
   The big reason why it needed an AppendBools function instead of just using a loop with `Next()`, `Set()`, and `Clear()` is because `Next()` has to check for bitmask being 0 to update it along with the offset and position tracking, whereas having the function inside there allows for a single update of the bookkeeping and just looping through to set the bits. Either way, the result is a significant performance improvement so I'm happy :smiley:

##########
File path: go/parquet/internal/encoding/boolean_decoder.go
##########
@@ -0,0 +1,98 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"github.com/apache/arrow/go/arrow/bitutil"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+)
+
+// PlainBooleanDecoder is for the Plain Encoding type, there is no
+// dictionary decoding for bools.
+type PlainBooleanDecoder struct {
+	decoder
+
+	bitOffset int
+}
+
+// Type for the PlainBooleanDecoder is parquet.Types.Boolean
+func (PlainBooleanDecoder) Type() parquet.Type {
+	return parquet.Types.Boolean
+}
+
+// Decode fills out with bools decoded from the data at the current point
+// or until we reach the end of the data.
+//
+// Returns the number of values decoded
+func (dec *PlainBooleanDecoder) Decode(out []bool) (int, error) {
+	max := utils.MinInt(len(out), dec.nvals)
+
+	// if we aren't at a byte boundary, then get bools until we hit
+	// a byte boundary with the bit offset.
+	i := 0
+	for dec.bitOffset != 0 && dec.bitOffset < 8 && i < max {
+		out[i] = (dec.data[0] & byte(1<<dec.bitOffset)) != 0
+		dec.bitOffset++
+		i++
+	}
+	if dec.bitOffset == 8 {
+		dec.bitOffset = 0
+	}
+
+	// determine the number of full bytes worth of bits we can decode
+	// given the number of values we want to decode.
+	bitsRemain := max - i
+	batch := bitsRemain / 8 * 8
+	if batch > 0 { // only go in here if there's at least one full byte to decode
+		if i > 0 { // skip our data forward if we decoded anything above
+			dec.data = dec.data[1:]
+			out = out[i:]
+		}
+		// determine the number of aligned bytes we can grab using SIMD optimized
+		// functions to improve performance.
+		alignedBytes := bitutil.BytesForBits(int64(batch))
+		utils.BytesToBools(dec.data[:alignedBytes], out)
+		dec.data = dec.data[alignedBytes:]
+		out = out[alignedBytes*8:]
+	}
+
+	// grab any trailing bits now that we've got our aligned bytes.
+	for ; dec.bitOffset < (bitsRemain - batch); dec.bitOffset++ {
+		out[dec.bitOffset] = (dec.data[0] & byte(1<<dec.bitOffset)) != 0
+	}
+
+	dec.nvals -= max
+	return max, nil
+}
+
+// DecodeSpaced is like Decode except it expands the values to leave spaces for null
+// as determined by the validBits bitmap.
+func (dec *PlainBooleanDecoder) DecodeSpaced(out []bool, nullCount int, validBits []byte, validBitsOffset int64) (int, error) {
+	if nullCount > 0 {
+		toRead := len(out) - nullCount
+		valuesRead, err := dec.Decode(out[:toRead])
+		if err != nil {
+			return 0, err
+		}
+		if valuesRead != toRead {
+			panic("parquet: number of values / definition levels read did not match")

Review comment:
       updated to return an error, that's a good point.

##########
File path: go/parquet/internal/encoding/boolean_decoder.go
##########
@@ -0,0 +1,98 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"github.com/apache/arrow/go/arrow/bitutil"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+)
+
+// PlainBooleanDecoder is for the Plain Encoding type, there is no
+// dictionary decoding for bools.
+type PlainBooleanDecoder struct {
+	decoder
+
+	bitOffset int
+}
+
+// Type for the PlainBooleanDecoder is parquet.Types.Boolean
+func (PlainBooleanDecoder) Type() parquet.Type {
+	return parquet.Types.Boolean
+}
+
+// Decode fills out with bools decoded from the data at the current point
+// or until we reach the end of the data.
+//
+// Returns the number of values decoded
+func (dec *PlainBooleanDecoder) Decode(out []bool) (int, error) {
+	max := utils.MinInt(len(out), dec.nvals)
+
+	// if we aren't at a byte boundary, then get bools until we hit
+	// a byte boundary with the bit offset.
+	i := 0
+	for dec.bitOffset != 0 && dec.bitOffset < 8 && i < max {
+		out[i] = (dec.data[0] & byte(1<<dec.bitOffset)) != 0
+		dec.bitOffset++
+		i++
+	}
+	if dec.bitOffset == 8 {
+		dec.bitOffset = 0
+	}
+
+	// determine the number of full bytes worth of bits we can decode
+	// given the number of values we want to decode.
+	bitsRemain := max - i
+	batch := bitsRemain / 8 * 8
+	if batch > 0 { // only go in here if there's at least one full byte to decode
+		if i > 0 { // skip our data forward if we decoded anything above
+			dec.data = dec.data[1:]
+			out = out[i:]
+		}
+		// determine the number of aligned bytes we can grab using SIMD optimized
+		// functions to improve performance.
+		alignedBytes := bitutil.BytesForBits(int64(batch))
+		utils.BytesToBools(dec.data[:alignedBytes], out)
+		dec.data = dec.data[alignedBytes:]
+		out = out[alignedBytes*8:]
+	}
+
+	// grab any trailing bits now that we've got our aligned bytes.
+	for ; dec.bitOffset < (bitsRemain - batch); dec.bitOffset++ {
+		out[dec.bitOffset] = (dec.data[0] & byte(1<<dec.bitOffset)) != 0

Review comment:
       done.

##########
File path: go/parquet/internal/encoding/byte_array_decoder.go
##########
@@ -0,0 +1,84 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// PlainByteArrayDecoder decodes a data chunk for bytearrays according to
+// the plain encoding. The byte arrays will use slices to reference the
+// data rather than copying it.
+type PlainByteArrayDecoder struct {
+	decoder
+}
+
+// Type returns parquet.Types.ByteArray for this decoder
+func (PlainByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+// Decode will populate the slice of bytearrays in full or until the number
+// of values is emptied.
+//
+// Returns the number of values that were decoded.
+func (pbad *PlainByteArrayDecoder) Decode(out []parquet.ByteArray) (int, error) {
+	max := utils.MinInt(len(out), pbad.nvals)
+
+	for i := 0; i < max; i++ {
+		// there should always be at least four bytes which is the length of the
+		// next value in the data.
+		if len(pbad.data) < 4 {
+			return i, xerrors.New("parquet: eof reading bytearray")
+		}
+
+		// the first 4 bytes are a little endian uint32 length

Review comment:
       So the parquet-format spec only says "length in 4 bytes little endian" and does not specify signed or unsigned and the C++ implementation uses an unsigned int32 which is why I used uint32. 

##########
File path: go/parquet/internal/encoding/byte_array_decoder.go
##########
@@ -0,0 +1,84 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// PlainByteArrayDecoder decodes a data chunk for bytearrays according to
+// the plain encoding. The byte arrays will use slices to reference the
+// data rather than copying it.
+type PlainByteArrayDecoder struct {
+	decoder
+}
+
+// Type returns parquet.Types.ByteArray for this decoder
+func (PlainByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+// Decode will populate the slice of bytearrays in full or until the number
+// of values is emptied.
+//
+// Returns the number of values that were decoded.
+func (pbad *PlainByteArrayDecoder) Decode(out []parquet.ByteArray) (int, error) {
+	max := utils.MinInt(len(out), pbad.nvals)
+
+	for i := 0; i < max; i++ {
+		// there should always be at least four bytes which is the length of the
+		// next value in the data.
+		if len(pbad.data) < 4 {
+			return i, xerrors.New("parquet: eof reading bytearray")
+		}
+
+		// the first 4 bytes are a little endian uint32 length
+		nbytes := int32(binary.LittleEndian.Uint32(pbad.data[:4]))
+		if nbytes < 0 {
+			return i, xerrors.New("parquet: invalid BYTE_ARRAY value")
+		}
+
+		if int64(len(pbad.data)) < int64(nbytes)+4 {
+			return i, xerrors.New("parquet: eof reading bytearray")
+		}
+
+		out[i] = pbad.data[4 : nbytes+4]

Review comment:
       this is a view (in Go terms it's called a *slice*). It essentially creates an object which contains the pointer to the first byte, then length of the slice (so `nbytes`), and the capacity (the rest of the capacity that exists in `pbad.data`. 
   
   In theory it might actually be better to specify the capacity in this slice so that a consumer couldn't expand the individual slice and clobber other data. I'll make that update.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r647569748



##########
File path: go/parquet/internal/encoding/delta_bit_packing.go
##########
@@ -0,0 +1,514 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"bytes"
+	"math"
+	"math/bits"
+	"reflect"
+
+	"github.com/apache/arrow/go/arrow"
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// see the deltaBitPack encoder for a description of the encoding format that is
+// used for delta-bitpacking.
+type deltaBitPackDecoder struct {
+	decoder
+
+	mem memory.Allocator
+
+	usedFirst            bool
+	bitdecoder           *utils.BitReader
+	blockSize            uint64
+	currentBlockVals     uint64
+	miniBlocks           uint64
+	valsPerMini          uint64
+	currentMiniBlockVals uint64
+	minDelta             int64
+	miniBlockIdx         uint64
+
+	deltaBitWidths *memory.Buffer
+	deltaBitWidth  byte
+
+	lastVal int64
+}
+
+// returns the number of bytes read so far
+func (d *deltaBitPackDecoder) bytesRead() int64 {
+	return d.bitdecoder.CurOffset()
+}
+
+func (d *deltaBitPackDecoder) Allocator() memory.Allocator { return d.mem }
+
+// SetData sets in the data to be decoded and the expected number of values to decode

Review comment:
       "sets in the data" does quite make sense to me.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#issuecomment-859637836


   bump @emkornfield @fsaintjacques 
   
   thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#issuecomment-847495023


   @zeroshade apologies this is likely going to be another rough week in terms of availability to review.  If you don't hear anything by next tuesday please ping again.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r653979156



##########
File path: go/parquet/internal/encoding/decoder.go
##########
@@ -0,0 +1,180 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"bytes"
+	"reflect"
+
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/debug"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"github.com/apache/arrow/go/parquet/schema"
+)
+
+// DecoderTraits provides an interface for more easily interacting with types
+// to generate decoders for specific types.
+type DecoderTraits interface {
+	Decoder(parquet.Encoding, *schema.Column, bool, memory.Allocator) TypedDecoder
+	BytesRequired(int) int
+}
+
+// NewDecoder constructs a decoder for a given type and encoding
+func NewDecoder(t parquet.Type, e parquet.Encoding, descr *schema.Column, mem memory.Allocator) TypedDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	return traits.Decoder(e, descr, false, mem)
+}
+
+// NewDictDecoder is like NewDecoder but for dictionary encodings, panics if type is bool.

Review comment:
       I could definitely add a place above this in the code that will call this to recover from the panic and return an error, or just return the error here. I'll take a look and see which makes the most sense as far as code readability.

##########
File path: go/parquet/internal/encoding/decoder.go
##########
@@ -0,0 +1,180 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"bytes"
+	"reflect"
+
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/debug"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"github.com/apache/arrow/go/parquet/schema"
+)
+
+// DecoderTraits provides an interface for more easily interacting with types
+// to generate decoders for specific types.
+type DecoderTraits interface {
+	Decoder(parquet.Encoding, *schema.Column, bool, memory.Allocator) TypedDecoder
+	BytesRequired(int) int
+}
+
+// NewDecoder constructs a decoder for a given type and encoding
+func NewDecoder(t parquet.Type, e parquet.Encoding, descr *schema.Column, mem memory.Allocator) TypedDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	return traits.Decoder(e, descr, false, mem)
+}
+
+// NewDictDecoder is like NewDecoder but for dictionary encodings, panics if type is bool.
+//
+// if mem is nil, memory.DefaultAllocator will be used
+func NewDictDecoder(t parquet.Type, descr *schema.Column, mem memory.Allocator) DictDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	if mem == nil {
+		mem = memory.DefaultAllocator
+	}
+
+	return traits.Decoder(parquet.Encodings.RLEDict, descr, true, mem).(DictDecoder)
+}
+
+type decoder struct {
+	descr    *schema.Column
+	encoding format.Encoding
+	nvals    int
+	data     []byte
+	typeLen  int
+}
+
+// newDecoderBase constructs the base decoding object that is embedded in the
+// type specific decoders.
+func newDecoderBase(e format.Encoding, descr *schema.Column) decoder {
+	typeLen := -1
+	if descr != nil && descr.PhysicalType() == parquet.Types.FixedLenByteArray {
+		typeLen = int(descr.TypeLength())
+	}
+
+	return decoder{
+		descr:    descr,
+		encoding: e,
+		typeLen:  typeLen,
+	}
+}
+
+// SetData sets the data for decoding into the decoder to update the available
+// data bytes and number of values available.
+func (d *decoder) SetData(nvals int, data []byte) {
+	d.data = data
+	d.nvals = nvals
+}
+
+// ValuesLeft returns the number of remaining values that can be decoded
+func (d *decoder) ValuesLeft() int { return d.nvals }
+
+// Encoding returns the encoding type used by this decoder to decode the bytes.
+func (d *decoder) Encoding() parquet.Encoding { return parquet.Encoding(d.encoding) }
+
+type dictDecoder struct {
+	decoder
+	mem              memory.Allocator
+	dictValueDecoder utils.DictionaryConverter
+	idxDecoder       *utils.RleDecoder
+}
+
+// SetDict sets a decoder that can be used to decode the dictionary that is
+// used for this column in order to return the proper values.
+func (d *dictDecoder) SetDict(dict TypedDecoder) {
+	if dict.Type() != d.descr.PhysicalType() {
+		panic("parquet: mismatch dictionary and column data type")
+	}
+
+	d.dictValueDecoder = NewDictConverter(dict)
+}
+
+// SetData sets the index value data into the decoder.
+func (d *dictDecoder) SetData(nvals int, data []byte) {
+	d.nvals = nvals
+	if len(data) == 0 {
+		d.idxDecoder = utils.NewRleDecoder(bytes.NewReader(data), 1)
+		return
+	}
+
+	width := uint8(data[0])
+	if width >= 64 {
+		panic("parquet: invalid or corrupted bit width")

Review comment:
       the common rules of thumb for panic vs returning an error are along the lines of catastrophic un-recoverable errors (such as programmer errors ie: dereferencing nil, or going out of bounds on a slice) for panics, and to avoid using panic's as a flow control mechanism like say, exceptions in python. It should be for rare occurrences that are truly exceptional.....
   
   However, you can also end up with extremely overly verbose code if every thing you do has to handle an error output, and even the go standard library packages will opt to use panics sometimes in order to make code cleaner or more readable.
   
   So i guess the good question here to have is when we fail, do we want to propagate an error all the way up the stack or should we just stick a recover at a higher level and throw panics from below.
   
   For now, this particular function is easy to convert to returning an error without impacting much readability or verbosity of the code.

##########
File path: go/parquet/internal/encoding/decoder.go
##########
@@ -0,0 +1,180 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"bytes"
+	"reflect"
+
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/debug"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"github.com/apache/arrow/go/parquet/schema"
+)
+
+// DecoderTraits provides an interface for more easily interacting with types
+// to generate decoders for specific types.
+type DecoderTraits interface {
+	Decoder(parquet.Encoding, *schema.Column, bool, memory.Allocator) TypedDecoder
+	BytesRequired(int) int
+}
+
+// NewDecoder constructs a decoder for a given type and encoding
+func NewDecoder(t parquet.Type, e parquet.Encoding, descr *schema.Column, mem memory.Allocator) TypedDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	return traits.Decoder(e, descr, false, mem)
+}
+
+// NewDictDecoder is like NewDecoder but for dictionary encodings, panics if type is bool.
+//
+// if mem is nil, memory.DefaultAllocator will be used
+func NewDictDecoder(t parquet.Type, descr *schema.Column, mem memory.Allocator) DictDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	if mem == nil {
+		mem = memory.DefaultAllocator
+	}
+
+	return traits.Decoder(parquet.Encodings.RLEDict, descr, true, mem).(DictDecoder)
+}
+
+type decoder struct {
+	descr    *schema.Column
+	encoding format.Encoding
+	nvals    int
+	data     []byte
+	typeLen  int
+}
+
+// newDecoderBase constructs the base decoding object that is embedded in the
+// type specific decoders.
+func newDecoderBase(e format.Encoding, descr *schema.Column) decoder {
+	typeLen := -1
+	if descr != nil && descr.PhysicalType() == parquet.Types.FixedLenByteArray {
+		typeLen = int(descr.TypeLength())
+	}
+
+	return decoder{
+		descr:    descr,
+		encoding: e,
+		typeLen:  typeLen,
+	}
+}
+
+// SetData sets the data for decoding into the decoder to update the available
+// data bytes and number of values available.
+func (d *decoder) SetData(nvals int, data []byte) {
+	d.data = data
+	d.nvals = nvals
+}
+
+// ValuesLeft returns the number of remaining values that can be decoded
+func (d *decoder) ValuesLeft() int { return d.nvals }
+
+// Encoding returns the encoding type used by this decoder to decode the bytes.
+func (d *decoder) Encoding() parquet.Encoding { return parquet.Encoding(d.encoding) }
+
+type dictDecoder struct {
+	decoder
+	mem              memory.Allocator
+	dictValueDecoder utils.DictionaryConverter
+	idxDecoder       *utils.RleDecoder
+}
+
+// SetDict sets a decoder that can be used to decode the dictionary that is
+// used for this column in order to return the proper values.
+func (d *dictDecoder) SetDict(dict TypedDecoder) {
+	if dict.Type() != d.descr.PhysicalType() {
+		panic("parquet: mismatch dictionary and column data type")
+	}
+
+	d.dictValueDecoder = NewDictConverter(dict)
+}
+
+// SetData sets the index value data into the decoder.
+func (d *dictDecoder) SetData(nvals int, data []byte) {
+	d.nvals = nvals
+	if len(data) == 0 {
+		d.idxDecoder = utils.NewRleDecoder(bytes.NewReader(data), 1)
+		return
+	}
+
+	width := uint8(data[0])
+	if width >= 64 {
+		panic("parquet: invalid or corrupted bit width")
+	}
+
+	d.idxDecoder = utils.NewRleDecoder(bytes.NewReader(data[1:]), int(width))

Review comment:
       yea, the first byte is the bit width, so we slice it off after grabbing it to hand the decoder the actual data to decode, minus the width. i'll add a comment

##########
File path: go/parquet/internal/encoding/decoder.go
##########
@@ -0,0 +1,180 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"bytes"
+	"reflect"
+
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/debug"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"github.com/apache/arrow/go/parquet/schema"
+)
+
+// DecoderTraits provides an interface for more easily interacting with types
+// to generate decoders for specific types.
+type DecoderTraits interface {
+	Decoder(parquet.Encoding, *schema.Column, bool, memory.Allocator) TypedDecoder
+	BytesRequired(int) int
+}
+
+// NewDecoder constructs a decoder for a given type and encoding
+func NewDecoder(t parquet.Type, e parquet.Encoding, descr *schema.Column, mem memory.Allocator) TypedDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	return traits.Decoder(e, descr, false, mem)
+}
+
+// NewDictDecoder is like NewDecoder but for dictionary encodings, panics if type is bool.
+//
+// if mem is nil, memory.DefaultAllocator will be used
+func NewDictDecoder(t parquet.Type, descr *schema.Column, mem memory.Allocator) DictDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	if mem == nil {
+		mem = memory.DefaultAllocator
+	}
+
+	return traits.Decoder(parquet.Encodings.RLEDict, descr, true, mem).(DictDecoder)
+}
+
+type decoder struct {
+	descr    *schema.Column
+	encoding format.Encoding
+	nvals    int
+	data     []byte
+	typeLen  int
+}
+
+// newDecoderBase constructs the base decoding object that is embedded in the
+// type specific decoders.
+func newDecoderBase(e format.Encoding, descr *schema.Column) decoder {
+	typeLen := -1
+	if descr != nil && descr.PhysicalType() == parquet.Types.FixedLenByteArray {
+		typeLen = int(descr.TypeLength())
+	}
+
+	return decoder{
+		descr:    descr,
+		encoding: e,
+		typeLen:  typeLen,
+	}
+}
+
+// SetData sets the data for decoding into the decoder to update the available
+// data bytes and number of values available.
+func (d *decoder) SetData(nvals int, data []byte) {
+	d.data = data
+	d.nvals = nvals
+}
+
+// ValuesLeft returns the number of remaining values that can be decoded
+func (d *decoder) ValuesLeft() int { return d.nvals }
+
+// Encoding returns the encoding type used by this decoder to decode the bytes.
+func (d *decoder) Encoding() parquet.Encoding { return parquet.Encoding(d.encoding) }
+
+type dictDecoder struct {
+	decoder
+	mem              memory.Allocator
+	dictValueDecoder utils.DictionaryConverter
+	idxDecoder       *utils.RleDecoder
+}
+
+// SetDict sets a decoder that can be used to decode the dictionary that is
+// used for this column in order to return the proper values.
+func (d *dictDecoder) SetDict(dict TypedDecoder) {
+	if dict.Type() != d.descr.PhysicalType() {
+		panic("parquet: mismatch dictionary and column data type")
+	}
+
+	d.dictValueDecoder = NewDictConverter(dict)
+}
+
+// SetData sets the index value data into the decoder.
+func (d *dictDecoder) SetData(nvals int, data []byte) {
+	d.nvals = nvals
+	if len(data) == 0 {
+		d.idxDecoder = utils.NewRleDecoder(bytes.NewReader(data), 1)

Review comment:
       that's actually a good point, i think it can safely be 0 for the bitwidth

##########
File path: go/parquet/internal/encoding/delta_bit_packing.go
##########
@@ -0,0 +1,515 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"bytes"
+	"math"
+	"math/bits"
+	"reflect"
+
+	"github.com/apache/arrow/go/arrow"
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// see the deltaBitPack encoder for a description of the encoding format that is
+// used for delta-bitpacking.
+type deltaBitPackDecoder struct {
+	decoder
+
+	mem memory.Allocator
+
+	usedFirst            bool
+	bitdecoder           *utils.BitReader
+	blockSize            uint64
+	currentBlockVals     uint64
+	miniBlocks           uint64
+	valsPerMini          uint64
+	currentMiniBlockVals uint64
+	minDelta             int64
+	miniBlockIdx         uint64
+
+	deltaBitWidths *memory.Buffer
+	deltaBitWidth  byte
+
+	lastVal int64
+}
+
+// returns the number of bytes read so far
+func (d *deltaBitPackDecoder) bytesRead() int64 {
+	return d.bitdecoder.CurOffset()
+}
+
+func (d *deltaBitPackDecoder) Allocator() memory.Allocator { return d.mem }
+
+// SetData sets the bytes and the expected number of values to decode
+// into the decoder, updating the decoder and allowing it to be reused.
+func (d *deltaBitPackDecoder) SetData(nvalues int, data []byte) {
+	// set our data into the underlying decoder for the type
+	d.decoder.SetData(nvalues, data)
+	// create a bit reader for our decoder's values
+	d.bitdecoder = utils.NewBitReader(bytes.NewReader(d.data))
+	d.currentBlockVals = 0
+	d.currentMiniBlockVals = 0
+	if d.deltaBitWidths == nil {
+		d.deltaBitWidths = memory.NewResizableBuffer(d.mem)
+	}
+
+	var ok bool
+	d.blockSize, ok = d.bitdecoder.GetVlqInt()
+	if !ok {
+		panic("parquet: eof exception")
+	}
+
+	if d.miniBlocks, ok = d.bitdecoder.GetVlqInt(); !ok {
+		panic("parquet: eof exception")
+	}
+
+	var totalValues uint64
+	if totalValues, ok = d.bitdecoder.GetVlqInt(); !ok {
+		panic("parquet: eof exception")
+	}
+
+	if int(totalValues) != d.nvals {
+		panic("parquet: mismatch between number of values and count in data header")
+	}
+
+	if d.lastVal, ok = d.bitdecoder.GetZigZagVlqInt(); !ok {
+		panic("parquet: eof exception")
+	}
+
+	d.valsPerMini = uint64(d.blockSize / d.miniBlocks)

Review comment:
       huh, i didn't know that. I didn't see anything in the spec other than that it would be a multiple of 32. i'll change it to a uint32 then, thanks!

##########
File path: go/parquet/internal/encoding/delta_byte_array.go
##########
@@ -0,0 +1,203 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// DeltaByteArrayEncoder is an encoder for writing bytearrays which are delta encoded
+// this is also known as incremental encoding or front compression. For each element
+// in a sequence of strings, we store the prefix length of the previous entry plus the suffix
+// see https://en.wikipedia.org/wiki/Incremental_encoding for a longer description.
+//
+// This is stored as a sequence of delta-encoded prefix lengths followed by the suffixes
+// encoded as delta length byte arrays.
+type DeltaByteArrayEncoder struct {
+	encoder
+
+	prefixEncoder *DeltaBitPackInt32Encoder
+	suffixEncoder *DeltaLengthByteArrayEncoder
+
+	lastVal parquet.ByteArray
+}
+
+func (enc *DeltaByteArrayEncoder) initEncoders() {
+	enc.prefixEncoder = &DeltaBitPackInt32Encoder{
+		deltaBitPackEncoder: &deltaBitPackEncoder{encoder: newEncoderBase(enc.encoding, nil, enc.mem)}}
+	enc.suffixEncoder = &DeltaLengthByteArrayEncoder{
+		newEncoderBase(enc.encoding, nil, enc.mem),
+		&DeltaBitPackInt32Encoder{
+			deltaBitPackEncoder: &deltaBitPackEncoder{encoder: newEncoderBase(enc.encoding, nil, enc.mem)}}}
+}
+
+// Type returns the underlying physical type this operates on, in this case ByteArrays only
+func (DeltaByteArrayEncoder) Type() parquet.Type { return parquet.Types.ByteArray }
+
+// Put writes a slice of ByteArrays to the encoder
+func (enc *DeltaByteArrayEncoder) Put(in []parquet.ByteArray) {
+	if len(in) == 0 {
+		return
+	}
+
+	var suf [1]parquet.ByteArray
+	if enc.prefixEncoder == nil { // initialize our encoders if we haven't yet
+		enc.initEncoders()
+		enc.prefixEncoder.Put([]int32{0})
+		suf[0] = in[0]
+		enc.lastVal = append([]byte(nil), in[0]...)
+		enc.suffixEncoder.Put(suf[:])
+		in = in[1:]
+	}
+
+	// for each value, figure out the common prefix with the previous value
+	// and then write the prefix length and the suffix.
+	for _, val := range in {
+		l1 := enc.lastVal.Len()
+		l2 := val.Len()
+		j := 0
+		for j < l1 && j < l2 {
+			if enc.lastVal[j] != val[j] {
+				break
+			}
+			j++
+		}
+		enc.prefixEncoder.Put([]int32{int32(j)})
+		suf[0] = val[j:]

Review comment:
       because the `DeltaLengthByteArrayEncoder` only takes in a slice of `ByteArray`'s rather than having a function which can take a single `ByteArray`, I create an array of a single `ByteArray` and use that to pass into the suffix encoder by creating a slice of it `suf[:]`. So since i had an array of size 1 bytearray, i needed to index into it using [0].
   
   All of that being said, as I wrote that I realized that it doesn't actually save me the creations of the slice to pass into the encoder, since it still ends up creating a slice when i do `suf[:]`, so i might as well just have a single parquet.ByteArray that i use and make it more obvious rather than this roundabout way it currently is. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#issuecomment-877273045


   I'll double check but I think this is ok to merge, will try to do so this evening.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r664589184



##########
File path: go/parquet/internal/encoding/plain_encoder_types.gen.go
##########
@@ -0,0 +1,553 @@
+// Code generated by plain_encoder_types.gen.go.tmpl. DO NOT EDIT.
+
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"math"
+
+	"github.com/apache/arrow/go/arrow"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// PlainInt32Encoder is an encoder for int32 values using Plain Encoding
+// which in general is just storing the values as raw bytes of the appropriate size
+type PlainInt32Encoder struct {
+	encoder
+
+	bitSetReader utils.SetBitRunReader
+}
+
+// Put encodes a slice of values into the underlying buffer
+func (enc *PlainInt32Encoder) Put(in []int32) {
+	enc.append(arrow.Int32Traits.CastToBytes(in))

Review comment:
       that's actually a good point, i'll update this to fix that but this also means that we should file a JIRA card since the C++ implementation has the same issue currently as it does exactly the same logic and thus would have an issue on big endian (for numeric types it just memcpy's the bytes as is without accounting for big-endian)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r663312872



##########
File path: go/parquet/internal/encoding/delta_byte_array.go
##########
@@ -0,0 +1,206 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// DeltaByteArrayEncoder is an encoder for writing bytearrays which are delta encoded
+// this is also known as incremental encoding or front compression. For each element
+// in a sequence of strings, we store the prefix length of the previous entry plus the suffix
+// see https://en.wikipedia.org/wiki/Incremental_encoding for a longer description.
+//
+// This is stored as a sequence of delta-encoded prefix lengths followed by the suffixes
+// encoded as delta length byte arrays.
+type DeltaByteArrayEncoder struct {
+	encoder
+
+	prefixEncoder *DeltaBitPackInt32Encoder
+	suffixEncoder *DeltaLengthByteArrayEncoder
+
+	lastVal parquet.ByteArray
+}
+
+func (enc *DeltaByteArrayEncoder) initEncoders() {
+	enc.prefixEncoder = &DeltaBitPackInt32Encoder{
+		deltaBitPackEncoder: &deltaBitPackEncoder{encoder: newEncoderBase(enc.encoding, nil, enc.mem)}}
+	enc.suffixEncoder = &DeltaLengthByteArrayEncoder{
+		newEncoderBase(enc.encoding, nil, enc.mem),
+		&DeltaBitPackInt32Encoder{
+			deltaBitPackEncoder: &deltaBitPackEncoder{encoder: newEncoderBase(enc.encoding, nil, enc.mem)}}}
+}
+
+// Type returns the underlying physical type this operates on, in this case ByteArrays only
+func (DeltaByteArrayEncoder) Type() parquet.Type { return parquet.Types.ByteArray }
+
+// Put writes a slice of ByteArrays to the encoder
+func (enc *DeltaByteArrayEncoder) Put(in []parquet.ByteArray) {
+	if len(in) == 0 {
+		return
+	}
+
+	var suf parquet.ByteArray
+	if enc.prefixEncoder == nil { // initialize our encoders if we haven't yet
+		enc.initEncoders()
+		enc.prefixEncoder.Put([]int32{0})
+		suf = in[0]
+		enc.lastVal = append([]byte(nil), in[0]...)
+		enc.suffixEncoder.Put([]parquet.ByteArray{suf})
+		in = in[1:]
+	}
+
+	// for each value, figure out the common prefix with the previous value
+	// and then write the prefix length and the suffix.
+	for _, val := range in {
+		l1 := enc.lastVal.Len()
+		l2 := val.Len()
+		j := 0
+		for j < l1 && j < l2 {
+			if enc.lastVal[j] != val[j] {
+				break
+			}
+			j++
+		}
+		enc.prefixEncoder.Put([]int32{int32(j)})
+		suf = val[j:]
+		enc.suffixEncoder.Put([]parquet.ByteArray{suf})
+		enc.lastVal = append([]byte(nil), val...)
+	}
+}
+
+// PutSpaced is like Put, but assumes the data is already spaced for nulls and uses the bitmap provided and offset
+// to compress the data before writing it without the null slots.
+func (enc *DeltaByteArrayEncoder) PutSpaced(in []parquet.ByteArray, validBits []byte, validBitsOffset int64) {
+	if validBits != nil {
+		data := make([]parquet.ByteArray, len(in))
+		nvalid := spacedCompress(in, data, validBits, validBitsOffset)
+		enc.Put(data[:nvalid])
+	} else {
+		enc.Put(in)
+	}
+}
+
+// Flush flushes any remaining data out and returns the finished encoded buffer.
+func (enc *DeltaByteArrayEncoder) FlushValues() Buffer {
+	if enc.prefixEncoder == nil {
+		enc.initEncoders()
+	}
+	prefixBuf := enc.prefixEncoder.FlushValues()
+	defer prefixBuf.Release()
+
+	suffixBuf := enc.suffixEncoder.FlushValues()
+	defer suffixBuf.Release()
+
+	ret := bufferPool.Get().(*memory.Buffer)
+	ret.ResizeNoShrink(prefixBuf.Len() + suffixBuf.Len())
+	copy(ret.Bytes(), prefixBuf.Bytes())
+	copy(ret.Bytes()[prefixBuf.Len():], suffixBuf.Bytes())
+	return poolBuffer{ret}
+}
+
+// DeltaByteArrayDecoder is a decoder for a column of data encoded using incremental or prefix encoding.
+type DeltaByteArrayDecoder struct {
+	*DeltaLengthByteArrayDecoder
+
+	prefixLengths []int32
+	lastVal       parquet.ByteArray
+}
+
+// Type returns the underlying physical type this decoder operates on, in this case ByteArrays only
+func (DeltaByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+func (d *DeltaByteArrayDecoder) Allocator() memory.Allocator { return d.mem }
+
+// SetData expects the data passed in to be the prefix lengths, followed by the
+// blocks of suffix data in order to initialize the decoder.
+func (d *DeltaByteArrayDecoder) SetData(nvalues int, data []byte) error {
+	prefixLenDec := DeltaBitPackInt32Decoder{
+		deltaBitPackDecoder: &deltaBitPackDecoder{
+			decoder: newDecoderBase(d.encoding, d.descr),
+			mem:     d.mem}}
+
+	if err := prefixLenDec.SetData(nvalues, data); err != nil {
+		return err
+	}
+
+	d.prefixLengths = make([]int32, nvalues)
+	// decode all the prefix lengths first so we know how many bytes it took to get the
+	// prefix lengths for nvalues
+	prefixLenDec.Decode(d.prefixLengths)
+
+	// now that we know how many bytes we needed for the prefix lengths, the rest are the
+	// delta length byte array encoding.
+	return d.DeltaLengthByteArrayDecoder.SetData(nvalues, data[int(prefixLenDec.bytesRead()):])
+}
+
+// Decode decodes byte arrays into the slice provided and returns the number of values actually decoded
+func (d *DeltaByteArrayDecoder) Decode(out []parquet.ByteArray) (int, error) {
+	max := utils.MinInt(len(out), d.nvals)
+	if max == 0 {
+		return 0, nil
+	}
+	out = out[:max]
+
+	var err error
+	if d.lastVal == nil {
+		_, err = d.DeltaLengthByteArrayDecoder.Decode(out[:1])
+		if err != nil {
+			return 0, err
+		}
+		d.lastVal = out[0]
+		out = out[1:]
+		d.prefixLengths = d.prefixLengths[1:]
+	}
+
+	var prefixLen int32
+	suffixHolder := make([]parquet.ByteArray, 1)
+	for len(out) > 0 {
+		prefixLen, d.prefixLengths = d.prefixLengths[0], d.prefixLengths[1:]
+
+		prefix := d.lastVal[:prefixLen]
+		_, err = d.DeltaLengthByteArrayDecoder.Decode(suffixHolder)
+		if err != nil {
+			return 0, err
+		}
+
+		d.lastVal = make([]byte, 0, int(prefixLen)+len(suffixHolder[0]))
+		d.lastVal = append([]byte{}, prefix...)
+		d.lastVal = append(d.lastVal, suffixHolder[0]...)
+		out[0], out = d.lastVal, out[1:]

Review comment:
       how expensive is slicing?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r647554565



##########
File path: go/parquet/internal/encoding/decoder.go
##########
@@ -0,0 +1,178 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"bytes"
+	"reflect"
+
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"github.com/apache/arrow/go/parquet/schema"
+)
+
+// DecoderTraits provides an interface for more easily interacting with types
+// to generate decoders for specific types.
+type DecoderTraits interface {
+	Decoder(parquet.Encoding, *schema.Column, bool, memory.Allocator) TypedDecoder
+	BytesRequired(int) int
+}
+
+// NewDecoder constructs a decoder for a given type and encoding
+func NewDecoder(t parquet.Type, e parquet.Encoding, descr *schema.Column, mem memory.Allocator) TypedDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	return traits.Decoder(e, descr, false, mem)
+}
+
+// NewDictDecoder is like NewDecoder but for dictionary encodings, panics if type is bool.
+//
+// if mem is nil, memory.DefaultAllocator will be used
+func NewDictDecoder(t parquet.Type, descr *schema.Column, mem memory.Allocator) DictDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	if mem == nil {
+		mem = memory.DefaultAllocator
+	}
+
+	return traits.Decoder(parquet.Encodings.RLEDict, descr, true, mem).(DictDecoder)
+}
+
+type decoder struct {
+	descr    *schema.Column
+	encoding format.Encoding
+	nvals    int
+	data     []byte
+	typeLen  int
+}
+
+// newDecoderBase constructs the base decoding object that is embedded in the
+// type specific decoders.
+func newDecoderBase(e format.Encoding, descr *schema.Column) decoder {
+	typeLen := -1
+	if descr != nil && descr.PhysicalType() == parquet.Types.FixedLenByteArray {
+		typeLen = int(descr.TypeLength())
+	}
+
+	return decoder{
+		descr:    descr,
+		encoding: e,
+		typeLen:  typeLen,
+	}
+}
+
+// SetData sets the data for decoding into the decoder to update the available
+// data bytes and number of values available.
+func (d *decoder) SetData(nvals int, data []byte) {
+	d.data = data
+	d.nvals = nvals
+}
+
+// ValuesLeft returns the number of remaining values that can be decoded
+func (d *decoder) ValuesLeft() int { return d.nvals }
+
+// Encoding returns the encoding type used by this decoder to decode the bytes.
+func (d *decoder) Encoding() parquet.Encoding { return parquet.Encoding(d.encoding) }
+
+type dictDecoder struct {
+	decoder
+	mem              memory.Allocator
+	dictValueDecoder utils.DictionaryConverter
+	idxDecoder       *utils.RleDecoder
+}
+
+// SetDict sets a decoder that can be used to decode the dictionary that is
+// used for this column in order to return the proper values.
+func (d *dictDecoder) SetDict(dict TypedDecoder) {
+	if dict.Type() != d.descr.PhysicalType() {
+		panic("parquet: mismatch dictionary and column data type")
+	}
+
+	d.dictValueDecoder = NewDictConverter(dict)
+}
+
+// SetData sets the index value data into the decoder.
+func (d *dictDecoder) SetData(nvals int, data []byte) {
+	d.nvals = nvals
+	if len(data) == 0 {
+		d.idxDecoder = utils.NewRleDecoder(bytes.NewReader(data), 1)
+		return
+	}
+
+	width := uint8(data[0])
+	if width >= 64 {
+		panic("parquet: invalid or corrupted bit width")
+	}
+
+	d.idxDecoder = utils.NewRleDecoder(bytes.NewReader(data[1:]), int(width))
+}
+
+func (d *dictDecoder) decode(out interface{}) (int, error) {
+	return d.idxDecoder.GetBatchWithDict(d.dictValueDecoder, out)
+}
+
+func (d *dictDecoder) decodeSpaced(out interface{}, nullCount int, validBits []byte, validBitsOffset int64) (int, error) {
+	return d.idxDecoder.GetBatchWithDictSpaced(d.dictValueDecoder, out, nullCount, validBits, validBitsOffset)
+}
+
+var empty = [1]byte{0}
+
+// spacedExpand is used to take a slice of data and utilize the bitmap provided to fill in nulls into the
+// correct slots according to the bitmap in order to produce a fully expanded result slice with nulls
+// in the correct slots.
+func spacedExpand(buffer interface{}, nullCount int, validBits []byte, validBitsOffset int64) int {
+	bufferRef := reflect.ValueOf(buffer)
+	if bufferRef.Kind() != reflect.Slice {
+		panic("invalid spacedexpand type, not slice")
+	}
+
+	var (
+		numValues int = bufferRef.Len()
+	)
+
+	idxDecode := int32(numValues - nullCount)
+	if idxDecode == 0 { // if there's nothing to decode there's nothing to do.
+		return numValues
+	}
+
+	// read the bitmap in reverse grabbing runs of valid bits where possible.
+	rdr := utils.NewReverseSetBitRunReader(validBits, validBitsOffset, int64(numValues))
+	for {
+		run := rdr.NextRun()
+		if run.Length == 0 {
+			break
+		}
+
+		// copy data from the end of the slice to it's proper location in the slice after accounting for the nulls
+		// because we technically don't care what is in the null slots we don't actually have to clean
+		// up after ourselves because we're doing this in reverse to guarantee that we'll always simply
+		// overwrite any existing data with the correctly spaced data. Any data that happens to be left in the null
+		// slots is fine since it shouldn't matter and saves us work.
+		idxDecode -= int32(run.Length)
+		reflect.Copy(bufferRef.Slice(int(run.Pos), bufferRef.Len()), bufferRef.Slice(int(idxDecode), int(int64(idxDecode)+run.Length)))

Review comment:
       does copy require non-overlapping ranges (it is possible for ranges to be exactly equal here.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield closed pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield closed pull request #10379:
URL: https://github.com/apache/arrow/pull/10379


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#issuecomment-872292123


   @emkornfield Pinging as requested since I didn't hear anything by wednesday :)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r663312637



##########
File path: go/parquet/internal/encoding/delta_byte_array.go
##########
@@ -0,0 +1,206 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// DeltaByteArrayEncoder is an encoder for writing bytearrays which are delta encoded
+// this is also known as incremental encoding or front compression. For each element
+// in a sequence of strings, we store the prefix length of the previous entry plus the suffix
+// see https://en.wikipedia.org/wiki/Incremental_encoding for a longer description.
+//
+// This is stored as a sequence of delta-encoded prefix lengths followed by the suffixes
+// encoded as delta length byte arrays.
+type DeltaByteArrayEncoder struct {
+	encoder
+
+	prefixEncoder *DeltaBitPackInt32Encoder
+	suffixEncoder *DeltaLengthByteArrayEncoder
+
+	lastVal parquet.ByteArray
+}
+
+func (enc *DeltaByteArrayEncoder) initEncoders() {
+	enc.prefixEncoder = &DeltaBitPackInt32Encoder{
+		deltaBitPackEncoder: &deltaBitPackEncoder{encoder: newEncoderBase(enc.encoding, nil, enc.mem)}}
+	enc.suffixEncoder = &DeltaLengthByteArrayEncoder{
+		newEncoderBase(enc.encoding, nil, enc.mem),
+		&DeltaBitPackInt32Encoder{
+			deltaBitPackEncoder: &deltaBitPackEncoder{encoder: newEncoderBase(enc.encoding, nil, enc.mem)}}}
+}
+
+// Type returns the underlying physical type this operates on, in this case ByteArrays only
+func (DeltaByteArrayEncoder) Type() parquet.Type { return parquet.Types.ByteArray }
+
+// Put writes a slice of ByteArrays to the encoder
+func (enc *DeltaByteArrayEncoder) Put(in []parquet.ByteArray) {
+	if len(in) == 0 {
+		return
+	}
+
+	var suf parquet.ByteArray
+	if enc.prefixEncoder == nil { // initialize our encoders if we haven't yet
+		enc.initEncoders()
+		enc.prefixEncoder.Put([]int32{0})
+		suf = in[0]
+		enc.lastVal = append([]byte(nil), in[0]...)
+		enc.suffixEncoder.Put([]parquet.ByteArray{suf})
+		in = in[1:]
+	}
+
+	// for each value, figure out the common prefix with the previous value
+	// and then write the prefix length and the suffix.
+	for _, val := range in {
+		l1 := enc.lastVal.Len()
+		l2 := val.Len()
+		j := 0
+		for j < l1 && j < l2 {
+			if enc.lastVal[j] != val[j] {
+				break
+			}
+			j++
+		}
+		enc.prefixEncoder.Put([]int32{int32(j)})
+		suf = val[j:]
+		enc.suffixEncoder.Put([]parquet.ByteArray{suf})
+		enc.lastVal = append([]byte(nil), val...)
+	}
+}
+
+// PutSpaced is like Put, but assumes the data is already spaced for nulls and uses the bitmap provided and offset
+// to compress the data before writing it without the null slots.
+func (enc *DeltaByteArrayEncoder) PutSpaced(in []parquet.ByteArray, validBits []byte, validBitsOffset int64) {
+	if validBits != nil {
+		data := make([]parquet.ByteArray, len(in))
+		nvalid := spacedCompress(in, data, validBits, validBitsOffset)
+		enc.Put(data[:nvalid])
+	} else {
+		enc.Put(in)
+	}
+}
+
+// Flush flushes any remaining data out and returns the finished encoded buffer.
+func (enc *DeltaByteArrayEncoder) FlushValues() Buffer {
+	if enc.prefixEncoder == nil {
+		enc.initEncoders()
+	}
+	prefixBuf := enc.prefixEncoder.FlushValues()
+	defer prefixBuf.Release()
+
+	suffixBuf := enc.suffixEncoder.FlushValues()
+	defer suffixBuf.Release()
+
+	ret := bufferPool.Get().(*memory.Buffer)
+	ret.ResizeNoShrink(prefixBuf.Len() + suffixBuf.Len())
+	copy(ret.Bytes(), prefixBuf.Bytes())
+	copy(ret.Bytes()[prefixBuf.Len():], suffixBuf.Bytes())
+	return poolBuffer{ret}
+}
+
+// DeltaByteArrayDecoder is a decoder for a column of data encoded using incremental or prefix encoding.
+type DeltaByteArrayDecoder struct {
+	*DeltaLengthByteArrayDecoder
+
+	prefixLengths []int32
+	lastVal       parquet.ByteArray
+}
+
+// Type returns the underlying physical type this decoder operates on, in this case ByteArrays only
+func (DeltaByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+func (d *DeltaByteArrayDecoder) Allocator() memory.Allocator { return d.mem }
+
+// SetData expects the data passed in to be the prefix lengths, followed by the
+// blocks of suffix data in order to initialize the decoder.
+func (d *DeltaByteArrayDecoder) SetData(nvalues int, data []byte) error {
+	prefixLenDec := DeltaBitPackInt32Decoder{
+		deltaBitPackDecoder: &deltaBitPackDecoder{
+			decoder: newDecoderBase(d.encoding, d.descr),
+			mem:     d.mem}}
+
+	if err := prefixLenDec.SetData(nvalues, data); err != nil {
+		return err
+	}
+
+	d.prefixLengths = make([]int32, nvalues)
+	// decode all the prefix lengths first so we know how many bytes it took to get the
+	// prefix lengths for nvalues
+	prefixLenDec.Decode(d.prefixLengths)
+
+	// now that we know how many bytes we needed for the prefix lengths, the rest are the
+	// delta length byte array encoding.
+	return d.DeltaLengthByteArrayDecoder.SetData(nvalues, data[int(prefixLenDec.bytesRead()):])
+}
+
+// Decode decodes byte arrays into the slice provided and returns the number of values actually decoded
+func (d *DeltaByteArrayDecoder) Decode(out []parquet.ByteArray) (int, error) {
+	max := utils.MinInt(len(out), d.nvals)
+	if max == 0 {
+		return 0, nil
+	}
+	out = out[:max]
+
+	var err error
+	if d.lastVal == nil {
+		_, err = d.DeltaLengthByteArrayDecoder.Decode(out[:1])
+		if err != nil {
+			return 0, err
+		}
+		d.lastVal = out[0]
+		out = out[1:]
+		d.prefixLengths = d.prefixLengths[1:]

Review comment:
       was this reverse engineered fro java (i.e. is zero actually stored for the first length?)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r645674860



##########
File path: go/parquet/internal/encoding/byte_array_decoder.go
##########
@@ -0,0 +1,88 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// PlainByteArrayDecoder decodes a data chunk for bytearrays according to
+// the plain encoding. The byte arrays will use slices to reference the
+// data rather than copying it.
+//
+// The parquet spec defines Plain encoding for ByteArrays as a 4 byte little
+// endian integer containing the length of the bytearray followed by that many
+// bytes being the raw data of the byte array.
+type PlainByteArrayDecoder struct {
+	decoder
+}
+
+// Type returns parquet.Types.ByteArray for this decoder
+func (PlainByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+// Decode will populate the slice of bytearrays in full or until the number
+// of values is consumed.
+//
+// Returns the number of values that were decoded.
+func (pbad *PlainByteArrayDecoder) Decode(out []parquet.ByteArray) (int, error) {
+	max := utils.MinInt(len(out), pbad.nvals)
+
+	for i := 0; i < max; i++ {
+		// there should always be at least four bytes which is the length of the
+		// next value in the data.
+		if len(pbad.data) < 4 {
+			return i, xerrors.New("parquet: eof reading bytearray")
+		}
+
+		// the first 4 bytes are a little endian int32 length
+		byteLen := int32(binary.LittleEndian.Uint32(pbad.data[:4]))
+		if byteLen < 0 {
+			return i, xerrors.New("parquet: invalid BYTE_ARRAY value")
+		}
+
+		if int64(len(pbad.data)) < int64(byteLen)+4 {
+			return i, xerrors.New("parquet: eof reading bytearray")
+		}
+
+		out[i] = pbad.data[4 : byteLen+4 : byteLen+4]
+		pbad.data = pbad.data[byteLen+4:]
+	}
+
+	pbad.nvals -= max
+	return max, nil
+}
+
+// DecodeSpaced is like Decode, but expands the slice out to leave empty values
+// where the validBits bitmap has 0s
+func (pbad *PlainByteArrayDecoder) DecodeSpaced(out []parquet.ByteArray, nullCount int, validBits []byte, validBitsOffset int64) (int, error) {

Review comment:
       one more thought, I'm not sure how you've organized the remaining code, but spacing values might more naturally fit with Arrow (not critical, and might not make sense)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r663313252



##########
File path: go/parquet/internal/encoding/delta_byte_array.go
##########
@@ -0,0 +1,206 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// DeltaByteArrayEncoder is an encoder for writing bytearrays which are delta encoded
+// this is also known as incremental encoding or front compression. For each element
+// in a sequence of strings, we store the prefix length of the previous entry plus the suffix
+// see https://en.wikipedia.org/wiki/Incremental_encoding for a longer description.
+//
+// This is stored as a sequence of delta-encoded prefix lengths followed by the suffixes
+// encoded as delta length byte arrays.
+type DeltaByteArrayEncoder struct {
+	encoder
+
+	prefixEncoder *DeltaBitPackInt32Encoder
+	suffixEncoder *DeltaLengthByteArrayEncoder
+
+	lastVal parquet.ByteArray
+}
+
+func (enc *DeltaByteArrayEncoder) initEncoders() {
+	enc.prefixEncoder = &DeltaBitPackInt32Encoder{
+		deltaBitPackEncoder: &deltaBitPackEncoder{encoder: newEncoderBase(enc.encoding, nil, enc.mem)}}
+	enc.suffixEncoder = &DeltaLengthByteArrayEncoder{
+		newEncoderBase(enc.encoding, nil, enc.mem),
+		&DeltaBitPackInt32Encoder{
+			deltaBitPackEncoder: &deltaBitPackEncoder{encoder: newEncoderBase(enc.encoding, nil, enc.mem)}}}
+}
+
+// Type returns the underlying physical type this operates on, in this case ByteArrays only
+func (DeltaByteArrayEncoder) Type() parquet.Type { return parquet.Types.ByteArray }
+
+// Put writes a slice of ByteArrays to the encoder
+func (enc *DeltaByteArrayEncoder) Put(in []parquet.ByteArray) {
+	if len(in) == 0 {
+		return
+	}
+
+	var suf parquet.ByteArray
+	if enc.prefixEncoder == nil { // initialize our encoders if we haven't yet
+		enc.initEncoders()
+		enc.prefixEncoder.Put([]int32{0})
+		suf = in[0]
+		enc.lastVal = append([]byte(nil), in[0]...)
+		enc.suffixEncoder.Put([]parquet.ByteArray{suf})
+		in = in[1:]
+	}
+
+	// for each value, figure out the common prefix with the previous value
+	// and then write the prefix length and the suffix.
+	for _, val := range in {
+		l1 := enc.lastVal.Len()
+		l2 := val.Len()
+		j := 0
+		for j < l1 && j < l2 {
+			if enc.lastVal[j] != val[j] {
+				break
+			}
+			j++
+		}
+		enc.prefixEncoder.Put([]int32{int32(j)})
+		suf = val[j:]
+		enc.suffixEncoder.Put([]parquet.ByteArray{suf})
+		enc.lastVal = append([]byte(nil), val...)
+	}
+}
+
+// PutSpaced is like Put, but assumes the data is already spaced for nulls and uses the bitmap provided and offset
+// to compress the data before writing it without the null slots.
+func (enc *DeltaByteArrayEncoder) PutSpaced(in []parquet.ByteArray, validBits []byte, validBitsOffset int64) {
+	if validBits != nil {
+		data := make([]parquet.ByteArray, len(in))
+		nvalid := spacedCompress(in, data, validBits, validBitsOffset)
+		enc.Put(data[:nvalid])
+	} else {
+		enc.Put(in)
+	}
+}
+
+// Flush flushes any remaining data out and returns the finished encoded buffer.
+func (enc *DeltaByteArrayEncoder) FlushValues() Buffer {
+	if enc.prefixEncoder == nil {
+		enc.initEncoders()
+	}
+	prefixBuf := enc.prefixEncoder.FlushValues()
+	defer prefixBuf.Release()
+
+	suffixBuf := enc.suffixEncoder.FlushValues()
+	defer suffixBuf.Release()
+
+	ret := bufferPool.Get().(*memory.Buffer)
+	ret.ResizeNoShrink(prefixBuf.Len() + suffixBuf.Len())
+	copy(ret.Bytes(), prefixBuf.Bytes())
+	copy(ret.Bytes()[prefixBuf.Len():], suffixBuf.Bytes())
+	return poolBuffer{ret}
+}
+
+// DeltaByteArrayDecoder is a decoder for a column of data encoded using incremental or prefix encoding.
+type DeltaByteArrayDecoder struct {
+	*DeltaLengthByteArrayDecoder
+
+	prefixLengths []int32
+	lastVal       parquet.ByteArray
+}
+
+// Type returns the underlying physical type this decoder operates on, in this case ByteArrays only
+func (DeltaByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+func (d *DeltaByteArrayDecoder) Allocator() memory.Allocator { return d.mem }
+
+// SetData expects the data passed in to be the prefix lengths, followed by the
+// blocks of suffix data in order to initialize the decoder.
+func (d *DeltaByteArrayDecoder) SetData(nvalues int, data []byte) error {
+	prefixLenDec := DeltaBitPackInt32Decoder{
+		deltaBitPackDecoder: &deltaBitPackDecoder{
+			decoder: newDecoderBase(d.encoding, d.descr),
+			mem:     d.mem}}
+
+	if err := prefixLenDec.SetData(nvalues, data); err != nil {
+		return err
+	}
+
+	d.prefixLengths = make([]int32, nvalues)
+	// decode all the prefix lengths first so we know how many bytes it took to get the
+	// prefix lengths for nvalues
+	prefixLenDec.Decode(d.prefixLengths)
+
+	// now that we know how many bytes we needed for the prefix lengths, the rest are the
+	// delta length byte array encoding.
+	return d.DeltaLengthByteArrayDecoder.SetData(nvalues, data[int(prefixLenDec.bytesRead()):])
+}
+
+// Decode decodes byte arrays into the slice provided and returns the number of values actually decoded
+func (d *DeltaByteArrayDecoder) Decode(out []parquet.ByteArray) (int, error) {
+	max := utils.MinInt(len(out), d.nvals)
+	if max == 0 {
+		return 0, nil
+	}
+	out = out[:max]
+
+	var err error
+	if d.lastVal == nil {
+		_, err = d.DeltaLengthByteArrayDecoder.Decode(out[:1])
+		if err != nil {
+			return 0, err
+		}
+		d.lastVal = out[0]
+		out = out[1:]
+		d.prefixLengths = d.prefixLengths[1:]
+	}
+
+	var prefixLen int32
+	suffixHolder := make([]parquet.ByteArray, 1)
+	for len(out) > 0 {
+		prefixLen, d.prefixLengths = d.prefixLengths[0], d.prefixLengths[1:]
+
+		prefix := d.lastVal[:prefixLen]
+		_, err = d.DeltaLengthByteArrayDecoder.Decode(suffixHolder)
+		if err != nil {
+			return 0, err
+		}
+
+		d.lastVal = make([]byte, 0, int(prefixLen)+len(suffixHolder[0]))

Review comment:
       does it pay to try to handle the case where suffix legth = 0?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r664717594



##########
File path: go/parquet/internal/encoding/plain_encoder_types.gen.go
##########
@@ -0,0 +1,553 @@
+// Code generated by plain_encoder_types.gen.go.tmpl. DO NOT EDIT.
+
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"math"
+
+	"github.com/apache/arrow/go/arrow"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// PlainInt32Encoder is an encoder for int32 values using Plain Encoding
+// which in general is just storing the values as raw bytes of the appropriate size
+type PlainInt32Encoder struct {
+	encoder
+
+	bitSetReader utils.SetBitRunReader
+}
+
+// Put encodes a slice of values into the underlying buffer
+func (enc *PlainInt32Encoder) Put(in []int32) {
+	enc.append(arrow.Int32Traits.CastToBytes(in))

Review comment:
       that is quite possible.  CC @kiszk about big-endian issue in the C++ parquet implementation




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r645847411



##########
File path: go/parquet/internal/encoding/byte_array_encoder.go
##########
@@ -0,0 +1,111 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+	"unsafe"
+
+	"github.com/apache/arrow/go/arrow"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+)
+
+// PlainByteArrayEncoder encodes byte arrays according to the spec for Plain encoding
+// by encoding the length as a uint32 followed by the bytes of the value.
+type PlainByteArrayEncoder struct {
+	encoder
+}
+
+// PutByteArray writes out the 4 bytes for the length followed by the data
+func (enc *PlainByteArrayEncoder) PutByteArray(val parquet.ByteArray) {
+	inc := val.Len() + arrow.Uint32SizeBytes
+	enc.sink.Reserve(inc)
+	vlen := uint32(val.Len())
+	enc.sink.UnsafeWrite((*(*[4]byte)(unsafe.Pointer(&vlen)))[:])
+	enc.sink.UnsafeWrite(val)
+}
+
+// Put writes out all of the values in this slice to the buffer
+func (enc *PlainByteArrayEncoder) Put(in []parquet.ByteArray) {
+	for _, val := range in {
+		enc.PutByteArray(val)

Review comment:
       You're referring to the comment? Fair, I'll change `writes out all of the values in the slice to the buffer` to `writes out all of the values in the slice to the sink`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r644923055



##########
File path: go/parquet/internal/encoding/byte_array_decoder.go
##########
@@ -0,0 +1,84 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// PlainByteArrayDecoder decodes a data chunk for bytearrays according to
+// the plain encoding. The byte arrays will use slices to reference the

Review comment:
       might be worth adding a quick summary fo what plain encoding is




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r647548320



##########
File path: go/parquet/internal/encoding/byte_array_encoder.go
##########
@@ -0,0 +1,111 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+	"unsafe"
+
+	"github.com/apache/arrow/go/arrow"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+)
+
+// PlainByteArrayEncoder encodes byte arrays according to the spec for Plain encoding
+// by encoding the length as a uint32 followed by the bytes of the value.
+type PlainByteArrayEncoder struct {
+	encoder
+}
+
+// PutByteArray writes out the 4 bytes for the length followed by the data
+func (enc *PlainByteArrayEncoder) PutByteArray(val parquet.ByteArray) {
+	inc := val.Len() + arrow.Uint32SizeBytes
+	enc.sink.Reserve(inc)
+	vlen := uint32(val.Len())
+	enc.sink.UnsafeWrite((*(*[4]byte)(unsafe.Pointer(&vlen)))[:])
+	enc.sink.UnsafeWrite(val)
+}
+
+// Put writes out all of the values in this slice to the buffer
+func (enc *PlainByteArrayEncoder) Put(in []parquet.ByteArray) {
+	for _, val := range in {
+		enc.PutByteArray(val)
+	}
+}
+
+// PutSpaced uses the bitmap of validBits to leave out anything that is null according
+// to the bitmap.
+//
+// If validBits is nil, this is equivalent to calling Put
+func (enc *PlainByteArrayEncoder) PutSpaced(in []parquet.ByteArray, validBits []byte, validBitsOffset int64) {

Review comment:
       Yes, that was what I was thinking, its minor in the grand scheme of things, so refactoring/changing isn't necessary just thought I would call it out.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r644101301



##########
File path: go/parquet/internal/encoding/boolean_encoder.go
##########
@@ -0,0 +1,112 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"github.com/apache/arrow/go/arrow/bitutil"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+)
+
+const boolBufSize = 1024
+
+// PlainBooleanEncoder encodes bools as a bitmap as per the Plain Encoding
+type PlainBooleanEncoder struct {
+	encoder
+	nbits      int
+	bitsBuffer []byte
+}
+
+// Type for the PlainBooleanEncoder is parquet.Types.Boolean
+func (PlainBooleanEncoder) Type() parquet.Type {
+	return parquet.Types.Boolean
+}
+
+// Put encodes the contents of in into the underlying data buffer.
+func (enc *PlainBooleanEncoder) Put(in []bool) {
+	if enc.bitsBuffer == nil {
+		enc.bitsBuffer = make([]byte, boolBufSize)
+	}
+
+	bitOffset := 0
+	// first check if we are in the middle of a byte due to previous
+	// encoding of data and finish out that byte's bits.
+	if enc.nbits > 0 {
+		bitsToWrite := utils.MinInt(enc.nbits, len(in))
+		beg := (boolBufSize * 8) - enc.nbits
+		for i, val := range in[:bitsToWrite] {
+			bitmask := uint8(1 << uint((beg+i)%8))
+			if val {
+				enc.bitsBuffer[(beg+i)/8] |= bitmask
+			} else {
+				enc.bitsBuffer[(beg+i)/8] &= bitmask ^ 0xFF

Review comment:
       might just be pre-caffience but isn't this just `~bitmask` (or does go not have that concept




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#issuecomment-853129506


   Starting to take a look.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#issuecomment-869758621


   Just bumping this again in the hopes for more reviews so i can get this merged
   
   @emkornfield @sbinet @fsaintjacques @nickpoorman 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r647568724



##########
File path: go/parquet/internal/encoding/decoder.go
##########
@@ -0,0 +1,178 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"bytes"
+	"reflect"
+
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"github.com/apache/arrow/go/parquet/schema"
+)
+
+// DecoderTraits provides an interface for more easily interacting with types
+// to generate decoders for specific types.
+type DecoderTraits interface {
+	Decoder(parquet.Encoding, *schema.Column, bool, memory.Allocator) TypedDecoder
+	BytesRequired(int) int
+}
+
+// NewDecoder constructs a decoder for a given type and encoding
+func NewDecoder(t parquet.Type, e parquet.Encoding, descr *schema.Column, mem memory.Allocator) TypedDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	return traits.Decoder(e, descr, false, mem)
+}
+
+// NewDictDecoder is like NewDecoder but for dictionary encodings, panics if type is bool.
+//
+// if mem is nil, memory.DefaultAllocator will be used
+func NewDictDecoder(t parquet.Type, descr *schema.Column, mem memory.Allocator) DictDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	if mem == nil {
+		mem = memory.DefaultAllocator
+	}
+
+	return traits.Decoder(parquet.Encodings.RLEDict, descr, true, mem).(DictDecoder)
+}
+
+type decoder struct {
+	descr    *schema.Column
+	encoding format.Encoding
+	nvals    int
+	data     []byte
+	typeLen  int
+}
+
+// newDecoderBase constructs the base decoding object that is embedded in the
+// type specific decoders.
+func newDecoderBase(e format.Encoding, descr *schema.Column) decoder {
+	typeLen := -1
+	if descr != nil && descr.PhysicalType() == parquet.Types.FixedLenByteArray {
+		typeLen = int(descr.TypeLength())
+	}
+
+	return decoder{
+		descr:    descr,
+		encoding: e,
+		typeLen:  typeLen,
+	}
+}
+
+// SetData sets the data for decoding into the decoder to update the available
+// data bytes and number of values available.
+func (d *decoder) SetData(nvals int, data []byte) {
+	d.data = data
+	d.nvals = nvals
+}
+
+// ValuesLeft returns the number of remaining values that can be decoded
+func (d *decoder) ValuesLeft() int { return d.nvals }
+
+// Encoding returns the encoding type used by this decoder to decode the bytes.
+func (d *decoder) Encoding() parquet.Encoding { return parquet.Encoding(d.encoding) }
+
+type dictDecoder struct {
+	decoder
+	mem              memory.Allocator
+	dictValueDecoder utils.DictionaryConverter
+	idxDecoder       *utils.RleDecoder
+}
+
+// SetDict sets a decoder that can be used to decode the dictionary that is
+// used for this column in order to return the proper values.
+func (d *dictDecoder) SetDict(dict TypedDecoder) {
+	if dict.Type() != d.descr.PhysicalType() {
+		panic("parquet: mismatch dictionary and column data type")
+	}
+
+	d.dictValueDecoder = NewDictConverter(dict)
+}
+
+// SetData sets the index value data into the decoder.
+func (d *dictDecoder) SetData(nvals int, data []byte) {
+	d.nvals = nvals
+	if len(data) == 0 {
+		d.idxDecoder = utils.NewRleDecoder(bytes.NewReader(data), 1)
+		return
+	}
+
+	width := uint8(data[0])
+	if width >= 64 {
+		panic("parquet: invalid or corrupted bit width")
+	}
+
+	d.idxDecoder = utils.NewRleDecoder(bytes.NewReader(data[1:]), int(width))
+}
+
+func (d *dictDecoder) decode(out interface{}) (int, error) {
+	return d.idxDecoder.GetBatchWithDict(d.dictValueDecoder, out)
+}
+
+func (d *dictDecoder) decodeSpaced(out interface{}, nullCount int, validBits []byte, validBitsOffset int64) (int, error) {
+	return d.idxDecoder.GetBatchWithDictSpaced(d.dictValueDecoder, out, nullCount, validBits, validBitsOffset)
+}
+
+var empty = [1]byte{0}
+
+// spacedExpand is used to take a slice of data and utilize the bitmap provided to fill in nulls into the
+// correct slots according to the bitmap in order to produce a fully expanded result slice with nulls
+// in the correct slots.
+func spacedExpand(buffer interface{}, nullCount int, validBits []byte, validBitsOffset int64) int {
+	bufferRef := reflect.ValueOf(buffer)
+	if bufferRef.Kind() != reflect.Slice {
+		panic("invalid spacedexpand type, not slice")
+	}
+
+	var (
+		numValues int = bufferRef.Len()
+	)
+
+	idxDecode := int32(numValues - nullCount)
+	if idxDecode == 0 { // if there's nothing to decode there's nothing to do.
+		return numValues
+	}
+
+	// read the bitmap in reverse grabbing runs of valid bits where possible.
+	rdr := utils.NewReverseSetBitRunReader(validBits, validBitsOffset, int64(numValues))
+	for {
+		run := rdr.NextRun()
+		if run.Length == 0 {
+			break
+		}
+
+		// copy data from the end of the slice to it's proper location in the slice after accounting for the nulls
+		// because we technically don't care what is in the null slots we don't actually have to clean
+		// up after ourselves because we're doing this in reverse to guarantee that we'll always simply
+		// overwrite any existing data with the correctly spaced data. Any data that happens to be left in the null
+		// slots is fine since it shouldn't matter and saves us work.
+		idxDecode -= int32(run.Length)
+		reflect.Copy(bufferRef.Slice(int(run.Pos), bufferRef.Len()), bufferRef.Slice(int(idxDecode), int(int64(idxDecode)+run.Length)))

Review comment:
       nit: assert run.Length elements are copied?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r644345648



##########
File path: go/parquet/internal/encoding/boolean_encoder.go
##########
@@ -0,0 +1,112 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"github.com/apache/arrow/go/arrow/bitutil"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+)
+
+const boolBufSize = 1024
+
+// PlainBooleanEncoder encodes bools as a bitmap as per the Plain Encoding
+type PlainBooleanEncoder struct {
+	encoder
+	nbits      int
+	bitsBuffer []byte
+}
+
+// Type for the PlainBooleanEncoder is parquet.Types.Boolean
+func (PlainBooleanEncoder) Type() parquet.Type {
+	return parquet.Types.Boolean
+}
+
+// Put encodes the contents of in into the underlying data buffer.
+func (enc *PlainBooleanEncoder) Put(in []bool) {
+	if enc.bitsBuffer == nil {
+		enc.bitsBuffer = make([]byte, boolBufSize)
+	}
+
+	bitOffset := 0
+	// first check if we are in the middle of a byte due to previous
+	// encoding of data and finish out that byte's bits.
+	if enc.nbits > 0 {
+		bitsToWrite := utils.MinInt(enc.nbits, len(in))
+		beg := (boolBufSize * 8) - enc.nbits
+		for i, val := range in[:bitsToWrite] {
+			bitmask := uint8(1 << uint((beg+i)%8))
+			if val {
+				enc.bitsBuffer[(beg+i)/8] |= bitmask

Review comment:
       So the combination of the changes I made to how I handle the bitmapwriter and utilize an `AppendBools` function resulted in about a 40% improvement in speed of encoding bools! It also turns out that Go definitely does optimize the /8 and the *8 as far as i can tell since there was very little performance difference between using them or the shifts directly. 
   
   The primary benefits that resulted in the performance improvements came from switching to use `bitutil.SetBit` / `bitutil.ClearBit` since they used the constant slices with the bitmasks that we index into rather than constructing them on the fly as it converted a modulus+shift operation into a simple index lookup inside of a tight loop.
   
   The big reason why it needed an AppendBools function instead of just using a loop with `Next()`, `Set()`, and `Clear()` is because `Next()` has to check for bitmask being 0 to update it along with the offset and position tracking, whereas having the function inside there allows for a single update of the bookkeeping and just looping through to set the bits. Either way, the result is a significant performance improvement so I'm happy :smiley:




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r647566763



##########
File path: go/parquet/internal/encoding/decoder.go
##########
@@ -0,0 +1,178 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"bytes"
+	"reflect"
+
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"github.com/apache/arrow/go/parquet/schema"
+)
+
+// DecoderTraits provides an interface for more easily interacting with types
+// to generate decoders for specific types.
+type DecoderTraits interface {
+	Decoder(parquet.Encoding, *schema.Column, bool, memory.Allocator) TypedDecoder
+	BytesRequired(int) int
+}
+
+// NewDecoder constructs a decoder for a given type and encoding
+func NewDecoder(t parquet.Type, e parquet.Encoding, descr *schema.Column, mem memory.Allocator) TypedDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	return traits.Decoder(e, descr, false, mem)
+}
+
+// NewDictDecoder is like NewDecoder but for dictionary encodings, panics if type is bool.
+//
+// if mem is nil, memory.DefaultAllocator will be used
+func NewDictDecoder(t parquet.Type, descr *schema.Column, mem memory.Allocator) DictDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	if mem == nil {
+		mem = memory.DefaultAllocator
+	}
+
+	return traits.Decoder(parquet.Encodings.RLEDict, descr, true, mem).(DictDecoder)
+}
+
+type decoder struct {
+	descr    *schema.Column
+	encoding format.Encoding
+	nvals    int
+	data     []byte
+	typeLen  int
+}
+
+// newDecoderBase constructs the base decoding object that is embedded in the
+// type specific decoders.
+func newDecoderBase(e format.Encoding, descr *schema.Column) decoder {
+	typeLen := -1
+	if descr != nil && descr.PhysicalType() == parquet.Types.FixedLenByteArray {
+		typeLen = int(descr.TypeLength())
+	}
+
+	return decoder{
+		descr:    descr,
+		encoding: e,
+		typeLen:  typeLen,
+	}
+}
+
+// SetData sets the data for decoding into the decoder to update the available
+// data bytes and number of values available.
+func (d *decoder) SetData(nvals int, data []byte) {
+	d.data = data
+	d.nvals = nvals
+}
+
+// ValuesLeft returns the number of remaining values that can be decoded
+func (d *decoder) ValuesLeft() int { return d.nvals }
+
+// Encoding returns the encoding type used by this decoder to decode the bytes.
+func (d *decoder) Encoding() parquet.Encoding { return parquet.Encoding(d.encoding) }
+
+type dictDecoder struct {
+	decoder
+	mem              memory.Allocator
+	dictValueDecoder utils.DictionaryConverter
+	idxDecoder       *utils.RleDecoder
+}
+
+// SetDict sets a decoder that can be used to decode the dictionary that is
+// used for this column in order to return the proper values.
+func (d *dictDecoder) SetDict(dict TypedDecoder) {
+	if dict.Type() != d.descr.PhysicalType() {
+		panic("parquet: mismatch dictionary and column data type")
+	}
+
+	d.dictValueDecoder = NewDictConverter(dict)
+}
+
+// SetData sets the index value data into the decoder.
+func (d *dictDecoder) SetData(nvals int, data []byte) {
+	d.nvals = nvals
+	if len(data) == 0 {
+		d.idxDecoder = utils.NewRleDecoder(bytes.NewReader(data), 1)
+		return
+	}
+
+	width := uint8(data[0])
+	if width >= 64 {
+		panic("parquet: invalid or corrupted bit width")
+	}
+
+	d.idxDecoder = utils.NewRleDecoder(bytes.NewReader(data[1:]), int(width))
+}
+
+func (d *dictDecoder) decode(out interface{}) (int, error) {
+	return d.idxDecoder.GetBatchWithDict(d.dictValueDecoder, out)
+}
+
+func (d *dictDecoder) decodeSpaced(out interface{}, nullCount int, validBits []byte, validBitsOffset int64) (int, error) {
+	return d.idxDecoder.GetBatchWithDictSpaced(d.dictValueDecoder, out, nullCount, validBits, validBitsOffset)
+}
+
+var empty = [1]byte{0}
+
+// spacedExpand is used to take a slice of data and utilize the bitmap provided to fill in nulls into the
+// correct slots according to the bitmap in order to produce a fully expanded result slice with nulls
+// in the correct slots.
+func spacedExpand(buffer interface{}, nullCount int, validBits []byte, validBitsOffset int64) int {
+	bufferRef := reflect.ValueOf(buffer)
+	if bufferRef.Kind() != reflect.Slice {
+		panic("invalid spacedexpand type, not slice")
+	}
+
+	var (
+		numValues int = bufferRef.Len()
+	)
+
+	idxDecode := int32(numValues - nullCount)
+	if idxDecode == 0 { // if there's nothing to decode there's nothing to do.
+		return numValues
+	}
+
+	// read the bitmap in reverse grabbing runs of valid bits where possible.
+	rdr := utils.NewReverseSetBitRunReader(validBits, validBitsOffset, int64(numValues))
+	for {
+		run := rdr.NextRun()
+		if run.Length == 0 {
+			break
+		}
+
+		// copy data from the end of the slice to it's proper location in the slice after accounting for the nulls
+		// because we technically don't care what is in the null slots we don't actually have to clean
+		// up after ourselves because we're doing this in reverse to guarantee that we'll always simply
+		// overwrite any existing data with the correctly spaced data. Any data that happens to be left in the null
+		// slots is fine since it shouldn't matter and saves us work.
+		idxDecode -= int32(run.Length)

Review comment:
       total length? or just the run.Length? The total length, ie: `bufferRef.Len()` is a 64-bit int on 64-bit architectures above since it's using `int`, but I'd actually rather have `idxDecode` be an `int64` than add a comparison against int32 max in this loop as this is a low level tight loop so adding that comparison would actually affect performance more than just changing this to be an `int64` to avoid the potential for int32 overflow




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r644928045



##########
File path: go/parquet/internal/encoding/byte_array_decoder.go
##########
@@ -0,0 +1,84 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// PlainByteArrayDecoder decodes a data chunk for bytearrays according to
+// the plain encoding. The byte arrays will use slices to reference the
+// data rather than copying it.
+type PlainByteArrayDecoder struct {
+	decoder
+}
+
+// Type returns parquet.Types.ByteArray for this decoder
+func (PlainByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+// Decode will populate the slice of bytearrays in full or until the number
+// of values is emptied.
+//
+// Returns the number of values that were decoded.
+func (pbad *PlainByteArrayDecoder) Decode(out []parquet.ByteArray) (int, error) {
+	max := utils.MinInt(len(out), pbad.nvals)
+
+	for i := 0; i < max; i++ {
+		// there should always be at least four bytes which is the length of the
+		// next value in the data.
+		if len(pbad.data) < 4 {
+			return i, xerrors.New("parquet: eof reading bytearray")
+		}
+
+		// the first 4 bytes are a little endian uint32 length

Review comment:
       Is there a reference for unsigned?  I believe this is actually [signed](https://github.com/apache/parquet-mr/blob/master/parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesWriter.java#L54)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r664586508



##########
File path: go/parquet/internal/encoding/delta_length_byte_array.go
##########
@@ -0,0 +1,144 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// DeltaLengthByteArrayEncoder encodes data using by taking all of the byte array lengths
+// and encoding them in front using delta encoding, followed by all of the binary data
+// concatenated back to back. The expected savings is from the cost of encoding the lengths
+// and possibly better compression in the data which will no longer be interleaved with the lengths.
+//
+// This encoding is always preferred over PLAIN for byte array columns where possible.
+//
+// For example, if the data was "Hello", "World", "Foobar", "ABCDEF" the encoded data would be:
+// DeltaEncoding(5, 5, 6, 6) "HelloWorldFoobarABCDEF"
+type DeltaLengthByteArrayEncoder struct {
+	encoder
+
+	lengthEncoder *DeltaBitPackInt32Encoder
+}
+
+// Put writes the provided slice of byte arrays to the encoder
+func (enc *DeltaLengthByteArrayEncoder) Put(in []parquet.ByteArray) {
+	lengths := make([]int32, len(in))
+	totalLen := int(0)
+	for idx, val := range in {
+		lengths[idx] = int32(val.Len())
+		totalLen += val.Len()
+	}
+
+	enc.lengthEncoder.Put(lengths)
+	enc.sink.Reserve(totalLen)
+	for _, val := range in {
+		enc.sink.UnsafeWrite(val)
+	}
+}
+
+// PutSpaced is like Put, but the data is spaced out according to the bitmap provided and is compressed
+// accordingly before it is written to drop the null data from the write.
+func (enc *DeltaLengthByteArrayEncoder) PutSpaced(in []parquet.ByteArray, validBits []byte, validBitsOffset int64) {
+	if validBits != nil {
+		data := make([]parquet.ByteArray, len(in))
+		nvalid := spacedCompress(in, data, validBits, validBitsOffset)
+		enc.Put(data[:nvalid])
+	} else {
+		enc.Put(in)
+	}
+}
+
+// Type returns the underlying type which is handled by this encoder, ByteArrays only.
+func (DeltaLengthByteArrayEncoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+// FlushValues flushes any remaining data and returns the final encoded buffer of data.
+func (enc *DeltaLengthByteArrayEncoder) FlushValues() Buffer {
+	ret := enc.lengthEncoder.FlushValues()
+	defer ret.Release()
+
+	data := enc.sink.Finish()
+	defer data.Release()
+
+	output := bufferPool.Get().(*memory.Buffer)
+	output.ResizeNoShrink(ret.Len() + data.Len())
+	copy(output.Bytes(), ret.Bytes())
+	copy(output.Bytes()[ret.Len():], data.Bytes())
+	return poolBuffer{output}
+}
+
+// DeltaLengthByteArrayDecoder is a decoder for handling data produced by the corresponding
+// encoder which expects delta packed lengths followed by the bytes of data.
+type DeltaLengthByteArrayDecoder struct {
+	decoder
+
+	mem     memory.Allocator
+	lengths []int32
+}
+
+// Type returns the underlying type which is handled by this encoder, ByteArrays only.
+func (DeltaLengthByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+func (d *DeltaLengthByteArrayDecoder) Allocator() memory.Allocator { return d.mem }
+
+// SetData sets in the expected data to the decoder which should be nvalues delta packed lengths
+// followed by the rest of the byte array data immediately after.
+func (d *DeltaLengthByteArrayDecoder) SetData(nvalues int, data []byte) error {
+	dec := DeltaBitPackInt32Decoder{
+		deltaBitPackDecoder: &deltaBitPackDecoder{
+			decoder: newDecoderBase(d.encoding, d.descr),
+			mem:     d.mem}}
+
+	if err := dec.SetData(nvalues, data); err != nil {
+		return err
+	}
+	d.lengths = make([]int32, nvalues)
+	dec.Decode(d.lengths)
+
+	return d.decoder.SetData(nvalues, data[int(dec.bytesRead()):])
+}
+
+// Decode populates the passed in slice with data decoded until it hits the length of out
+// or runs out of values in the column to decode, then returns the number of values actually decoded.
+func (d *DeltaLengthByteArrayDecoder) Decode(out []parquet.ByteArray) (int, error) {
+	max := utils.MinInt(len(out), d.nvals)
+	for i := 0; i < max; i++ {
+		out[i] = d.data[:d.lengths[i]]

Review comment:
       do you mean making the contents of the slice unmodifiable? No.
   
   That said, it might make sense to change `d.data[:d.lengths[i]]` to `d.data[:d.lengths[i]:d.lengths[i]]` so that they can't be expanded further.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r645846526



##########
File path: go/parquet/internal/encoding/byte_array_encoder.go
##########
@@ -0,0 +1,111 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+	"unsafe"
+
+	"github.com/apache/arrow/go/arrow"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+)
+
+// PlainByteArrayEncoder encodes byte arrays according to the spec for Plain encoding
+// by encoding the length as a uint32 followed by the bytes of the value.
+type PlainByteArrayEncoder struct {
+	encoder
+}
+
+// PutByteArray writes out the 4 bytes for the length followed by the data
+func (enc *PlainByteArrayEncoder) PutByteArray(val parquet.ByteArray) {
+	inc := val.Len() + arrow.Uint32SizeBytes
+	enc.sink.Reserve(inc)
+	vlen := uint32(val.Len())
+	enc.sink.UnsafeWrite((*(*[4]byte)(unsafe.Pointer(&vlen)))[:])

Review comment:
       fair point, i should update this to do the byteswap if necessary so that a bigendian platform works correctly to write it as a little endian value.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r663313791



##########
File path: go/parquet/internal/encoding/plain_encoder_types.gen.go
##########
@@ -0,0 +1,553 @@
+// Code generated by plain_encoder_types.gen.go.tmpl. DO NOT EDIT.
+
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"math"
+
+	"github.com/apache/arrow/go/arrow"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// PlainInt32Encoder is an encoder for int32 values using Plain Encoding
+// which in general is just storing the values as raw bytes of the appropriate size
+type PlainInt32Encoder struct {
+	encoder
+
+	bitSetReader utils.SetBitRunReader
+}
+
+// Put encodes a slice of values into the underlying buffer
+func (enc *PlainInt32Encoder) Put(in []int32) {
+	enc.append(arrow.Int32Traits.CastToBytes(in))

Review comment:
       Will this work on bigendian?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r644346457



##########
File path: go/parquet/internal/encoding/boolean_decoder.go
##########
@@ -0,0 +1,98 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"github.com/apache/arrow/go/arrow/bitutil"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+)
+
+// PlainBooleanDecoder is for the Plain Encoding type, there is no
+// dictionary decoding for bools.
+type PlainBooleanDecoder struct {
+	decoder
+
+	bitOffset int
+}
+
+// Type for the PlainBooleanDecoder is parquet.Types.Boolean
+func (PlainBooleanDecoder) Type() parquet.Type {
+	return parquet.Types.Boolean
+}
+
+// Decode fills out with bools decoded from the data at the current point
+// or until we reach the end of the data.
+//
+// Returns the number of values decoded
+func (dec *PlainBooleanDecoder) Decode(out []bool) (int, error) {
+	max := utils.MinInt(len(out), dec.nvals)
+
+	// if we aren't at a byte boundary, then get bools until we hit
+	// a byte boundary with the bit offset.
+	i := 0
+	for dec.bitOffset != 0 && dec.bitOffset < 8 && i < max {
+		out[i] = (dec.data[0] & byte(1<<dec.bitOffset)) != 0
+		dec.bitOffset++
+		i++
+	}
+	if dec.bitOffset == 8 {
+		dec.bitOffset = 0
+	}
+
+	// determine the number of full bytes worth of bits we can decode
+	// given the number of values we want to decode.
+	bitsRemain := max - i
+	batch := bitsRemain / 8 * 8
+	if batch > 0 { // only go in here if there's at least one full byte to decode
+		if i > 0 { // skip our data forward if we decoded anything above
+			dec.data = dec.data[1:]
+			out = out[i:]
+		}
+		// determine the number of aligned bytes we can grab using SIMD optimized
+		// functions to improve performance.
+		alignedBytes := bitutil.BytesForBits(int64(batch))
+		utils.BytesToBools(dec.data[:alignedBytes], out)
+		dec.data = dec.data[alignedBytes:]
+		out = out[alignedBytes*8:]
+	}
+
+	// grab any trailing bits now that we've got our aligned bytes.
+	for ; dec.bitOffset < (bitsRemain - batch); dec.bitOffset++ {
+		out[dec.bitOffset] = (dec.data[0] & byte(1<<dec.bitOffset)) != 0
+	}
+
+	dec.nvals -= max
+	return max, nil
+}
+
+// DecodeSpaced is like Decode except it expands the values to leave spaces for null
+// as determined by the validBits bitmap.
+func (dec *PlainBooleanDecoder) DecodeSpaced(out []bool, nullCount int, validBits []byte, validBitsOffset int64) (int, error) {
+	if nullCount > 0 {
+		toRead := len(out) - nullCount
+		valuesRead, err := dec.Decode(out[:toRead])
+		if err != nil {
+			return 0, err
+		}
+		if valuesRead != toRead {
+			panic("parquet: number of values / definition levels read did not match")

Review comment:
       updated to return an error, that's a good point.

##########
File path: go/parquet/internal/encoding/boolean_decoder.go
##########
@@ -0,0 +1,98 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"github.com/apache/arrow/go/arrow/bitutil"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+)
+
+// PlainBooleanDecoder is for the Plain Encoding type, there is no
+// dictionary decoding for bools.
+type PlainBooleanDecoder struct {
+	decoder
+
+	bitOffset int
+}
+
+// Type for the PlainBooleanDecoder is parquet.Types.Boolean
+func (PlainBooleanDecoder) Type() parquet.Type {
+	return parquet.Types.Boolean
+}
+
+// Decode fills out with bools decoded from the data at the current point
+// or until we reach the end of the data.
+//
+// Returns the number of values decoded
+func (dec *PlainBooleanDecoder) Decode(out []bool) (int, error) {
+	max := utils.MinInt(len(out), dec.nvals)
+
+	// if we aren't at a byte boundary, then get bools until we hit
+	// a byte boundary with the bit offset.
+	i := 0
+	for dec.bitOffset != 0 && dec.bitOffset < 8 && i < max {
+		out[i] = (dec.data[0] & byte(1<<dec.bitOffset)) != 0
+		dec.bitOffset++
+		i++
+	}
+	if dec.bitOffset == 8 {
+		dec.bitOffset = 0
+	}
+
+	// determine the number of full bytes worth of bits we can decode
+	// given the number of values we want to decode.
+	bitsRemain := max - i
+	batch := bitsRemain / 8 * 8
+	if batch > 0 { // only go in here if there's at least one full byte to decode
+		if i > 0 { // skip our data forward if we decoded anything above
+			dec.data = dec.data[1:]
+			out = out[i:]
+		}
+		// determine the number of aligned bytes we can grab using SIMD optimized
+		// functions to improve performance.
+		alignedBytes := bitutil.BytesForBits(int64(batch))
+		utils.BytesToBools(dec.data[:alignedBytes], out)
+		dec.data = dec.data[alignedBytes:]
+		out = out[alignedBytes*8:]
+	}
+
+	// grab any trailing bits now that we've got our aligned bytes.
+	for ; dec.bitOffset < (bitsRemain - batch); dec.bitOffset++ {
+		out[dec.bitOffset] = (dec.data[0] & byte(1<<dec.bitOffset)) != 0

Review comment:
       done.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r647553533



##########
File path: go/parquet/internal/encoding/decoder.go
##########
@@ -0,0 +1,178 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"bytes"
+	"reflect"
+
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"github.com/apache/arrow/go/parquet/schema"
+)
+
+// DecoderTraits provides an interface for more easily interacting with types
+// to generate decoders for specific types.
+type DecoderTraits interface {
+	Decoder(parquet.Encoding, *schema.Column, bool, memory.Allocator) TypedDecoder
+	BytesRequired(int) int
+}
+
+// NewDecoder constructs a decoder for a given type and encoding
+func NewDecoder(t parquet.Type, e parquet.Encoding, descr *schema.Column, mem memory.Allocator) TypedDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	return traits.Decoder(e, descr, false, mem)
+}
+
+// NewDictDecoder is like NewDecoder but for dictionary encodings, panics if type is bool.
+//
+// if mem is nil, memory.DefaultAllocator will be used
+func NewDictDecoder(t parquet.Type, descr *schema.Column, mem memory.Allocator) DictDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	if mem == nil {
+		mem = memory.DefaultAllocator
+	}
+
+	return traits.Decoder(parquet.Encodings.RLEDict, descr, true, mem).(DictDecoder)
+}
+
+type decoder struct {
+	descr    *schema.Column
+	encoding format.Encoding
+	nvals    int
+	data     []byte
+	typeLen  int
+}
+
+// newDecoderBase constructs the base decoding object that is embedded in the
+// type specific decoders.
+func newDecoderBase(e format.Encoding, descr *schema.Column) decoder {
+	typeLen := -1
+	if descr != nil && descr.PhysicalType() == parquet.Types.FixedLenByteArray {
+		typeLen = int(descr.TypeLength())
+	}
+
+	return decoder{
+		descr:    descr,
+		encoding: e,
+		typeLen:  typeLen,
+	}
+}
+
+// SetData sets the data for decoding into the decoder to update the available
+// data bytes and number of values available.
+func (d *decoder) SetData(nvals int, data []byte) {
+	d.data = data
+	d.nvals = nvals
+}
+
+// ValuesLeft returns the number of remaining values that can be decoded
+func (d *decoder) ValuesLeft() int { return d.nvals }
+
+// Encoding returns the encoding type used by this decoder to decode the bytes.
+func (d *decoder) Encoding() parquet.Encoding { return parquet.Encoding(d.encoding) }
+
+type dictDecoder struct {
+	decoder
+	mem              memory.Allocator
+	dictValueDecoder utils.DictionaryConverter
+	idxDecoder       *utils.RleDecoder
+}
+
+// SetDict sets a decoder that can be used to decode the dictionary that is
+// used for this column in order to return the proper values.
+func (d *dictDecoder) SetDict(dict TypedDecoder) {
+	if dict.Type() != d.descr.PhysicalType() {
+		panic("parquet: mismatch dictionary and column data type")
+	}
+
+	d.dictValueDecoder = NewDictConverter(dict)
+}
+
+// SetData sets the index value data into the decoder.
+func (d *dictDecoder) SetData(nvals int, data []byte) {
+	d.nvals = nvals
+	if len(data) == 0 {
+		d.idxDecoder = utils.NewRleDecoder(bytes.NewReader(data), 1)
+		return
+	}
+
+	width := uint8(data[0])
+	if width >= 64 {
+		panic("parquet: invalid or corrupted bit width")
+	}
+
+	d.idxDecoder = utils.NewRleDecoder(bytes.NewReader(data[1:]), int(width))
+}
+
+func (d *dictDecoder) decode(out interface{}) (int, error) {
+	return d.idxDecoder.GetBatchWithDict(d.dictValueDecoder, out)
+}
+
+func (d *dictDecoder) decodeSpaced(out interface{}, nullCount int, validBits []byte, validBitsOffset int64) (int, error) {
+	return d.idxDecoder.GetBatchWithDictSpaced(d.dictValueDecoder, out, nullCount, validBits, validBitsOffset)
+}
+
+var empty = [1]byte{0}
+
+// spacedExpand is used to take a slice of data and utilize the bitmap provided to fill in nulls into the
+// correct slots according to the bitmap in order to produce a fully expanded result slice with nulls
+// in the correct slots.
+func spacedExpand(buffer interface{}, nullCount int, validBits []byte, validBitsOffset int64) int {
+	bufferRef := reflect.ValueOf(buffer)
+	if bufferRef.Kind() != reflect.Slice {
+		panic("invalid spacedexpand type, not slice")
+	}
+
+	var (
+		numValues int = bufferRef.Len()
+	)
+
+	idxDecode := int32(numValues - nullCount)
+	if idxDecode == 0 { // if there's nothing to decode there's nothing to do.
+		return numValues
+	}
+
+	// read the bitmap in reverse grabbing runs of valid bits where possible.
+	rdr := utils.NewReverseSetBitRunReader(validBits, validBitsOffset, int64(numValues))
+	for {
+		run := rdr.NextRun()
+		if run.Length == 0 {
+			break
+		}
+
+		// copy data from the end of the slice to it's proper location in the slice after accounting for the nulls
+		// because we technically don't care what is in the null slots we don't actually have to clean
+		// up after ourselves because we're doing this in reverse to guarantee that we'll always simply
+		// overwrite any existing data with the correctly spaced data. Any data that happens to be left in the null
+		// slots is fine since it shouldn't matter and saves us work.
+		idxDecode -= int32(run.Length)
+		reflect.Copy(bufferRef.Slice(int(run.Pos), bufferRef.Len()), bufferRef.Slice(int(idxDecode), int(int64(idxDecode)+run.Length)))

Review comment:
       does this work for bits (i.e. boolean?)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r644926066



##########
File path: go/parquet/internal/encoding/byte_array_decoder.go
##########
@@ -0,0 +1,84 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// PlainByteArrayDecoder decodes a data chunk for bytearrays according to
+// the plain encoding. The byte arrays will use slices to reference the
+// data rather than copying it.
+type PlainByteArrayDecoder struct {
+	decoder
+}
+
+// Type returns parquet.Types.ByteArray for this decoder
+func (PlainByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+// Decode will populate the slice of bytearrays in full or until the number
+// of values is emptied.
+//
+// Returns the number of values that were decoded.
+func (pbad *PlainByteArrayDecoder) Decode(out []parquet.ByteArray) (int, error) {
+	max := utils.MinInt(len(out), pbad.nvals)
+
+	for i := 0; i < max; i++ {
+		// there should always be at least four bytes which is the length of the
+		// next value in the data.
+		if len(pbad.data) < 4 {
+			return i, xerrors.New("parquet: eof reading bytearray")
+		}
+
+		// the first 4 bytes are a little endian uint32 length
+		nbytes := int32(binary.LittleEndian.Uint32(pbad.data[:4]))
+		if nbytes < 0 {

Review comment:
       byte_len or str_len might be a better variable name?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r645676237



##########
File path: go/parquet/internal/encoding/byte_array_encoder.go
##########
@@ -0,0 +1,111 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+	"unsafe"
+
+	"github.com/apache/arrow/go/arrow"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+)
+
+// PlainByteArrayEncoder encodes byte arrays according to the spec for Plain encoding
+// by encoding the length as a uint32 followed by the bytes of the value.

Review comment:
       ```suggestion
   // by encoding the length as a int32 followed by the bytes of the value.
   ```
   
   I'll start a thread to clarify this on dev@parquet.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r663312923



##########
File path: go/parquet/internal/encoding/delta_byte_array.go
##########
@@ -0,0 +1,206 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// DeltaByteArrayEncoder is an encoder for writing bytearrays which are delta encoded
+// this is also known as incremental encoding or front compression. For each element
+// in a sequence of strings, we store the prefix length of the previous entry plus the suffix
+// see https://en.wikipedia.org/wiki/Incremental_encoding for a longer description.
+//
+// This is stored as a sequence of delta-encoded prefix lengths followed by the suffixes
+// encoded as delta length byte arrays.
+type DeltaByteArrayEncoder struct {
+	encoder
+
+	prefixEncoder *DeltaBitPackInt32Encoder
+	suffixEncoder *DeltaLengthByteArrayEncoder
+
+	lastVal parquet.ByteArray
+}
+
+func (enc *DeltaByteArrayEncoder) initEncoders() {
+	enc.prefixEncoder = &DeltaBitPackInt32Encoder{
+		deltaBitPackEncoder: &deltaBitPackEncoder{encoder: newEncoderBase(enc.encoding, nil, enc.mem)}}
+	enc.suffixEncoder = &DeltaLengthByteArrayEncoder{
+		newEncoderBase(enc.encoding, nil, enc.mem),
+		&DeltaBitPackInt32Encoder{
+			deltaBitPackEncoder: &deltaBitPackEncoder{encoder: newEncoderBase(enc.encoding, nil, enc.mem)}}}
+}
+
+// Type returns the underlying physical type this operates on, in this case ByteArrays only
+func (DeltaByteArrayEncoder) Type() parquet.Type { return parquet.Types.ByteArray }
+
+// Put writes a slice of ByteArrays to the encoder
+func (enc *DeltaByteArrayEncoder) Put(in []parquet.ByteArray) {
+	if len(in) == 0 {
+		return
+	}
+
+	var suf parquet.ByteArray
+	if enc.prefixEncoder == nil { // initialize our encoders if we haven't yet
+		enc.initEncoders()
+		enc.prefixEncoder.Put([]int32{0})
+		suf = in[0]
+		enc.lastVal = append([]byte(nil), in[0]...)
+		enc.suffixEncoder.Put([]parquet.ByteArray{suf})
+		in = in[1:]
+	}
+
+	// for each value, figure out the common prefix with the previous value
+	// and then write the prefix length and the suffix.
+	for _, val := range in {
+		l1 := enc.lastVal.Len()
+		l2 := val.Len()
+		j := 0
+		for j < l1 && j < l2 {
+			if enc.lastVal[j] != val[j] {
+				break
+			}
+			j++
+		}
+		enc.prefixEncoder.Put([]int32{int32(j)})
+		suf = val[j:]
+		enc.suffixEncoder.Put([]parquet.ByteArray{suf})
+		enc.lastVal = append([]byte(nil), val...)
+	}
+}
+
+// PutSpaced is like Put, but assumes the data is already spaced for nulls and uses the bitmap provided and offset
+// to compress the data before writing it without the null slots.
+func (enc *DeltaByteArrayEncoder) PutSpaced(in []parquet.ByteArray, validBits []byte, validBitsOffset int64) {
+	if validBits != nil {
+		data := make([]parquet.ByteArray, len(in))
+		nvalid := spacedCompress(in, data, validBits, validBitsOffset)
+		enc.Put(data[:nvalid])
+	} else {
+		enc.Put(in)
+	}
+}
+
+// Flush flushes any remaining data out and returns the finished encoded buffer.
+func (enc *DeltaByteArrayEncoder) FlushValues() Buffer {
+	if enc.prefixEncoder == nil {
+		enc.initEncoders()
+	}
+	prefixBuf := enc.prefixEncoder.FlushValues()
+	defer prefixBuf.Release()
+
+	suffixBuf := enc.suffixEncoder.FlushValues()
+	defer suffixBuf.Release()
+
+	ret := bufferPool.Get().(*memory.Buffer)
+	ret.ResizeNoShrink(prefixBuf.Len() + suffixBuf.Len())
+	copy(ret.Bytes(), prefixBuf.Bytes())
+	copy(ret.Bytes()[prefixBuf.Len():], suffixBuf.Bytes())
+	return poolBuffer{ret}
+}
+
+// DeltaByteArrayDecoder is a decoder for a column of data encoded using incremental or prefix encoding.
+type DeltaByteArrayDecoder struct {
+	*DeltaLengthByteArrayDecoder
+
+	prefixLengths []int32
+	lastVal       parquet.ByteArray
+}
+
+// Type returns the underlying physical type this decoder operates on, in this case ByteArrays only
+func (DeltaByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+func (d *DeltaByteArrayDecoder) Allocator() memory.Allocator { return d.mem }
+
+// SetData expects the data passed in to be the prefix lengths, followed by the
+// blocks of suffix data in order to initialize the decoder.
+func (d *DeltaByteArrayDecoder) SetData(nvalues int, data []byte) error {
+	prefixLenDec := DeltaBitPackInt32Decoder{
+		deltaBitPackDecoder: &deltaBitPackDecoder{
+			decoder: newDecoderBase(d.encoding, d.descr),
+			mem:     d.mem}}
+
+	if err := prefixLenDec.SetData(nvalues, data); err != nil {
+		return err
+	}
+
+	d.prefixLengths = make([]int32, nvalues)
+	// decode all the prefix lengths first so we know how many bytes it took to get the
+	// prefix lengths for nvalues
+	prefixLenDec.Decode(d.prefixLengths)
+
+	// now that we know how many bytes we needed for the prefix lengths, the rest are the
+	// delta length byte array encoding.
+	return d.DeltaLengthByteArrayDecoder.SetData(nvalues, data[int(prefixLenDec.bytesRead()):])
+}
+
+// Decode decodes byte arrays into the slice provided and returns the number of values actually decoded
+func (d *DeltaByteArrayDecoder) Decode(out []parquet.ByteArray) (int, error) {
+	max := utils.MinInt(len(out), d.nvals)
+	if max == 0 {
+		return 0, nil
+	}
+	out = out[:max]
+
+	var err error
+	if d.lastVal == nil {
+		_, err = d.DeltaLengthByteArrayDecoder.Decode(out[:1])
+		if err != nil {
+			return 0, err
+		}
+		d.lastVal = out[0]
+		out = out[1:]
+		d.prefixLengths = d.prefixLengths[1:]
+	}
+
+	var prefixLen int32
+	suffixHolder := make([]parquet.ByteArray, 1)
+	for len(out) > 0 {
+		prefixLen, d.prefixLengths = d.prefixLengths[0], d.prefixLengths[1:]
+
+		prefix := d.lastVal[:prefixLen]
+		_, err = d.DeltaLengthByteArrayDecoder.Decode(suffixHolder)
+		if err != nil {
+			return 0, err
+		}
+
+		d.lastVal = make([]byte, 0, int(prefixLen)+len(suffixHolder[0]))

Review comment:
       this preallocates the target array?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r664656768



##########
File path: go/parquet/internal/encoding/delta_byte_array.go
##########
@@ -0,0 +1,206 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// DeltaByteArrayEncoder is an encoder for writing bytearrays which are delta encoded
+// this is also known as incremental encoding or front compression. For each element
+// in a sequence of strings, we store the prefix length of the previous entry plus the suffix
+// see https://en.wikipedia.org/wiki/Incremental_encoding for a longer description.
+//
+// This is stored as a sequence of delta-encoded prefix lengths followed by the suffixes
+// encoded as delta length byte arrays.
+type DeltaByteArrayEncoder struct {
+	encoder
+
+	prefixEncoder *DeltaBitPackInt32Encoder
+	suffixEncoder *DeltaLengthByteArrayEncoder
+
+	lastVal parquet.ByteArray
+}
+
+func (enc *DeltaByteArrayEncoder) initEncoders() {
+	enc.prefixEncoder = &DeltaBitPackInt32Encoder{
+		deltaBitPackEncoder: &deltaBitPackEncoder{encoder: newEncoderBase(enc.encoding, nil, enc.mem)}}
+	enc.suffixEncoder = &DeltaLengthByteArrayEncoder{
+		newEncoderBase(enc.encoding, nil, enc.mem),
+		&DeltaBitPackInt32Encoder{
+			deltaBitPackEncoder: &deltaBitPackEncoder{encoder: newEncoderBase(enc.encoding, nil, enc.mem)}}}
+}
+
+// Type returns the underlying physical type this operates on, in this case ByteArrays only
+func (DeltaByteArrayEncoder) Type() parquet.Type { return parquet.Types.ByteArray }
+
+// Put writes a slice of ByteArrays to the encoder
+func (enc *DeltaByteArrayEncoder) Put(in []parquet.ByteArray) {
+	if len(in) == 0 {
+		return
+	}
+
+	var suf parquet.ByteArray
+	if enc.prefixEncoder == nil { // initialize our encoders if we haven't yet
+		enc.initEncoders()
+		enc.prefixEncoder.Put([]int32{0})
+		suf = in[0]
+		enc.lastVal = append([]byte(nil), in[0]...)
+		enc.suffixEncoder.Put([]parquet.ByteArray{suf})
+		in = in[1:]
+	}
+
+	// for each value, figure out the common prefix with the previous value
+	// and then write the prefix length and the suffix.
+	for _, val := range in {
+		l1 := enc.lastVal.Len()
+		l2 := val.Len()
+		j := 0
+		for j < l1 && j < l2 {
+			if enc.lastVal[j] != val[j] {
+				break
+			}
+			j++
+		}
+		enc.prefixEncoder.Put([]int32{int32(j)})
+		suf = val[j:]
+		enc.suffixEncoder.Put([]parquet.ByteArray{suf})
+		enc.lastVal = append([]byte(nil), val...)
+	}
+}
+
+// PutSpaced is like Put, but assumes the data is already spaced for nulls and uses the bitmap provided and offset
+// to compress the data before writing it without the null slots.
+func (enc *DeltaByteArrayEncoder) PutSpaced(in []parquet.ByteArray, validBits []byte, validBitsOffset int64) {
+	if validBits != nil {
+		data := make([]parquet.ByteArray, len(in))
+		nvalid := spacedCompress(in, data, validBits, validBitsOffset)
+		enc.Put(data[:nvalid])
+	} else {
+		enc.Put(in)
+	}
+}
+
+// Flush flushes any remaining data out and returns the finished encoded buffer.
+func (enc *DeltaByteArrayEncoder) FlushValues() Buffer {
+	if enc.prefixEncoder == nil {
+		enc.initEncoders()
+	}
+	prefixBuf := enc.prefixEncoder.FlushValues()
+	defer prefixBuf.Release()
+
+	suffixBuf := enc.suffixEncoder.FlushValues()
+	defer suffixBuf.Release()
+
+	ret := bufferPool.Get().(*memory.Buffer)
+	ret.ResizeNoShrink(prefixBuf.Len() + suffixBuf.Len())
+	copy(ret.Bytes(), prefixBuf.Bytes())
+	copy(ret.Bytes()[prefixBuf.Len():], suffixBuf.Bytes())
+	return poolBuffer{ret}
+}
+
+// DeltaByteArrayDecoder is a decoder for a column of data encoded using incremental or prefix encoding.
+type DeltaByteArrayDecoder struct {
+	*DeltaLengthByteArrayDecoder
+
+	prefixLengths []int32
+	lastVal       parquet.ByteArray
+}
+
+// Type returns the underlying physical type this decoder operates on, in this case ByteArrays only
+func (DeltaByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+func (d *DeltaByteArrayDecoder) Allocator() memory.Allocator { return d.mem }
+
+// SetData expects the data passed in to be the prefix lengths, followed by the
+// blocks of suffix data in order to initialize the decoder.
+func (d *DeltaByteArrayDecoder) SetData(nvalues int, data []byte) error {
+	prefixLenDec := DeltaBitPackInt32Decoder{
+		deltaBitPackDecoder: &deltaBitPackDecoder{
+			decoder: newDecoderBase(d.encoding, d.descr),
+			mem:     d.mem}}
+
+	if err := prefixLenDec.SetData(nvalues, data); err != nil {
+		return err
+	}
+
+	d.prefixLengths = make([]int32, nvalues)
+	// decode all the prefix lengths first so we know how many bytes it took to get the
+	// prefix lengths for nvalues
+	prefixLenDec.Decode(d.prefixLengths)
+
+	// now that we know how many bytes we needed for the prefix lengths, the rest are the
+	// delta length byte array encoding.
+	return d.DeltaLengthByteArrayDecoder.SetData(nvalues, data[int(prefixLenDec.bytesRead()):])
+}
+
+// Decode decodes byte arrays into the slice provided and returns the number of values actually decoded
+func (d *DeltaByteArrayDecoder) Decode(out []parquet.ByteArray) (int, error) {
+	max := utils.MinInt(len(out), d.nvals)
+	if max == 0 {
+		return 0, nil
+	}
+	out = out[:max]
+
+	var err error
+	if d.lastVal == nil {
+		_, err = d.DeltaLengthByteArrayDecoder.Decode(out[:1])
+		if err != nil {
+			return 0, err
+		}
+		d.lastVal = out[0]
+		out = out[1:]
+		d.prefixLengths = d.prefixLengths[1:]
+	}
+
+	var prefixLen int32
+	suffixHolder := make([]parquet.ByteArray, 1)
+	for len(out) > 0 {
+		prefixLen, d.prefixLengths = d.prefixLengths[0], d.prefixLengths[1:]
+
+		prefix := d.lastVal[:prefixLen]
+		_, err = d.DeltaLengthByteArrayDecoder.Decode(suffixHolder)
+		if err != nil {
+			return 0, err
+		}
+
+		d.lastVal = make([]byte, 0, int(prefixLen)+len(suffixHolder[0]))

Review comment:
       oh i see, you're saying to just optimize and grab a reference to the prefix slice when suffix len == 0 rather than doing a copy, i think that should work fine. i'll see.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r644089074



##########
File path: go/parquet/internal/encoding/boolean_decoder.go
##########
@@ -0,0 +1,98 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"github.com/apache/arrow/go/arrow/bitutil"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+)
+
+// PlainBooleanDecoder is for the Plain Encoding type, there is no
+// dictionary decoding for bools.
+type PlainBooleanDecoder struct {
+	decoder
+
+	bitOffset int
+}
+
+// Type for the PlainBooleanDecoder is parquet.Types.Boolean
+func (PlainBooleanDecoder) Type() parquet.Type {
+	return parquet.Types.Boolean
+}
+
+// Decode fills out with bools decoded from the data at the current point
+// or until we reach the end of the data.
+//
+// Returns the number of values decoded
+func (dec *PlainBooleanDecoder) Decode(out []bool) (int, error) {
+	max := utils.MinInt(len(out), dec.nvals)
+
+	// if we aren't at a byte boundary, then get bools until we hit
+	// a byte boundary with the bit offset.
+	i := 0
+	for dec.bitOffset != 0 && dec.bitOffset < 8 && i < max {
+		out[i] = (dec.data[0] & byte(1<<dec.bitOffset)) != 0
+		dec.bitOffset++
+		i++
+	}
+	if dec.bitOffset == 8 {
+		dec.bitOffset = 0
+	}
+
+	// determine the number of full bytes worth of bits we can decode
+	// given the number of values we want to decode.
+	bitsRemain := max - i
+	batch := bitsRemain / 8 * 8
+	if batch > 0 { // only go in here if there's at least one full byte to decode
+		if i > 0 { // skip our data forward if we decoded anything above
+			dec.data = dec.data[1:]
+			out = out[i:]
+		}
+		// determine the number of aligned bytes we can grab using SIMD optimized
+		// functions to improve performance.
+		alignedBytes := bitutil.BytesForBits(int64(batch))
+		utils.BytesToBools(dec.data[:alignedBytes], out)
+		dec.data = dec.data[alignedBytes:]
+		out = out[alignedBytes*8:]
+	}
+
+	// grab any trailing bits now that we've got our aligned bytes.
+	for ; dec.bitOffset < (bitsRemain - batch); dec.bitOffset++ {
+		out[dec.bitOffset] = (dec.data[0] & byte(1<<dec.bitOffset)) != 0
+	}
+
+	dec.nvals -= max
+	return max, nil
+}
+
+// DecodeSpaced is like Decode except it expands the values to leave spaces for null
+// as determined by the validBits bitmap.
+func (dec *PlainBooleanDecoder) DecodeSpaced(out []bool, nullCount int, validBits []byte, validBitsOffset int64) (int, error) {
+	if nullCount > 0 {
+		toRead := len(out) - nullCount
+		valuesRead, err := dec.Decode(out[:toRead])
+		if err != nil {
+			return 0, err
+		}
+		if valuesRead != toRead {
+			panic("parquet: number of values / definition levels read did not match")

Review comment:
       unless there is validation elsewhere (and even if there is) it is probably a good idea to return an error here instead of panic, since this could be a bad file instead of code a bug?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r644100191



##########
File path: go/parquet/internal/encoding/boolean_encoder.go
##########
@@ -0,0 +1,112 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"github.com/apache/arrow/go/arrow/bitutil"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+)
+
+const boolBufSize = 1024
+
+// PlainBooleanEncoder encodes bools as a bitmap as per the Plain Encoding
+type PlainBooleanEncoder struct {
+	encoder
+	nbits      int
+	bitsBuffer []byte
+}
+
+// Type for the PlainBooleanEncoder is parquet.Types.Boolean
+func (PlainBooleanEncoder) Type() parquet.Type {
+	return parquet.Types.Boolean
+}
+
+// Put encodes the contents of in into the underlying data buffer.
+func (enc *PlainBooleanEncoder) Put(in []bool) {
+	if enc.bitsBuffer == nil {
+		enc.bitsBuffer = make([]byte, boolBufSize)
+	}
+
+	bitOffset := 0
+	// first check if we are in the middle of a byte due to previous
+	// encoding of data and finish out that byte's bits.
+	if enc.nbits > 0 {
+		bitsToWrite := utils.MinInt(enc.nbits, len(in))
+		beg := (boolBufSize * 8) - enc.nbits
+		for i, val := range in[:bitsToWrite] {
+			bitmask := uint8(1 << uint((beg+i)%8))
+			if val {
+				enc.bitsBuffer[(beg+i)/8] |= bitmask

Review comment:
       general, comment, I'm not sure how good go is about optimizing away "/8" and "*8" operation into shifts for signed ints.  regardless it might pay to factor out some of these operations into ByteForBit.
   
   One other comment, did you port over the BitMapWriter (might be off on the name) from C++, this appears to be exactly what the class was meant for?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#issuecomment-878524594


   @emkornfield  just poking for update to get this merged so i can put the other half up


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r663313304



##########
File path: go/parquet/internal/encoding/delta_length_byte_array.go
##########
@@ -0,0 +1,144 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// DeltaLengthByteArrayEncoder encodes data using by taking all of the byte array lengths
+// and encoding them in front using delta encoding, followed by all of the binary data
+// concatenated back to back. The expected savings is from the cost of encoding the lengths
+// and possibly better compression in the data which will no longer be interleaved with the lengths.
+//
+// This encoding is always preferred over PLAIN for byte array columns where possible.
+//
+// For example, if the data was "Hello", "World", "Foobar", "ABCDEF" the encoded data would be:
+// DeltaEncoding(5, 5, 6, 6) "HelloWorldFoobarABCDEF"
+type DeltaLengthByteArrayEncoder struct {
+	encoder
+
+	lengthEncoder *DeltaBitPackInt32Encoder
+}
+
+// Put writes the provided slice of byte arrays to the encoder
+func (enc *DeltaLengthByteArrayEncoder) Put(in []parquet.ByteArray) {
+	lengths := make([]int32, len(in))
+	totalLen := int(0)
+	for idx, val := range in {
+		lengths[idx] = int32(val.Len())
+		totalLen += val.Len()
+	}
+
+	enc.lengthEncoder.Put(lengths)
+	enc.sink.Reserve(totalLen)
+	for _, val := range in {
+		enc.sink.UnsafeWrite(val)
+	}
+}
+
+// PutSpaced is like Put, but the data is spaced out according to the bitmap provided and is compressed
+// accordingly before it is written to drop the null data from the write.
+func (enc *DeltaLengthByteArrayEncoder) PutSpaced(in []parquet.ByteArray, validBits []byte, validBitsOffset int64) {
+	if validBits != nil {
+		data := make([]parquet.ByteArray, len(in))
+		nvalid := spacedCompress(in, data, validBits, validBitsOffset)
+		enc.Put(data[:nvalid])
+	} else {
+		enc.Put(in)
+	}
+}
+
+// Type returns the underlying type which is handled by this encoder, ByteArrays only.
+func (DeltaLengthByteArrayEncoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+// FlushValues flushes any remaining data and returns the final encoded buffer of data.
+func (enc *DeltaLengthByteArrayEncoder) FlushValues() Buffer {
+	ret := enc.lengthEncoder.FlushValues()
+	defer ret.Release()
+
+	data := enc.sink.Finish()
+	defer data.Release()
+
+	output := bufferPool.Get().(*memory.Buffer)
+	output.ResizeNoShrink(ret.Len() + data.Len())
+	copy(output.Bytes(), ret.Bytes())
+	copy(output.Bytes()[ret.Len():], data.Bytes())
+	return poolBuffer{output}
+}
+
+// DeltaLengthByteArrayDecoder is a decoder for handling data produced by the corresponding
+// encoder which expects delta packed lengths followed by the bytes of data.
+type DeltaLengthByteArrayDecoder struct {
+	decoder
+
+	mem     memory.Allocator
+	lengths []int32
+}
+
+// Type returns the underlying type which is handled by this encoder, ByteArrays only.
+func (DeltaLengthByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+func (d *DeltaLengthByteArrayDecoder) Allocator() memory.Allocator { return d.mem }
+
+// SetData sets in the expected data to the decoder which should be nvalues delta packed lengths
+// followed by the rest of the byte array data immediately after.
+func (d *DeltaLengthByteArrayDecoder) SetData(nvalues int, data []byte) error {
+	dec := DeltaBitPackInt32Decoder{
+		deltaBitPackDecoder: &deltaBitPackDecoder{
+			decoder: newDecoderBase(d.encoding, d.descr),
+			mem:     d.mem}}
+
+	if err := dec.SetData(nvalues, data); err != nil {
+		return err
+	}
+	d.lengths = make([]int32, nvalues)
+	dec.Decode(d.lengths)
+
+	return d.decoder.SetData(nvalues, data[int(dec.bytesRead()):])
+}
+
+// Decode populates the passed in slice with data decoded until it hits the length of out
+// or runs out of values in the column to decode, then returns the number of values actually decoded.
+func (d *DeltaLengthByteArrayDecoder) Decode(out []parquet.ByteArray) (int, error) {
+	max := utils.MinInt(len(out), d.nvals)
+	for i := 0; i < max; i++ {
+		out[i] = d.data[:d.lengths[i]]

Review comment:
       was there something to make the slices unmodifiable?  do we care here?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r645845745



##########
File path: go/parquet/internal/encoding/byte_array_decoder.go
##########
@@ -0,0 +1,88 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// PlainByteArrayDecoder decodes a data chunk for bytearrays according to
+// the plain encoding. The byte arrays will use slices to reference the
+// data rather than copying it.
+//
+// The parquet spec defines Plain encoding for ByteArrays as a 4 byte little
+// endian integer containing the length of the bytearray followed by that many
+// bytes being the raw data of the byte array.
+type PlainByteArrayDecoder struct {
+	decoder
+}
+
+// Type returns parquet.Types.ByteArray for this decoder
+func (PlainByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+// Decode will populate the slice of bytearrays in full or until the number
+// of values is consumed.
+//
+// Returns the number of values that were decoded.
+func (pbad *PlainByteArrayDecoder) Decode(out []parquet.ByteArray) (int, error) {
+	max := utils.MinInt(len(out), pbad.nvals)
+
+	for i := 0; i < max; i++ {
+		// there should always be at least four bytes which is the length of the
+		// next value in the data.
+		if len(pbad.data) < 4 {
+			return i, xerrors.New("parquet: eof reading bytearray")
+		}
+
+		// the first 4 bytes are a little endian int32 length
+		byteLen := int32(binary.LittleEndian.Uint32(pbad.data[:4]))
+		if byteLen < 0 {
+			return i, xerrors.New("parquet: invalid BYTE_ARRAY value")
+		}
+
+		if int64(len(pbad.data)) < int64(byteLen)+4 {
+			return i, xerrors.New("parquet: eof reading bytearray")
+		}
+
+		out[i] = pbad.data[4 : byteLen+4 : byteLen+4]
+		pbad.data = pbad.data[byteLen+4:]
+	}
+
+	pbad.nvals -= max
+	return max, nil
+}
+
+// DecodeSpaced is like Decode, but expands the slice out to leave empty values
+// where the validBits bitmap has 0s
+func (pbad *PlainByteArrayDecoder) DecodeSpaced(out []parquet.ByteArray, nullCount int, validBits []byte, validBitsOffset int64) (int, error) {

Review comment:
       For the most part all of the code that interacts with Arrow arrays is isolated to a single package that i haven't put up yet, which utilizes decoders by being able to call the `DecodeSpaced` functions in order to easily populate arrow array bytes.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r644261917



##########
File path: go/parquet/internal/encoding/boolean_encoder.go
##########
@@ -0,0 +1,112 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"github.com/apache/arrow/go/arrow/bitutil"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+)
+
+const boolBufSize = 1024
+
+// PlainBooleanEncoder encodes bools as a bitmap as per the Plain Encoding
+type PlainBooleanEncoder struct {
+	encoder
+	nbits      int
+	bitsBuffer []byte
+}
+
+// Type for the PlainBooleanEncoder is parquet.Types.Boolean
+func (PlainBooleanEncoder) Type() parquet.Type {
+	return parquet.Types.Boolean
+}
+
+// Put encodes the contents of in into the underlying data buffer.
+func (enc *PlainBooleanEncoder) Put(in []bool) {
+	if enc.bitsBuffer == nil {
+		enc.bitsBuffer = make([]byte, boolBufSize)
+	}
+
+	bitOffset := 0
+	// first check if we are in the middle of a byte due to previous
+	// encoding of data and finish out that byte's bits.
+	if enc.nbits > 0 {
+		bitsToWrite := utils.MinInt(enc.nbits, len(in))
+		beg := (boolBufSize * 8) - enc.nbits
+		for i, val := range in[:bitsToWrite] {
+			bitmask := uint8(1 << uint((beg+i)%8))
+			if val {
+				enc.bitsBuffer[(beg+i)/8] |= bitmask

Review comment:
       I did port the bitmap writer, and originally i was doing that, i need to do some digging to confirm, but if i remember correctly, I ended up doing it this way instead of using the bitmapwriter for this because it ultimately ended up being faster since I had bools rather than appending words, and thus avoided significant function calls by calling `Next()`, `Set()` and `Clear()`. 
   
   I'll take another look at the benchmarks and maybe add a function into the bitmapwriter that takes a bunch of bools to optimize that workflow




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r644923055



##########
File path: go/parquet/internal/encoding/byte_array_decoder.go
##########
@@ -0,0 +1,84 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// PlainByteArrayDecoder decodes a data chunk for bytearrays according to
+// the plain encoding. The byte arrays will use slices to reference the

Review comment:
       might be worth adding a quick summary fo what plain encoding is

##########
File path: go/parquet/internal/encoding/byte_array_decoder.go
##########
@@ -0,0 +1,84 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// PlainByteArrayDecoder decodes a data chunk for bytearrays according to
+// the plain encoding. The byte arrays will use slices to reference the
+// data rather than copying it.
+type PlainByteArrayDecoder struct {
+	decoder
+}
+
+// Type returns parquet.Types.ByteArray for this decoder
+func (PlainByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+// Decode will populate the slice of bytearrays in full or until the number
+// of values is emptied.

Review comment:
       ```suggestion
   // of values is consumed.
   ```
   Is this more accurate?

##########
File path: go/parquet/internal/encoding/byte_array_decoder.go
##########
@@ -0,0 +1,84 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// PlainByteArrayDecoder decodes a data chunk for bytearrays according to
+// the plain encoding. The byte arrays will use slices to reference the
+// data rather than copying it.
+type PlainByteArrayDecoder struct {
+	decoder
+}
+
+// Type returns parquet.Types.ByteArray for this decoder
+func (PlainByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+// Decode will populate the slice of bytearrays in full or until the number
+// of values is emptied.
+//
+// Returns the number of values that were decoded.
+func (pbad *PlainByteArrayDecoder) Decode(out []parquet.ByteArray) (int, error) {
+	max := utils.MinInt(len(out), pbad.nvals)
+
+	for i := 0; i < max; i++ {
+		// there should always be at least four bytes which is the length of the
+		// next value in the data.
+		if len(pbad.data) < 4 {
+			return i, xerrors.New("parquet: eof reading bytearray")
+		}
+
+		// the first 4 bytes are a little endian uint32 length
+		nbytes := int32(binary.LittleEndian.Uint32(pbad.data[:4]))
+		if nbytes < 0 {

Review comment:
       byte_len or str_len might be a better variable name?

##########
File path: go/parquet/internal/encoding/byte_array_decoder.go
##########
@@ -0,0 +1,84 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// PlainByteArrayDecoder decodes a data chunk for bytearrays according to
+// the plain encoding. The byte arrays will use slices to reference the
+// data rather than copying it.
+type PlainByteArrayDecoder struct {
+	decoder
+}
+
+// Type returns parquet.Types.ByteArray for this decoder
+func (PlainByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+// Decode will populate the slice of bytearrays in full or until the number
+// of values is emptied.
+//
+// Returns the number of values that were decoded.
+func (pbad *PlainByteArrayDecoder) Decode(out []parquet.ByteArray) (int, error) {
+	max := utils.MinInt(len(out), pbad.nvals)
+
+	for i := 0; i < max; i++ {
+		// there should always be at least four bytes which is the length of the
+		// next value in the data.
+		if len(pbad.data) < 4 {
+			return i, xerrors.New("parquet: eof reading bytearray")
+		}
+
+		// the first 4 bytes are a little endian uint32 length

Review comment:
       Is there a reference for unsigned?  I believe this is actually [signed](https://github.com/apache/parquet-mr/blob/master/parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesWriter.java#L54)

##########
File path: go/parquet/internal/encoding/byte_array_decoder.go
##########
@@ -0,0 +1,84 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// PlainByteArrayDecoder decodes a data chunk for bytearrays according to
+// the plain encoding. The byte arrays will use slices to reference the
+// data rather than copying it.
+type PlainByteArrayDecoder struct {
+	decoder
+}
+
+// Type returns parquet.Types.ByteArray for this decoder
+func (PlainByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+// Decode will populate the slice of bytearrays in full or until the number
+// of values is emptied.
+//
+// Returns the number of values that were decoded.
+func (pbad *PlainByteArrayDecoder) Decode(out []parquet.ByteArray) (int, error) {
+	max := utils.MinInt(len(out), pbad.nvals)
+
+	for i := 0; i < max; i++ {
+		// there should always be at least four bytes which is the length of the
+		// next value in the data.
+		if len(pbad.data) < 4 {
+			return i, xerrors.New("parquet: eof reading bytearray")
+		}
+
+		// the first 4 bytes are a little endian uint32 length

Review comment:
       it seems that you also check below whether it is less then zero which would be a good indication that it is signed.

##########
File path: go/parquet/internal/encoding/byte_array_decoder.go
##########
@@ -0,0 +1,84 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// PlainByteArrayDecoder decodes a data chunk for bytearrays according to
+// the plain encoding. The byte arrays will use slices to reference the
+// data rather than copying it.
+type PlainByteArrayDecoder struct {
+	decoder
+}
+
+// Type returns parquet.Types.ByteArray for this decoder
+func (PlainByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+// Decode will populate the slice of bytearrays in full or until the number
+// of values is emptied.
+//
+// Returns the number of values that were decoded.
+func (pbad *PlainByteArrayDecoder) Decode(out []parquet.ByteArray) (int, error) {
+	max := utils.MinInt(len(out), pbad.nvals)
+
+	for i := 0; i < max; i++ {
+		// there should always be at least four bytes which is the length of the
+		// next value in the data.
+		if len(pbad.data) < 4 {
+			return i, xerrors.New("parquet: eof reading bytearray")
+		}
+
+		// the first 4 bytes are a little endian uint32 length
+		nbytes := int32(binary.LittleEndian.Uint32(pbad.data[:4]))
+		if nbytes < 0 {
+			return i, xerrors.New("parquet: invalid BYTE_ARRAY value")
+		}
+
+		if int64(len(pbad.data)) < int64(nbytes)+4 {
+			return i, xerrors.New("parquet: eof reading bytearray")
+		}
+
+		out[i] = pbad.data[4 : nbytes+4]

Review comment:
       just curious on Go semantics here, is this a copy or a view?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r647560029



##########
File path: go/parquet/internal/encoding/decoder.go
##########
@@ -0,0 +1,178 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"bytes"
+	"reflect"
+
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"github.com/apache/arrow/go/parquet/schema"
+)
+
+// DecoderTraits provides an interface for more easily interacting with types
+// to generate decoders for specific types.
+type DecoderTraits interface {
+	Decoder(parquet.Encoding, *schema.Column, bool, memory.Allocator) TypedDecoder
+	BytesRequired(int) int
+}
+
+// NewDecoder constructs a decoder for a given type and encoding
+func NewDecoder(t parquet.Type, e parquet.Encoding, descr *schema.Column, mem memory.Allocator) TypedDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	return traits.Decoder(e, descr, false, mem)
+}
+
+// NewDictDecoder is like NewDecoder but for dictionary encodings, panics if type is bool.
+//
+// if mem is nil, memory.DefaultAllocator will be used
+func NewDictDecoder(t parquet.Type, descr *schema.Column, mem memory.Allocator) DictDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	if mem == nil {
+		mem = memory.DefaultAllocator
+	}
+
+	return traits.Decoder(parquet.Encodings.RLEDict, descr, true, mem).(DictDecoder)
+}
+
+type decoder struct {
+	descr    *schema.Column
+	encoding format.Encoding
+	nvals    int
+	data     []byte
+	typeLen  int
+}
+
+// newDecoderBase constructs the base decoding object that is embedded in the
+// type specific decoders.
+func newDecoderBase(e format.Encoding, descr *schema.Column) decoder {
+	typeLen := -1
+	if descr != nil && descr.PhysicalType() == parquet.Types.FixedLenByteArray {
+		typeLen = int(descr.TypeLength())
+	}
+
+	return decoder{
+		descr:    descr,
+		encoding: e,
+		typeLen:  typeLen,
+	}
+}
+
+// SetData sets the data for decoding into the decoder to update the available
+// data bytes and number of values available.
+func (d *decoder) SetData(nvals int, data []byte) {
+	d.data = data
+	d.nvals = nvals
+}
+
+// ValuesLeft returns the number of remaining values that can be decoded
+func (d *decoder) ValuesLeft() int { return d.nvals }
+
+// Encoding returns the encoding type used by this decoder to decode the bytes.
+func (d *decoder) Encoding() parquet.Encoding { return parquet.Encoding(d.encoding) }
+
+type dictDecoder struct {
+	decoder
+	mem              memory.Allocator
+	dictValueDecoder utils.DictionaryConverter
+	idxDecoder       *utils.RleDecoder
+}
+
+// SetDict sets a decoder that can be used to decode the dictionary that is
+// used for this column in order to return the proper values.
+func (d *dictDecoder) SetDict(dict TypedDecoder) {
+	if dict.Type() != d.descr.PhysicalType() {
+		panic("parquet: mismatch dictionary and column data type")
+	}
+
+	d.dictValueDecoder = NewDictConverter(dict)
+}
+
+// SetData sets the index value data into the decoder.
+func (d *dictDecoder) SetData(nvals int, data []byte) {
+	d.nvals = nvals
+	if len(data) == 0 {
+		d.idxDecoder = utils.NewRleDecoder(bytes.NewReader(data), 1)
+		return
+	}
+
+	width := uint8(data[0])
+	if width >= 64 {
+		panic("parquet: invalid or corrupted bit width")
+	}
+
+	d.idxDecoder = utils.NewRleDecoder(bytes.NewReader(data[1:]), int(width))
+}
+
+func (d *dictDecoder) decode(out interface{}) (int, error) {
+	return d.idxDecoder.GetBatchWithDict(d.dictValueDecoder, out)
+}
+
+func (d *dictDecoder) decodeSpaced(out interface{}, nullCount int, validBits []byte, validBitsOffset int64) (int, error) {
+	return d.idxDecoder.GetBatchWithDictSpaced(d.dictValueDecoder, out, nullCount, validBits, validBitsOffset)
+}
+
+var empty = [1]byte{0}
+
+// spacedExpand is used to take a slice of data and utilize the bitmap provided to fill in nulls into the
+// correct slots according to the bitmap in order to produce a fully expanded result slice with nulls
+// in the correct slots.
+func spacedExpand(buffer interface{}, nullCount int, validBits []byte, validBitsOffset int64) int {
+	bufferRef := reflect.ValueOf(buffer)
+	if bufferRef.Kind() != reflect.Slice {
+		panic("invalid spacedexpand type, not slice")
+	}
+
+	var (
+		numValues int = bufferRef.Len()
+	)
+
+	idxDecode := int32(numValues - nullCount)
+	if idxDecode == 0 { // if there's nothing to decode there's nothing to do.
+		return numValues
+	}
+
+	// read the bitmap in reverse grabbing runs of valid bits where possible.
+	rdr := utils.NewReverseSetBitRunReader(validBits, validBitsOffset, int64(numValues))
+	for {
+		run := rdr.NextRun()
+		if run.Length == 0 {
+			break
+		}
+
+		// copy data from the end of the slice to it's proper location in the slice after accounting for the nulls
+		// because we technically don't care what is in the null slots we don't actually have to clean
+		// up after ourselves because we're doing this in reverse to guarantee that we'll always simply
+		// overwrite any existing data with the correctly spaced data. Any data that happens to be left in the null
+		// slots is fine since it shouldn't matter and saves us work.
+		idxDecode -= int32(run.Length)
+		reflect.Copy(bufferRef.Slice(int(run.Pos), bufferRef.Len()), bufferRef.Slice(int(idxDecode), int(int64(idxDecode)+run.Length)))

Review comment:
       they are allowed to overlap




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r647562458



##########
File path: go/parquet/internal/encoding/decoder.go
##########
@@ -0,0 +1,178 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"bytes"
+	"reflect"
+
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"github.com/apache/arrow/go/parquet/schema"
+)
+
+// DecoderTraits provides an interface for more easily interacting with types
+// to generate decoders for specific types.
+type DecoderTraits interface {
+	Decoder(parquet.Encoding, *schema.Column, bool, memory.Allocator) TypedDecoder
+	BytesRequired(int) int
+}
+
+// NewDecoder constructs a decoder for a given type and encoding
+func NewDecoder(t parquet.Type, e parquet.Encoding, descr *schema.Column, mem memory.Allocator) TypedDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	return traits.Decoder(e, descr, false, mem)
+}
+
+// NewDictDecoder is like NewDecoder but for dictionary encodings, panics if type is bool.
+//
+// if mem is nil, memory.DefaultAllocator will be used
+func NewDictDecoder(t parquet.Type, descr *schema.Column, mem memory.Allocator) DictDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	if mem == nil {
+		mem = memory.DefaultAllocator
+	}
+
+	return traits.Decoder(parquet.Encodings.RLEDict, descr, true, mem).(DictDecoder)
+}
+
+type decoder struct {
+	descr    *schema.Column
+	encoding format.Encoding
+	nvals    int
+	data     []byte
+	typeLen  int
+}
+
+// newDecoderBase constructs the base decoding object that is embedded in the
+// type specific decoders.
+func newDecoderBase(e format.Encoding, descr *schema.Column) decoder {
+	typeLen := -1
+	if descr != nil && descr.PhysicalType() == parquet.Types.FixedLenByteArray {
+		typeLen = int(descr.TypeLength())
+	}
+
+	return decoder{
+		descr:    descr,
+		encoding: e,
+		typeLen:  typeLen,
+	}
+}
+
+// SetData sets the data for decoding into the decoder to update the available
+// data bytes and number of values available.
+func (d *decoder) SetData(nvals int, data []byte) {
+	d.data = data
+	d.nvals = nvals
+}
+
+// ValuesLeft returns the number of remaining values that can be decoded
+func (d *decoder) ValuesLeft() int { return d.nvals }
+
+// Encoding returns the encoding type used by this decoder to decode the bytes.
+func (d *decoder) Encoding() parquet.Encoding { return parquet.Encoding(d.encoding) }
+
+type dictDecoder struct {
+	decoder
+	mem              memory.Allocator
+	dictValueDecoder utils.DictionaryConverter
+	idxDecoder       *utils.RleDecoder
+}
+
+// SetDict sets a decoder that can be used to decode the dictionary that is
+// used for this column in order to return the proper values.
+func (d *dictDecoder) SetDict(dict TypedDecoder) {
+	if dict.Type() != d.descr.PhysicalType() {
+		panic("parquet: mismatch dictionary and column data type")
+	}
+
+	d.dictValueDecoder = NewDictConverter(dict)
+}
+
+// SetData sets the index value data into the decoder.
+func (d *dictDecoder) SetData(nvals int, data []byte) {
+	d.nvals = nvals
+	if len(data) == 0 {
+		d.idxDecoder = utils.NewRleDecoder(bytes.NewReader(data), 1)
+		return
+	}
+
+	width := uint8(data[0])
+	if width >= 64 {
+		panic("parquet: invalid or corrupted bit width")
+	}
+
+	d.idxDecoder = utils.NewRleDecoder(bytes.NewReader(data[1:]), int(width))
+}
+
+func (d *dictDecoder) decode(out interface{}) (int, error) {
+	return d.idxDecoder.GetBatchWithDict(d.dictValueDecoder, out)
+}
+
+func (d *dictDecoder) decodeSpaced(out interface{}, nullCount int, validBits []byte, validBitsOffset int64) (int, error) {
+	return d.idxDecoder.GetBatchWithDictSpaced(d.dictValueDecoder, out, nullCount, validBits, validBitsOffset)
+}
+
+var empty = [1]byte{0}
+
+// spacedExpand is used to take a slice of data and utilize the bitmap provided to fill in nulls into the
+// correct slots according to the bitmap in order to produce a fully expanded result slice with nulls
+// in the correct slots.
+func spacedExpand(buffer interface{}, nullCount int, validBits []byte, validBitsOffset int64) int {
+	bufferRef := reflect.ValueOf(buffer)
+	if bufferRef.Kind() != reflect.Slice {
+		panic("invalid spacedexpand type, not slice")
+	}
+
+	var (
+		numValues int = bufferRef.Len()
+	)
+
+	idxDecode := int32(numValues - nullCount)
+	if idxDecode == 0 { // if there's nothing to decode there's nothing to do.
+		return numValues
+	}
+
+	// read the bitmap in reverse grabbing runs of valid bits where possible.
+	rdr := utils.NewReverseSetBitRunReader(validBits, validBitsOffset, int64(numValues))
+	for {
+		run := rdr.NextRun()
+		if run.Length == 0 {
+			break
+		}
+
+		// copy data from the end of the slice to it's proper location in the slice after accounting for the nulls
+		// because we technically don't care what is in the null slots we don't actually have to clean
+		// up after ourselves because we're doing this in reverse to guarantee that we'll always simply
+		// overwrite any existing data with the correctly spaced data. Any data that happens to be left in the null
+		// slots is fine since it shouldn't matter and saves us work.
+		idxDecode -= int32(run.Length)

Review comment:
       small nit: there should be a check to make total length is less then int32 max?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#issuecomment-863603639






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r644964327



##########
File path: go/parquet/internal/encoding/byte_array_decoder.go
##########
@@ -0,0 +1,84 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// PlainByteArrayDecoder decodes a data chunk for bytearrays according to
+// the plain encoding. The byte arrays will use slices to reference the
+// data rather than copying it.
+type PlainByteArrayDecoder struct {
+	decoder
+}
+
+// Type returns parquet.Types.ByteArray for this decoder
+func (PlainByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+// Decode will populate the slice of bytearrays in full or until the number
+// of values is emptied.
+//
+// Returns the number of values that were decoded.
+func (pbad *PlainByteArrayDecoder) Decode(out []parquet.ByteArray) (int, error) {
+	max := utils.MinInt(len(out), pbad.nvals)
+
+	for i := 0; i < max; i++ {
+		// there should always be at least four bytes which is the length of the
+		// next value in the data.
+		if len(pbad.data) < 4 {
+			return i, xerrors.New("parquet: eof reading bytearray")
+		}
+
+		// the first 4 bytes are a little endian uint32 length
+		nbytes := int32(binary.LittleEndian.Uint32(pbad.data[:4]))
+		if nbytes < 0 {
+			return i, xerrors.New("parquet: invalid BYTE_ARRAY value")
+		}
+
+		if int64(len(pbad.data)) < int64(nbytes)+4 {
+			return i, xerrors.New("parquet: eof reading bytearray")
+		}
+
+		out[i] = pbad.data[4 : nbytes+4]

Review comment:
       this is a view (in Go terms it's called a *slice*). It essentially creates an object which contains the pointer to the first byte, then length of the slice (so `nbytes`), and the capacity (the rest of the capacity that exists in `pbad.data`. 
   
   In theory it might actually be better to specify the capacity in this slice so that a consumer couldn't expand the individual slice and clobber other data. I'll make that update.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#issuecomment-866859172


   @emkornfield bump


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r664713411



##########
File path: go/parquet/internal/encoding/delta_byte_array.go
##########
@@ -0,0 +1,206 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// DeltaByteArrayEncoder is an encoder for writing bytearrays which are delta encoded
+// this is also known as incremental encoding or front compression. For each element
+// in a sequence of strings, we store the prefix length of the previous entry plus the suffix
+// see https://en.wikipedia.org/wiki/Incremental_encoding for a longer description.
+//
+// This is stored as a sequence of delta-encoded prefix lengths followed by the suffixes
+// encoded as delta length byte arrays.
+type DeltaByteArrayEncoder struct {
+	encoder
+
+	prefixEncoder *DeltaBitPackInt32Encoder
+	suffixEncoder *DeltaLengthByteArrayEncoder
+
+	lastVal parquet.ByteArray
+}
+
+func (enc *DeltaByteArrayEncoder) initEncoders() {
+	enc.prefixEncoder = &DeltaBitPackInt32Encoder{
+		deltaBitPackEncoder: &deltaBitPackEncoder{encoder: newEncoderBase(enc.encoding, nil, enc.mem)}}
+	enc.suffixEncoder = &DeltaLengthByteArrayEncoder{
+		newEncoderBase(enc.encoding, nil, enc.mem),
+		&DeltaBitPackInt32Encoder{
+			deltaBitPackEncoder: &deltaBitPackEncoder{encoder: newEncoderBase(enc.encoding, nil, enc.mem)}}}
+}
+
+// Type returns the underlying physical type this operates on, in this case ByteArrays only
+func (DeltaByteArrayEncoder) Type() parquet.Type { return parquet.Types.ByteArray }
+
+// Put writes a slice of ByteArrays to the encoder
+func (enc *DeltaByteArrayEncoder) Put(in []parquet.ByteArray) {
+	if len(in) == 0 {
+		return
+	}
+
+	var suf parquet.ByteArray
+	if enc.prefixEncoder == nil { // initialize our encoders if we haven't yet
+		enc.initEncoders()
+		enc.prefixEncoder.Put([]int32{0})
+		suf = in[0]
+		enc.lastVal = append([]byte(nil), in[0]...)
+		enc.suffixEncoder.Put([]parquet.ByteArray{suf})
+		in = in[1:]
+	}
+
+	// for each value, figure out the common prefix with the previous value
+	// and then write the prefix length and the suffix.
+	for _, val := range in {
+		l1 := enc.lastVal.Len()
+		l2 := val.Len()
+		j := 0
+		for j < l1 && j < l2 {
+			if enc.lastVal[j] != val[j] {
+				break
+			}
+			j++
+		}
+		enc.prefixEncoder.Put([]int32{int32(j)})
+		suf = val[j:]
+		enc.suffixEncoder.Put([]parquet.ByteArray{suf})
+		enc.lastVal = append([]byte(nil), val...)
+	}
+}
+
+// PutSpaced is like Put, but assumes the data is already spaced for nulls and uses the bitmap provided and offset
+// to compress the data before writing it without the null slots.
+func (enc *DeltaByteArrayEncoder) PutSpaced(in []parquet.ByteArray, validBits []byte, validBitsOffset int64) {
+	if validBits != nil {
+		data := make([]parquet.ByteArray, len(in))
+		nvalid := spacedCompress(in, data, validBits, validBitsOffset)
+		enc.Put(data[:nvalid])
+	} else {
+		enc.Put(in)
+	}
+}
+
+// Flush flushes any remaining data out and returns the finished encoded buffer.
+func (enc *DeltaByteArrayEncoder) FlushValues() Buffer {
+	if enc.prefixEncoder == nil {
+		enc.initEncoders()
+	}
+	prefixBuf := enc.prefixEncoder.FlushValues()
+	defer prefixBuf.Release()
+
+	suffixBuf := enc.suffixEncoder.FlushValues()
+	defer suffixBuf.Release()
+
+	ret := bufferPool.Get().(*memory.Buffer)
+	ret.ResizeNoShrink(prefixBuf.Len() + suffixBuf.Len())
+	copy(ret.Bytes(), prefixBuf.Bytes())
+	copy(ret.Bytes()[prefixBuf.Len():], suffixBuf.Bytes())
+	return poolBuffer{ret}
+}
+
+// DeltaByteArrayDecoder is a decoder for a column of data encoded using incremental or prefix encoding.
+type DeltaByteArrayDecoder struct {
+	*DeltaLengthByteArrayDecoder
+
+	prefixLengths []int32
+	lastVal       parquet.ByteArray
+}
+
+// Type returns the underlying physical type this decoder operates on, in this case ByteArrays only
+func (DeltaByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+func (d *DeltaByteArrayDecoder) Allocator() memory.Allocator { return d.mem }
+
+// SetData expects the data passed in to be the prefix lengths, followed by the
+// blocks of suffix data in order to initialize the decoder.
+func (d *DeltaByteArrayDecoder) SetData(nvalues int, data []byte) error {
+	prefixLenDec := DeltaBitPackInt32Decoder{
+		deltaBitPackDecoder: &deltaBitPackDecoder{
+			decoder: newDecoderBase(d.encoding, d.descr),
+			mem:     d.mem}}
+
+	if err := prefixLenDec.SetData(nvalues, data); err != nil {
+		return err
+	}
+
+	d.prefixLengths = make([]int32, nvalues)
+	// decode all the prefix lengths first so we know how many bytes it took to get the
+	// prefix lengths for nvalues
+	prefixLenDec.Decode(d.prefixLengths)
+
+	// now that we know how many bytes we needed for the prefix lengths, the rest are the
+	// delta length byte array encoding.
+	return d.DeltaLengthByteArrayDecoder.SetData(nvalues, data[int(prefixLenDec.bytesRead()):])
+}
+
+// Decode decodes byte arrays into the slice provided and returns the number of values actually decoded
+func (d *DeltaByteArrayDecoder) Decode(out []parquet.ByteArray) (int, error) {
+	max := utils.MinInt(len(out), d.nvals)
+	if max == 0 {
+		return 0, nil
+	}
+	out = out[:max]
+
+	var err error
+	if d.lastVal == nil {
+		_, err = d.DeltaLengthByteArrayDecoder.Decode(out[:1])
+		if err != nil {
+			return 0, err
+		}
+		d.lastVal = out[0]
+		out = out[1:]
+		d.prefixLengths = d.prefixLengths[1:]
+	}
+
+	var prefixLen int32
+	suffixHolder := make([]parquet.ByteArray, 1)
+	for len(out) > 0 {
+		prefixLen, d.prefixLengths = d.prefixLengths[0], d.prefixLengths[1:]
+
+		prefix := d.lastVal[:prefixLen]
+		_, err = d.DeltaLengthByteArrayDecoder.Decode(suffixHolder)
+		if err != nil {
+			return 0, err
+		}
+
+		d.lastVal = make([]byte, 0, int(prefixLen)+len(suffixHolder[0]))
+		d.lastVal = append([]byte{}, prefix...)
+		d.lastVal = append(d.lastVal, suffixHolder[0]...)
+		out[0], out = d.lastVal, out[1:]

Review comment:
       OK, in C++ to keep references alive for something like shared_ptr, is actually relatively expensive which is why I asked.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#issuecomment-873337382


   Thanks for the ping, sorry this week went a  bit haywire, going to review some more now.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#issuecomment-869878256


   Sorry will try to finish it off this week.  Please ping me again if you don't hear back anything by Wednesday morning.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r644924084



##########
File path: go/parquet/internal/encoding/byte_array_decoder.go
##########
@@ -0,0 +1,84 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// PlainByteArrayDecoder decodes a data chunk for bytearrays according to
+// the plain encoding. The byte arrays will use slices to reference the
+// data rather than copying it.
+type PlainByteArrayDecoder struct {
+	decoder
+}
+
+// Type returns parquet.Types.ByteArray for this decoder
+func (PlainByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+// Decode will populate the slice of bytearrays in full or until the number
+// of values is emptied.

Review comment:
       ```suggestion
   // of values is consumed.
   ```
   Is this more accurate?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r647557581



##########
File path: go/parquet/internal/encoding/byte_array_encoder.go
##########
@@ -0,0 +1,111 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+	"unsafe"
+
+	"github.com/apache/arrow/go/arrow"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+)
+
+// PlainByteArrayEncoder encodes byte arrays according to the spec for Plain encoding
+// by encoding the length as a uint32 followed by the bytes of the value.
+type PlainByteArrayEncoder struct {
+	encoder
+}
+
+// PutByteArray writes out the 4 bytes for the length followed by the data
+func (enc *PlainByteArrayEncoder) PutByteArray(val parquet.ByteArray) {
+	inc := val.Len() + arrow.Uint32SizeBytes
+	enc.sink.Reserve(inc)
+	vlen := uint32(val.Len())
+	enc.sink.UnsafeWrite((*(*[4]byte)(unsafe.Pointer(&vlen)))[:])
+	enc.sink.UnsafeWrite(val)
+}
+
+// Put writes out all of the values in this slice to the buffer
+func (enc *PlainByteArrayEncoder) Put(in []parquet.ByteArray) {
+	for _, val := range in {
+		enc.PutByteArray(val)
+	}
+}
+
+// PutSpaced uses the bitmap of validBits to leave out anything that is null according
+// to the bitmap.
+//
+// If validBits is nil, this is equivalent to calling Put
+func (enc *PlainByteArrayEncoder) PutSpaced(in []parquet.ByteArray, validBits []byte, validBitsOffset int64) {

Review comment:
       It's an idea that I can look into as a potential simplification refactor definitely. But i agree it's minor in the grand scheme of things :)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#issuecomment-846452513


   @emkornfield @sbinet @nickpoorman And here we go with the next chunk! time for the implementation of the encoding types, we're in the home stretch now, i think i've only got about 2 or 3 more of these: metadata, file reading/writing, and integration with arrow


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#issuecomment-869878256


   Sorry will try to finish it off this week.  Please ping me again if you don't hear back anything by Wednesday morning.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r664709513



##########
File path: go/parquet/internal/encoding/delta_byte_array.go
##########
@@ -0,0 +1,206 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// DeltaByteArrayEncoder is an encoder for writing bytearrays which are delta encoded
+// this is also known as incremental encoding or front compression. For each element
+// in a sequence of strings, we store the prefix length of the previous entry plus the suffix
+// see https://en.wikipedia.org/wiki/Incremental_encoding for a longer description.
+//
+// This is stored as a sequence of delta-encoded prefix lengths followed by the suffixes
+// encoded as delta length byte arrays.
+type DeltaByteArrayEncoder struct {
+	encoder
+
+	prefixEncoder *DeltaBitPackInt32Encoder
+	suffixEncoder *DeltaLengthByteArrayEncoder
+
+	lastVal parquet.ByteArray
+}
+
+func (enc *DeltaByteArrayEncoder) initEncoders() {
+	enc.prefixEncoder = &DeltaBitPackInt32Encoder{
+		deltaBitPackEncoder: &deltaBitPackEncoder{encoder: newEncoderBase(enc.encoding, nil, enc.mem)}}
+	enc.suffixEncoder = &DeltaLengthByteArrayEncoder{
+		newEncoderBase(enc.encoding, nil, enc.mem),
+		&DeltaBitPackInt32Encoder{
+			deltaBitPackEncoder: &deltaBitPackEncoder{encoder: newEncoderBase(enc.encoding, nil, enc.mem)}}}
+}
+
+// Type returns the underlying physical type this operates on, in this case ByteArrays only
+func (DeltaByteArrayEncoder) Type() parquet.Type { return parquet.Types.ByteArray }
+
+// Put writes a slice of ByteArrays to the encoder
+func (enc *DeltaByteArrayEncoder) Put(in []parquet.ByteArray) {
+	if len(in) == 0 {
+		return
+	}
+
+	var suf parquet.ByteArray
+	if enc.prefixEncoder == nil { // initialize our encoders if we haven't yet
+		enc.initEncoders()
+		enc.prefixEncoder.Put([]int32{0})
+		suf = in[0]
+		enc.lastVal = append([]byte(nil), in[0]...)
+		enc.suffixEncoder.Put([]parquet.ByteArray{suf})
+		in = in[1:]
+	}
+
+	// for each value, figure out the common prefix with the previous value
+	// and then write the prefix length and the suffix.
+	for _, val := range in {
+		l1 := enc.lastVal.Len()
+		l2 := val.Len()
+		j := 0
+		for j < l1 && j < l2 {
+			if enc.lastVal[j] != val[j] {
+				break
+			}
+			j++
+		}
+		enc.prefixEncoder.Put([]int32{int32(j)})
+		suf = val[j:]
+		enc.suffixEncoder.Put([]parquet.ByteArray{suf})
+		enc.lastVal = append([]byte(nil), val...)

Review comment:
       that's a good point, i can hold the reference to the slice and only do the copy at the end. updated.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] github-actions[bot] commented on pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#issuecomment-846452439


   https://issues.apache.org/jira/browse/ARROW-12851


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r647551884



##########
File path: go/parquet/internal/encoding/decoder.go
##########
@@ -0,0 +1,178 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"bytes"
+	"reflect"
+
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"github.com/apache/arrow/go/parquet/schema"
+)
+
+// DecoderTraits provides an interface for more easily interacting with types
+// to generate decoders for specific types.
+type DecoderTraits interface {
+	Decoder(parquet.Encoding, *schema.Column, bool, memory.Allocator) TypedDecoder
+	BytesRequired(int) int
+}
+
+// NewDecoder constructs a decoder for a given type and encoding
+func NewDecoder(t parquet.Type, e parquet.Encoding, descr *schema.Column, mem memory.Allocator) TypedDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	return traits.Decoder(e, descr, false, mem)
+}
+
+// NewDictDecoder is like NewDecoder but for dictionary encodings, panics if type is bool.
+//
+// if mem is nil, memory.DefaultAllocator will be used
+func NewDictDecoder(t parquet.Type, descr *schema.Column, mem memory.Allocator) DictDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	if mem == nil {
+		mem = memory.DefaultAllocator
+	}
+
+	return traits.Decoder(parquet.Encodings.RLEDict, descr, true, mem).(DictDecoder)
+}
+
+type decoder struct {
+	descr    *schema.Column
+	encoding format.Encoding
+	nvals    int
+	data     []byte
+	typeLen  int
+}
+
+// newDecoderBase constructs the base decoding object that is embedded in the
+// type specific decoders.
+func newDecoderBase(e format.Encoding, descr *schema.Column) decoder {
+	typeLen := -1
+	if descr != nil && descr.PhysicalType() == parquet.Types.FixedLenByteArray {
+		typeLen = int(descr.TypeLength())
+	}
+
+	return decoder{
+		descr:    descr,
+		encoding: e,
+		typeLen:  typeLen,
+	}
+}
+
+// SetData sets the data for decoding into the decoder to update the available
+// data bytes and number of values available.
+func (d *decoder) SetData(nvals int, data []byte) {
+	d.data = data
+	d.nvals = nvals
+}
+
+// ValuesLeft returns the number of remaining values that can be decoded
+func (d *decoder) ValuesLeft() int { return d.nvals }
+
+// Encoding returns the encoding type used by this decoder to decode the bytes.
+func (d *decoder) Encoding() parquet.Encoding { return parquet.Encoding(d.encoding) }
+
+type dictDecoder struct {
+	decoder
+	mem              memory.Allocator
+	dictValueDecoder utils.DictionaryConverter
+	idxDecoder       *utils.RleDecoder
+}
+
+// SetDict sets a decoder that can be used to decode the dictionary that is
+// used for this column in order to return the proper values.
+func (d *dictDecoder) SetDict(dict TypedDecoder) {
+	if dict.Type() != d.descr.PhysicalType() {
+		panic("parquet: mismatch dictionary and column data type")
+	}
+
+	d.dictValueDecoder = NewDictConverter(dict)
+}
+
+// SetData sets the index value data into the decoder.
+func (d *dictDecoder) SetData(nvals int, data []byte) {
+	d.nvals = nvals
+	if len(data) == 0 {
+		d.idxDecoder = utils.NewRleDecoder(bytes.NewReader(data), 1)
+		return
+	}
+
+	width := uint8(data[0])
+	if width >= 64 {
+		panic("parquet: invalid or corrupted bit width")
+	}
+
+	d.idxDecoder = utils.NewRleDecoder(bytes.NewReader(data[1:]), int(width))
+}
+
+func (d *dictDecoder) decode(out interface{}) (int, error) {
+	return d.idxDecoder.GetBatchWithDict(d.dictValueDecoder, out)
+}
+
+func (d *dictDecoder) decodeSpaced(out interface{}, nullCount int, validBits []byte, validBitsOffset int64) (int, error) {
+	return d.idxDecoder.GetBatchWithDictSpaced(d.dictValueDecoder, out, nullCount, validBits, validBitsOffset)
+}
+
+var empty = [1]byte{0}
+
+// spacedExpand is used to take a slice of data and utilize the bitmap provided to fill in nulls into the
+// correct slots according to the bitmap in order to produce a fully expanded result slice with nulls
+// in the correct slots.
+func spacedExpand(buffer interface{}, nullCount int, validBits []byte, validBitsOffset int64) int {
+	bufferRef := reflect.ValueOf(buffer)
+	if bufferRef.Kind() != reflect.Slice {
+		panic("invalid spacedexpand type, not slice")
+	}
+
+	var (
+		numValues int = bufferRef.Len()
+	)
+
+	idxDecode := int32(numValues - nullCount)
+	if idxDecode == 0 { // if there's nothing to decode there's nothing to do.
+		return numValues
+	}
+
+	// read the bitmap in reverse grabbing runs of valid bits where possible.
+	rdr := utils.NewReverseSetBitRunReader(validBits, validBitsOffset, int64(numValues))
+	for {
+		run := rdr.NextRun()
+		if run.Length == 0 {
+			break
+		}
+
+		// copy data from the end of the slice to it's proper location in the slice after accounting for the nulls
+		// because we technically don't care what is in the null slots we don't actually have to clean
+		// up after ourselves because we're doing this in reverse to guarantee that we'll always simply
+		// overwrite any existing data with the correctly spaced data. Any data that happens to be left in the null
+		// slots is fine since it shouldn't matter and saves us work.

Review comment:
       does Go guarantee zero initialization of Arrays?  If not there is potential for reuse of memory which can "leak" data




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r645677836



##########
File path: go/parquet/internal/encoding/byte_array_encoder.go
##########
@@ -0,0 +1,111 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+	"unsafe"
+
+	"github.com/apache/arrow/go/arrow"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+)
+
+// PlainByteArrayEncoder encodes byte arrays according to the spec for Plain encoding
+// by encoding the length as a uint32 followed by the bytes of the value.
+type PlainByteArrayEncoder struct {
+	encoder
+}
+
+// PutByteArray writes out the 4 bytes for the length followed by the data
+func (enc *PlainByteArrayEncoder) PutByteArray(val parquet.ByteArray) {
+	inc := val.Len() + arrow.Uint32SizeBytes
+	enc.sink.Reserve(inc)
+	vlen := uint32(val.Len())
+	enc.sink.UnsafeWrite((*(*[4]byte)(unsafe.Pointer(&vlen)))[:])
+	enc.sink.UnsafeWrite(val)
+}
+
+// Put writes out all of the values in this slice to the buffer
+func (enc *PlainByteArrayEncoder) Put(in []parquet.ByteArray) {
+	for _, val := range in {
+		enc.PutByteArray(val)

Review comment:
       buffering would be a property of the sink?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#issuecomment-853150678


   @zeroshade still working my way through but at least I've started, will try to continue to review over the next few days.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#issuecomment-879201418


   Sorry, for the delay merging now.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r647554565



##########
File path: go/parquet/internal/encoding/decoder.go
##########
@@ -0,0 +1,178 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"bytes"
+	"reflect"
+
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"github.com/apache/arrow/go/parquet/schema"
+)
+
+// DecoderTraits provides an interface for more easily interacting with types
+// to generate decoders for specific types.
+type DecoderTraits interface {
+	Decoder(parquet.Encoding, *schema.Column, bool, memory.Allocator) TypedDecoder
+	BytesRequired(int) int
+}
+
+// NewDecoder constructs a decoder for a given type and encoding
+func NewDecoder(t parquet.Type, e parquet.Encoding, descr *schema.Column, mem memory.Allocator) TypedDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	return traits.Decoder(e, descr, false, mem)
+}
+
+// NewDictDecoder is like NewDecoder but for dictionary encodings, panics if type is bool.
+//
+// if mem is nil, memory.DefaultAllocator will be used
+func NewDictDecoder(t parquet.Type, descr *schema.Column, mem memory.Allocator) DictDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	if mem == nil {
+		mem = memory.DefaultAllocator
+	}
+
+	return traits.Decoder(parquet.Encodings.RLEDict, descr, true, mem).(DictDecoder)
+}
+
+type decoder struct {
+	descr    *schema.Column
+	encoding format.Encoding
+	nvals    int
+	data     []byte
+	typeLen  int
+}
+
+// newDecoderBase constructs the base decoding object that is embedded in the
+// type specific decoders.
+func newDecoderBase(e format.Encoding, descr *schema.Column) decoder {
+	typeLen := -1
+	if descr != nil && descr.PhysicalType() == parquet.Types.FixedLenByteArray {
+		typeLen = int(descr.TypeLength())
+	}
+
+	return decoder{
+		descr:    descr,
+		encoding: e,
+		typeLen:  typeLen,
+	}
+}
+
+// SetData sets the data for decoding into the decoder to update the available
+// data bytes and number of values available.
+func (d *decoder) SetData(nvals int, data []byte) {
+	d.data = data
+	d.nvals = nvals
+}
+
+// ValuesLeft returns the number of remaining values that can be decoded
+func (d *decoder) ValuesLeft() int { return d.nvals }
+
+// Encoding returns the encoding type used by this decoder to decode the bytes.
+func (d *decoder) Encoding() parquet.Encoding { return parquet.Encoding(d.encoding) }
+
+type dictDecoder struct {
+	decoder
+	mem              memory.Allocator
+	dictValueDecoder utils.DictionaryConverter
+	idxDecoder       *utils.RleDecoder
+}
+
+// SetDict sets a decoder that can be used to decode the dictionary that is
+// used for this column in order to return the proper values.
+func (d *dictDecoder) SetDict(dict TypedDecoder) {
+	if dict.Type() != d.descr.PhysicalType() {
+		panic("parquet: mismatch dictionary and column data type")
+	}
+
+	d.dictValueDecoder = NewDictConverter(dict)
+}
+
+// SetData sets the index value data into the decoder.
+func (d *dictDecoder) SetData(nvals int, data []byte) {
+	d.nvals = nvals
+	if len(data) == 0 {
+		d.idxDecoder = utils.NewRleDecoder(bytes.NewReader(data), 1)
+		return
+	}
+
+	width := uint8(data[0])
+	if width >= 64 {
+		panic("parquet: invalid or corrupted bit width")
+	}
+
+	d.idxDecoder = utils.NewRleDecoder(bytes.NewReader(data[1:]), int(width))
+}
+
+func (d *dictDecoder) decode(out interface{}) (int, error) {
+	return d.idxDecoder.GetBatchWithDict(d.dictValueDecoder, out)
+}
+
+func (d *dictDecoder) decodeSpaced(out interface{}, nullCount int, validBits []byte, validBitsOffset int64) (int, error) {
+	return d.idxDecoder.GetBatchWithDictSpaced(d.dictValueDecoder, out, nullCount, validBits, validBitsOffset)
+}
+
+var empty = [1]byte{0}
+
+// spacedExpand is used to take a slice of data and utilize the bitmap provided to fill in nulls into the
+// correct slots according to the bitmap in order to produce a fully expanded result slice with nulls
+// in the correct slots.
+func spacedExpand(buffer interface{}, nullCount int, validBits []byte, validBitsOffset int64) int {
+	bufferRef := reflect.ValueOf(buffer)
+	if bufferRef.Kind() != reflect.Slice {
+		panic("invalid spacedexpand type, not slice")
+	}
+
+	var (
+		numValues int = bufferRef.Len()
+	)
+
+	idxDecode := int32(numValues - nullCount)
+	if idxDecode == 0 { // if there's nothing to decode there's nothing to do.
+		return numValues
+	}
+
+	// read the bitmap in reverse grabbing runs of valid bits where possible.
+	rdr := utils.NewReverseSetBitRunReader(validBits, validBitsOffset, int64(numValues))
+	for {
+		run := rdr.NextRun()
+		if run.Length == 0 {
+			break
+		}
+
+		// copy data from the end of the slice to it's proper location in the slice after accounting for the nulls
+		// because we technically don't care what is in the null slots we don't actually have to clean
+		// up after ourselves because we're doing this in reverse to guarantee that we'll always simply
+		// overwrite any existing data with the correctly spaced data. Any data that happens to be left in the null
+		// slots is fine since it shouldn't matter and saves us work.
+		idxDecode -= int32(run.Length)
+		reflect.Copy(bufferRef.Slice(int(run.Pos), bufferRef.Len()), bufferRef.Slice(int(idxDecode), int(int64(idxDecode)+run.Length)))

Review comment:
       does copy require non-overlapping ranges?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r664709759



##########
File path: go/parquet/internal/encoding/delta_byte_array.go
##########
@@ -0,0 +1,206 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// DeltaByteArrayEncoder is an encoder for writing bytearrays which are delta encoded
+// this is also known as incremental encoding or front compression. For each element
+// in a sequence of strings, we store the prefix length of the previous entry plus the suffix
+// see https://en.wikipedia.org/wiki/Incremental_encoding for a longer description.
+//
+// This is stored as a sequence of delta-encoded prefix lengths followed by the suffixes
+// encoded as delta length byte arrays.
+type DeltaByteArrayEncoder struct {
+	encoder
+
+	prefixEncoder *DeltaBitPackInt32Encoder
+	suffixEncoder *DeltaLengthByteArrayEncoder
+
+	lastVal parquet.ByteArray
+}
+
+func (enc *DeltaByteArrayEncoder) initEncoders() {
+	enc.prefixEncoder = &DeltaBitPackInt32Encoder{
+		deltaBitPackEncoder: &deltaBitPackEncoder{encoder: newEncoderBase(enc.encoding, nil, enc.mem)}}
+	enc.suffixEncoder = &DeltaLengthByteArrayEncoder{
+		newEncoderBase(enc.encoding, nil, enc.mem),
+		&DeltaBitPackInt32Encoder{
+			deltaBitPackEncoder: &deltaBitPackEncoder{encoder: newEncoderBase(enc.encoding, nil, enc.mem)}}}
+}
+
+// Type returns the underlying physical type this operates on, in this case ByteArrays only
+func (DeltaByteArrayEncoder) Type() parquet.Type { return parquet.Types.ByteArray }
+
+// Put writes a slice of ByteArrays to the encoder
+func (enc *DeltaByteArrayEncoder) Put(in []parquet.ByteArray) {
+	if len(in) == 0 {
+		return
+	}
+
+	var suf parquet.ByteArray
+	if enc.prefixEncoder == nil { // initialize our encoders if we haven't yet
+		enc.initEncoders()
+		enc.prefixEncoder.Put([]int32{0})
+		suf = in[0]
+		enc.lastVal = append([]byte(nil), in[0]...)
+		enc.suffixEncoder.Put([]parquet.ByteArray{suf})
+		in = in[1:]
+	}
+
+	// for each value, figure out the common prefix with the previous value
+	// and then write the prefix length and the suffix.
+	for _, val := range in {
+		l1 := enc.lastVal.Len()
+		l2 := val.Len()
+		j := 0
+		for j < l1 && j < l2 {
+			if enc.lastVal[j] != val[j] {
+				break
+			}
+			j++
+		}
+		enc.prefixEncoder.Put([]int32{int32(j)})
+		suf = val[j:]
+		enc.suffixEncoder.Put([]parquet.ByteArray{suf})
+		enc.lastVal = append([]byte(nil), val...)
+	}
+}
+
+// PutSpaced is like Put, but assumes the data is already spaced for nulls and uses the bitmap provided and offset
+// to compress the data before writing it without the null slots.
+func (enc *DeltaByteArrayEncoder) PutSpaced(in []parquet.ByteArray, validBits []byte, validBitsOffset int64) {
+	if validBits != nil {
+		data := make([]parquet.ByteArray, len(in))
+		nvalid := spacedCompress(in, data, validBits, validBitsOffset)
+		enc.Put(data[:nvalid])
+	} else {
+		enc.Put(in)
+	}
+}
+
+// Flush flushes any remaining data out and returns the finished encoded buffer.
+func (enc *DeltaByteArrayEncoder) FlushValues() Buffer {
+	if enc.prefixEncoder == nil {
+		enc.initEncoders()
+	}
+	prefixBuf := enc.prefixEncoder.FlushValues()
+	defer prefixBuf.Release()
+
+	suffixBuf := enc.suffixEncoder.FlushValues()
+	defer suffixBuf.Release()
+
+	ret := bufferPool.Get().(*memory.Buffer)
+	ret.ResizeNoShrink(prefixBuf.Len() + suffixBuf.Len())
+	copy(ret.Bytes(), prefixBuf.Bytes())
+	copy(ret.Bytes()[prefixBuf.Len():], suffixBuf.Bytes())
+	return poolBuffer{ret}
+}
+
+// DeltaByteArrayDecoder is a decoder for a column of data encoded using incremental or prefix encoding.
+type DeltaByteArrayDecoder struct {
+	*DeltaLengthByteArrayDecoder
+
+	prefixLengths []int32
+	lastVal       parquet.ByteArray
+}
+
+// Type returns the underlying physical type this decoder operates on, in this case ByteArrays only
+func (DeltaByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+func (d *DeltaByteArrayDecoder) Allocator() memory.Allocator { return d.mem }
+
+// SetData expects the data passed in to be the prefix lengths, followed by the
+// blocks of suffix data in order to initialize the decoder.
+func (d *DeltaByteArrayDecoder) SetData(nvalues int, data []byte) error {
+	prefixLenDec := DeltaBitPackInt32Decoder{
+		deltaBitPackDecoder: &deltaBitPackDecoder{
+			decoder: newDecoderBase(d.encoding, d.descr),
+			mem:     d.mem}}
+
+	if err := prefixLenDec.SetData(nvalues, data); err != nil {
+		return err
+	}
+
+	d.prefixLengths = make([]int32, nvalues)
+	// decode all the prefix lengths first so we know how many bytes it took to get the
+	// prefix lengths for nvalues
+	prefixLenDec.Decode(d.prefixLengths)
+
+	// now that we know how many bytes we needed for the prefix lengths, the rest are the
+	// delta length byte array encoding.
+	return d.DeltaLengthByteArrayDecoder.SetData(nvalues, data[int(prefixLenDec.bytesRead()):])
+}
+
+// Decode decodes byte arrays into the slice provided and returns the number of values actually decoded
+func (d *DeltaByteArrayDecoder) Decode(out []parquet.ByteArray) (int, error) {
+	max := utils.MinInt(len(out), d.nvals)
+	if max == 0 {
+		return 0, nil
+	}
+	out = out[:max]
+
+	var err error
+	if d.lastVal == nil {
+		_, err = d.DeltaLengthByteArrayDecoder.Decode(out[:1])
+		if err != nil {
+			return 0, err
+		}
+		d.lastVal = out[0]
+		out = out[1:]
+		d.prefixLengths = d.prefixLengths[1:]
+	}
+
+	var prefixLen int32
+	suffixHolder := make([]parquet.ByteArray, 1)
+	for len(out) > 0 {
+		prefixLen, d.prefixLengths = d.prefixLengths[0], d.prefixLengths[1:]
+
+		prefix := d.lastVal[:prefixLen]
+		_, err = d.DeltaLengthByteArrayDecoder.Decode(suffixHolder)
+		if err != nil {
+			return 0, err
+		}
+
+		d.lastVal = make([]byte, 0, int(prefixLen)+len(suffixHolder[0]))

Review comment:
       updated.

##########
File path: go/parquet/internal/encoding/delta_length_byte_array.go
##########
@@ -0,0 +1,144 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// DeltaLengthByteArrayEncoder encodes data using by taking all of the byte array lengths
+// and encoding them in front using delta encoding, followed by all of the binary data
+// concatenated back to back. The expected savings is from the cost of encoding the lengths
+// and possibly better compression in the data which will no longer be interleaved with the lengths.
+//
+// This encoding is always preferred over PLAIN for byte array columns where possible.
+//
+// For example, if the data was "Hello", "World", "Foobar", "ABCDEF" the encoded data would be:
+// DeltaEncoding(5, 5, 6, 6) "HelloWorldFoobarABCDEF"
+type DeltaLengthByteArrayEncoder struct {
+	encoder
+
+	lengthEncoder *DeltaBitPackInt32Encoder
+}
+
+// Put writes the provided slice of byte arrays to the encoder
+func (enc *DeltaLengthByteArrayEncoder) Put(in []parquet.ByteArray) {
+	lengths := make([]int32, len(in))
+	totalLen := int(0)
+	for idx, val := range in {
+		lengths[idx] = int32(val.Len())
+		totalLen += val.Len()
+	}
+
+	enc.lengthEncoder.Put(lengths)
+	enc.sink.Reserve(totalLen)
+	for _, val := range in {
+		enc.sink.UnsafeWrite(val)
+	}
+}
+
+// PutSpaced is like Put, but the data is spaced out according to the bitmap provided and is compressed
+// accordingly before it is written to drop the null data from the write.
+func (enc *DeltaLengthByteArrayEncoder) PutSpaced(in []parquet.ByteArray, validBits []byte, validBitsOffset int64) {
+	if validBits != nil {
+		data := make([]parquet.ByteArray, len(in))
+		nvalid := spacedCompress(in, data, validBits, validBitsOffset)
+		enc.Put(data[:nvalid])
+	} else {
+		enc.Put(in)
+	}
+}
+
+// Type returns the underlying type which is handled by this encoder, ByteArrays only.
+func (DeltaLengthByteArrayEncoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+// FlushValues flushes any remaining data and returns the final encoded buffer of data.
+func (enc *DeltaLengthByteArrayEncoder) FlushValues() Buffer {
+	ret := enc.lengthEncoder.FlushValues()
+	defer ret.Release()
+
+	data := enc.sink.Finish()
+	defer data.Release()
+
+	output := bufferPool.Get().(*memory.Buffer)
+	output.ResizeNoShrink(ret.Len() + data.Len())
+	copy(output.Bytes(), ret.Bytes())
+	copy(output.Bytes()[ret.Len():], data.Bytes())
+	return poolBuffer{output}
+}
+
+// DeltaLengthByteArrayDecoder is a decoder for handling data produced by the corresponding
+// encoder which expects delta packed lengths followed by the bytes of data.
+type DeltaLengthByteArrayDecoder struct {
+	decoder
+
+	mem     memory.Allocator
+	lengths []int32
+}
+
+// Type returns the underlying type which is handled by this encoder, ByteArrays only.
+func (DeltaLengthByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+func (d *DeltaLengthByteArrayDecoder) Allocator() memory.Allocator { return d.mem }
+
+// SetData sets in the expected data to the decoder which should be nvalues delta packed lengths
+// followed by the rest of the byte array data immediately after.
+func (d *DeltaLengthByteArrayDecoder) SetData(nvalues int, data []byte) error {
+	dec := DeltaBitPackInt32Decoder{
+		deltaBitPackDecoder: &deltaBitPackDecoder{
+			decoder: newDecoderBase(d.encoding, d.descr),
+			mem:     d.mem}}
+
+	if err := dec.SetData(nvalues, data); err != nil {
+		return err
+	}
+	d.lengths = make([]int32, nvalues)
+	dec.Decode(d.lengths)
+
+	return d.decoder.SetData(nvalues, data[int(dec.bytesRead()):])
+}
+
+// Decode populates the passed in slice with data decoded until it hits the length of out
+// or runs out of values in the column to decode, then returns the number of values actually decoded.
+func (d *DeltaLengthByteArrayDecoder) Decode(out []parquet.ByteArray) (int, error) {
+	max := utils.MinInt(len(out), d.nvals)
+	for i := 0; i < max; i++ {
+		out[i] = d.data[:d.lengths[i]]

Review comment:
       done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r645849749



##########
File path: go/parquet/internal/encoding/byte_array_encoder.go
##########
@@ -0,0 +1,111 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+	"unsafe"
+
+	"github.com/apache/arrow/go/arrow"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+)
+
+// PlainByteArrayEncoder encodes byte arrays according to the spec for Plain encoding
+// by encoding the length as a uint32 followed by the bytes of the value.
+type PlainByteArrayEncoder struct {
+	encoder
+}
+
+// PutByteArray writes out the 4 bytes for the length followed by the data
+func (enc *PlainByteArrayEncoder) PutByteArray(val parquet.ByteArray) {
+	inc := val.Len() + arrow.Uint32SizeBytes
+	enc.sink.Reserve(inc)
+	vlen := uint32(val.Len())
+	enc.sink.UnsafeWrite((*(*[4]byte)(unsafe.Pointer(&vlen)))[:])
+	enc.sink.UnsafeWrite(val)
+}
+
+// Put writes out all of the values in this slice to the buffer
+func (enc *PlainByteArrayEncoder) Put(in []parquet.ByteArray) {
+	for _, val := range in {
+		enc.PutByteArray(val)
+	}
+}
+
+// PutSpaced uses the bitmap of validBits to leave out anything that is null according
+// to the bitmap.
+//
+// If validBits is nil, this is equivalent to calling Put
+func (enc *PlainByteArrayEncoder) PutSpaced(in []parquet.ByteArray, validBits []byte, validBitsOffset int64) {
+	if validBits != nil {
+		data := make([]parquet.ByteArray, len(in))
+		nvalid := spacedCompress(in, data, validBits, validBitsOffset)
+		enc.Put(data[:nvalid])
+	} else {
+		enc.Put(in)
+	}
+}
+
+// Type returns parquet.Types.ByteArray for the bytearray encoder
+func (PlainByteArrayEncoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+// WriteDict writes the dictionary out to the provided slice, out should be
+// at least DictEncodedSize() bytes
+func (enc *DictByteArrayEncoder) WriteDict(out []byte) {
+	enc.memo.(BinaryMemoTable).VisitValues(0, func(v []byte) {
+		binary.LittleEndian.PutUint32(out, uint32(len(v)))

Review comment:
       in this case, it's equivalent to doing `memcpy(out, &static_cast<uint32>(intvalue), 4)` in c++, even though it's writing a `uint32` it's just a cast, the byte pattern isn't being changed by the cast so when read back it will still correctly be read as a little endian int32 value.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r664580404



##########
File path: go/parquet/internal/encoding/delta_byte_array.go
##########
@@ -0,0 +1,206 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// DeltaByteArrayEncoder is an encoder for writing bytearrays which are delta encoded
+// this is also known as incremental encoding or front compression. For each element
+// in a sequence of strings, we store the prefix length of the previous entry plus the suffix
+// see https://en.wikipedia.org/wiki/Incremental_encoding for a longer description.
+//
+// This is stored as a sequence of delta-encoded prefix lengths followed by the suffixes
+// encoded as delta length byte arrays.
+type DeltaByteArrayEncoder struct {
+	encoder
+
+	prefixEncoder *DeltaBitPackInt32Encoder
+	suffixEncoder *DeltaLengthByteArrayEncoder
+
+	lastVal parquet.ByteArray
+}
+
+func (enc *DeltaByteArrayEncoder) initEncoders() {
+	enc.prefixEncoder = &DeltaBitPackInt32Encoder{
+		deltaBitPackEncoder: &deltaBitPackEncoder{encoder: newEncoderBase(enc.encoding, nil, enc.mem)}}
+	enc.suffixEncoder = &DeltaLengthByteArrayEncoder{
+		newEncoderBase(enc.encoding, nil, enc.mem),
+		&DeltaBitPackInt32Encoder{
+			deltaBitPackEncoder: &deltaBitPackEncoder{encoder: newEncoderBase(enc.encoding, nil, enc.mem)}}}
+}
+
+// Type returns the underlying physical type this operates on, in this case ByteArrays only
+func (DeltaByteArrayEncoder) Type() parquet.Type { return parquet.Types.ByteArray }
+
+// Put writes a slice of ByteArrays to the encoder
+func (enc *DeltaByteArrayEncoder) Put(in []parquet.ByteArray) {
+	if len(in) == 0 {
+		return
+	}
+
+	var suf parquet.ByteArray
+	if enc.prefixEncoder == nil { // initialize our encoders if we haven't yet
+		enc.initEncoders()
+		enc.prefixEncoder.Put([]int32{0})
+		suf = in[0]
+		enc.lastVal = append([]byte(nil), in[0]...)
+		enc.suffixEncoder.Put([]parquet.ByteArray{suf})
+		in = in[1:]
+	}
+
+	// for each value, figure out the common prefix with the previous value
+	// and then write the prefix length and the suffix.
+	for _, val := range in {
+		l1 := enc.lastVal.Len()
+		l2 := val.Len()
+		j := 0
+		for j < l1 && j < l2 {
+			if enc.lastVal[j] != val[j] {
+				break
+			}
+			j++
+		}
+		enc.prefixEncoder.Put([]int32{int32(j)})
+		suf = val[j:]
+		enc.suffixEncoder.Put([]parquet.ByteArray{suf})
+		enc.lastVal = append([]byte(nil), val...)
+	}
+}
+
+// PutSpaced is like Put, but assumes the data is already spaced for nulls and uses the bitmap provided and offset
+// to compress the data before writing it without the null slots.
+func (enc *DeltaByteArrayEncoder) PutSpaced(in []parquet.ByteArray, validBits []byte, validBitsOffset int64) {
+	if validBits != nil {
+		data := make([]parquet.ByteArray, len(in))
+		nvalid := spacedCompress(in, data, validBits, validBitsOffset)
+		enc.Put(data[:nvalid])
+	} else {
+		enc.Put(in)
+	}
+}
+
+// Flush flushes any remaining data out and returns the finished encoded buffer.
+func (enc *DeltaByteArrayEncoder) FlushValues() Buffer {
+	if enc.prefixEncoder == nil {
+		enc.initEncoders()
+	}
+	prefixBuf := enc.prefixEncoder.FlushValues()
+	defer prefixBuf.Release()
+
+	suffixBuf := enc.suffixEncoder.FlushValues()
+	defer suffixBuf.Release()
+
+	ret := bufferPool.Get().(*memory.Buffer)
+	ret.ResizeNoShrink(prefixBuf.Len() + suffixBuf.Len())
+	copy(ret.Bytes(), prefixBuf.Bytes())
+	copy(ret.Bytes()[prefixBuf.Len():], suffixBuf.Bytes())
+	return poolBuffer{ret}
+}
+
+// DeltaByteArrayDecoder is a decoder for a column of data encoded using incremental or prefix encoding.
+type DeltaByteArrayDecoder struct {
+	*DeltaLengthByteArrayDecoder
+
+	prefixLengths []int32
+	lastVal       parquet.ByteArray
+}
+
+// Type returns the underlying physical type this decoder operates on, in this case ByteArrays only
+func (DeltaByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+func (d *DeltaByteArrayDecoder) Allocator() memory.Allocator { return d.mem }
+
+// SetData expects the data passed in to be the prefix lengths, followed by the
+// blocks of suffix data in order to initialize the decoder.
+func (d *DeltaByteArrayDecoder) SetData(nvalues int, data []byte) error {
+	prefixLenDec := DeltaBitPackInt32Decoder{
+		deltaBitPackDecoder: &deltaBitPackDecoder{
+			decoder: newDecoderBase(d.encoding, d.descr),
+			mem:     d.mem}}
+
+	if err := prefixLenDec.SetData(nvalues, data); err != nil {
+		return err
+	}
+
+	d.prefixLengths = make([]int32, nvalues)
+	// decode all the prefix lengths first so we know how many bytes it took to get the
+	// prefix lengths for nvalues
+	prefixLenDec.Decode(d.prefixLengths)
+
+	// now that we know how many bytes we needed for the prefix lengths, the rest are the
+	// delta length byte array encoding.
+	return d.DeltaLengthByteArrayDecoder.SetData(nvalues, data[int(prefixLenDec.bytesRead()):])
+}
+
+// Decode decodes byte arrays into the slice provided and returns the number of values actually decoded
+func (d *DeltaByteArrayDecoder) Decode(out []parquet.ByteArray) (int, error) {
+	max := utils.MinInt(len(out), d.nvals)
+	if max == 0 {
+		return 0, nil
+	}
+	out = out[:max]
+
+	var err error
+	if d.lastVal == nil {
+		_, err = d.DeltaLengthByteArrayDecoder.Decode(out[:1])
+		if err != nil {
+			return 0, err
+		}
+		d.lastVal = out[0]
+		out = out[1:]
+		d.prefixLengths = d.prefixLengths[1:]
+	}
+
+	var prefixLen int32
+	suffixHolder := make([]parquet.ByteArray, 1)
+	for len(out) > 0 {
+		prefixLen, d.prefixLengths = d.prefixLengths[0], d.prefixLengths[1:]
+
+		prefix := d.lastVal[:prefixLen]
+		_, err = d.DeltaLengthByteArrayDecoder.Decode(suffixHolder)
+		if err != nil {
+			return 0, err
+		}
+
+		d.lastVal = make([]byte, 0, int(prefixLen)+len(suffixHolder[0]))

Review comment:
       Yes, this pre-allocates the target array.
   
   I'm not sure what you mean about handling the case for suffix length = 0, in the case where suffixlength = 0 then it will just allocate and copy the prefix only.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r663312168



##########
File path: go/parquet/internal/encoding/delta_byte_array.go
##########
@@ -0,0 +1,206 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// DeltaByteArrayEncoder is an encoder for writing bytearrays which are delta encoded
+// this is also known as incremental encoding or front compression. For each element
+// in a sequence of strings, we store the prefix length of the previous entry plus the suffix
+// see https://en.wikipedia.org/wiki/Incremental_encoding for a longer description.
+//
+// This is stored as a sequence of delta-encoded prefix lengths followed by the suffixes
+// encoded as delta length byte arrays.
+type DeltaByteArrayEncoder struct {
+	encoder
+
+	prefixEncoder *DeltaBitPackInt32Encoder
+	suffixEncoder *DeltaLengthByteArrayEncoder
+
+	lastVal parquet.ByteArray
+}
+
+func (enc *DeltaByteArrayEncoder) initEncoders() {
+	enc.prefixEncoder = &DeltaBitPackInt32Encoder{
+		deltaBitPackEncoder: &deltaBitPackEncoder{encoder: newEncoderBase(enc.encoding, nil, enc.mem)}}
+	enc.suffixEncoder = &DeltaLengthByteArrayEncoder{
+		newEncoderBase(enc.encoding, nil, enc.mem),
+		&DeltaBitPackInt32Encoder{
+			deltaBitPackEncoder: &deltaBitPackEncoder{encoder: newEncoderBase(enc.encoding, nil, enc.mem)}}}
+}
+
+// Type returns the underlying physical type this operates on, in this case ByteArrays only
+func (DeltaByteArrayEncoder) Type() parquet.Type { return parquet.Types.ByteArray }
+
+// Put writes a slice of ByteArrays to the encoder
+func (enc *DeltaByteArrayEncoder) Put(in []parquet.ByteArray) {
+	if len(in) == 0 {
+		return
+	}
+
+	var suf parquet.ByteArray
+	if enc.prefixEncoder == nil { // initialize our encoders if we haven't yet
+		enc.initEncoders()
+		enc.prefixEncoder.Put([]int32{0})
+		suf = in[0]
+		enc.lastVal = append([]byte(nil), in[0]...)
+		enc.suffixEncoder.Put([]parquet.ByteArray{suf})
+		in = in[1:]
+	}
+
+	// for each value, figure out the common prefix with the previous value
+	// and then write the prefix length and the suffix.
+	for _, val := range in {
+		l1 := enc.lastVal.Len()
+		l2 := val.Len()
+		j := 0
+		for j < l1 && j < l2 {
+			if enc.lastVal[j] != val[j] {
+				break
+			}
+			j++
+		}
+		enc.prefixEncoder.Put([]int32{int32(j)})
+		suf = val[j:]
+		enc.suffixEncoder.Put([]parquet.ByteArray{suf})
+		enc.lastVal = append([]byte(nil), val...)

Review comment:
       this do memcpy copy on each run?  does it make sense to use reference and only make the copy after existing the loop?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#issuecomment-877270990


   @emkornfield Just giving another poke here for any other feedback, thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r645676869



##########
File path: go/parquet/internal/encoding/byte_array_encoder.go
##########
@@ -0,0 +1,111 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+	"unsafe"
+
+	"github.com/apache/arrow/go/arrow"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+)
+
+// PlainByteArrayEncoder encodes byte arrays according to the spec for Plain encoding
+// by encoding the length as a uint32 followed by the bytes of the value.
+type PlainByteArrayEncoder struct {
+	encoder
+}
+
+// PutByteArray writes out the 4 bytes for the length followed by the data
+func (enc *PlainByteArrayEncoder) PutByteArray(val parquet.ByteArray) {
+	inc := val.Len() + arrow.Uint32SizeBytes
+	enc.sink.Reserve(inc)
+	vlen := uint32(val.Len())
+	enc.sink.UnsafeWrite((*(*[4]byte)(unsafe.Pointer(&vlen)))[:])

Review comment:
       do you want to support bigendian platforms here?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r654006657



##########
File path: go/parquet/internal/encoding/delta_bit_packing.go
##########
@@ -0,0 +1,515 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"bytes"
+	"math"
+	"math/bits"
+	"reflect"
+
+	"github.com/apache/arrow/go/arrow"
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// see the deltaBitPack encoder for a description of the encoding format that is
+// used for delta-bitpacking.
+type deltaBitPackDecoder struct {
+	decoder
+
+	mem memory.Allocator
+
+	usedFirst            bool
+	bitdecoder           *utils.BitReader
+	blockSize            uint64
+	currentBlockVals     uint64
+	miniBlocks           uint64
+	valsPerMini          uint64
+	currentMiniBlockVals uint64
+	minDelta             int64
+	miniBlockIdx         uint64
+
+	deltaBitWidths *memory.Buffer
+	deltaBitWidth  byte
+
+	lastVal int64
+}
+
+// returns the number of bytes read so far
+func (d *deltaBitPackDecoder) bytesRead() int64 {
+	return d.bitdecoder.CurOffset()
+}
+
+func (d *deltaBitPackDecoder) Allocator() memory.Allocator { return d.mem }
+
+// SetData sets the bytes and the expected number of values to decode
+// into the decoder, updating the decoder and allowing it to be reused.
+func (d *deltaBitPackDecoder) SetData(nvalues int, data []byte) {
+	// set our data into the underlying decoder for the type
+	d.decoder.SetData(nvalues, data)
+	// create a bit reader for our decoder's values
+	d.bitdecoder = utils.NewBitReader(bytes.NewReader(d.data))
+	d.currentBlockVals = 0
+	d.currentMiniBlockVals = 0
+	if d.deltaBitWidths == nil {
+		d.deltaBitWidths = memory.NewResizableBuffer(d.mem)
+	}
+
+	var ok bool
+	d.blockSize, ok = d.bitdecoder.GetVlqInt()
+	if !ok {
+		panic("parquet: eof exception")
+	}
+
+	if d.miniBlocks, ok = d.bitdecoder.GetVlqInt(); !ok {
+		panic("parquet: eof exception")
+	}
+
+	var totalValues uint64
+	if totalValues, ok = d.bitdecoder.GetVlqInt(); !ok {
+		panic("parquet: eof exception")
+	}
+
+	if int(totalValues) != d.nvals {
+		panic("parquet: mismatch between number of values and count in data header")
+	}
+
+	if d.lastVal, ok = d.bitdecoder.GetZigZagVlqInt(); !ok {
+		panic("parquet: eof exception")
+	}
+
+	d.valsPerMini = uint64(d.blockSize / d.miniBlocks)

Review comment:
       This is my intuition.  I believe values per page are limited to 2^32 - 1.  I was assuming at worst 1 mini block per value.

##########
File path: go/parquet/internal/encoding/delta_bit_packing.go
##########
@@ -0,0 +1,515 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"bytes"
+	"math"
+	"math/bits"
+	"reflect"
+
+	"github.com/apache/arrow/go/arrow"
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// see the deltaBitPack encoder for a description of the encoding format that is
+// used for delta-bitpacking.
+type deltaBitPackDecoder struct {
+	decoder
+
+	mem memory.Allocator
+
+	usedFirst            bool
+	bitdecoder           *utils.BitReader
+	blockSize            uint64
+	currentBlockVals     uint64
+	miniBlocks           uint64
+	valsPerMini          uint64
+	currentMiniBlockVals uint64
+	minDelta             int64
+	miniBlockIdx         uint64
+
+	deltaBitWidths *memory.Buffer
+	deltaBitWidth  byte
+
+	lastVal int64
+}
+
+// returns the number of bytes read so far
+func (d *deltaBitPackDecoder) bytesRead() int64 {
+	return d.bitdecoder.CurOffset()
+}
+
+func (d *deltaBitPackDecoder) Allocator() memory.Allocator { return d.mem }
+
+// SetData sets the bytes and the expected number of values to decode
+// into the decoder, updating the decoder and allowing it to be reused.
+func (d *deltaBitPackDecoder) SetData(nvalues int, data []byte) {
+	// set our data into the underlying decoder for the type
+	d.decoder.SetData(nvalues, data)
+	// create a bit reader for our decoder's values
+	d.bitdecoder = utils.NewBitReader(bytes.NewReader(d.data))
+	d.currentBlockVals = 0
+	d.currentMiniBlockVals = 0
+	if d.deltaBitWidths == nil {
+		d.deltaBitWidths = memory.NewResizableBuffer(d.mem)
+	}
+
+	var ok bool
+	d.blockSize, ok = d.bitdecoder.GetVlqInt()
+	if !ok {
+		panic("parquet: eof exception")
+	}
+
+	if d.miniBlocks, ok = d.bitdecoder.GetVlqInt(); !ok {
+		panic("parquet: eof exception")
+	}
+
+	var totalValues uint64
+	if totalValues, ok = d.bitdecoder.GetVlqInt(); !ok {
+		panic("parquet: eof exception")
+	}
+
+	if int(totalValues) != d.nvals {
+		panic("parquet: mismatch between number of values and count in data header")
+	}
+
+	if d.lastVal, ok = d.bitdecoder.GetZigZagVlqInt(); !ok {
+		panic("parquet: eof exception")
+	}
+
+	d.valsPerMini = uint64(d.blockSize / d.miniBlocks)

Review comment:
       https://github.com/apache/parquet-format/blob/master/src/main/thrift/parquet.thrift#L555 limits number of values.
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#issuecomment-852297082


   @emkornfield Pinging for tuesday update :smiley: 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r645848592



##########
File path: go/parquet/internal/encoding/byte_array_encoder.go
##########
@@ -0,0 +1,111 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+	"unsafe"
+
+	"github.com/apache/arrow/go/arrow"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+)
+
+// PlainByteArrayEncoder encodes byte arrays according to the spec for Plain encoding
+// by encoding the length as a uint32 followed by the bytes of the value.
+type PlainByteArrayEncoder struct {
+	encoder
+}
+
+// PutByteArray writes out the 4 bytes for the length followed by the data
+func (enc *PlainByteArrayEncoder) PutByteArray(val parquet.ByteArray) {
+	inc := val.Len() + arrow.Uint32SizeBytes
+	enc.sink.Reserve(inc)
+	vlen := uint32(val.Len())
+	enc.sink.UnsafeWrite((*(*[4]byte)(unsafe.Pointer(&vlen)))[:])
+	enc.sink.UnsafeWrite(val)
+}
+
+// Put writes out all of the values in this slice to the buffer
+func (enc *PlainByteArrayEncoder) Put(in []parquet.ByteArray) {
+	for _, val := range in {
+		enc.PutByteArray(val)
+	}
+}
+
+// PutSpaced uses the bitmap of validBits to leave out anything that is null according
+// to the bitmap.
+//
+// If validBits is nil, this is equivalent to calling Put
+func (enc *PlainByteArrayEncoder) PutSpaced(in []parquet.ByteArray, validBits []byte, validBitsOffset int64) {

Review comment:
       I'm not sure what you mean, are you saying that the PutSpaced/DecodeSpaced functions only really make sense in the context of being used with Arrow Arrays? and basically having a layer above the encoders that implements the *Spaced functions by performing the compress/expansion and then just calling `Put` without having a PutSpaced/DecodeSpaced on every encoder?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r645678942



##########
File path: go/parquet/internal/encoding/byte_array_encoder.go
##########
@@ -0,0 +1,111 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+	"unsafe"
+
+	"github.com/apache/arrow/go/arrow"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+)
+
+// PlainByteArrayEncoder encodes byte arrays according to the spec for Plain encoding
+// by encoding the length as a uint32 followed by the bytes of the value.
+type PlainByteArrayEncoder struct {
+	encoder
+}
+
+// PutByteArray writes out the 4 bytes for the length followed by the data
+func (enc *PlainByteArrayEncoder) PutByteArray(val parquet.ByteArray) {
+	inc := val.Len() + arrow.Uint32SizeBytes
+	enc.sink.Reserve(inc)
+	vlen := uint32(val.Len())
+	enc.sink.UnsafeWrite((*(*[4]byte)(unsafe.Pointer(&vlen)))[:])
+	enc.sink.UnsafeWrite(val)
+}
+
+// Put writes out all of the values in this slice to the buffer
+func (enc *PlainByteArrayEncoder) Put(in []parquet.ByteArray) {
+	for _, val := range in {
+		enc.PutByteArray(val)
+	}
+}
+
+// PutSpaced uses the bitmap of validBits to leave out anything that is null according
+// to the bitmap.
+//
+// If validBits is nil, this is equivalent to calling Put
+func (enc *PlainByteArrayEncoder) PutSpaced(in []parquet.ByteArray, validBits []byte, validBitsOffset int64) {

Review comment:
       not an issue for now, but it seems a nicer way of doing this (don't know about the performance implications for Go) is some sort of callback/visitor on Arrow arrays instead of bundling it here.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r647559147



##########
File path: go/parquet/internal/encoding/decoder.go
##########
@@ -0,0 +1,178 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"bytes"
+	"reflect"
+
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"github.com/apache/arrow/go/parquet/schema"
+)
+
+// DecoderTraits provides an interface for more easily interacting with types
+// to generate decoders for specific types.
+type DecoderTraits interface {
+	Decoder(parquet.Encoding, *schema.Column, bool, memory.Allocator) TypedDecoder
+	BytesRequired(int) int
+}
+
+// NewDecoder constructs a decoder for a given type and encoding
+func NewDecoder(t parquet.Type, e parquet.Encoding, descr *schema.Column, mem memory.Allocator) TypedDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	return traits.Decoder(e, descr, false, mem)
+}
+
+// NewDictDecoder is like NewDecoder but for dictionary encodings, panics if type is bool.
+//
+// if mem is nil, memory.DefaultAllocator will be used
+func NewDictDecoder(t parquet.Type, descr *schema.Column, mem memory.Allocator) DictDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	if mem == nil {
+		mem = memory.DefaultAllocator
+	}
+
+	return traits.Decoder(parquet.Encodings.RLEDict, descr, true, mem).(DictDecoder)
+}
+
+type decoder struct {
+	descr    *schema.Column
+	encoding format.Encoding
+	nvals    int
+	data     []byte
+	typeLen  int
+}
+
+// newDecoderBase constructs the base decoding object that is embedded in the
+// type specific decoders.
+func newDecoderBase(e format.Encoding, descr *schema.Column) decoder {
+	typeLen := -1
+	if descr != nil && descr.PhysicalType() == parquet.Types.FixedLenByteArray {
+		typeLen = int(descr.TypeLength())
+	}
+
+	return decoder{
+		descr:    descr,
+		encoding: e,
+		typeLen:  typeLen,
+	}
+}
+
+// SetData sets the data for decoding into the decoder to update the available
+// data bytes and number of values available.
+func (d *decoder) SetData(nvals int, data []byte) {
+	d.data = data
+	d.nvals = nvals
+}
+
+// ValuesLeft returns the number of remaining values that can be decoded
+func (d *decoder) ValuesLeft() int { return d.nvals }
+
+// Encoding returns the encoding type used by this decoder to decode the bytes.
+func (d *decoder) Encoding() parquet.Encoding { return parquet.Encoding(d.encoding) }
+
+type dictDecoder struct {
+	decoder
+	mem              memory.Allocator
+	dictValueDecoder utils.DictionaryConverter
+	idxDecoder       *utils.RleDecoder
+}
+
+// SetDict sets a decoder that can be used to decode the dictionary that is
+// used for this column in order to return the proper values.
+func (d *dictDecoder) SetDict(dict TypedDecoder) {
+	if dict.Type() != d.descr.PhysicalType() {
+		panic("parquet: mismatch dictionary and column data type")
+	}
+
+	d.dictValueDecoder = NewDictConverter(dict)
+}
+
+// SetData sets the index value data into the decoder.
+func (d *dictDecoder) SetData(nvals int, data []byte) {
+	d.nvals = nvals
+	if len(data) == 0 {
+		d.idxDecoder = utils.NewRleDecoder(bytes.NewReader(data), 1)
+		return
+	}
+
+	width := uint8(data[0])
+	if width >= 64 {
+		panic("parquet: invalid or corrupted bit width")
+	}
+
+	d.idxDecoder = utils.NewRleDecoder(bytes.NewReader(data[1:]), int(width))
+}
+
+func (d *dictDecoder) decode(out interface{}) (int, error) {
+	return d.idxDecoder.GetBatchWithDict(d.dictValueDecoder, out)
+}
+
+func (d *dictDecoder) decodeSpaced(out interface{}, nullCount int, validBits []byte, validBitsOffset int64) (int, error) {
+	return d.idxDecoder.GetBatchWithDictSpaced(d.dictValueDecoder, out, nullCount, validBits, validBitsOffset)
+}
+
+var empty = [1]byte{0}
+
+// spacedExpand is used to take a slice of data and utilize the bitmap provided to fill in nulls into the
+// correct slots according to the bitmap in order to produce a fully expanded result slice with nulls
+// in the correct slots.
+func spacedExpand(buffer interface{}, nullCount int, validBits []byte, validBitsOffset int64) int {
+	bufferRef := reflect.ValueOf(buffer)
+	if bufferRef.Kind() != reflect.Slice {
+		panic("invalid spacedexpand type, not slice")
+	}
+
+	var (
+		numValues int = bufferRef.Len()
+	)
+
+	idxDecode := int32(numValues - nullCount)
+	if idxDecode == 0 { // if there's nothing to decode there's nothing to do.
+		return numValues
+	}
+
+	// read the bitmap in reverse grabbing runs of valid bits where possible.
+	rdr := utils.NewReverseSetBitRunReader(validBits, validBitsOffset, int64(numValues))
+	for {
+		run := rdr.NextRun()
+		if run.Length == 0 {
+			break
+		}
+
+		// copy data from the end of the slice to it's proper location in the slice after accounting for the nulls
+		// because we technically don't care what is in the null slots we don't actually have to clean
+		// up after ourselves because we're doing this in reverse to guarantee that we'll always simply
+		// overwrite any existing data with the correctly spaced data. Any data that happens to be left in the null
+		// slots is fine since it shouldn't matter and saves us work.
+		idxDecode -= int32(run.Length)
+		reflect.Copy(bufferRef.Slice(int(run.Pos), bufferRef.Len()), bufferRef.Slice(int(idxDecode), int(int64(idxDecode)+run.Length)))

Review comment:
       like C++, in Go booleans are represented in memory as a full byte, so this works for a slice of bools without issue because this would be called on the decoded bool slice, not on the bit packed bytes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] kiszk commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
kiszk commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r664735916



##########
File path: go/parquet/internal/encoding/plain_encoder_types.gen.go
##########
@@ -0,0 +1,553 @@
+// Code generated by plain_encoder_types.gen.go.tmpl. DO NOT EDIT.
+
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"math"
+
+	"github.com/apache/arrow/go/arrow"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// PlainInt32Encoder is an encoder for int32 values using Plain Encoding
+// which in general is just storing the values as raw bytes of the appropriate size
+type PlainInt32Encoder struct {
+	encoder
+
+	bitSetReader utils.SetBitRunReader
+}
+
+// Put encodes a slice of values into the underlying buffer
+func (enc *PlainInt32Encoder) Put(in []int32) {
+	enc.append(arrow.Int32Traits.CastToBytes(in))

Review comment:
       Yes, I know that C++ implementation does not support big-endian yet. Thank you for supporting big-endian from the beginning. We will follow this implementation to fix C++ implementation.
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r647558061



##########
File path: go/parquet/internal/encoding/decoder.go
##########
@@ -0,0 +1,178 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"bytes"
+	"reflect"
+
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"github.com/apache/arrow/go/parquet/schema"
+)
+
+// DecoderTraits provides an interface for more easily interacting with types
+// to generate decoders for specific types.
+type DecoderTraits interface {
+	Decoder(parquet.Encoding, *schema.Column, bool, memory.Allocator) TypedDecoder
+	BytesRequired(int) int
+}
+
+// NewDecoder constructs a decoder for a given type and encoding
+func NewDecoder(t parquet.Type, e parquet.Encoding, descr *schema.Column, mem memory.Allocator) TypedDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	return traits.Decoder(e, descr, false, mem)
+}
+
+// NewDictDecoder is like NewDecoder but for dictionary encodings, panics if type is bool.
+//
+// if mem is nil, memory.DefaultAllocator will be used
+func NewDictDecoder(t parquet.Type, descr *schema.Column, mem memory.Allocator) DictDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	if mem == nil {
+		mem = memory.DefaultAllocator
+	}
+
+	return traits.Decoder(parquet.Encodings.RLEDict, descr, true, mem).(DictDecoder)
+}
+
+type decoder struct {
+	descr    *schema.Column
+	encoding format.Encoding
+	nvals    int
+	data     []byte
+	typeLen  int
+}
+
+// newDecoderBase constructs the base decoding object that is embedded in the
+// type specific decoders.
+func newDecoderBase(e format.Encoding, descr *schema.Column) decoder {
+	typeLen := -1
+	if descr != nil && descr.PhysicalType() == parquet.Types.FixedLenByteArray {
+		typeLen = int(descr.TypeLength())
+	}
+
+	return decoder{
+		descr:    descr,
+		encoding: e,
+		typeLen:  typeLen,
+	}
+}
+
+// SetData sets the data for decoding into the decoder to update the available
+// data bytes and number of values available.
+func (d *decoder) SetData(nvals int, data []byte) {
+	d.data = data
+	d.nvals = nvals
+}
+
+// ValuesLeft returns the number of remaining values that can be decoded
+func (d *decoder) ValuesLeft() int { return d.nvals }
+
+// Encoding returns the encoding type used by this decoder to decode the bytes.
+func (d *decoder) Encoding() parquet.Encoding { return parquet.Encoding(d.encoding) }
+
+type dictDecoder struct {
+	decoder
+	mem              memory.Allocator
+	dictValueDecoder utils.DictionaryConverter
+	idxDecoder       *utils.RleDecoder
+}
+
+// SetDict sets a decoder that can be used to decode the dictionary that is
+// used for this column in order to return the proper values.
+func (d *dictDecoder) SetDict(dict TypedDecoder) {
+	if dict.Type() != d.descr.PhysicalType() {
+		panic("parquet: mismatch dictionary and column data type")
+	}
+
+	d.dictValueDecoder = NewDictConverter(dict)
+}
+
+// SetData sets the index value data into the decoder.
+func (d *dictDecoder) SetData(nvals int, data []byte) {
+	d.nvals = nvals
+	if len(data) == 0 {
+		d.idxDecoder = utils.NewRleDecoder(bytes.NewReader(data), 1)
+		return
+	}
+
+	width := uint8(data[0])
+	if width >= 64 {
+		panic("parquet: invalid or corrupted bit width")
+	}
+
+	d.idxDecoder = utils.NewRleDecoder(bytes.NewReader(data[1:]), int(width))
+}
+
+func (d *dictDecoder) decode(out interface{}) (int, error) {
+	return d.idxDecoder.GetBatchWithDict(d.dictValueDecoder, out)
+}
+
+func (d *dictDecoder) decodeSpaced(out interface{}, nullCount int, validBits []byte, validBitsOffset int64) (int, error) {
+	return d.idxDecoder.GetBatchWithDictSpaced(d.dictValueDecoder, out, nullCount, validBits, validBitsOffset)
+}
+
+var empty = [1]byte{0}
+
+// spacedExpand is used to take a slice of data and utilize the bitmap provided to fill in nulls into the
+// correct slots according to the bitmap in order to produce a fully expanded result slice with nulls
+// in the correct slots.
+func spacedExpand(buffer interface{}, nullCount int, validBits []byte, validBitsOffset int64) int {
+	bufferRef := reflect.ValueOf(buffer)
+	if bufferRef.Kind() != reflect.Slice {
+		panic("invalid spacedexpand type, not slice")
+	}
+
+	var (
+		numValues int = bufferRef.Len()
+	)
+
+	idxDecode := int32(numValues - nullCount)
+	if idxDecode == 0 { // if there's nothing to decode there's nothing to do.
+		return numValues
+	}
+
+	// read the bitmap in reverse grabbing runs of valid bits where possible.
+	rdr := utils.NewReverseSetBitRunReader(validBits, validBitsOffset, int64(numValues))
+	for {
+		run := rdr.NextRun()
+		if run.Length == 0 {
+			break
+		}
+
+		// copy data from the end of the slice to it's proper location in the slice after accounting for the nulls
+		// because we technically don't care what is in the null slots we don't actually have to clean
+		// up after ourselves because we're doing this in reverse to guarantee that we'll always simply
+		// overwrite any existing data with the correctly spaced data. Any data that happens to be left in the null
+		// slots is fine since it shouldn't matter and saves us work.

Review comment:
       yes, Go guarantees all variables are zero initialized, including slices and arrays




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r647547644



##########
File path: go/parquet/internal/encoding/byte_array_encoder.go
##########
@@ -0,0 +1,111 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+	"unsafe"
+
+	"github.com/apache/arrow/go/arrow"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+)
+
+// PlainByteArrayEncoder encodes byte arrays according to the spec for Plain encoding
+// by encoding the length as a uint32 followed by the bytes of the value.
+type PlainByteArrayEncoder struct {
+	encoder
+}
+
+// PutByteArray writes out the 4 bytes for the length followed by the data
+func (enc *PlainByteArrayEncoder) PutByteArray(val parquet.ByteArray) {
+	inc := val.Len() + arrow.Uint32SizeBytes
+	enc.sink.Reserve(inc)
+	vlen := uint32(val.Len())
+	enc.sink.UnsafeWrite((*(*[4]byte)(unsafe.Pointer(&vlen)))[:])
+	enc.sink.UnsafeWrite(val)
+}
+
+// Put writes out all of the values in this slice to the buffer
+func (enc *PlainByteArrayEncoder) Put(in []parquet.ByteArray) {
+	for _, val := range in {
+		enc.PutByteArray(val)

Review comment:
       I was just making sure buffering was happening someplace :)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r645857966



##########
File path: go/parquet/internal/encoding/byte_array_encoder.go
##########
@@ -0,0 +1,111 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+	"unsafe"
+
+	"github.com/apache/arrow/go/arrow"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+)
+
+// PlainByteArrayEncoder encodes byte arrays according to the spec for Plain encoding
+// by encoding the length as a uint32 followed by the bytes of the value.
+type PlainByteArrayEncoder struct {
+	encoder
+}
+
+// PutByteArray writes out the 4 bytes for the length followed by the data
+func (enc *PlainByteArrayEncoder) PutByteArray(val parquet.ByteArray) {
+	inc := val.Len() + arrow.Uint32SizeBytes
+	enc.sink.Reserve(inc)
+	vlen := uint32(val.Len())
+	enc.sink.UnsafeWrite((*(*[4]byte)(unsafe.Pointer(&vlen)))[:])

Review comment:
       updated, current version properly will support big-endian platforms now by reversing the bytes for big endian platforms and leaving them as is for little-endian platforms.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r644930435



##########
File path: go/parquet/internal/encoding/byte_array_decoder.go
##########
@@ -0,0 +1,84 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"golang.org/x/xerrors"
+)
+
+// PlainByteArrayDecoder decodes a data chunk for bytearrays according to
+// the plain encoding. The byte arrays will use slices to reference the
+// data rather than copying it.
+type PlainByteArrayDecoder struct {
+	decoder
+}
+
+// Type returns parquet.Types.ByteArray for this decoder
+func (PlainByteArrayDecoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+// Decode will populate the slice of bytearrays in full or until the number
+// of values is emptied.
+//
+// Returns the number of values that were decoded.
+func (pbad *PlainByteArrayDecoder) Decode(out []parquet.ByteArray) (int, error) {
+	max := utils.MinInt(len(out), pbad.nvals)
+
+	for i := 0; i < max; i++ {
+		// there should always be at least four bytes which is the length of the
+		// next value in the data.
+		if len(pbad.data) < 4 {
+			return i, xerrors.New("parquet: eof reading bytearray")
+		}
+
+		// the first 4 bytes are a little endian uint32 length
+		nbytes := int32(binary.LittleEndian.Uint32(pbad.data[:4]))
+		if nbytes < 0 {
+			return i, xerrors.New("parquet: invalid BYTE_ARRAY value")
+		}
+
+		if int64(len(pbad.data)) < int64(nbytes)+4 {
+			return i, xerrors.New("parquet: eof reading bytearray")
+		}
+
+		out[i] = pbad.data[4 : nbytes+4]

Review comment:
       just curious on Go semantics here, is this a copy or a view?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r648390452



##########
File path: go/parquet/internal/encoding/decoder.go
##########
@@ -0,0 +1,178 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"bytes"
+	"reflect"
+
+	"github.com/apache/arrow/go/arrow/memory"
+	"github.com/apache/arrow/go/parquet"
+	format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+	"github.com/apache/arrow/go/parquet/schema"
+)
+
+// DecoderTraits provides an interface for more easily interacting with types
+// to generate decoders for specific types.
+type DecoderTraits interface {
+	Decoder(parquet.Encoding, *schema.Column, bool, memory.Allocator) TypedDecoder
+	BytesRequired(int) int
+}
+
+// NewDecoder constructs a decoder for a given type and encoding
+func NewDecoder(t parquet.Type, e parquet.Encoding, descr *schema.Column, mem memory.Allocator) TypedDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	return traits.Decoder(e, descr, false, mem)
+}
+
+// NewDictDecoder is like NewDecoder but for dictionary encodings, panics if type is bool.
+//
+// if mem is nil, memory.DefaultAllocator will be used
+func NewDictDecoder(t parquet.Type, descr *schema.Column, mem memory.Allocator) DictDecoder {
+	traits := getDecodingTraits(t)
+	if traits == nil {
+		return nil
+	}
+
+	if mem == nil {
+		mem = memory.DefaultAllocator
+	}
+
+	return traits.Decoder(parquet.Encodings.RLEDict, descr, true, mem).(DictDecoder)
+}
+
+type decoder struct {
+	descr    *schema.Column
+	encoding format.Encoding
+	nvals    int
+	data     []byte
+	typeLen  int
+}
+
+// newDecoderBase constructs the base decoding object that is embedded in the
+// type specific decoders.
+func newDecoderBase(e format.Encoding, descr *schema.Column) decoder {
+	typeLen := -1
+	if descr != nil && descr.PhysicalType() == parquet.Types.FixedLenByteArray {
+		typeLen = int(descr.TypeLength())
+	}
+
+	return decoder{
+		descr:    descr,
+		encoding: e,
+		typeLen:  typeLen,
+	}
+}
+
+// SetData sets the data for decoding into the decoder to update the available
+// data bytes and number of values available.
+func (d *decoder) SetData(nvals int, data []byte) {
+	d.data = data
+	d.nvals = nvals
+}
+
+// ValuesLeft returns the number of remaining values that can be decoded
+func (d *decoder) ValuesLeft() int { return d.nvals }
+
+// Encoding returns the encoding type used by this decoder to decode the bytes.
+func (d *decoder) Encoding() parquet.Encoding { return parquet.Encoding(d.encoding) }
+
+type dictDecoder struct {
+	decoder
+	mem              memory.Allocator
+	dictValueDecoder utils.DictionaryConverter
+	idxDecoder       *utils.RleDecoder
+}
+
+// SetDict sets a decoder that can be used to decode the dictionary that is
+// used for this column in order to return the proper values.
+func (d *dictDecoder) SetDict(dict TypedDecoder) {
+	if dict.Type() != d.descr.PhysicalType() {
+		panic("parquet: mismatch dictionary and column data type")
+	}
+
+	d.dictValueDecoder = NewDictConverter(dict)
+}
+
+// SetData sets the index value data into the decoder.
+func (d *dictDecoder) SetData(nvals int, data []byte) {
+	d.nvals = nvals
+	if len(data) == 0 {
+		d.idxDecoder = utils.NewRleDecoder(bytes.NewReader(data), 1)
+		return
+	}
+
+	width := uint8(data[0])
+	if width >= 64 {
+		panic("parquet: invalid or corrupted bit width")
+	}
+
+	d.idxDecoder = utils.NewRleDecoder(bytes.NewReader(data[1:]), int(width))
+}
+
+func (d *dictDecoder) decode(out interface{}) (int, error) {
+	return d.idxDecoder.GetBatchWithDict(d.dictValueDecoder, out)
+}
+
+func (d *dictDecoder) decodeSpaced(out interface{}, nullCount int, validBits []byte, validBitsOffset int64) (int, error) {
+	return d.idxDecoder.GetBatchWithDictSpaced(d.dictValueDecoder, out, nullCount, validBits, validBitsOffset)
+}
+
+var empty = [1]byte{0}
+
+// spacedExpand is used to take a slice of data and utilize the bitmap provided to fill in nulls into the
+// correct slots according to the bitmap in order to produce a fully expanded result slice with nulls
+// in the correct slots.
+func spacedExpand(buffer interface{}, nullCount int, validBits []byte, validBitsOffset int64) int {
+	bufferRef := reflect.ValueOf(buffer)
+	if bufferRef.Kind() != reflect.Slice {
+		panic("invalid spacedexpand type, not slice")
+	}
+
+	var (
+		numValues int = bufferRef.Len()
+	)
+
+	idxDecode := int32(numValues - nullCount)
+	if idxDecode == 0 { // if there's nothing to decode there's nothing to do.
+		return numValues
+	}
+
+	// read the bitmap in reverse grabbing runs of valid bits where possible.
+	rdr := utils.NewReverseSetBitRunReader(validBits, validBitsOffset, int64(numValues))
+	for {
+		run := rdr.NextRun()
+		if run.Length == 0 {
+			break
+		}
+
+		// copy data from the end of the slice to it's proper location in the slice after accounting for the nulls
+		// because we technically don't care what is in the null slots we don't actually have to clean
+		// up after ourselves because we're doing this in reverse to guarantee that we'll always simply
+		// overwrite any existing data with the correctly spaced data. Any data that happens to be left in the null
+		// slots is fine since it shouldn't matter and saves us work.
+		idxDecode -= int32(run.Length)
+		reflect.Copy(bufferRef.Slice(int(run.Pos), bufferRef.Len()), bufferRef.Slice(int(idxDecode), int(int64(idxDecode)+run.Length)))

Review comment:
       added




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] emkornfield commented on a change in pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#discussion_r645679912



##########
File path: go/parquet/internal/encoding/byte_array_encoder.go
##########
@@ -0,0 +1,111 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package encoding
+
+import (
+	"encoding/binary"
+	"unsafe"
+
+	"github.com/apache/arrow/go/arrow"
+	"github.com/apache/arrow/go/parquet"
+	"github.com/apache/arrow/go/parquet/internal/utils"
+)
+
+// PlainByteArrayEncoder encodes byte arrays according to the spec for Plain encoding
+// by encoding the length as a uint32 followed by the bytes of the value.
+type PlainByteArrayEncoder struct {
+	encoder
+}
+
+// PutByteArray writes out the 4 bytes for the length followed by the data
+func (enc *PlainByteArrayEncoder) PutByteArray(val parquet.ByteArray) {
+	inc := val.Len() + arrow.Uint32SizeBytes
+	enc.sink.Reserve(inc)
+	vlen := uint32(val.Len())
+	enc.sink.UnsafeWrite((*(*[4]byte)(unsafe.Pointer(&vlen)))[:])
+	enc.sink.UnsafeWrite(val)
+}
+
+// Put writes out all of the values in this slice to the buffer
+func (enc *PlainByteArrayEncoder) Put(in []parquet.ByteArray) {
+	for _, val := range in {
+		enc.PutByteArray(val)
+	}
+}
+
+// PutSpaced uses the bitmap of validBits to leave out anything that is null according
+// to the bitmap.
+//
+// If validBits is nil, this is equivalent to calling Put
+func (enc *PlainByteArrayEncoder) PutSpaced(in []parquet.ByteArray, validBits []byte, validBitsOffset int64) {
+	if validBits != nil {
+		data := make([]parquet.ByteArray, len(in))
+		nvalid := spacedCompress(in, data, validBits, validBitsOffset)
+		enc.Put(data[:nvalid])
+	} else {
+		enc.Put(in)
+	}
+}
+
+// Type returns parquet.Types.ByteArray for the bytearray encoder
+func (PlainByteArrayEncoder) Type() parquet.Type {
+	return parquet.Types.ByteArray
+}
+
+// WriteDict writes the dictionary out to the provided slice, out should be
+// at least DictEncodedSize() bytes
+func (enc *DictByteArrayEncoder) WriteDict(out []byte) {
+	enc.memo.(BinaryMemoTable).VisitValues(0, func(v []byte) {
+		binary.LittleEndian.PutUint32(out, uint32(len(v)))

Review comment:
       another int vs uint spot.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] zeroshade commented on pull request #10379: ARROW-12851: [Go][Parquet] Add Golang Parquet encoding package

Posted by GitBox <gi...@apache.org>.
zeroshade commented on pull request #10379:
URL: https://github.com/apache/arrow/pull/10379#issuecomment-869758621


   Just bumping this again in the hopes for more reviews so i can get this merged
   
   @emkornfield @sbinet @fsaintjacques @nickpoorman 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org