You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by ze...@apache.org on 2022/06/16 17:03:30 UTC

[arrow] branch master updated: ARROW-16831: [Go] panic in ipc.Reader when string array offsets are invalid

This is an automated email from the ASF dual-hosted git repository.

zeroshade pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow.git


The following commit(s) were added to refs/heads/master by this push:
     new 771ad83a6d ARROW-16831: [Go] panic in ipc.Reader when string array offsets are invalid
771ad83a6d is described below

commit 771ad83a6d11aa32f58434c02c73646837412fc4
Author: Chris Casola <cc...@factset.com>
AuthorDate: Thu Jun 16 13:03:17 2022 -0400

    ARROW-16831: [Go] panic in ipc.Reader when string array offsets are invalid
    
    Add a check for invalid offsets in a string array and panic. This
    prevents panic'ing later when accessing the column data or
    attempting to write it with ipc.Writer.
    
    Closes #13381 from chriscasola/choff--go-validate-string-offsets
    
    Authored-by: Chris Casola <cc...@factset.com>
    Signed-off-by: Matthew Topol <mt...@factset.com>
---
 go/arrow/array/binary.go      | 13 ++++++++
 go/arrow/array/binary_test.go | 71 +++++++++++++++++++++++++++++++++++++++++++
 go/arrow/array/concat.go      | 16 +++++++---
 go/arrow/array/concat_test.go | 11 -------
 go/arrow/array/string.go      | 13 ++++++++
 go/arrow/array/string_test.go | 71 +++++++++++++++++++++++++++++++++++++++++++
 go/arrow/ipc/reader.go        | 15 +++++++--
 go/arrow/ipc/reader_test.go   | 58 +++++++++++++++++++++++++++++++++++
 go/arrow/ipc/writer.go        | 10 ++++--
 go/arrow/ipc/writer_test.go   | 23 ++++++++++++++
 10 files changed, 281 insertions(+), 20 deletions(-)

diff --git a/go/arrow/array/binary.go b/go/arrow/array/binary.go
index e33181cf1b..b2281f4df9 100644
--- a/go/arrow/array/binary.go
+++ b/go/arrow/array/binary.go
@@ -116,6 +116,19 @@ func (a *Binary) setData(data *Data) {
 	if valueOffsets := data.buffers[1]; valueOffsets != nil {
 		a.valueOffsets = arrow.Int32Traits.CastFromBytes(valueOffsets.Bytes())
 	}
+
+	if a.array.data.length < 1 {
+		return
+	}
+
+	expNumOffsets := a.array.data.offset + a.array.data.length + 1
+	if len(a.valueOffsets) < expNumOffsets {
+		panic(fmt.Errorf("arrow/array: binary offset buffer must have at least %d values", expNumOffsets))
+	}
+
+	if int(a.valueOffsets[expNumOffsets-1]) > len(a.valueBytes) {
+		panic("arrow/array: binary offsets out of bounds of data buffer")
+	}
 }
 
 func (a *Binary) getOneForMarshal(i int) interface{} {
diff --git a/go/arrow/array/binary_test.go b/go/arrow/array/binary_test.go
index ddcc903178..776d679acf 100644
--- a/go/arrow/array/binary_test.go
+++ b/go/arrow/array/binary_test.go
@@ -23,6 +23,7 @@ import (
 	"github.com/stretchr/testify/assert"
 
 	"github.com/apache/arrow/go/v9/arrow"
+	"github.com/apache/arrow/go/v9/arrow/bitutil"
 	"github.com/apache/arrow/go/v9/arrow/memory"
 )
 
@@ -428,3 +429,73 @@ func TestBinaryStringer(t *testing.T) {
 		t.Fatalf("invalid stringer:\ngot= %s\nwant=%s\n", got, want)
 	}
 }
+
+func TestBinaryInvalidOffsets(t *testing.T) {
+	const expectedPanic = "arrow/array: binary offsets out of bounds of data buffer"
+
+	makeBuffers := func(valids []bool, offsets []int32, data string) []*memory.Buffer {
+		offsetBuf := memory.NewBufferBytes(arrow.Int32Traits.CastToBytes(offsets))
+		var nullBufBytes []byte
+		var nullBuf *memory.Buffer
+		if valids != nil {
+			nullBufBytes = make([]byte, bitutil.BytesForBits(int64(len(valids))))
+			for i, v := range valids {
+				bitutil.SetBitTo(nullBufBytes, i, v)
+			}
+			nullBuf = memory.NewBufferBytes(nullBufBytes)
+		}
+		return []*memory.Buffer{nullBuf, offsetBuf, memory.NewBufferBytes([]byte(data))}
+	}
+
+	assert.NotPanics(t, func() {
+		buffers := makeBuffers(nil, []int32{}, "")
+		NewBinaryData(NewData(arrow.BinaryTypes.Binary, 0, buffers, nil, 0, 0))
+	}, "empty array with no offsets")
+
+	assert.NotPanics(t, func() {
+		buffers := makeBuffers(nil, []int32{0, 5}, "")
+		NewBinaryData(NewData(arrow.BinaryTypes.Binary, 0, buffers, nil, 0, 0))
+	}, "empty array, offsets ignored")
+
+	assert.NotPanics(t, func() {
+		buffers := makeBuffers(nil, []int32{0, 3, 4, 9}, "oooabcdef")
+		NewBinaryData(NewData(arrow.BinaryTypes.Binary, 1, buffers, nil, 0, 2))
+	}, "data has offset and value offsets are valid")
+
+	assert.NotPanics(t, func() {
+		buffers := makeBuffers(nil, []int32{0, 3, 6, 9, 9}, "012345678")
+		arr := NewBinaryData(NewData(arrow.BinaryTypes.Binary, 4, buffers, nil, 0, 0))
+		if assert.Equal(t, 4, arr.Len()) && assert.Zero(t, arr.NullN()) {
+			assert.EqualValues(t, "012", arr.Value(0))
+			assert.EqualValues(t, "345", arr.Value(1))
+			assert.EqualValues(t, "678", arr.Value(2))
+			assert.EqualValues(t, "", arr.Value(3), "trailing empty binary value will have offset past end")
+		}
+	}, "simple valid case")
+
+	assert.NotPanics(t, func() {
+		buffers := makeBuffers([]bool{true, false, true, false}, []int32{0, 3, 4, 9, 9}, "oooabcdef")
+		arr := NewBinaryData(NewData(arrow.BinaryTypes.Binary, 4, buffers, nil, 2, 0))
+		if assert.Equal(t, 4, arr.Len()) && assert.Equal(t, 2, arr.NullN()) {
+			assert.EqualValues(t, "ooo", arr.Value(0))
+			assert.True(t, arr.IsNull(1))
+			assert.EqualValues(t, "bcdef", arr.Value(2))
+			assert.True(t, arr.IsNull(3))
+		}
+	}, "simple valid case with nulls")
+
+	assert.PanicsWithValue(t, expectedPanic, func() {
+		buffers := makeBuffers(nil, []int32{0, 5}, "abc")
+		NewBinaryData(NewData(arrow.BinaryTypes.Binary, 1, buffers, nil, 0, 0))
+	}, "last offset is overflowing")
+
+	assert.PanicsWithError(t, "arrow/array: binary offset buffer must have at least 2 values", func() {
+		buffers := makeBuffers(nil, []int32{0}, "abc")
+		NewBinaryData(NewData(arrow.BinaryTypes.Binary, 1, buffers, nil, 0, 0))
+	}, "last offset is missing")
+
+	assert.PanicsWithValue(t, expectedPanic, func() {
+		buffers := makeBuffers(nil, []int32{0, 3, 10, 15}, "oooabcdef")
+		NewBinaryData(NewData(arrow.BinaryTypes.Binary, 1, buffers, nil, 0, 2))
+	}, "data has offset and value offset is overflowing")
+}
diff --git a/go/arrow/array/concat.go b/go/arrow/array/concat.go
index 38344cfda8..2842757859 100644
--- a/go/arrow/array/concat.go
+++ b/go/arrow/array/concat.go
@@ -17,6 +17,7 @@
 package array
 
 import (
+	"errors"
 	"fmt"
 	"math"
 	"math/bits"
@@ -25,7 +26,6 @@ import (
 	"github.com/apache/arrow/go/v9/arrow/bitutil"
 	"github.com/apache/arrow/go/v9/arrow/internal/debug"
 	"github.com/apache/arrow/go/v9/arrow/memory"
-	"golang.org/x/xerrors"
 )
 
 // Concatenate creates a new arrow.Array which is the concatenation of the
@@ -33,11 +33,17 @@ import (
 //
 // The passed in arrays still need to be released manually, and will not be
 // released by this function.
-func Concatenate(arrs []arrow.Array, mem memory.Allocator) (arrow.Array, error) {
+func Concatenate(arrs []arrow.Array, mem memory.Allocator) (result arrow.Array, err error) {
 	if len(arrs) == 0 {
-		return nil, xerrors.New("array/concat: must pass at least one array")
+		return nil, errors.New("array/concat: must pass at least one array")
 	}
 
+	defer func() {
+		if pErr := recover(); pErr != nil {
+			err = fmt.Errorf("arrow/concat: unknown error: %v", pErr)
+		}
+	}()
+
 	// gather Data of inputs
 	data := make([]arrow.ArrayData, len(arrs))
 	for i, ar := range arrs {
@@ -202,7 +208,7 @@ func concatOffsets(buffers []*memory.Buffer, mem memory.Allocator) (*memory.Buff
 		valuesRanges[i].len = int(expand[len(src)]) - valuesRanges[i].offset
 
 		if nextOffset > math.MaxInt32-int32(valuesRanges[i].len) {
-			return nil, nil, xerrors.New("offset overflow while concatenating arrays")
+			return nil, nil, errors.New("offset overflow while concatenating arrays")
 		}
 
 		// adjust each offset by the difference between our last ending point and our starting point
@@ -347,7 +353,7 @@ func concatBitmaps(bitmaps []bitmap, mem memory.Allocator) (*memory.Buffer, erro
 
 	for _, bm := range bitmaps {
 		if outlen, overflow = addOvf(outlen, bm.rng.len); overflow {
-			return nil, xerrors.New("length overflow when concatenating arrays")
+			return nil, errors.New("length overflow when concatenating arrays")
 		}
 	}
 
diff --git a/go/arrow/array/concat_test.go b/go/arrow/array/concat_test.go
index 4ad1abbf2d..8beee43306 100644
--- a/go/arrow/array/concat_test.go
+++ b/go/arrow/array/concat_test.go
@@ -18,7 +18,6 @@ package array_test
 
 import (
 	"fmt"
-	"math"
 	"sort"
 	"testing"
 
@@ -289,13 +288,3 @@ func (cts *ConcatTestSuite) TestCheckConcat() {
 		})
 	}
 }
-
-func TestOffsetOverflow(t *testing.T) {
-	fakeOffsets := memory.NewBufferBytes(arrow.Int32Traits.CastToBytes([]int32{0, math.MaxInt32}))
-	fakeArr := array.NewStringData(array.NewData(arrow.BinaryTypes.String, 1, []*memory.Buffer{nil, fakeOffsets, memory.NewBufferBytes([]byte{})}, nil, 0, 0))
-	var err error
-	assert.NotPanics(t, func() {
-		_, err = array.Concatenate([]arrow.Array{fakeArr, fakeArr}, memory.DefaultAllocator)
-	})
-	assert.EqualError(t, err, "offset overflow while concatenating arrays")
-}
diff --git a/go/arrow/array/string.go b/go/arrow/array/string.go
index ee7a07813e..4c033118a6 100644
--- a/go/arrow/array/string.go
+++ b/go/arrow/array/string.go
@@ -113,6 +113,19 @@ func (a *String) setData(data *Data) {
 	if offsets := data.buffers[1]; offsets != nil {
 		a.offsets = arrow.Int32Traits.CastFromBytes(offsets.Bytes())
 	}
+
+	if a.array.data.length < 1 {
+		return
+	}
+
+	expNumOffsets := a.array.data.offset + a.array.data.length + 1
+	if len(a.offsets) < expNumOffsets {
+		panic(fmt.Errorf("arrow/array: string offset buffer must have at least %d values", expNumOffsets))
+	}
+
+	if int(a.offsets[expNumOffsets-1]) > len(a.values) {
+		panic("arrow/array: string offsets out of bounds of data buffer")
+	}
 }
 
 func (a *String) getOneForMarshal(i int) interface{} {
diff --git a/go/arrow/array/string_test.go b/go/arrow/array/string_test.go
index 2d841eeba9..f0e0325772 100644
--- a/go/arrow/array/string_test.go
+++ b/go/arrow/array/string_test.go
@@ -23,6 +23,7 @@ import (
 
 	"github.com/apache/arrow/go/v9/arrow"
 	"github.com/apache/arrow/go/v9/arrow/array"
+	"github.com/apache/arrow/go/v9/arrow/bitutil"
 	"github.com/apache/arrow/go/v9/arrow/memory"
 	"github.com/stretchr/testify/assert"
 )
@@ -206,3 +207,73 @@ func TestStringReset(t *testing.T) {
 
 	assert.Equal(t, "string1", string2.Value(0))
 }
+
+func TestStringInvalidOffsets(t *testing.T) {
+	const expectedPanic = "arrow/array: string offsets out of bounds of data buffer"
+
+	makeBuffers := func(valids []bool, offsets []int32, data string) []*memory.Buffer {
+		offsetBuf := memory.NewBufferBytes(arrow.Int32Traits.CastToBytes(offsets))
+		var nullBufBytes []byte
+		var nullBuf *memory.Buffer
+		if valids != nil {
+			nullBufBytes = make([]byte, bitutil.BytesForBits(int64(len(valids))))
+			for i, v := range valids {
+				bitutil.SetBitTo(nullBufBytes, i, v)
+			}
+			nullBuf = memory.NewBufferBytes(nullBufBytes)
+		}
+		return []*memory.Buffer{nullBuf, offsetBuf, memory.NewBufferBytes([]byte(data))}
+	}
+
+	assert.NotPanics(t, func() {
+		buffers := makeBuffers(nil, []int32{}, "")
+		array.NewStringData(array.NewData(arrow.BinaryTypes.String, 0, buffers, nil, 0, 0))
+	}, "empty array with no offsets")
+
+	assert.NotPanics(t, func() {
+		buffers := makeBuffers(nil, []int32{0, 5}, "")
+		array.NewStringData(array.NewData(arrow.BinaryTypes.String, 0, buffers, nil, 0, 0))
+	}, "empty array, offsets ignored")
+
+	assert.NotPanics(t, func() {
+		buffers := makeBuffers(nil, []int32{0, 3, 4, 9}, "oooabcdef")
+		array.NewStringData(array.NewData(arrow.BinaryTypes.String, 1, buffers, nil, 0, 2))
+	}, "data has offset and value offsets are valid")
+
+	assert.NotPanics(t, func() {
+		buffers := makeBuffers(nil, []int32{0, 3, 6, 9, 9}, "012345678")
+		arr := array.NewStringData(array.NewData(arrow.BinaryTypes.String, 4, buffers, nil, 0, 0))
+		if assert.Equal(t, 4, arr.Len()) && assert.Zero(t, arr.NullN()) {
+			assert.Equal(t, "012", arr.Value(0))
+			assert.Equal(t, "345", arr.Value(1))
+			assert.Equal(t, "678", arr.Value(2))
+			assert.Equal(t, "", arr.Value(3), "trailing empty string value will have offset past end")
+		}
+	}, "simple valid case")
+
+	assert.NotPanics(t, func() {
+		buffers := makeBuffers([]bool{true, false, true, false}, []int32{0, 3, 4, 9, 9}, "oooabcdef")
+		arr := array.NewStringData(array.NewData(arrow.BinaryTypes.String, 4, buffers, nil, 2, 0))
+		if assert.Equal(t, 4, arr.Len()) && assert.Equal(t, 2, arr.NullN()) {
+			assert.Equal(t, "ooo", arr.Value(0))
+			assert.True(t, arr.IsNull(1))
+			assert.Equal(t, "bcdef", arr.Value(2))
+			assert.True(t, arr.IsNull(3))
+		}
+	}, "simple valid case with nulls")
+
+	assert.PanicsWithValue(t, expectedPanic, func() {
+		buffers := makeBuffers(nil, []int32{0, 5}, "abc")
+		array.NewStringData(array.NewData(arrow.BinaryTypes.String, 1, buffers, nil, 0, 0))
+	}, "last offset is overflowing")
+
+	assert.PanicsWithError(t, "arrow/array: string offset buffer must have at least 2 values", func() {
+		buffers := makeBuffers(nil, []int32{0}, "abc")
+		array.NewStringData(array.NewData(arrow.BinaryTypes.String, 1, buffers, nil, 0, 0))
+	}, "last offset is missing")
+
+	assert.PanicsWithValue(t, expectedPanic, func() {
+		buffers := makeBuffers(nil, []int32{0, 3, 10, 15}, "oooabcdef")
+		array.NewStringData(array.NewData(arrow.BinaryTypes.String, 1, buffers, nil, 0, 2))
+	}, "data has offset and value offset is overflowing")
+}
diff --git a/go/arrow/ipc/reader.go b/go/arrow/ipc/reader.go
index 200160f7a9..69f1097eac 100644
--- a/go/arrow/ipc/reader.go
+++ b/go/arrow/ipc/reader.go
@@ -54,7 +54,12 @@ type Reader struct {
 // NewReaderFromMessageReader allows constructing a new reader object with the
 // provided MessageReader allowing injection of reading messages other than
 // by simple streaming bytes such as Arrow Flight which receives a protobuf message
-func NewReaderFromMessageReader(r MessageReader, opts ...Option) (*Reader, error) {
+func NewReaderFromMessageReader(r MessageReader, opts ...Option) (reader *Reader, err error) {
+	defer func() {
+		if pErr := recover(); pErr != nil {
+			err = fmt.Errorf("arrow/ipc: unknown error while reading: %v", pErr)
+		}
+	}()
 	cfg := newConfig()
 	for _, opt := range opts {
 		opt(cfg)
@@ -68,7 +73,7 @@ func NewReaderFromMessageReader(r MessageReader, opts ...Option) (*Reader, error
 		mem:  cfg.alloc,
 	}
 
-	err := rr.readSchema(cfg.schema)
+	err = rr.readSchema(cfg.schema)
 	if err != nil {
 		return nil, fmt.Errorf("arrow/ipc: could not read schema from stream: %w", err)
 	}
@@ -186,6 +191,12 @@ func (r *Reader) getInitialDicts() bool {
 }
 
 func (r *Reader) next() bool {
+	defer func() {
+		if pErr := recover(); pErr != nil {
+			r.err = fmt.Errorf("arrow/ipc: unknown error while reading: %v", pErr)
+		}
+	}()
+
 	if !r.readInitialDicts && !r.getInitialDicts() {
 		return false
 	}
diff --git a/go/arrow/ipc/reader_test.go b/go/arrow/ipc/reader_test.go
new file mode 100644
index 0000000000..76ef0c139a
--- /dev/null
+++ b/go/arrow/ipc/reader_test.go
@@ -0,0 +1,58 @@
+// 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 ipc
+
+import (
+	"bytes"
+	"testing"
+
+	"github.com/apache/arrow/go/v9/arrow"
+	"github.com/apache/arrow/go/v9/arrow/array"
+	"github.com/apache/arrow/go/v9/arrow/memory"
+	"github.com/stretchr/testify/assert"
+	"github.com/stretchr/testify/require"
+)
+
+func TestReaderCatchPanic(t *testing.T) {
+	alloc := memory.NewGoAllocator()
+	schema := arrow.NewSchema([]arrow.Field{
+		{Name: "s", Type: arrow.BinaryTypes.String},
+	}, nil)
+
+	b := array.NewRecordBuilder(alloc, schema)
+	defer b.Release()
+
+	b.Field(0).(*array.StringBuilder).AppendValues([]string{"foo", "bar", "baz"}, nil)
+	rec := b.NewRecord()
+	defer rec.Release()
+
+	buf := new(bytes.Buffer)
+	writer := NewWriter(buf, WithSchema(schema))
+	require.NoError(t, writer.Write(rec))
+
+	for i := buf.Len() - 100; i < buf.Len(); i++ {
+		buf.Bytes()[i] = 0
+	}
+
+	reader, err := NewReader(buf)
+	require.NoError(t, err)
+
+	_, err = reader.Read()
+	if assert.Error(t, err) {
+		assert.Contains(t, err.Error(), "arrow/ipc: unknown error while reading")
+	}
+}
diff --git a/go/arrow/ipc/writer.go b/go/arrow/ipc/writer.go
index f022422320..7a288af258 100644
--- a/go/arrow/ipc/writer.go
+++ b/go/arrow/ipc/writer.go
@@ -141,7 +141,13 @@ func (w *Writer) Close() error {
 	return nil
 }
 
-func (w *Writer) Write(rec arrow.Record) error {
+func (w *Writer) Write(rec arrow.Record) (err error) {
+	defer func() {
+		if pErr := recover(); pErr != nil {
+			err = fmt.Errorf("arrow/ipc: unknown error while writing: %v", pErr)
+		}
+	}()
+
 	if !w.started {
 		err := w.start()
 		if err != nil {
@@ -161,7 +167,7 @@ func (w *Writer) Write(rec arrow.Record) error {
 	)
 	defer data.Release()
 
-	err := writeDictionaryPayloads(w.mem, rec, false, w.emitDictDeltas, &w.mapper, w.lastWrittenDicts, w.pw, enc)
+	err = writeDictionaryPayloads(w.mem, rec, false, w.emitDictDeltas, &w.mapper, w.lastWrittenDicts, w.pw, enc)
 	if err != nil {
 		return fmt.Errorf("arrow/ipc: failure writing dictionary batches: %w", err)
 	}
diff --git a/go/arrow/ipc/writer_test.go b/go/arrow/ipc/writer_test.go
index 7d4ce41c8d..ae15e838b6 100644
--- a/go/arrow/ipc/writer_test.go
+++ b/go/arrow/ipc/writer_test.go
@@ -121,3 +121,26 @@ func TestGetZeroBasedValueOffsets(t *testing.T) {
 	defer offsets.Release()
 	assert.Equal(t, 20, offsets.Len(), "trim trailing offsets after slice")
 }
+
+func TestWriterCatchPanic(t *testing.T) {
+	alloc := memory.NewGoAllocator()
+	schema := arrow.NewSchema([]arrow.Field{
+		{Name: "s", Type: arrow.BinaryTypes.String},
+	}, nil)
+
+	b := array.NewRecordBuilder(alloc, schema)
+	defer b.Release()
+
+	b.Field(0).(*array.StringBuilder).AppendValues([]string{"foo", "bar", "baz"}, nil)
+	rec := b.NewRecord()
+	defer rec.Release()
+
+	// mess up the first offset for the string column
+	offsetBuf := rec.Column(0).Data().Buffers()[1]
+	bitutil.SetBitsTo(offsetBuf.Bytes(), 0, 32, true)
+
+	buf := new(bytes.Buffer)
+
+	writer := NewWriter(buf, WithSchema(schema))
+	assert.EqualError(t, writer.Write(rec), "arrow/ipc: unknown error while writing: runtime error: slice bounds out of range [-1:]")
+}