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/14 17:34:49 UTC

[arrow] branch master updated: ARROW-3037: [Go] implement Null array

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 08fa38b  ARROW-3037: [Go] implement Null array
08fa38b is described below

commit 08fa38b7b659ce22abe368f8fa86d51171058d10
Author: Sebastien Binet <bi...@cern.ch>
AuthorDate: Tue Aug 14 13:34:44 2018 -0400

    ARROW-3037: [Go] implement Null array
    
    needs https://github.com/apache/arrow/pull/2419.
    
    Author: Sebastien Binet <bi...@cern.ch>
    
    Closes #2421 from sbinet/issue-3037 and squashes the following commits:
    
    39d0bc79 <Sebastien Binet> ARROW-3037:  implement Null array
---
 go/README.md                   |   2 +-
 go/arrow/array/array.go        |   2 +-
 go/arrow/array/array_test.go   |   2 +-
 go/arrow/array/null.go         | 123 +++++++++++++++++++++++++++++++++++++++++
 go/arrow/array/null_test.go    |  77 ++++++++++++++++++++++++++
 go/arrow/datatype_null.go      |  28 ++++++++++
 go/arrow/datatype_null_test.go |  34 ++++++++++++
 7 files changed, 265 insertions(+), 3 deletions(-)

diff --git a/go/README.md b/go/README.md
index 90d55e4..9b801f7 100644
--- a/go/README.md
+++ b/go/README.md
@@ -133,7 +133,7 @@ them internally in the [ifql][] execution engine and storage layers of [InfluxDB
 - [x] Variable-length binary
 - [ ] String (valid UTF-8)
 - [ ] Half-float (16-bit)
-- [ ] Null (no physical storage)
+- [x] Null (no physical storage)
 
 **Parametric types**
 
diff --git a/go/arrow/array/array.go b/go/arrow/array/array.go
index 9b49ea0..7aa1295 100644
--- a/go/arrow/array/array.go
+++ b/go/arrow/array/array.go
@@ -134,7 +134,7 @@ type arrayConstructorFn func(*Data) Interface
 
 var (
 	makeArrayFn = [...]arrayConstructorFn{
-		arrow.NULL:              unsupportedArrayType,
+		arrow.NULL:              func(data *Data) Interface { return NewNullData(data) },
 		arrow.BOOL:              func(data *Data) Interface { return NewBooleanData(data) },
 		arrow.UINT8:             func(data *Data) Interface { return NewUint8Data(data) },
 		arrow.INT8:              func(data *Data) Interface { return NewInt8Data(data) },
diff --git a/go/arrow/array/array_test.go b/go/arrow/array/array_test.go
index 5c82709..02f588a 100644
--- a/go/arrow/array/array_test.go
+++ b/go/arrow/array/array_test.go
@@ -43,10 +43,10 @@ func TestMakeFromData(t *testing.T) {
 		expError string
 	}{
 		// unsupported types
-		{name: "null", d: &testDataType{arrow.NULL}, expPanic: true, expError: "unsupported data type: NULL"},
 		{name: "map", d: &testDataType{arrow.MAP}, expPanic: true, expError: "unsupported data type: MAP"},
 
 		// supported types
+		{name: "null", d: &testDataType{arrow.NULL}},
 		{name: "bool", d: &testDataType{arrow.BOOL}},
 		{name: "uint8", d: &testDataType{arrow.UINT8}},
 		{name: "uint16", d: &testDataType{arrow.UINT16}},
diff --git a/go/arrow/array/null.go b/go/arrow/array/null.go
new file mode 100644
index 0000000..5cab192
--- /dev/null
+++ b/go/arrow/array/null.go
@@ -0,0 +1,123 @@
+// 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/debug"
+	"github.com/apache/arrow/go/arrow/memory"
+)
+
+// Null represents an immutable, degenerate array with no physical storage.
+type Null struct {
+	array
+}
+
+// NewNull returns a new Null array value of size n.
+func NewNull(n int) *Null {
+	a := &Null{}
+	a.refCount = 1
+	data := NewData(
+		arrow.Null, n,
+		[]*memory.Buffer{nil},
+		nil,
+		n,
+		0,
+	)
+	a.setData(data)
+	data.Release()
+	return a
+}
+
+// NewNullData returns a new Null array value, from data.
+func NewNullData(data *Data) *Null {
+	a := &Null{}
+	a.refCount = 1
+	a.setData(data)
+	return a
+}
+
+func (a *Null) setData(data *Data) {
+	a.array.setData(data)
+	a.array.nullBitmapBytes = nil
+	a.array.data.nulls = a.array.data.length
+}
+
+type NullBuilder struct {
+	builder
+}
+
+// NewNullBuilder returns a builder, using the provided memory allocator.
+func NewNullBuilder(mem memory.Allocator) *NullBuilder {
+	return &NullBuilder{builder: builder{refCount: 1, mem: mem}}
+}
+
+// Release decreases the reference count by 1.
+// When the reference count goes to zero, the memory is freed.
+func (b *NullBuilder) 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
+		}
+	}
+}
+
+func (b *NullBuilder) AppendNull() {
+	b.builder.length++
+	b.builder.nulls++
+}
+
+func (*NullBuilder) init(cap int)                       {}
+func (*NullBuilder) resize(newBits int, init func(int)) {}
+
+// NewArray creates a List array from the memory buffers used by the builder and resets the NullBuilder
+// so it can be used to build a new array.
+func (b *NullBuilder) NewArray() Interface {
+	return b.NewNullArray()
+}
+
+// NewNullArray creates a Null array from the memory buffers used by the builder and resets the NullBuilder
+// so it can be used to build a new array.
+func (b *NullBuilder) NewNullArray() (a *Null) {
+	data := b.newData()
+	a = NewNullData(data)
+	data.Release()
+	return
+}
+
+func (b *NullBuilder) newData() (data *Data) {
+	data = NewData(
+		arrow.Null, b.length,
+		[]*memory.Buffer{nil},
+		nil,
+		b.nulls,
+		0,
+	)
+	b.reset()
+
+	return
+}
+
+var (
+	_ Interface = (*Null)(nil)
+	_ Builder   = (*NullBuilder)(nil)
+)
diff --git a/go/arrow/array/null_test.go b/go/arrow/array/null_test.go
new file mode 100644
index 0000000..1339c4c
--- /dev/null
+++ b/go/arrow/array/null_test.go
@@ -0,0 +1,77 @@
+// 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 (
+	"testing"
+
+	"github.com/apache/arrow/go/arrow"
+	"github.com/apache/arrow/go/arrow/array"
+	"github.com/apache/arrow/go/arrow/memory"
+)
+
+func TestNullArray(t *testing.T) {
+	pool := memory.NewCheckedAllocator(memory.NewGoAllocator())
+	defer pool.AssertSize(t, 0)
+
+	b := array.NewNullBuilder(pool)
+	defer b.Release()
+
+	b.AppendNull()
+	b.AppendNull()
+
+	arr1 := b.NewArray().(*array.Null)
+	defer arr1.Release()
+
+	if got, want := arr1.Len(), 2; got != want {
+		t.Fatalf("invalid null array length: got=%d, want=%d", got, want)
+	}
+
+	if got, want := arr1.NullN(), 2; got != want {
+		t.Fatalf("invalid number of nulls: got=%d, want=%d", got, want)
+	}
+
+	if got, want := arr1.DataType(), arrow.Null; got != want {
+		t.Fatalf("invalid null data type: got=%v, want=%v", got, want)
+	}
+
+	arr1.Retain()
+	arr1.Release()
+
+	if arr1.Data() == nil {
+		t.Fatalf("invalid null data")
+	}
+
+	arr2 := b.NewNullArray()
+	defer arr2.Release()
+
+	if got, want := arr2.Len(), 0; got != want {
+		t.Fatalf("invalid null array length: got=%d, want=%d", got, want)
+	}
+
+	arr3 := array.NewNull(10)
+	defer arr3.Release()
+
+	if got, want := arr3.Len(), 10; got != want {
+		t.Fatalf("invalid null array length: got=%d, want=%d", got, want)
+	}
+
+	if got, want := arr3.NullN(), 10; got != want {
+		t.Fatalf("invalid number of nulls: got=%d, want=%d", got, want)
+	}
+
+}
diff --git a/go/arrow/datatype_null.go b/go/arrow/datatype_null.go
new file mode 100644
index 0000000..7f0d8bf
--- /dev/null
+++ b/go/arrow/datatype_null.go
@@ -0,0 +1,28 @@
+// 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
+
+// NullType describes a degenerate array, with zero physical storage.
+type NullType struct{}
+
+func (*NullType) ID() Type     { return NULL }
+func (*NullType) Name() string { return "null" }
+
+var (
+	Null *NullType
+	_    DataType = Null
+)
diff --git a/go/arrow/datatype_null_test.go b/go/arrow/datatype_null_test.go
new file mode 100644
index 0000000..e773cc4
--- /dev/null
+++ b/go/arrow/datatype_null_test.go
@@ -0,0 +1,34 @@
+// 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_test
+
+import (
+	"testing"
+
+	"github.com/apache/arrow/go/arrow"
+)
+
+func TestNullType(t *testing.T) {
+	var nt *arrow.NullType
+	if got, want := nt.ID(), arrow.NULL; got != want {
+		t.Fatalf("invalid null type id. got=%v, want=%v", got, want)
+	}
+
+	if got, want := nt.Name(), "null"; got != want {
+		t.Fatalf("invalid null type name. got=%q, want=%q", got, want)
+	}
+}