You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by we...@apache.org on 2018/08/08 21:02:33 UTC

[arrow] branch master updated: ARROW-3021: [Go] add support for List arrays

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

wesm 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 67aae05  ARROW-3021: [Go] add support for List arrays
67aae05 is described below

commit 67aae050c45a2d37504c4cc15f04afeb5dccf063
Author: Sebastien Binet <bi...@cern.ch>
AuthorDate: Wed Aug 8 17:02:28 2018 -0400

    ARROW-3021: [Go] add support for List arrays
    
    This CL introduces array.Builder, a generic interface to build
    array.Interface.
    It is needed to be able to build Lists' array values.
    
    array.NewListArray and array.NewListBuilder are introduced to,
    respectively, create a new List array value and a builder value to
    populate the contents of a List array.
    
    Author: Sebastien Binet <bi...@cern.ch>
    
    Closes #2402 from sbinet/issue-3021 and squashes the following commits:
    
    255b425a <Sebastien Binet> ARROW-3021:  add support for List arrays
---
 go/arrow/array/array.go                            |  18 ++-
 go/arrow/array/array_test.go                       |  10 +-
 go/arrow/array/binarybuilder.go                    |   8 +-
 go/arrow/array/boolean.go                          |   6 +-
 go/arrow/array/booleanbuilder.go                   |  12 +-
 go/arrow/array/builder.go                          |  81 +++++++++-
 go/arrow/array/data.go                             |  27 ++--
 go/arrow/array/{numeric_test.go => list.go}        |  44 ++++--
 go/arrow/array/list_test.go                        | 136 +++++++++++++++++
 go/arrow/array/listbuilder.go                      | 170 +++++++++++++++++++++
 go/arrow/array/numeric_test.go                     |   2 +-
 go/arrow/array/numericbuilder.gen.go               | 130 +++++++++++++---
 go/arrow/array/numericbuilder.gen.go.tmpl          |  26 +++-
 .../{array/numeric_test.go => datatype_nested.go}  |  38 +++--
 go/arrow/datatype_nested_test.go                   |  58 +++++++
 go/arrow/example_test.go                           |  77 ++++++++++
 go/arrow/type_string.go                            |   4 +-
 17 files changed, 753 insertions(+), 94 deletions(-)

diff --git a/go/arrow/array/array.go b/go/arrow/array/array.go
index d897690..61cefac 100644
--- a/go/arrow/array/array.go
+++ b/go/arrow/array/array.go
@@ -88,14 +88,14 @@ func (a *array) Release() {
 }
 
 // DataType returns the type metadata for this instance.
-func (a *array) DataType() arrow.DataType { return a.data.typE }
+func (a *array) DataType() arrow.DataType { return a.data.dtype }
 
 // NullN returns the number of null values in the array.
 func (a *array) NullN() int {
-	if a.data.nullN < 0 {
-		a.data.nullN = a.data.length - bitutil.CountSetBits(a.nullBitmapBytes, a.data.length)
+	if a.data.nulls < 0 {
+		a.data.nulls = a.data.length - bitutil.CountSetBits(a.nullBitmapBytes, a.data.length)
 	}
-	return a.data.nullN
+	return a.data.nulls
 }
 
 // NullBitmapBytes returns a byte slice of the validity bitmap.
@@ -172,14 +172,18 @@ var (
 )
 
 func unsupportedArrayType(data *Data) Interface {
-	panic("unsupported data type: " + data.typE.ID().String())
+	panic("unsupported data type: " + data.dtype.ID().String())
 }
 
 func invalidDataType(data *Data) Interface {
-	panic("invalid data type: " + data.typE.ID().String())
+	panic("invalid data type: " + data.dtype.ID().String())
 }
 
 // MakeFromData constructs a strongly-typed array instance from generic Data.
 func MakeFromData(data *Data) Interface {
-	return makeArrayFn[byte(data.typE.ID()&0x1f)](data)
+	return makeArrayFn[byte(data.dtype.ID()&0x1f)](data)
+}
+
+func init() {
+	makeArrayFn[arrow.LIST] = func(data *Data) Interface { return NewListData(data) }
 }
diff --git a/go/arrow/array/array_test.go b/go/arrow/array/array_test.go
index 4704016..cd051ee 100644
--- a/go/arrow/array/array_test.go
+++ b/go/arrow/array/array_test.go
@@ -38,6 +38,7 @@ func TestMakeFromData(t *testing.T) {
 		name     string
 		d        arrow.DataType
 		size     int
+		child    []*array.Data
 		expPanic bool
 		expError string
 	}{
@@ -60,6 +61,11 @@ func TestMakeFromData(t *testing.T) {
 		{name: "binary", d: &testDataType{arrow.BINARY}, size: 3},
 		{name: "timestamp", d: &testDataType{arrow.TIMESTAMP}},
 
+		{name: "list", d: &testDataType{arrow.LIST}, child: []*array.Data{
+			array.NewData(&testDataType{arrow.INT64}, 0, make([]*memory.Buffer, 4), nil, 0),
+			array.NewData(&testDataType{arrow.INT64}, 0, make([]*memory.Buffer, 4), nil, 0),
+		}},
+
 		// invalid types
 		{name: "invalid(-1)", d: &testDataType{arrow.Type(-1)}, expPanic: true, expError: "invalid data type: Type(-1)"},
 		{name: "invalid(28)", d: &testDataType{arrow.Type(28)}, expPanic: true, expError: "invalid data type: Type(28)"},
@@ -71,7 +77,7 @@ func TestMakeFromData(t *testing.T) {
 			if test.size != 0 {
 				n = test.size
 			}
-			data := array.NewData(test.d, 0, b[:n], 0)
+			data := array.NewData(test.d, 0, b[:n], test.child, 0)
 
 			if test.expPanic {
 				assert.PanicsWithValue(t, test.expError, func() {
@@ -104,7 +110,7 @@ func TestArray_NullN(t *testing.T) {
 	for _, test := range tests {
 		t.Run(test.name, func(t *testing.T) {
 			buf := memory.NewBufferBytes(test.bm)
-			data := array.NewData(arrow.FixedWidthTypes.Boolean, test.l, []*memory.Buffer{buf, nil}, test.n)
+			data := array.NewData(arrow.FixedWidthTypes.Boolean, test.l, []*memory.Buffer{buf, nil}, nil, test.n)
 			buf.Release()
 			ar := array.MakeFromData(data)
 			data.Release()
diff --git a/go/arrow/array/binarybuilder.go b/go/arrow/array/binarybuilder.go
index 71aa4cc..8534d86 100644
--- a/go/arrow/array/binarybuilder.go
+++ b/go/arrow/array/binarybuilder.go
@@ -33,15 +33,15 @@ const (
 type BinaryBuilder struct {
 	builder
 
-	typE    arrow.BinaryDataType
+	dtype   arrow.BinaryDataType
 	offsets *int32BufferBuilder
 	values  *byteBufferBuilder
 }
 
-func NewBinaryBuilder(mem memory.Allocator, typE arrow.BinaryDataType) *BinaryBuilder {
+func NewBinaryBuilder(mem memory.Allocator, dtype arrow.BinaryDataType) *BinaryBuilder {
 	b := &BinaryBuilder{
 		builder: builder{refCount: 1, mem: mem},
-		typE:    typE,
+		dtype:   dtype,
 		offsets: newInt32BufferBuilder(mem),
 		values:  newByteBufferBuilder(mem),
 	}
@@ -163,7 +163,7 @@ func (b *BinaryBuilder) NewBinaryArray() (a *Binary) {
 func (b *BinaryBuilder) newData() (data *Data) {
 	b.appendNextOffset()
 	offsets, values := b.offsets.Finish(), b.values.Finish()
-	data = NewData(b.typE, b.length, []*memory.Buffer{b.nullBitmap, offsets, values}, b.nullN)
+	data = NewData(b.dtype, b.length, []*memory.Buffer{b.nullBitmap, offsets, values}, nil, b.nulls)
 	if offsets != nil {
 		offsets.Release()
 	}
diff --git a/go/arrow/array/boolean.go b/go/arrow/array/boolean.go
index 0d41381..49468e5 100644
--- a/go/arrow/array/boolean.go
+++ b/go/arrow/array/boolean.go
@@ -30,9 +30,9 @@ type Boolean struct {
 
 // NewBoolean creates a boolean array from the data memory.Buffer and contains length elements.
 // The nullBitmap buffer can be nil of there are no null values.
-// If nullN is not known, use UnknownNullCount to calculate the value of NullN at runtime from the nullBitmap buffer.
-func NewBoolean(length int, data *memory.Buffer, nullBitmap *memory.Buffer, nullN int) *Boolean {
-	return NewBooleanData(NewData(arrow.FixedWidthTypes.Boolean, length, []*memory.Buffer{nullBitmap, data}, nullN))
+// If nulls is not known, use UnknownNullCount to calculate the value of NullN at runtime from the nullBitmap buffer.
+func NewBoolean(length int, data *memory.Buffer, nullBitmap *memory.Buffer, nulls int) *Boolean {
+	return NewBooleanData(NewData(arrow.FixedWidthTypes.Boolean, length, []*memory.Buffer{nullBitmap, data}, nil, nulls))
 }
 
 func NewBooleanData(data *Data) *Boolean {
diff --git a/go/arrow/array/booleanbuilder.go b/go/arrow/array/booleanbuilder.go
index 05e506f..8ec96ac 100644
--- a/go/arrow/array/booleanbuilder.go
+++ b/go/arrow/array/booleanbuilder.go
@@ -122,6 +122,12 @@ func (b *BooleanBuilder) Resize(n int) {
 	}
 }
 
+// NewArray creates a Boolean array from the memory buffers used by the builder and resets the BooleanBuilder
+// so it can be used to build a new array.
+func (b *BooleanBuilder) NewArray() Interface {
+	return b.NewBooleanArray()
+}
+
 // NewBooleanArray creates a Boolean array from the memory buffers used by the builder and resets the BooleanBuilder
 // so it can be used to build a new array.
 func (b *BooleanBuilder) NewBooleanArray() (a *Boolean) {
@@ -137,7 +143,7 @@ func (b *BooleanBuilder) newData() *Data {
 		// trim buffers
 		b.data.Resize(bytesRequired)
 	}
-	res := NewData(arrow.FixedWidthTypes.Boolean, b.length, []*memory.Buffer{b.nullBitmap, b.data}, b.nullN)
+	res := NewData(arrow.FixedWidthTypes.Boolean, b.length, []*memory.Buffer{b.nullBitmap, b.data}, nil, b.nulls)
 	b.reset()
 
 	if b.data != nil {
@@ -148,3 +154,7 @@ func (b *BooleanBuilder) newData() *Data {
 
 	return res
 }
+
+var (
+	_ Builder = (*BooleanBuilder)(nil)
+)
diff --git a/go/arrow/array/builder.go b/go/arrow/array/builder.go
index bde12fb..a20c303 100644
--- a/go/arrow/array/builder.go
+++ b/go/arrow/array/builder.go
@@ -17,8 +17,10 @@
 package array
 
 import (
+	"fmt"
 	"sync/atomic"
 
+	"github.com/apache/arrow/go/arrow"
 	"github.com/apache/arrow/go/arrow/internal/bitutil"
 	"github.com/apache/arrow/go/arrow/memory"
 )
@@ -27,12 +29,33 @@ const (
 	minBuilderCapacity = 1 << 5
 )
 
+// Builder provides an interface to build arrow arrays.
+type Builder interface {
+	// Release decreases the reference count by 1.
+	Release()
+
+	// Len returns the number of elements in the array builder.
+	Len() int
+
+	// Cap returns the total number of elements that can be stored
+	// without allocating additional memory.
+	Cap() int
+
+	// NullN returns the number of null values in the array builder.
+	NullN() int
+
+	// NewArray creates a new array from the memory buffers used
+	// by the builder and resets the Builder so it can be used to build
+	// a new array.
+	NewArray() Interface
+}
+
 // builder provides common functionality for managing the validity bitmap (nulls) when building arrays.
 type builder struct {
 	refCount   int64
 	mem        memory.Allocator
 	nullBitmap *memory.Buffer
-	nullN      int
+	nulls      int
 	length     int
 	capacity   int
 }
@@ -50,7 +73,7 @@ func (b *builder) Len() int { return b.length }
 func (b *builder) Cap() int { return b.capacity }
 
 // NullN returns the number of null values in the array builder.
-func (b *builder) NullN() int { return b.nullN }
+func (b *builder) NullN() int { return b.nulls }
 
 func (b *builder) init(capacity int) {
 	toAlloc := bitutil.CeilByte(capacity) / 8
@@ -66,7 +89,7 @@ func (b *builder) reset() {
 		b.nullBitmap = nil
 	}
 
-	b.nullN = 0
+	b.nulls = 0
 	b.length = 0
 	b.capacity = 0
 }
@@ -119,7 +142,7 @@ func (b *builder) unsafeAppendBoolsToBitmap(valid []bool, length int) {
 			bitSet |= bitutil.BitMask[bitOffset]
 		} else {
 			bitSet &= bitutil.FlippedBitMask[bitOffset]
-			b.nullN++
+			b.nulls++
 		}
 		bitOffset++
 	}
@@ -158,7 +181,55 @@ func (b *builder) UnsafeAppendBoolToBitmap(isValid bool) {
 	if isValid {
 		bitutil.SetBit(b.nullBitmap.Bytes(), b.length)
 	} else {
-		b.nullN++
+		b.nulls++
 	}
 	b.length++
 }
+
+func newBuilder(mem memory.Allocator, dtype arrow.DataType) Builder {
+	// FIXME(sbinet): use a type switch on dtype instead?
+	switch dtype.ID() {
+	case arrow.NULL:
+	case arrow.BOOL:
+		return NewBooleanBuilder(mem)
+	case arrow.UINT8:
+		return NewUint8Builder(mem)
+	case arrow.INT8:
+		return NewInt8Builder(mem)
+	case arrow.UINT16:
+		return NewUint16Builder(mem)
+	case arrow.INT16:
+		return NewInt16Builder(mem)
+	case arrow.UINT32:
+		return NewUint32Builder(mem)
+	case arrow.INT32:
+		return NewInt32Builder(mem)
+	case arrow.UINT64:
+		return NewUint64Builder(mem)
+	case arrow.INT64:
+		return NewInt64Builder(mem)
+	case arrow.HALF_FLOAT:
+	case arrow.FLOAT32:
+		return NewFloat32Builder(mem)
+	case arrow.FLOAT64:
+		return NewFloat64Builder(mem)
+	case arrow.STRING:
+	case arrow.BINARY:
+	case arrow.FIXED_SIZE_BINARY:
+	case arrow.DATE32:
+	case arrow.DATE64:
+	case arrow.TIMESTAMP:
+	case arrow.TIME32:
+	case arrow.TIME64:
+	case arrow.INTERVAL:
+	case arrow.DECIMAL:
+	case arrow.LIST:
+		typ := dtype.(*arrow.ListType)
+		return NewListBuilder(mem, typ.Elem())
+	case arrow.STRUCT:
+	case arrow.UNION:
+	case arrow.DICTIONARY:
+	case arrow.MAP:
+	}
+	panic(fmt.Errorf("arrow/array: unsupported builder for %T", dtype))
+}
diff --git a/go/arrow/array/data.go b/go/arrow/array/data.go
index e689f0f..72a88ed 100644
--- a/go/arrow/array/data.go
+++ b/go/arrow/array/data.go
@@ -27,26 +27,33 @@ import (
 // A type which represents the memory and metadata for an Arrow array.
 type Data struct {
 	refCount  int64
-	typE      arrow.DataType
-	nullN     int
+	dtype     arrow.DataType
+	nulls     int
 	length    int
 	buffers   []*memory.Buffer // TODO(sgc): should this be an interface?
 	childData []*Data          // TODO(sgc): managed by ListArray, StructArray and UnionArray types
 }
 
-func NewData(typE arrow.DataType, length int, buffers []*memory.Buffer, nullN int) *Data {
+func NewData(dtype arrow.DataType, length int, buffers []*memory.Buffer, childData []*Data, nulls int) *Data {
 	for _, b := range buffers {
 		if b != nil {
 			b.Retain()
 		}
 	}
 
+	for _, child := range childData {
+		if child != nil {
+			child.Retain()
+		}
+	}
+
 	return &Data{
-		refCount: 1,
-		typE:     typE,
-		nullN:    nullN,
-		length:   length,
-		buffers:  buffers,
+		refCount:  1,
+		dtype:     dtype,
+		nulls:     nulls,
+		length:    length,
+		buffers:   buffers,
+		childData: childData,
 	}
 }
 
@@ -76,6 +83,6 @@ func (d *Data) Release() {
 	}
 }
 
-func (d *Data) DataType() arrow.DataType { return d.typE }
-func (d *Data) NullN() int               { return d.nullN }
+func (d *Data) DataType() arrow.DataType { return d.dtype }
+func (d *Data) NullN() int               { return d.nulls }
 func (d *Data) Len() int                 { return d.length }
diff --git a/go/arrow/array/numeric_test.go b/go/arrow/array/list.go
similarity index 53%
copy from go/arrow/array/numeric_test.go
copy to go/arrow/array/list.go
index d5b798f..344c137 100644
--- a/go/arrow/array/numeric_test.go
+++ b/go/arrow/array/list.go
@@ -14,23 +14,43 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-package array_test
+package array
 
 import (
-	"testing"
-
 	"github.com/apache/arrow/go/arrow"
-	"github.com/apache/arrow/go/arrow/array"
-	"github.com/apache/arrow/go/arrow/memory"
-	"github.com/stretchr/testify/assert"
 )
 
-func TestNewFloat64Data(t *testing.T) {
-	exp := []float64{1.0, 2.0, 4.0, 8.0, 16.0}
+// List represents an immutable sequence of array values.
+type List struct {
+	array
+	values  Interface
+	offsets []int32
+}
+
+// NewListData returns a new List array value, from data.
+func NewListData(data *Data) *List {
+	a := &List{}
+	a.refCount = 1
+	a.setData(data)
+	return a
+}
 
-	ad := array.NewData(arrow.PrimitiveTypes.Float64, len(exp), []*memory.Buffer{nil, memory.NewBufferBytes(arrow.Float64Traits.CastToBytes(exp))}, 0)
-	fa := array.NewFloat64Data(ad)
+func (a *List) ListValues() Interface { return a.values }
 
-	assert.Equal(t, len(exp), fa.Len(), "unexpected Len()")
-	assert.Equal(t, exp, fa.Float64Values(), "unexpected Float64Values()")
+func (a *List) setData(data *Data) {
+	a.array.setData(data)
+	vals := data.buffers[1]
+	if vals != nil {
+		a.offsets = arrow.Int32Traits.CastFromBytes(vals.Bytes())
+	}
+	a.values = MakeFromData(data.childData[0])
 }
+
+// Len returns the number of elements in the array.
+func (a *List) Len() int { return a.array.Len() }
+
+func (a *List) Offsets() []int32 { return a.offsets }
+
+var (
+	_ Interface = (*List)(nil)
+)
diff --git a/go/arrow/array/list_test.go b/go/arrow/array/list_test.go
new file mode 100644
index 0000000..0c30ecd
--- /dev/null
+++ b/go/arrow/array/list_test.go
@@ -0,0 +1,136 @@
+// 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 array_test
+
+import (
+	"reflect"
+	"testing"
+
+	"github.com/apache/arrow/go/arrow"
+	"github.com/apache/arrow/go/arrow/array"
+	"github.com/apache/arrow/go/arrow/memory"
+)
+
+func TestListArray(t *testing.T) {
+	var (
+		pool    = memory.NewGoAllocator()
+		vs      = []int32{0, 1, 2, 3, 4, 5, 6}
+		lengths = []int{3, 0, 4}
+		isValid = []bool{true, false, true}
+		offsets = []int32{0, 3, 3, 7}
+	)
+
+	lb := array.NewListBuilder(pool, arrow.PrimitiveTypes.Int32)
+	defer lb.Release()
+	for i := 0; i < 10; i++ {
+		vb := lb.ValueBuilder().(*array.Int32Builder)
+		vb.Reserve(len(vs))
+
+		pos := 0
+		for i, length := range lengths {
+			lb.Append(isValid[i])
+			for j := 0; j < length; j++ {
+				vb.Append(vs[pos])
+				pos++
+			}
+		}
+
+		arr := lb.NewArray().(*array.List)
+		if got, want := arr.DataType().ID(), arrow.LIST; got != want {
+			t.Fatalf("got=%v, want=%v", got, want)
+		}
+
+		if got, want := arr.Len(), len(isValid); got != want {
+			t.Fatalf("got=%d, want=%d", got, want)
+		}
+
+		for i := range lengths {
+			if got, want := arr.IsValid(i), isValid[i]; got != want {
+				t.Fatalf("got[%d]=%v, want[%d]=%v", i, got, i, want)
+			}
+			if got, want := arr.IsNull(i), lengths[i] == 0; got != want {
+				t.Fatalf("got[%d]=%v, want[%d]=%v", i, got, i, want)
+			}
+		}
+
+		if got, want := arr.Offsets(), offsets; !reflect.DeepEqual(got, want) {
+			t.Fatalf("got=%v, want=%v", got, want)
+		}
+
+		varr := arr.ListValues().(*array.Int32)
+		if got, want := varr.Int32Values(), vs; !reflect.DeepEqual(got, want) {
+			t.Fatalf("got=%v, want=%v", got, want)
+		}
+	}
+}
+
+func TestListArrayEmpty(t *testing.T) {
+	pool := memory.NewGoAllocator()
+	lb := array.NewListBuilder(pool, arrow.PrimitiveTypes.Int32)
+	defer lb.Release()
+	arr := lb.NewArray().(*array.List)
+	if got, want := arr.Len(), 0; got != want {
+		t.Fatalf("got=%d, want=%d", got, want)
+	}
+}
+
+func TestListArrayBulkAppend(t *testing.T) {
+	var (
+		pool    = memory.NewGoAllocator()
+		vs      = []int32{0, 1, 2, 3, 4, 5, 6}
+		lengths = []int{3, 0, 4}
+		isValid = []bool{true, false, true}
+		offsets = []int32{0, 3, 3, 7}
+	)
+
+	lb := array.NewListBuilder(pool, arrow.PrimitiveTypes.Int32)
+	defer lb.Release()
+	vb := lb.ValueBuilder().(*array.Int32Builder)
+	vb.Reserve(len(vs))
+
+	lb.AppendValues(offsets, isValid)
+	for _, v := range vs {
+		vb.Append(v)
+	}
+
+	arr := lb.NewArray().(*array.List)
+	if got, want := arr.DataType().ID(), arrow.LIST; got != want {
+		t.Fatalf("got=%v, want=%v", got, want)
+	}
+
+	if got, want := arr.Len(), len(isValid); got != want {
+		t.Fatalf("got=%d, want=%d", got, want)
+	}
+
+	for i := range lengths {
+		if got, want := arr.IsValid(i), isValid[i]; got != want {
+			t.Fatalf("got[%d]=%v, want[%d]=%v", i, got, i, want)
+		}
+		if got, want := arr.IsNull(i), lengths[i] == 0; got != want {
+			t.Fatalf("got[%d]=%v, want[%d]=%v", i, got, i, want)
+		}
+	}
+
+	if got, want := arr.Offsets(), offsets; !reflect.DeepEqual(got, want) {
+		t.Fatalf("got=%v, want=%v", got, want)
+	}
+
+	varr := arr.ListValues().(*array.Int32)
+	if got, want := varr.Int32Values(), vs; !reflect.DeepEqual(got, want) {
+		t.Fatalf("got=%v, want=%v", got, want)
+	}
+}
diff --git a/go/arrow/array/listbuilder.go b/go/arrow/array/listbuilder.go
new file mode 100644
index 0000000..2bef68a
--- /dev/null
+++ b/go/arrow/array/listbuilder.go
@@ -0,0 +1,170 @@
+// 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 array
+
+import (
+	"sync/atomic"
+
+	"github.com/apache/arrow/go/arrow"
+	"github.com/apache/arrow/go/arrow/internal/bitutil"
+	"github.com/apache/arrow/go/arrow/internal/debug"
+	"github.com/apache/arrow/go/arrow/memory"
+)
+
+type ListBuilder struct {
+	builder
+
+	etype   arrow.DataType // data type of the list's elements.
+	values  Builder        // value builder for the list's elements.
+	offsets *Int32Builder
+}
+
+// NewListBuilder returns a builder, using the provided memory allocator.
+// The created list builder will create a list whose elements will be of type etype.
+func NewListBuilder(mem memory.Allocator, etype arrow.DataType) *ListBuilder {
+	return &ListBuilder{
+		builder: builder{refCount: 1, mem: mem},
+		etype:   etype,
+		values:  newBuilder(mem, etype),
+		offsets: NewInt32Builder(mem),
+	}
+}
+
+// Release decreases the reference count by 1.
+// When the reference count goes to zero, the memory is freed.
+func (b *ListBuilder) Release() {
+	debug.Assert(atomic.LoadInt64(&b.refCount) > 0, "too many releases")
+
+	if atomic.AddInt64(&b.refCount, -1) == 0 {
+		if b.nullBitmap != nil {
+			b.nullBitmap.Release()
+			b.nullBitmap = nil
+		}
+	}
+
+	b.values.Release()
+	b.offsets.Release()
+}
+
+func (b *ListBuilder) appendNextOffset() {
+	b.offsets.Append(int32(b.values.Len()))
+}
+
+func (b *ListBuilder) Append(v bool) {
+	b.Reserve(1)
+	b.unsafeAppendBoolToBitmap(v)
+	b.appendNextOffset()
+}
+
+func (b *ListBuilder) AppendNull() {
+	b.Reserve(1)
+	b.unsafeAppendBoolToBitmap(false)
+	b.appendNextOffset()
+}
+
+func (b *ListBuilder) AppendValues(offsets []int32, valid []bool) {
+	b.Reserve(len(valid))
+	b.offsets.AppendValues(offsets, nil)
+	b.builder.unsafeAppendBoolsToBitmap(valid, len(valid))
+}
+
+func (b *ListBuilder) unsafeAppend(v bool) {
+	bitutil.SetBit(b.nullBitmap.Bytes(), b.length)
+	b.length++
+}
+
+func (b *ListBuilder) unsafeAppendBoolToBitmap(isValid bool) {
+	if isValid {
+		bitutil.SetBit(b.nullBitmap.Bytes(), b.length)
+	} else {
+		b.nulls++
+	}
+	b.length++
+}
+
+func (b *ListBuilder) init(capacity int) {
+	b.builder.init(capacity)
+	b.offsets.init(capacity + 1)
+}
+
+// Reserve ensures there is enough space for appending n elements
+// by checking the capacity and calling Resize if necessary.
+func (b *ListBuilder) Reserve(n int) {
+	b.builder.reserve(n, b.Resize)
+}
+
+// Resize adjusts the space allocated by b to n elements. If n is greater than b.Cap(),
+// additional memory will be allocated. If n is smaller, the allocated memory may reduced.
+func (b *ListBuilder) Resize(n int) {
+	if n < minBuilderCapacity {
+		n = minBuilderCapacity
+	}
+
+	if b.capacity == 0 {
+		b.init(n)
+	} else {
+		b.builder.resize(n, b.builder.init)
+		b.offsets.resize(n+1, b.offsets.init)
+	}
+}
+
+func (b *ListBuilder) ValueBuilder() Builder {
+	return b.values
+}
+
+// NewArray creates a List array from the memory buffers used by the builder and resets the ListBuilder
+// so it can be used to build a new array.
+func (b *ListBuilder) NewArray() Interface {
+	return b.NewListArray()
+}
+
+// NewListArray creates a List array from the memory buffers used by the builder and resets the ListBuilder
+// so it can be used to build a new array.
+func (b *ListBuilder) NewListArray() (a *List) {
+	if b.offsets.Len() != b.length+1 {
+		b.appendNextOffset()
+	}
+	data := b.newData()
+	a = NewListData(data)
+	data.Release()
+	return
+}
+
+func (b *ListBuilder) newData() (data *Data) {
+	values := b.values.NewArray()
+	defer values.Release()
+
+	var offsets *memory.Buffer
+	if b.offsets != nil {
+		arr := b.offsets.NewInt32Array()
+		defer arr.Release()
+		offsets = arr.Data().buffers[1]
+	}
+
+	data = NewData(
+		arrow.ListOf(b.etype), b.length,
+		[]*memory.Buffer{
+			b.nullBitmap,
+			offsets,
+		},
+		[]*Data{values.Data()},
+		b.nulls,
+	)
+	b.reset()
+
+	return
+}
diff --git a/go/arrow/array/numeric_test.go b/go/arrow/array/numeric_test.go
index d5b798f..9f29394 100644
--- a/go/arrow/array/numeric_test.go
+++ b/go/arrow/array/numeric_test.go
@@ -28,7 +28,7 @@ import (
 func TestNewFloat64Data(t *testing.T) {
 	exp := []float64{1.0, 2.0, 4.0, 8.0, 16.0}
 
-	ad := array.NewData(arrow.PrimitiveTypes.Float64, len(exp), []*memory.Buffer{nil, memory.NewBufferBytes(arrow.Float64Traits.CastToBytes(exp))}, 0)
+	ad := array.NewData(arrow.PrimitiveTypes.Float64, len(exp), []*memory.Buffer{nil, memory.NewBufferBytes(arrow.Float64Traits.CastToBytes(exp))}, nil, 0)
 	fa := array.NewFloat64Data(ad)
 
 	assert.Equal(t, len(exp), fa.Len(), "unexpected Len()")
diff --git a/go/arrow/array/numericbuilder.gen.go b/go/arrow/array/numericbuilder.gen.go
index c4e3508..06ec4ba 100644
--- a/go/arrow/array/numericbuilder.gen.go
+++ b/go/arrow/array/numericbuilder.gen.go
@@ -75,7 +75,7 @@ func (b *Int64Builder) UnsafeAppendBoolToBitmap(isValid bool) {
 	if isValid {
 		bitutil.SetBit(b.nullBitmap.Bytes(), b.length)
 	} else {
-		b.nullN++
+		b.nulls++
 	}
 	b.length++
 }
@@ -126,6 +126,12 @@ func (b *Int64Builder) Resize(n int) {
 	}
 }
 
+// NewArray creates a Int64 array from the memory buffers used by the builder and resets the Int64Builder
+// so it can be used to build a new array.
+func (b *Int64Builder) NewArray() Interface {
+	return b.NewInt64Array()
+}
+
 // NewInt64Array creates a Int64 array from the memory buffers used by the builder and resets the Int64Builder
 // so it can be used to build a new array.
 func (b *Int64Builder) NewInt64Array() (a *Int64) {
@@ -141,7 +147,7 @@ func (b *Int64Builder) newData() (data *Data) {
 		// trim buffers
 		b.data.Resize(bytesRequired)
 	}
-	data = NewData(arrow.PrimitiveTypes.Int64, b.length, []*memory.Buffer{b.nullBitmap, b.data}, b.nullN)
+	data = NewData(arrow.PrimitiveTypes.Int64, b.length, []*memory.Buffer{b.nullBitmap, b.data}, nil, b.nulls)
 	b.reset()
 
 	if b.data != nil {
@@ -201,7 +207,7 @@ func (b *Uint64Builder) UnsafeAppendBoolToBitmap(isValid bool) {
 	if isValid {
 		bitutil.SetBit(b.nullBitmap.Bytes(), b.length)
 	} else {
-		b.nullN++
+		b.nulls++
 	}
 	b.length++
 }
@@ -252,6 +258,12 @@ func (b *Uint64Builder) Resize(n int) {
 	}
 }
 
+// NewArray creates a Uint64 array from the memory buffers used by the builder and resets the Uint64Builder
+// so it can be used to build a new array.
+func (b *Uint64Builder) NewArray() Interface {
+	return b.NewUint64Array()
+}
+
 // NewUint64Array creates a Uint64 array from the memory buffers used by the builder and resets the Uint64Builder
 // so it can be used to build a new array.
 func (b *Uint64Builder) NewUint64Array() (a *Uint64) {
@@ -267,7 +279,7 @@ func (b *Uint64Builder) newData() (data *Data) {
 		// trim buffers
 		b.data.Resize(bytesRequired)
 	}
-	data = NewData(arrow.PrimitiveTypes.Uint64, b.length, []*memory.Buffer{b.nullBitmap, b.data}, b.nullN)
+	data = NewData(arrow.PrimitiveTypes.Uint64, b.length, []*memory.Buffer{b.nullBitmap, b.data}, nil, b.nulls)
 	b.reset()
 
 	if b.data != nil {
@@ -327,7 +339,7 @@ func (b *Float64Builder) UnsafeAppendBoolToBitmap(isValid bool) {
 	if isValid {
 		bitutil.SetBit(b.nullBitmap.Bytes(), b.length)
 	} else {
-		b.nullN++
+		b.nulls++
 	}
 	b.length++
 }
@@ -378,6 +390,12 @@ func (b *Float64Builder) Resize(n int) {
 	}
 }
 
+// NewArray creates a Float64 array from the memory buffers used by the builder and resets the Float64Builder
+// so it can be used to build a new array.
+func (b *Float64Builder) NewArray() Interface {
+	return b.NewFloat64Array()
+}
+
 // NewFloat64Array creates a Float64 array from the memory buffers used by the builder and resets the Float64Builder
 // so it can be used to build a new array.
 func (b *Float64Builder) NewFloat64Array() (a *Float64) {
@@ -393,7 +411,7 @@ func (b *Float64Builder) newData() (data *Data) {
 		// trim buffers
 		b.data.Resize(bytesRequired)
 	}
-	data = NewData(arrow.PrimitiveTypes.Float64, b.length, []*memory.Buffer{b.nullBitmap, b.data}, b.nullN)
+	data = NewData(arrow.PrimitiveTypes.Float64, b.length, []*memory.Buffer{b.nullBitmap, b.data}, nil, b.nulls)
 	b.reset()
 
 	if b.data != nil {
@@ -453,7 +471,7 @@ func (b *Int32Builder) UnsafeAppendBoolToBitmap(isValid bool) {
 	if isValid {
 		bitutil.SetBit(b.nullBitmap.Bytes(), b.length)
 	} else {
-		b.nullN++
+		b.nulls++
 	}
 	b.length++
 }
@@ -504,6 +522,12 @@ func (b *Int32Builder) Resize(n int) {
 	}
 }
 
+// NewArray creates a Int32 array from the memory buffers used by the builder and resets the Int32Builder
+// so it can be used to build a new array.
+func (b *Int32Builder) NewArray() Interface {
+	return b.NewInt32Array()
+}
+
 // NewInt32Array creates a Int32 array from the memory buffers used by the builder and resets the Int32Builder
 // so it can be used to build a new array.
 func (b *Int32Builder) NewInt32Array() (a *Int32) {
@@ -519,7 +543,7 @@ func (b *Int32Builder) newData() (data *Data) {
 		// trim buffers
 		b.data.Resize(bytesRequired)
 	}
-	data = NewData(arrow.PrimitiveTypes.Int32, b.length, []*memory.Buffer{b.nullBitmap, b.data}, b.nullN)
+	data = NewData(arrow.PrimitiveTypes.Int32, b.length, []*memory.Buffer{b.nullBitmap, b.data}, nil, b.nulls)
 	b.reset()
 
 	if b.data != nil {
@@ -579,7 +603,7 @@ func (b *Uint32Builder) UnsafeAppendBoolToBitmap(isValid bool) {
 	if isValid {
 		bitutil.SetBit(b.nullBitmap.Bytes(), b.length)
 	} else {
-		b.nullN++
+		b.nulls++
 	}
 	b.length++
 }
@@ -630,6 +654,12 @@ func (b *Uint32Builder) Resize(n int) {
 	}
 }
 
+// NewArray creates a Uint32 array from the memory buffers used by the builder and resets the Uint32Builder
+// so it can be used to build a new array.
+func (b *Uint32Builder) NewArray() Interface {
+	return b.NewUint32Array()
+}
+
 // NewUint32Array creates a Uint32 array from the memory buffers used by the builder and resets the Uint32Builder
 // so it can be used to build a new array.
 func (b *Uint32Builder) NewUint32Array() (a *Uint32) {
@@ -645,7 +675,7 @@ func (b *Uint32Builder) newData() (data *Data) {
 		// trim buffers
 		b.data.Resize(bytesRequired)
 	}
-	data = NewData(arrow.PrimitiveTypes.Uint32, b.length, []*memory.Buffer{b.nullBitmap, b.data}, b.nullN)
+	data = NewData(arrow.PrimitiveTypes.Uint32, b.length, []*memory.Buffer{b.nullBitmap, b.data}, nil, b.nulls)
 	b.reset()
 
 	if b.data != nil {
@@ -705,7 +735,7 @@ func (b *Float32Builder) UnsafeAppendBoolToBitmap(isValid bool) {
 	if isValid {
 		bitutil.SetBit(b.nullBitmap.Bytes(), b.length)
 	} else {
-		b.nullN++
+		b.nulls++
 	}
 	b.length++
 }
@@ -756,6 +786,12 @@ func (b *Float32Builder) Resize(n int) {
 	}
 }
 
+// NewArray creates a Float32 array from the memory buffers used by the builder and resets the Float32Builder
+// so it can be used to build a new array.
+func (b *Float32Builder) NewArray() Interface {
+	return b.NewFloat32Array()
+}
+
 // NewFloat32Array creates a Float32 array from the memory buffers used by the builder and resets the Float32Builder
 // so it can be used to build a new array.
 func (b *Float32Builder) NewFloat32Array() (a *Float32) {
@@ -771,7 +807,7 @@ func (b *Float32Builder) newData() (data *Data) {
 		// trim buffers
 		b.data.Resize(bytesRequired)
 	}
-	data = NewData(arrow.PrimitiveTypes.Float32, b.length, []*memory.Buffer{b.nullBitmap, b.data}, b.nullN)
+	data = NewData(arrow.PrimitiveTypes.Float32, b.length, []*memory.Buffer{b.nullBitmap, b.data}, nil, b.nulls)
 	b.reset()
 
 	if b.data != nil {
@@ -831,7 +867,7 @@ func (b *Int16Builder) UnsafeAppendBoolToBitmap(isValid bool) {
 	if isValid {
 		bitutil.SetBit(b.nullBitmap.Bytes(), b.length)
 	} else {
-		b.nullN++
+		b.nulls++
 	}
 	b.length++
 }
@@ -882,6 +918,12 @@ func (b *Int16Builder) Resize(n int) {
 	}
 }
 
+// NewArray creates a Int16 array from the memory buffers used by the builder and resets the Int16Builder
+// so it can be used to build a new array.
+func (b *Int16Builder) NewArray() Interface {
+	return b.NewInt16Array()
+}
+
 // NewInt16Array creates a Int16 array from the memory buffers used by the builder and resets the Int16Builder
 // so it can be used to build a new array.
 func (b *Int16Builder) NewInt16Array() (a *Int16) {
@@ -897,7 +939,7 @@ func (b *Int16Builder) newData() (data *Data) {
 		// trim buffers
 		b.data.Resize(bytesRequired)
 	}
-	data = NewData(arrow.PrimitiveTypes.Int16, b.length, []*memory.Buffer{b.nullBitmap, b.data}, b.nullN)
+	data = NewData(arrow.PrimitiveTypes.Int16, b.length, []*memory.Buffer{b.nullBitmap, b.data}, nil, b.nulls)
 	b.reset()
 
 	if b.data != nil {
@@ -957,7 +999,7 @@ func (b *Uint16Builder) UnsafeAppendBoolToBitmap(isValid bool) {
 	if isValid {
 		bitutil.SetBit(b.nullBitmap.Bytes(), b.length)
 	} else {
-		b.nullN++
+		b.nulls++
 	}
 	b.length++
 }
@@ -1008,6 +1050,12 @@ func (b *Uint16Builder) Resize(n int) {
 	}
 }
 
+// NewArray creates a Uint16 array from the memory buffers used by the builder and resets the Uint16Builder
+// so it can be used to build a new array.
+func (b *Uint16Builder) NewArray() Interface {
+	return b.NewUint16Array()
+}
+
 // NewUint16Array creates a Uint16 array from the memory buffers used by the builder and resets the Uint16Builder
 // so it can be used to build a new array.
 func (b *Uint16Builder) NewUint16Array() (a *Uint16) {
@@ -1023,7 +1071,7 @@ func (b *Uint16Builder) newData() (data *Data) {
 		// trim buffers
 		b.data.Resize(bytesRequired)
 	}
-	data = NewData(arrow.PrimitiveTypes.Uint16, b.length, []*memory.Buffer{b.nullBitmap, b.data}, b.nullN)
+	data = NewData(arrow.PrimitiveTypes.Uint16, b.length, []*memory.Buffer{b.nullBitmap, b.data}, nil, b.nulls)
 	b.reset()
 
 	if b.data != nil {
@@ -1083,7 +1131,7 @@ func (b *Int8Builder) UnsafeAppendBoolToBitmap(isValid bool) {
 	if isValid {
 		bitutil.SetBit(b.nullBitmap.Bytes(), b.length)
 	} else {
-		b.nullN++
+		b.nulls++
 	}
 	b.length++
 }
@@ -1134,6 +1182,12 @@ func (b *Int8Builder) Resize(n int) {
 	}
 }
 
+// NewArray creates a Int8 array from the memory buffers used by the builder and resets the Int8Builder
+// so it can be used to build a new array.
+func (b *Int8Builder) NewArray() Interface {
+	return b.NewInt8Array()
+}
+
 // NewInt8Array creates a Int8 array from the memory buffers used by the builder and resets the Int8Builder
 // so it can be used to build a new array.
 func (b *Int8Builder) NewInt8Array() (a *Int8) {
@@ -1149,7 +1203,7 @@ func (b *Int8Builder) newData() (data *Data) {
 		// trim buffers
 		b.data.Resize(bytesRequired)
 	}
-	data = NewData(arrow.PrimitiveTypes.Int8, b.length, []*memory.Buffer{b.nullBitmap, b.data}, b.nullN)
+	data = NewData(arrow.PrimitiveTypes.Int8, b.length, []*memory.Buffer{b.nullBitmap, b.data}, nil, b.nulls)
 	b.reset()
 
 	if b.data != nil {
@@ -1209,7 +1263,7 @@ func (b *Uint8Builder) UnsafeAppendBoolToBitmap(isValid bool) {
 	if isValid {
 		bitutil.SetBit(b.nullBitmap.Bytes(), b.length)
 	} else {
-		b.nullN++
+		b.nulls++
 	}
 	b.length++
 }
@@ -1260,6 +1314,12 @@ func (b *Uint8Builder) Resize(n int) {
 	}
 }
 
+// NewArray creates a Uint8 array from the memory buffers used by the builder and resets the Uint8Builder
+// so it can be used to build a new array.
+func (b *Uint8Builder) NewArray() Interface {
+	return b.NewUint8Array()
+}
+
 // NewUint8Array creates a Uint8 array from the memory buffers used by the builder and resets the Uint8Builder
 // so it can be used to build a new array.
 func (b *Uint8Builder) NewUint8Array() (a *Uint8) {
@@ -1275,7 +1335,7 @@ func (b *Uint8Builder) newData() (data *Data) {
 		// trim buffers
 		b.data.Resize(bytesRequired)
 	}
-	data = NewData(arrow.PrimitiveTypes.Uint8, b.length, []*memory.Buffer{b.nullBitmap, b.data}, b.nullN)
+	data = NewData(arrow.PrimitiveTypes.Uint8, b.length, []*memory.Buffer{b.nullBitmap, b.data}, nil, b.nulls)
 	b.reset()
 
 	if b.data != nil {
@@ -1290,13 +1350,13 @@ func (b *Uint8Builder) newData() (data *Data) {
 type TimestampBuilder struct {
 	builder
 
-	typE    *arrow.TimestampType
+	dtype   *arrow.TimestampType
 	data    *memory.Buffer
 	rawData []arrow.Timestamp
 }
 
-func NewTimestampBuilder(mem memory.Allocator, typE *arrow.TimestampType) *TimestampBuilder {
-	return &TimestampBuilder{builder: builder{refCount: 1, mem: mem}, typE: typE}
+func NewTimestampBuilder(mem memory.Allocator, dtype *arrow.TimestampType) *TimestampBuilder {
+	return &TimestampBuilder{builder: builder{refCount: 1, mem: mem}, dtype: dtype}
 }
 
 // Release decreases the reference count by 1.
@@ -1336,7 +1396,7 @@ func (b *TimestampBuilder) UnsafeAppendBoolToBitmap(isValid bool) {
 	if isValid {
 		bitutil.SetBit(b.nullBitmap.Bytes(), b.length)
 	} else {
-		b.nullN++
+		b.nulls++
 	}
 	b.length++
 }
@@ -1387,6 +1447,12 @@ func (b *TimestampBuilder) Resize(n int) {
 	}
 }
 
+// NewArray creates a Timestamp array from the memory buffers used by the builder and resets the TimestampBuilder
+// so it can be used to build a new array.
+func (b *TimestampBuilder) NewArray() Interface {
+	return b.NewTimestampArray()
+}
+
 // NewTimestampArray creates a Timestamp array from the memory buffers used by the builder and resets the TimestampBuilder
 // so it can be used to build a new array.
 func (b *TimestampBuilder) NewTimestampArray() (a *Timestamp) {
@@ -1402,7 +1468,7 @@ func (b *TimestampBuilder) newData() (data *Data) {
 		// trim buffers
 		b.data.Resize(bytesRequired)
 	}
-	data = NewData(b.typE, b.length, []*memory.Buffer{b.nullBitmap, b.data}, b.nullN)
+	data = NewData(b.dtype, b.length, []*memory.Buffer{b.nullBitmap, b.data}, nil, b.nulls)
 	b.reset()
 
 	if b.data != nil {
@@ -1413,3 +1479,17 @@ func (b *TimestampBuilder) newData() (data *Data) {
 
 	return
 }
+
+var (
+	_ Builder = (*Int64Builder)(nil)
+	_ Builder = (*Uint64Builder)(nil)
+	_ Builder = (*Float64Builder)(nil)
+	_ Builder = (*Int32Builder)(nil)
+	_ Builder = (*Uint32Builder)(nil)
+	_ Builder = (*Float32Builder)(nil)
+	_ Builder = (*Int16Builder)(nil)
+	_ Builder = (*Uint16Builder)(nil)
+	_ Builder = (*Int8Builder)(nil)
+	_ Builder = (*Uint8Builder)(nil)
+	_ Builder = (*TimestampBuilder)(nil)
+)
diff --git a/go/arrow/array/numericbuilder.gen.go.tmpl b/go/arrow/array/numericbuilder.gen.go.tmpl
index 1b2aaf0..c840530 100644
--- a/go/arrow/array/numericbuilder.gen.go.tmpl
+++ b/go/arrow/array/numericbuilder.gen.go.tmpl
@@ -28,15 +28,15 @@ type {{.Name}}Builder struct {
 	builder
 
 {{if .Opt.Parametric -}}
-	typE    *arrow.{{.Name}}Type
+	dtype    *arrow.{{.Name}}Type
 {{end -}}
 	data    *memory.Buffer
 	rawData []{{or .QualifiedType .Type}}
 }
 
 {{if .Opt.Parametric}}
-func New{{.Name}}Builder(mem memory.Allocator, typE *arrow.{{.Name}}Type) *{{.Name}}Builder {
-	return &{{.Name}}Builder{builder: builder{refCount:1, mem: mem}, typE: typE}
+func New{{.Name}}Builder(mem memory.Allocator, dtype *arrow.{{.Name}}Type) *{{.Name}}Builder {
+	return &{{.Name}}Builder{builder: builder{refCount:1, mem: mem}, dtype: dtype}
 }
 {{else}}
 func New{{.Name}}Builder(mem memory.Allocator) *{{.Name}}Builder {
@@ -81,7 +81,7 @@ func (b *{{.Name}}Builder) UnsafeAppendBoolToBitmap(isValid bool) {
 	if isValid {
 		bitutil.SetBit(b.nullBitmap.Bytes(), b.length)
 	} else {
-		b.nullN++
+		b.nulls++
 	}
 	b.length++
 }
@@ -132,6 +132,12 @@ func (b *{{.Name}}Builder) Resize(n int) {
 	}
 }
 
+// NewArray creates a {{.Name}} array from the memory buffers used by the builder and resets the {{.Name}}Builder
+// so it can be used to build a new array.
+func (b *{{.Name}}Builder) NewArray() Interface {
+	return b.New{{.Name}}Array()
+}
+
 // New{{.Name}}Array creates a {{.Name}} array from the memory buffers used by the builder and resets the {{.Name}}Builder
 // so it can be used to build a new array.
 func (b *{{.Name}}Builder) New{{.Name}}Array() (a *{{.Name}}) {
@@ -148,9 +154,9 @@ func (b *{{.Name}}Builder) newData() (data *Data) {
 		b.data.Resize(bytesRequired)
 	}
 {{if .Opt.Parametric -}}
-	data = NewData(b.typE, b.length, []*memory.Buffer{b.nullBitmap, b.data}, b.nullN)
+	data = NewData(b.dtype, b.length, []*memory.Buffer{b.nullBitmap, b.data}, nil, b.nulls)
 {{else -}}
-	data = NewData(arrow.PrimitiveTypes.{{.Name}}, b.length, []*memory.Buffer{b.nullBitmap, b.data}, b.nullN)
+	data = NewData(arrow.PrimitiveTypes.{{.Name}}, b.length, []*memory.Buffer{b.nullBitmap, b.data}, nil, b.nulls)
 {{end -}}
 	b.reset()
 
@@ -162,4 +168,10 @@ func (b *{{.Name}}Builder) newData() (data *Data) {
 
 	return
 }
-{{end}}
\ No newline at end of file
+{{end}}
+
+var (
+{{- range .In}}
+	_ Builder = (*{{.Name}}Builder)(nil)
+{{- end}}
+)
diff --git a/go/arrow/array/numeric_test.go b/go/arrow/datatype_nested.go
similarity index 52%
copy from go/arrow/array/numeric_test.go
copy to go/arrow/datatype_nested.go
index d5b798f..d34c096 100644
--- a/go/arrow/array/numeric_test.go
+++ b/go/arrow/datatype_nested.go
@@ -14,23 +14,31 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-package array_test
+package arrow
 
-import (
-	"testing"
+// ListType describes a nested type in which each array slot contains
+// a variable-size sequence of values, all having the same relative type.
+type ListType struct {
+	elem DataType // DataType of the list's elements
+}
 
-	"github.com/apache/arrow/go/arrow"
-	"github.com/apache/arrow/go/arrow/array"
-	"github.com/apache/arrow/go/arrow/memory"
-	"github.com/stretchr/testify/assert"
-)
+// ListOf returns the list type with element type t.
+// For example, if t represents int32, ListOf(t) represents []int32.
+//
+// ListOf panics if t is nil or invalid.
+func ListOf(t DataType) *ListType {
+	if t == nil {
+		panic("arrow: nil DataType")
+	}
+	return &ListType{elem: t}
+}
 
-func TestNewFloat64Data(t *testing.T) {
-	exp := []float64{1.0, 2.0, 4.0, 8.0, 16.0}
+func (*ListType) ID() Type     { return LIST }
+func (*ListType) Name() string { return "list" }
 
-	ad := array.NewData(arrow.PrimitiveTypes.Float64, len(exp), []*memory.Buffer{nil, memory.NewBufferBytes(arrow.Float64Traits.CastToBytes(exp))}, 0)
-	fa := array.NewFloat64Data(ad)
+// Elem returns the ListType's element type.
+func (t *ListType) Elem() DataType { return t.elem }
 
-	assert.Equal(t, len(exp), fa.Len(), "unexpected Len()")
-	assert.Equal(t, exp, fa.Float64Values(), "unexpected Float64Values()")
-}
+var (
+	_ DataType = (*ListType)(nil)
+)
diff --git a/go/arrow/datatype_nested_test.go b/go/arrow/datatype_nested_test.go
new file mode 100644
index 0000000..54ffb02
--- /dev/null
+++ b/go/arrow/datatype_nested_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 arrow
+
+import (
+	"reflect"
+	"testing"
+)
+
+func TestListOf(t *testing.T) {
+	for _, tc := range []DataType{
+		FixedWidthTypes.Boolean,
+		PrimitiveTypes.Int8,
+		PrimitiveTypes.Int16,
+		PrimitiveTypes.Int32,
+		PrimitiveTypes.Int64,
+		PrimitiveTypes.Uint8,
+		PrimitiveTypes.Uint16,
+		PrimitiveTypes.Uint32,
+		PrimitiveTypes.Uint64,
+		PrimitiveTypes.Float32,
+		PrimitiveTypes.Float64,
+	} {
+		t.Run(tc.Name(), func(t *testing.T) {
+			got := ListOf(tc)
+			want := &ListType{elem: tc}
+			if !reflect.DeepEqual(got, want) {
+				t.Fatalf("got=%#v, want=%#v", got, want)
+			}
+
+			if got, want := got.Name(), "list"; got != want {
+				t.Fatalf("got=%q, want=%q", got, want)
+			}
+
+			if got, want := got.ID(), LIST; got != want {
+				t.Fatalf("got=%v, want=%v", got, want)
+			}
+
+			if got, want := got.Elem(), tc; got != want {
+				t.Fatalf("got=%v, want=%v", got, want)
+			}
+		})
+	}
+}
diff --git a/go/arrow/example_test.go b/go/arrow/example_test.go
index 96fd3e5..19a6630 100644
--- a/go/arrow/example_test.go
+++ b/go/arrow/example_test.go
@@ -19,6 +19,7 @@ package arrow_test
 import (
 	"fmt"
 
+	"github.com/apache/arrow/go/arrow"
 	"github.com/apache/arrow/go/arrow/array"
 	"github.com/apache/arrow/go/arrow/memory"
 )
@@ -114,3 +115,79 @@ func Example_fromMemory() {
 	// bools[14] = false
 	// bools[15] = (null)
 }
+
+// This example shows how to create a List array.
+// The resulting array should be:
+//  [[0, 1, 2], [], [3], [4, 5], [6, 7, 8], [], [9]]
+func Example_listArray() {
+	pool := memory.NewGoAllocator()
+
+	lb := array.NewListBuilder(pool, arrow.PrimitiveTypes.Int64)
+	defer lb.Release()
+
+	vb := lb.ValueBuilder().(*array.Int64Builder)
+	defer vb.Release()
+
+	vb.Reserve(10)
+
+	lb.Append(true)
+	vb.Append(0)
+	vb.Append(1)
+	vb.Append(2)
+
+	lb.AppendNull()
+
+	lb.Append(true)
+	vb.Append(3)
+
+	lb.Append(true)
+	vb.Append(4)
+	vb.Append(5)
+
+	lb.Append(true)
+	vb.Append(6)
+	vb.Append(7)
+	vb.Append(8)
+
+	lb.AppendNull()
+
+	lb.Append(true)
+	vb.Append(9)
+
+	arr := lb.NewArray().(*array.List)
+	fmt.Printf("NullN()   = %d\n", arr.NullN())
+	fmt.Printf("Len()     = %d\n", arr.Len())
+	fmt.Printf("Offsets() = %v\n", arr.Offsets())
+
+	offsets := arr.Offsets()[1:]
+
+	varr := arr.ListValues().(*array.Int64)
+	pos := 0
+	for i := 0; i < arr.Len(); i++ {
+		if !arr.IsValid(i) {
+			fmt.Printf("List[%d]   = (null)\n", i)
+			continue
+		}
+		fmt.Printf("List[%d]   = [", i)
+		for j := pos; j < int(offsets[i]); j++ {
+			if j != pos {
+				fmt.Printf(", ")
+			}
+			fmt.Printf("%v", varr.Value(j))
+		}
+		pos = int(offsets[i])
+		fmt.Printf("]\n")
+	}
+
+	// Output:
+	// NullN()   = 2
+	// Len()     = 7
+	// Offsets() = [0 3 3 4 6 9 9 10]
+	// List[0]   = [0, 1, 2]
+	// List[1]   = (null)
+	// List[2]   = [3]
+	// List[3]   = [4, 5]
+	// List[4]   = [6, 7, 8]
+	// List[5]   = (null)
+	// List[6]   = [9]
+}
diff --git a/go/arrow/type_string.go b/go/arrow/type_string.go
index 9c82dba..b41c5ac 100644
--- a/go/arrow/type_string.go
+++ b/go/arrow/type_string.go
@@ -2,7 +2,7 @@
 
 package arrow
 
-import "fmt"
+import "strconv"
 
 const _Type_name = "NULLBOOLUINT8INT8UINT16INT16UINT32INT32UINT64INT64HALF_FLOATFLOAT32FLOAT64STRINGBINARYFIXED_SIZE_BINARYDATE32DATE64TIMESTAMPTIME32TIME64INTERVALDECIMALLISTSTRUCTUNIONDICTIONARYMAP"
 
@@ -10,7 +10,7 @@ var _Type_index = [...]uint8{0, 4, 8, 13, 17, 23, 28, 34, 39, 45, 50, 60, 67, 74
 
 func (i Type) String() string {
 	if i < 0 || i >= Type(len(_Type_index)-1) {
-		return fmt.Sprintf("Type(%d)", i)
+		return "Type(" + strconv.FormatInt(int64(i), 10) + ")"
 	}
 	return _Type_name[_Type_index[i]:_Type_index[i+1]]
 }