You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by ra...@apache.org on 2023/05/30 08:49:03 UTC

[arrow] 02/12: GH-35337: [Go] ASAN tests fail with Go1.20+ (#35338)

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

raulcd pushed a commit to branch maint-12.0.x
in repository https://gitbox.apache.org/repos/asf/arrow.git

commit 35a67867fab80b0c0b224cef7b5cd6576f9c030a
Author: Matt Topol <zo...@gmail.com>
AuthorDate: Thu Apr 27 22:59:21 2023 -0400

    GH-35337: [Go] ASAN tests fail with Go1.20+ (#35338)
    
    
    
    ### Rationale for this change
    Fixing crashes when using ASAN to run unit tests under Go1.20+, requires features that were only added in go1.20. For versions <=go.19, the code remains unchanged.
    
    The only way to properly test these changes would be to increase the matrix of jobs for Go, but we already have quite a few jobs. @ assignUser Do you have any issues with adding go1.20 (or maybe just telling it to use "latest") as another set of Go jobs to the GHA config?
    * Closes: #35337
    
    Lead-authored-by: Matt Topol <zo...@gmail.com>
    Co-authored-by: Sutou Kouhei <ko...@cozmixng.org>
    Signed-off-by: Sutou Kouhei <ko...@clear-code.com>
---
 .github/workflows/go.yml                           |  2 +-
 dev/tasks/tasks.yml                                |  7 +-
 docker-compose.yml                                 |  6 +-
 go/arrow/array/numeric.gen.go                      | 11 ++-
 go/arrow/array/numericbuilder.gen.go               | 21 ++---
 go/arrow/array/numericbuilder.gen.go.tmpl          |  8 +-
 go/arrow/arrio/arrio_test.go                       |  5 +-
 go/arrow/compute/arithmetic.go                     |  9 +--
 go/arrow/csv/reader_test.go                        |  5 +-
 go/arrow/csv/writer_test.go                        |  4 +-
 go/arrow/flight/flightsql/example/sqlite_server.go |  6 +-
 go/arrow/internal/arrjson/arrjson_test.go          |  8 +-
 go/arrow/ipc/cmd/arrow-cat/main_test.go            |  5 +-
 go/arrow/ipc/cmd/arrow-file-to-stream/main_test.go |  6 +-
 .../cmd/arrow-json-integration-test/main_test.go   |  7 +-
 go/arrow/ipc/cmd/arrow-ls/main_test.go             |  5 +-
 go/arrow/ipc/cmd/arrow-stream-to-file/main_test.go |  6 +-
 go/arrow/ipc/file_test.go                          |  6 +-
 go/arrow/ipc/stream_test.go                        |  6 +-
 go/go.mod                                          |  2 +-
 go/internal/hashing/hash_funcs.go                  | 90 ++++++++++++++++++++++
 go/internal/hashing/hash_string.go                 | 26 +++++++
 go/internal/hashing/hash_string_go1.19.go          | 30 ++++++++
 go/internal/hashing/xxh3_memo_table.go             | 75 ------------------
 go/parquet/compress/brotli.go                      |  5 +-
 go/parquet/compress/compress.go                    |  3 +-
 go/parquet/compress/compress_test.go               |  4 +-
 go/parquet/compress/gzip.go                        |  3 +-
 go/parquet/compress/snappy.go                      |  3 +-
 go/parquet/encryption_write_config_test.go         |  3 +-
 go/parquet/file/file_reader_test.go                |  2 +-
 31 files changed, 223 insertions(+), 156 deletions(-)

diff --git a/.github/workflows/go.yml b/.github/workflows/go.yml
index b0ea7f9397..eee2c8cdfc 100644
--- a/.github/workflows/go.yml
+++ b/.github/workflows/go.yml
@@ -126,7 +126,7 @@ jobs:
     name: AMD64 Debian 11 Go ${{ matrix.go }} - CGO
     runs-on: ubuntu-latest
     if: ${{ !contains(github.event.pull_request.title, 'WIP') }}
-    timeout-minutes: 15
+    timeout-minutes: 20
     strategy:
       fail-fast: false
       matrix:
diff --git a/dev/tasks/tasks.yml b/dev/tasks/tasks.yml
index e601346780..998f2aecdd 100644
--- a/dev/tasks/tasks.yml
+++ b/dev/tasks/tasks.yml
@@ -1411,14 +1411,17 @@ tasks:
         R_PRUNE_DEPS: TRUE
       image: fedora-r-clang-sanitizer
 
-  test-debian-11-go-1.17:
+  {% for go_version, staticcheck in [("1.17", "v0.2.2"), ("1.20", "latest")] %}
+  test-debian-11-go-{{ go_version }}:
     ci: azure
     template: docker-tests/azure.linux.yml
     params:
       env:
         DEBIAN: 11
-        GO: 1.17
+        GO: "{{go_version}}"
+        STATICCHECK: "{{ staticcheck }}"
       image: debian-go
+  {% endfor %}
 
   test-ubuntu-default-docs:
     ci: azure
diff --git a/docker-compose.yml b/docker-compose.yml
index 46024c43cb..a7f5f99278 100644
--- a/docker-compose.yml
+++ b/docker-compose.yml
@@ -1508,6 +1508,7 @@ services:
     volumes: *debian-volumes
     command: &go-command >
       /bin/bash -c "
+        git config --global --add safe.directory /arrow &&
         /arrow/ci/scripts/go_build.sh /arrow &&
         /arrow/ci/scripts/go_test.sh /arrow"
 
@@ -1544,8 +1545,9 @@ services:
     shm_size: *shm-size
     volumes: *debian-volumes
     command: &go-cgo-python-command >
-        /bin/bash -c "
-          /arrow/ci/scripts/go_cgo_python_test.sh /arrow"
+      /bin/bash -c "
+        git config --global --add safe.directory /arrow &&
+        /arrow/ci/scripts/go_cgo_python_test.sh /arrow"
 
   ############################# JavaScript ####################################
 
diff --git a/go/arrow/array/numeric.gen.go b/go/arrow/array/numeric.gen.go
index 5e9d5c7560..72db2d9d77 100644
--- a/go/arrow/array/numeric.gen.go
+++ b/go/arrow/array/numeric.gen.go
@@ -1126,8 +1126,7 @@ func (a *Time32) ValueStr(i int) string {
 	if a.IsNull(i) {
 		return NullValueStr
 	}
-	a.values[i].FormattedString(a.DataType().(*arrow.Time32Type).Unit)
-	return a.values[i].ToTime(a.DataType().(*arrow.Time32Type).Unit).Format("15:04:05.999999999")
+	return a.values[i].FormattedString(a.DataType().(*arrow.Time32Type).Unit)
 }
 
 func (a *Time32) GetOneForMarshal(i int) interface{} {
@@ -1217,7 +1216,7 @@ func (a *Time64) ValueStr(i int) string {
 	if a.IsNull(i) {
 		return NullValueStr
 	}
-	return a.values[i].ToTime(a.DataType().(*arrow.Time64Type).Unit).Format("15:04:05.999999999")
+	return a.values[i].FormattedString(a.DataType().(*arrow.Time64Type).Unit)
 }
 
 func (a *Time64) GetOneForMarshal(i int) interface{} {
@@ -1307,7 +1306,7 @@ func (a *Date32) ValueStr(i int) string {
 	if a.IsNull(i) {
 		return NullValueStr
 	}
-	return a.values[i].ToTime().Format("2006-01-02")
+	return a.values[i].FormattedString()
 }
 
 func (a *Date32) GetOneForMarshal(i int) interface{} {
@@ -1397,7 +1396,7 @@ func (a *Date64) ValueStr(i int) string {
 	if a.IsNull(i) {
 		return NullValueStr
 	}
-	return a.values[i].ToTime().Format("2006-01-02")
+	return a.values[i].FormattedString()
 }
 
 func (a *Date64) GetOneForMarshal(i int) interface{} {
@@ -1488,7 +1487,7 @@ func (a *Duration) ValueStr(i int) string {
 		return NullValueStr
 	}
 	// return value and suffix as a string such as "12345ms"
-	return fmt.Sprintf("%d%s", a.values[i], a.DataType().(*arrow.DurationType).Unit.String())
+	return fmt.Sprintf("%d%s", a.values[i], a.DataType().(*arrow.DurationType).Unit)
 }
 
 func (a *Duration) GetOneForMarshal(i int) interface{} {
diff --git a/go/arrow/array/numericbuilder.gen.go b/go/arrow/array/numericbuilder.gen.go
index 0cff3a581f..f893a633cf 100644
--- a/go/arrow/array/numericbuilder.gen.go
+++ b/go/arrow/array/numericbuilder.gen.go
@@ -181,7 +181,7 @@ func (b *Int64Builder) AppendValueFromString(s string) error {
 		b.AppendNull()
 		return nil
 	}
-	v, err := strconv.ParseInt(s, 10, 64)
+	v, err := strconv.ParseInt(s, 10, 8*8)
 	if err != nil {
 		b.AppendNull()
 		return err
@@ -404,7 +404,7 @@ func (b *Uint64Builder) AppendValueFromString(s string) error {
 		b.AppendNull()
 		return nil
 	}
-	v, err := strconv.ParseUint(s, 10, 64)
+	v, err := strconv.ParseUint(s, 10, 8*8)
 	if err != nil {
 		b.AppendNull()
 		return err
@@ -627,7 +627,7 @@ func (b *Float64Builder) AppendValueFromString(s string) error {
 		b.AppendNull()
 		return nil
 	}
-	v, err := strconv.ParseFloat(s, 64)
+	v, err := strconv.ParseFloat(s, 8*8)
 	if err != nil {
 		b.AppendNull()
 		return err
@@ -850,7 +850,7 @@ func (b *Int32Builder) AppendValueFromString(s string) error {
 		b.AppendNull()
 		return nil
 	}
-	v, err := strconv.ParseInt(s, 10, 32)
+	v, err := strconv.ParseInt(s, 10, 4*8)
 	if err != nil {
 		b.AppendNull()
 		return err
@@ -1073,7 +1073,7 @@ func (b *Uint32Builder) AppendValueFromString(s string) error {
 		b.AppendNull()
 		return nil
 	}
-	v, err := strconv.ParseUint(s, 10, 32)
+	v, err := strconv.ParseUint(s, 10, 4*8)
 	if err != nil {
 		b.AppendNull()
 		return err
@@ -1296,7 +1296,7 @@ func (b *Float32Builder) AppendValueFromString(s string) error {
 		b.AppendNull()
 		return nil
 	}
-	v, err := strconv.ParseFloat(s, 32)
+	v, err := strconv.ParseFloat(s, 4*8)
 	if err != nil {
 		b.AppendNull()
 		return err
@@ -1519,7 +1519,7 @@ func (b *Int16Builder) AppendValueFromString(s string) error {
 		b.AppendNull()
 		return nil
 	}
-	v, err := strconv.ParseInt(s, 10, 16)
+	v, err := strconv.ParseInt(s, 10, 2*8)
 	if err != nil {
 		b.AppendNull()
 		return err
@@ -1742,7 +1742,7 @@ func (b *Uint16Builder) AppendValueFromString(s string) error {
 		b.AppendNull()
 		return nil
 	}
-	v, err := strconv.ParseUint(s, 10, 16)
+	v, err := strconv.ParseUint(s, 10, 2*8)
 	if err != nil {
 		b.AppendNull()
 		return err
@@ -1965,7 +1965,7 @@ func (b *Int8Builder) AppendValueFromString(s string) error {
 		b.AppendNull()
 		return nil
 	}
-	v, err := strconv.ParseInt(s, 10, 8)
+	v, err := strconv.ParseInt(s, 10, 1*8)
 	if err != nil {
 		b.AppendNull()
 		return err
@@ -2188,7 +2188,7 @@ func (b *Uint8Builder) AppendValueFromString(s string) error {
 		b.AppendNull()
 		return nil
 	}
-	v, err := strconv.ParseUint(s, 10, 8)
+	v, err := strconv.ParseUint(s, 10, 1*8)
 	if err != nil {
 		b.AppendNull()
 		return err
@@ -3535,6 +3535,7 @@ func (b *DurationBuilder) AppendValueFromString(s string) error {
 		return nil
 	}
 	return fmt.Errorf("%w: AppendValueFromString not implemented for Duration", arrow.ErrNotImplemented)
+	return nil
 }
 
 func (b *DurationBuilder) UnmarshalOne(dec *json.Decoder) error {
diff --git a/go/arrow/array/numericbuilder.gen.go.tmpl b/go/arrow/array/numericbuilder.gen.go.tmpl
index d414945ac0..3778153870 100644
--- a/go/arrow/array/numericbuilder.gen.go.tmpl
+++ b/go/arrow/array/numericbuilder.gen.go.tmpl
@@ -225,23 +225,23 @@ func (b *{{.Name}}Builder) AppendValueFromString(s string) error {
     }
     b.Append(v)
   {{else if (eq .Name "Duration") -}}	
-    return fmt.Errorf("%w: AppendValueFromString not implemented for Duration", ErrNotImplemented)
+    return fmt.Errorf("%w: AppendValueFromString not implemented for Duration", arrow.ErrNotImplemented)
   {{else if or (eq .Name "Int8") (eq .Name "Int16") (eq .Name "Int32") (eq .Name "Int64") -}}
-    v, err := strconv.ParseInt(s, 10, {{.Size}})
+    v, err := strconv.ParseInt(s, 10, {{.Size}} * 8)
     if err != nil {
         b.AppendNull()
         return err
     }
     b.Append({{.name}}(v))
   {{else if or (eq .Name "Uint8") (eq .Name "Uint16") (eq .Name "Uint32") (eq .Name "Uint64") -}}
-    v, err := strconv.ParseUint(s, 10, {{.Size}})
+    v, err := strconv.ParseUint(s, 10, {{.Size}} * 8)
     if err != nil {
         b.AppendNull()
         return err
     }
     b.Append({{.name}}(v))
   {{else if or (eq .Name "Float32") (eq .Name "Float64") -}}
-    v, err := strconv.ParseFloat(s, {{.Size}})
+    v, err := strconv.ParseFloat(s, {{.Size}} * 8)
     if err != nil {
       b.AppendNull()
       return err
diff --git a/go/arrow/arrio/arrio_test.go b/go/arrow/arrio/arrio_test.go
index 707114c5ad..38d1cfc10a 100644
--- a/go/arrow/arrio/arrio_test.go
+++ b/go/arrow/arrio/arrio_test.go
@@ -19,7 +19,6 @@ package arrio_test
 import (
 	"fmt"
 	"io"
-	"io/ioutil"
 	"os"
 	"testing"
 
@@ -93,13 +92,13 @@ func TestCopy(t *testing.T) {
 							mem := memory.NewCheckedAllocator(memory.NewGoAllocator())
 							defer mem.AssertSize(t, 0)
 
-							f, err := ioutil.TempFile(tempDir, "go-arrow-copy-")
+							f, err := os.CreateTemp(tempDir, "go-arrow-copy-")
 							if err != nil {
 								t.Fatal(err)
 							}
 							defer f.Close()
 
-							o, err := ioutil.TempFile(tempDir, "go-arrow-copy-")
+							o, err := os.CreateTemp(tempDir, "go-arrow-copy-")
 							if err != nil {
 								t.Fatal(err)
 							}
diff --git a/go/arrow/compute/arithmetic.go b/go/arrow/compute/arithmetic.go
index 45d8d77382..56eb38df76 100644
--- a/go/arrow/compute/arithmetic.go
+++ b/go/arrow/compute/arithmetic.go
@@ -909,7 +909,7 @@ func RegisterScalarArithmetic(reg FunctionRegistry) {
 		reg.AddFunction(fn, false)
 	}
 
-	fn = &arithmeticFunction{*NewScalarFunction("bit_wise_not", Unary(), EmptyFuncDoc), decPromoteNone}
+	fn = &arithmeticFunction{*NewScalarFunction("bit_wise_not", Unary(), bitWiseNotDoc), decPromoteNone}
 	for _, k := range kernels.GetBitwiseUnaryKernels() {
 		if err := fn.AddKernel(k); err != nil {
 			panic(err)
@@ -1087,10 +1087,9 @@ func Negate(ctx context.Context, opts ArithmeticOptions, input Datum) (Datum, er
 // Sign returns -1, 0, or 1 depending on the sign of each element in the
 // input. For x in the input:
 //
-//	if x > 0: 1
-//  if x < 0: -1
-//  if x == 0: 0
-//
+//		if x > 0: 1
+//	 if x < 0: -1
+//	 if x == 0: 0
 func Sign(ctx context.Context, input Datum) (Datum, error) {
 	return CallFunction(ctx, "sign", nil, input)
 }
diff --git a/go/arrow/csv/reader_test.go b/go/arrow/csv/reader_test.go
index 560354fb64..af00525d8a 100644
--- a/go/arrow/csv/reader_test.go
+++ b/go/arrow/csv/reader_test.go
@@ -20,7 +20,6 @@ import (
 	"bytes"
 	stdcsv "encoding/csv"
 	"fmt"
-	"io/ioutil"
 	"log"
 	"os"
 	"strings"
@@ -315,7 +314,7 @@ func testCSVReader(t *testing.T, filepath string, withHeader bool) {
 	mem := memory.NewCheckedAllocator(memory.NewGoAllocator())
 	defer mem.AssertSize(t, 0)
 
-	raw, err := ioutil.ReadFile(filepath)
+	raw, err := os.ReadFile(filepath)
 	if err != nil {
 		t.Fatal(err)
 	}
@@ -448,7 +447,7 @@ func TestCSVReaderWithChunk(t *testing.T) {
 	mem := memory.NewCheckedAllocator(memory.NewGoAllocator())
 	defer mem.AssertSize(t, 0)
 
-	raw, err := ioutil.ReadFile("testdata/simple.csv")
+	raw, err := os.ReadFile("testdata/simple.csv")
 	if err != nil {
 		t.Fatal(err)
 	}
diff --git a/go/arrow/csv/writer_test.go b/go/arrow/csv/writer_test.go
index 9f1b524ce7..61021c849e 100644
--- a/go/arrow/csv/writer_test.go
+++ b/go/arrow/csv/writer_test.go
@@ -21,7 +21,7 @@ import (
 	"bytes"
 	ecsv "encoding/csv"
 	"fmt"
-	"io/ioutil"
+	"io"
 	"log"
 	"testing"
 
@@ -379,7 +379,7 @@ func BenchmarkWrite(b *testing.B) {
 	rec := bldr.NewRecord()
 	defer rec.Release()
 
-	w := csv.NewWriter(ioutil.Discard, schema, csv.WithComma(';'), csv.WithCRLF(false))
+	w := csv.NewWriter(io.Discard, schema, csv.WithComma(';'), csv.WithCRLF(false))
 
 	b.ResetTimer()
 	for i := 0; i < b.N; i++ {
diff --git a/go/arrow/flight/flightsql/example/sqlite_server.go b/go/arrow/flight/flightsql/example/sqlite_server.go
index 303b6e3de4..23d9a3b99b 100644
--- a/go/arrow/flight/flightsql/example/sqlite_server.go
+++ b/go/arrow/flight/flightsql/example/sqlite_server.go
@@ -60,7 +60,9 @@ import (
 func genRandomString() []byte {
 	const length = 16
 	max := int('z')
-	min := int('0')
+	// don't include ':' as a valid byte to generate
+	// because we use it as a separator for the transactions
+	min := int('<')
 
 	out := make([]byte, length)
 	for i := range out {
@@ -249,7 +251,7 @@ func (s *SQLiteFlightSQLServer) DoGetStatement(ctx context.Context, cmd flightsq
 	if txnid != "" {
 		tx, loaded := s.openTransactions.Load(txnid)
 		if !loaded {
-			return nil, nil, fmt.Errorf("%w: invalid transaction id specified", arrow.ErrInvalid)
+			return nil, nil, fmt.Errorf("%w: invalid transaction id specified: %s", arrow.ErrInvalid, txnid)
 		}
 		db = tx.(*sql.Tx)
 	}
diff --git a/go/arrow/internal/arrjson/arrjson_test.go b/go/arrow/internal/arrjson/arrjson_test.go
index d1ac46a160..688bc2a9a5 100644
--- a/go/arrow/internal/arrjson/arrjson_test.go
+++ b/go/arrow/internal/arrjson/arrjson_test.go
@@ -19,7 +19,7 @@ package arrjson
 import (
 	"errors"
 	"io"
-	"io/ioutil"
+	"os"
 	"testing"
 
 	"github.com/apache/arrow/go/v12/arrow/array"
@@ -53,7 +53,7 @@ func TestReadWrite(t *testing.T) {
 			mem := memory.NewCheckedAllocator(memory.NewGoAllocator())
 			defer mem.AssertSize(t, 0)
 
-			f, err := ioutil.TempFile(tempDir, "go-arrow-read-write-")
+			f, err := os.CreateTemp(tempDir, "go-arrow-read-write-")
 			if err != nil {
 				t.Fatal(err)
 			}
@@ -82,7 +82,7 @@ func TestReadWrite(t *testing.T) {
 				t.Fatalf("could not sync data to disk: %v", err)
 			}
 
-			fileBytes, _ := ioutil.ReadFile(f.Name())
+			fileBytes, _ := os.ReadFile(f.Name())
 			assert.JSONEq(t, wantJSONs[name], string(fileBytes))
 
 			_, err = f.Seek(0, io.SeekStart)
@@ -92,7 +92,7 @@ func TestReadWrite(t *testing.T) {
 
 			r, err := NewReader(f, WithAllocator(mem), WithSchema(recs[0].Schema()))
 			if err != nil {
-				raw, _ := ioutil.ReadFile(f.Name())
+				raw, _ := os.ReadFile(f.Name())
 				t.Fatalf("could not read JSON file: %v\n%v\n", err, string(raw))
 			}
 			defer r.Release()
diff --git a/go/arrow/ipc/cmd/arrow-cat/main_test.go b/go/arrow/ipc/cmd/arrow-cat/main_test.go
index 98a239d538..0c35dae55c 100644
--- a/go/arrow/ipc/cmd/arrow-cat/main_test.go
+++ b/go/arrow/ipc/cmd/arrow-cat/main_test.go
@@ -20,7 +20,6 @@ import (
 	"bytes"
 	"fmt"
 	"io"
-	"io/ioutil"
 	"os"
 	"testing"
 
@@ -176,7 +175,7 @@ record 3...
 			defer mem.AssertSize(t, 0)
 
 			fname := func() string {
-				f, err := ioutil.TempFile(tempDir, "go-arrow-cat-stream-")
+				f, err := os.CreateTemp(tempDir, "go-arrow-cat-stream-")
 				if err != nil {
 					t.Fatal(err)
 				}
@@ -518,7 +517,7 @@ record 3/3...
 			defer mem.AssertSize(t, 0)
 
 			fname := func() string {
-				f, err := ioutil.TempFile(tempDir, "go-arrow-cat-file-")
+				f, err := os.CreateTemp(tempDir, "go-arrow-cat-file-")
 				if err != nil {
 					t.Fatal(err)
 				}
diff --git a/go/arrow/ipc/cmd/arrow-file-to-stream/main_test.go b/go/arrow/ipc/cmd/arrow-file-to-stream/main_test.go
index 7f25ca5494..a00c9262f1 100644
--- a/go/arrow/ipc/cmd/arrow-file-to-stream/main_test.go
+++ b/go/arrow/ipc/cmd/arrow-file-to-stream/main_test.go
@@ -18,7 +18,7 @@ package main
 
 import (
 	"io"
-	"io/ioutil"
+	"os"
 	"testing"
 
 	"github.com/apache/arrow/go/v12/arrow/internal/arrdata"
@@ -33,7 +33,7 @@ func TestFileToStream(t *testing.T) {
 			mem := memory.NewCheckedAllocator(memory.NewGoAllocator())
 			defer mem.AssertSize(t, 0)
 
-			f, err := ioutil.TempFile(tempDir, "go-arrow-file-to-stream-")
+			f, err := os.CreateTemp(tempDir, "go-arrow-file-to-stream-")
 			if err != nil {
 				t.Fatal(err)
 			}
@@ -41,7 +41,7 @@ func TestFileToStream(t *testing.T) {
 
 			arrdata.WriteFile(t, f, mem, recs[0].Schema(), recs)
 
-			o, err := ioutil.TempFile(tempDir, "go-arrow-file-to-stream-")
+			o, err := os.CreateTemp(tempDir, "go-arrow-file-to-stream-")
 			if err != nil {
 				t.Fatal(err)
 			}
diff --git a/go/arrow/ipc/cmd/arrow-json-integration-test/main_test.go b/go/arrow/ipc/cmd/arrow-json-integration-test/main_test.go
index 68e8e1c58b..f7b889a140 100644
--- a/go/arrow/ipc/cmd/arrow-json-integration-test/main_test.go
+++ b/go/arrow/ipc/cmd/arrow-json-integration-test/main_test.go
@@ -17,7 +17,6 @@
 package main
 
 import (
-	"io/ioutil"
 	"os"
 	"testing"
 
@@ -37,7 +36,7 @@ func TestIntegration(t *testing.T) {
 			mem := memory.NewCheckedAllocator(memory.NewGoAllocator())
 			defer mem.AssertSize(t, 0)
 
-			af1, err := ioutil.TempFile(tempDir, "go-arrow-integration-")
+			af1, err := os.CreateTemp(tempDir, "go-arrow-integration-")
 			if err != nil {
 				t.Fatal(err)
 			}
@@ -46,7 +45,7 @@ func TestIntegration(t *testing.T) {
 			arrdata.WriteFile(t, af1, mem, recs[0].Schema(), recs)
 			arrdata.CheckArrowFile(t, af1, mem, recs[0].Schema(), recs)
 
-			aj, err := ioutil.TempFile(tempDir, "arrow-json-integration-")
+			aj, err := os.CreateTemp(tempDir, "arrow-json-integration-")
 			if err != nil {
 				t.Fatal(err)
 			}
@@ -62,7 +61,7 @@ func TestIntegration(t *testing.T) {
 				t.Fatal(err)
 			}
 
-			af2, err := ioutil.TempFile(tempDir, "go-arrow-integration-")
+			af2, err := os.CreateTemp(tempDir, "go-arrow-integration-")
 			if err != nil {
 				t.Fatal(err)
 			}
diff --git a/go/arrow/ipc/cmd/arrow-ls/main_test.go b/go/arrow/ipc/cmd/arrow-ls/main_test.go
index 57ed91534e..cb5bdc6b42 100644
--- a/go/arrow/ipc/cmd/arrow-ls/main_test.go
+++ b/go/arrow/ipc/cmd/arrow-ls/main_test.go
@@ -20,7 +20,6 @@ import (
 	"bytes"
 	"fmt"
 	"io"
-	"io/ioutil"
 	"os"
 	"testing"
 
@@ -121,7 +120,7 @@ records: 3
 			defer mem.AssertSize(t, 0)
 
 			fname := func() string {
-				f, err := ioutil.TempFile(tempDir, "go-arrow-ls-stream-")
+				f, err := os.CreateTemp(tempDir, "go-arrow-ls-stream-")
 				if err != nil {
 					t.Fatal(err)
 				}
@@ -277,7 +276,7 @@ records: 3
 			defer mem.AssertSize(t, 0)
 
 			fname := func() string {
-				f, err := ioutil.TempFile(tempDir, "go-arrow-ls-file-")
+				f, err := os.CreateTemp(tempDir, "go-arrow-ls-file-")
 				if err != nil {
 					t.Fatal(err)
 				}
diff --git a/go/arrow/ipc/cmd/arrow-stream-to-file/main_test.go b/go/arrow/ipc/cmd/arrow-stream-to-file/main_test.go
index 4a145ba145..b86dec7b7a 100644
--- a/go/arrow/ipc/cmd/arrow-stream-to-file/main_test.go
+++ b/go/arrow/ipc/cmd/arrow-stream-to-file/main_test.go
@@ -18,7 +18,7 @@ package main
 
 import (
 	"io"
-	"io/ioutil"
+	"os"
 	"testing"
 
 	"github.com/apache/arrow/go/v12/arrow/internal/arrdata"
@@ -33,7 +33,7 @@ func TestStreamToFile(t *testing.T) {
 			mem := memory.NewCheckedAllocator(memory.NewGoAllocator())
 			defer mem.AssertSize(t, 0)
 
-			f, err := ioutil.TempFile(tempDir, "go-arrow-stream-to-file-")
+			f, err := os.CreateTemp(tempDir, "go-arrow-stream-to-file-")
 			if err != nil {
 				t.Fatal(err)
 			}
@@ -51,7 +51,7 @@ func TestStreamToFile(t *testing.T) {
 				t.Fatal(err)
 			}
 
-			o, err := ioutil.TempFile(tempDir, "go-arrow-stream-to-file-")
+			o, err := os.CreateTemp(tempDir, "go-arrow-stream-to-file-")
 			if err != nil {
 				t.Fatal(err)
 			}
diff --git a/go/arrow/ipc/file_test.go b/go/arrow/ipc/file_test.go
index 81845d84be..6b85abedd5 100644
--- a/go/arrow/ipc/file_test.go
+++ b/go/arrow/ipc/file_test.go
@@ -18,7 +18,7 @@ package ipc_test
 
 import (
 	"fmt"
-	"io/ioutil"
+	"os"
 	"testing"
 
 	"github.com/apache/arrow/go/v12/arrow/internal/arrdata"
@@ -34,7 +34,7 @@ func TestFile(t *testing.T) {
 			mem := memory.NewCheckedAllocator(memory.NewGoAllocator())
 			defer mem.AssertSize(t, 0)
 
-			f, err := ioutil.TempFile(tempDir, "go-arrow-file-")
+			f, err := os.CreateTemp(tempDir, "go-arrow-file-")
 			if err != nil {
 				t.Fatal(err)
 			}
@@ -61,7 +61,7 @@ func TestFileCompressed(t *testing.T) {
 					mem := memory.NewCheckedAllocator(memory.NewGoAllocator())
 					defer mem.AssertSize(t, 0)
 
-					f, err := ioutil.TempFile(tempDir, "go-arrow-file-")
+					f, err := os.CreateTemp(tempDir, "go-arrow-file-")
 					if err != nil {
 						t.Fatal(err)
 					}
diff --git a/go/arrow/ipc/stream_test.go b/go/arrow/ipc/stream_test.go
index d46489a595..9504a79022 100644
--- a/go/arrow/ipc/stream_test.go
+++ b/go/arrow/ipc/stream_test.go
@@ -18,7 +18,7 @@ package ipc_test
 
 import (
 	"io"
-	"io/ioutil"
+	"os"
 	"strconv"
 	"testing"
 
@@ -35,7 +35,7 @@ func TestStream(t *testing.T) {
 			mem := memory.NewCheckedAllocator(memory.NewGoAllocator())
 			defer mem.AssertSize(t, 0)
 
-			f, err := ioutil.TempFile(tempDir, "go-arrow-stream-")
+			f, err := os.CreateTemp(tempDir, "go-arrow-stream-")
 			if err != nil {
 				t.Fatal(err)
 			}
@@ -74,7 +74,7 @@ func TestStreamCompressed(t *testing.T) {
 							mem := memory.NewCheckedAllocator(memory.NewGoAllocator())
 							defer mem.AssertSize(t, 0)
 
-							f, err := ioutil.TempFile(tempDir, "go-arrow-stream-")
+							f, err := os.CreateTemp(tempDir, "go-arrow-stream-")
 							if err != nil {
 								t.Fatal(err)
 							}
diff --git a/go/go.mod b/go/go.mod
index 2b1b8d2c67..aa4efdc1dc 100644
--- a/go/go.mod
+++ b/go/go.mod
@@ -16,7 +16,7 @@
 
 module github.com/apache/arrow/go/v12
 
-go 1.18
+go 1.20
 
 require (
 	github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c
diff --git a/go/internal/hashing/hash_funcs.go b/go/internal/hashing/hash_funcs.go
new file mode 100644
index 0000000000..1a859198e9
--- /dev/null
+++ b/go/internal/hashing/hash_funcs.go
@@ -0,0 +1,90 @@
+// 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 hashing
+
+import (
+	"math/bits"
+	"unsafe"
+
+	"github.com/zeebo/xxh3"
+)
+
+func hashInt(val uint64, alg uint64) uint64 {
+	// Two of xxhash's prime multipliers (which are chosen for their
+	// bit dispersion properties)
+	var multipliers = [2]uint64{11400714785074694791, 14029467366897019727}
+	// Multiplying by the prime number mixes the low bits into the high bits,
+	// then byte-swapping (which is a single CPU instruction) allows the
+	// combined high and low bits to participate in the initial hash table index.
+	return bits.ReverseBytes64(multipliers[alg] * val)
+}
+
+func hashFloat32(val float32, alg uint64) uint64 {
+	// grab the raw byte pattern of the
+	bt := *(*[4]byte)(unsafe.Pointer(&val))
+	x := uint64(*(*uint32)(unsafe.Pointer(&bt[0])))
+	hx := hashInt(x, alg)
+	hy := hashInt(x, alg^1)
+	return 4 ^ hx ^ hy
+}
+
+func hashFloat64(val float64, alg uint64) uint64 {
+	bt := *(*[8]byte)(unsafe.Pointer(&val))
+	hx := hashInt(uint64(*(*uint32)(unsafe.Pointer(&bt[4]))), alg)
+	hy := hashInt(uint64(*(*uint32)(unsafe.Pointer(&bt[0]))), alg^1)
+	return 8 ^ hx ^ hy
+}
+
+// prime constants used for slightly increasing the hash quality further
+var exprimes = [2]uint64{1609587929392839161, 9650029242287828579}
+
+// for smaller amounts of bytes this is faster than even calling into
+// xxh3 to do the hash, so we specialize in order to get the benefits
+// of that performance.
+func hash(b []byte, alg uint64) uint64 {
+	n := uint32(len(b))
+	if n <= 16 {
+		switch {
+		case n > 8:
+			// 8 < length <= 16
+			// apply same principle as above, but as two 64-bit ints
+			x := *(*uint64)(unsafe.Pointer(&b[n-8]))
+			y := *(*uint64)(unsafe.Pointer(&b[0]))
+			hx := hashInt(x, alg)
+			hy := hashInt(y, alg^1)
+			return uint64(n) ^ hx ^ hy
+		case n >= 4:
+			// 4 < length <= 8
+			// we can read the bytes as two overlapping 32-bit ints, apply different
+			// hash functions to each in parallel
+			// then xor the results
+			x := *(*uint32)(unsafe.Pointer(&b[n-4]))
+			y := *(*uint32)(unsafe.Pointer(&b[0]))
+			hx := hashInt(uint64(x), alg)
+			hy := hashInt(uint64(y), alg^1)
+			return uint64(n) ^ hx ^ hy
+		case n > 0:
+			x := uint32((n << 24) ^ (uint32(b[0]) << 16) ^ (uint32(b[n/2]) << 8) ^ uint32(b[n-1]))
+			return hashInt(uint64(x), alg)
+		case n == 0:
+			return 1
+		}
+	}
+
+	// increase differentiation enough to improve hash quality
+	return xxh3.Hash(b) + exprimes[alg]
+}
diff --git a/go/internal/hashing/hash_string.go b/go/internal/hashing/hash_string.go
new file mode 100644
index 0000000000..6630010ba0
--- /dev/null
+++ b/go/internal/hashing/hash_string.go
@@ -0,0 +1,26 @@
+// 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.
+
+//go:build go1.20
+
+package hashing
+
+import "unsafe"
+
+func hashString(val string, alg uint64) uint64 {
+	buf := unsafe.Slice(unsafe.StringData(val), len(val))
+	return hash(buf, alg)
+}
diff --git a/go/internal/hashing/hash_string_go1.19.go b/go/internal/hashing/hash_string_go1.19.go
new file mode 100644
index 0000000000..8a79906293
--- /dev/null
+++ b/go/internal/hashing/hash_string_go1.19.go
@@ -0,0 +1,30 @@
+// 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.
+
+//go:build !go1.20
+
+package hashing
+
+import (
+	"reflect"
+	"unsafe"
+)
+
+func hashString(val string, alg uint64) uint64 {
+	buf := *(*[]byte)(unsafe.Pointer(&val))
+	(*reflect.SliceHeader)(unsafe.Pointer(&buf)).Cap = len(val)
+	return hash(buf, alg)
+}
diff --git a/go/internal/hashing/xxh3_memo_table.go b/go/internal/hashing/xxh3_memo_table.go
index fe4f37f23f..c67b56bcf4 100644
--- a/go/internal/hashing/xxh3_memo_table.go
+++ b/go/internal/hashing/xxh3_memo_table.go
@@ -22,13 +22,10 @@ package hashing
 import (
 	"bytes"
 	"math"
-	"math/bits"
 	"reflect"
 	"unsafe"
 
 	"github.com/apache/arrow/go/v12/parquet"
-
-	"github.com/zeebo/xxh3"
 )
 
 //go:generate go run ../../arrow/_tools/tmpl/main.go -i -data=types.tmpldata xxh3_memo_table.gen.go.tmpl
@@ -76,78 +73,6 @@ type NumericMemoTable interface {
 	WriteOutSubsetLE(offset int, out []byte)
 }
 
-func hashInt(val uint64, alg uint64) uint64 {
-	// Two of xxhash's prime multipliers (which are chosen for their
-	// bit dispersion properties)
-	var multipliers = [2]uint64{11400714785074694791, 14029467366897019727}
-	// Multiplying by the prime number mixes the low bits into the high bits,
-	// then byte-swapping (which is a single CPU instruction) allows the
-	// combined high and low bits to participate in the initial hash table index.
-	return bits.ReverseBytes64(multipliers[alg] * val)
-}
-
-func hashFloat32(val float32, alg uint64) uint64 {
-	// grab the raw byte pattern of the
-	bt := *(*[4]byte)(unsafe.Pointer(&val))
-	x := uint64(*(*uint32)(unsafe.Pointer(&bt[0])))
-	hx := hashInt(x, alg)
-	hy := hashInt(x, alg^1)
-	return 4 ^ hx ^ hy
-}
-
-func hashFloat64(val float64, alg uint64) uint64 {
-	bt := *(*[8]byte)(unsafe.Pointer(&val))
-	hx := hashInt(uint64(*(*uint32)(unsafe.Pointer(&bt[4]))), alg)
-	hy := hashInt(uint64(*(*uint32)(unsafe.Pointer(&bt[0]))), alg^1)
-	return 8 ^ hx ^ hy
-}
-
-func hashString(val string, alg uint64) uint64 {
-	buf := *(*[]byte)(unsafe.Pointer(&val))
-	(*reflect.SliceHeader)(unsafe.Pointer(&buf)).Cap = len(val)
-	return hash(buf, alg)
-}
-
-// prime constants used for slightly increasing the hash quality further
-var exprimes = [2]uint64{1609587929392839161, 9650029242287828579}
-
-// for smaller amounts of bytes this is faster than even calling into
-// xxh3 to do the hash, so we specialize in order to get the benefits
-// of that performance.
-func hash(b []byte, alg uint64) uint64 {
-	n := uint32(len(b))
-	if n <= 16 {
-		switch {
-		case n > 8:
-			// 8 < length <= 16
-			// apply same principle as above, but as two 64-bit ints
-			x := *(*uint64)(unsafe.Pointer(&b[n-8]))
-			y := *(*uint64)(unsafe.Pointer(&b[0]))
-			hx := hashInt(x, alg)
-			hy := hashInt(y, alg^1)
-			return uint64(n) ^ hx ^ hy
-		case n >= 4:
-			// 4 < length <= 8
-			// we can read the bytes as two overlapping 32-bit ints, apply different
-			// hash functions to each in parallel
-			// then xor the results
-			x := *(*uint32)(unsafe.Pointer(&b[n-4]))
-			y := *(*uint32)(unsafe.Pointer(&b[0]))
-			hx := hashInt(uint64(x), alg)
-			hy := hashInt(uint64(y), alg^1)
-			return uint64(n) ^ hx ^ hy
-		case n > 0:
-			x := uint32((n << 24) ^ (uint32(b[0]) << 16) ^ (uint32(b[n/2]) << 8) ^ uint32(b[n-1]))
-			return hashInt(uint64(x), alg)
-		case n == 0:
-			return 1
-		}
-	}
-
-	// increase differentiation enough to improve hash quality
-	return xxh3.Hash(b) + exprimes[alg]
-}
-
 const (
 	sentinel   uint64 = 0
 	loadFactor int64  = 2
diff --git a/go/parquet/compress/brotli.go b/go/parquet/compress/brotli.go
index a79a4f8991..864cde268e 100644
--- a/go/parquet/compress/brotli.go
+++ b/go/parquet/compress/brotli.go
@@ -19,7 +19,6 @@ package compress
 import (
 	"bytes"
 	"io"
-	"io/ioutil"
 
 	"github.com/andybalholm/brotli"
 	"github.com/apache/arrow/go/v12/parquet/internal/debug"
@@ -28,7 +27,7 @@ import (
 type brotliCodec struct{}
 
 func (brotliCodec) NewReader(r io.Reader) io.ReadCloser {
-	return ioutil.NopCloser(brotli.NewReader(r))
+	return io.NopCloser(brotli.NewReader(r))
 }
 
 func (b brotliCodec) EncodeLevel(dst, src []byte, level int) []byte {
@@ -74,7 +73,7 @@ func (brotliCodec) Decode(dst, src []byte) []byte {
 		return dst[:sofar]
 	}
 
-	dst, err := ioutil.ReadAll(rdr)
+	dst, err := io.ReadAll(rdr)
 	if err != nil {
 		panic(err)
 	}
diff --git a/go/parquet/compress/compress.go b/go/parquet/compress/compress.go
index 09282a3190..6bc0f1eb10 100644
--- a/go/parquet/compress/compress.go
+++ b/go/parquet/compress/compress.go
@@ -22,7 +22,6 @@ import (
 	"compress/flate"
 	"fmt"
 	"io"
-	"io/ioutil"
 
 	"github.com/apache/arrow/go/v12/parquet/internal/gen-go/parquet"
 )
@@ -98,7 +97,7 @@ type nocodec struct{}
 func (nocodec) NewReader(r io.Reader) io.ReadCloser {
 	ret, ok := r.(io.ReadCloser)
 	if !ok {
-		return ioutil.NopCloser(r)
+		return io.NopCloser(r)
 	}
 	return ret
 }
diff --git a/go/parquet/compress/compress_test.go b/go/parquet/compress/compress_test.go
index cf13fcb0cb..d4db87c2d0 100644
--- a/go/parquet/compress/compress_test.go
+++ b/go/parquet/compress/compress_test.go
@@ -18,7 +18,7 @@ package compress_test
 
 import (
 	"bytes"
-	"io/ioutil"
+	"io"
 	"math/rand"
 	"testing"
 
@@ -130,7 +130,7 @@ func TestCompressReaderWriter(t *testing.T) {
 			wr.Close()
 
 			rdr := codec.NewReader(&buf)
-			out, err := ioutil.ReadAll(rdr)
+			out, err := io.ReadAll(rdr)
 			assert.NoError(t, err)
 			assert.Exactly(t, data, out)
 		})
diff --git a/go/parquet/compress/gzip.go b/go/parquet/compress/gzip.go
index c751438023..31f1729e9b 100644
--- a/go/parquet/compress/gzip.go
+++ b/go/parquet/compress/gzip.go
@@ -20,7 +20,6 @@ import (
 	"bytes"
 	"fmt"
 	"io"
-	"io/ioutil"
 
 	"github.com/klauspost/compress/gzip"
 )
@@ -49,7 +48,7 @@ func (gzipCodec) Decode(dst, src []byte) []byte {
 		return dst[:n]
 	}
 
-	dst, err = ioutil.ReadAll(rdr)
+	dst, err = io.ReadAll(rdr)
 	if err != nil {
 		panic(err)
 	}
diff --git a/go/parquet/compress/snappy.go b/go/parquet/compress/snappy.go
index 6468df780a..b7fa1142c3 100644
--- a/go/parquet/compress/snappy.go
+++ b/go/parquet/compress/snappy.go
@@ -18,7 +18,6 @@ package compress
 
 import (
 	"io"
-	"io/ioutil"
 
 	"github.com/golang/snappy"
 )
@@ -42,7 +41,7 @@ func (snappyCodec) Decode(dst, src []byte) []byte {
 }
 
 func (snappyCodec) NewReader(r io.Reader) io.ReadCloser {
-	return ioutil.NopCloser(snappy.NewReader(r))
+	return io.NopCloser(snappy.NewReader(r))
 }
 
 func (snappyCodec) CompressBound(len int64) int64 {
diff --git a/go/parquet/encryption_write_config_test.go b/go/parquet/encryption_write_config_test.go
index 4eb40a3432..4cfc17b568 100644
--- a/go/parquet/encryption_write_config_test.go
+++ b/go/parquet/encryption_write_config_test.go
@@ -19,7 +19,6 @@ package parquet_test
 import (
 	"encoding/binary"
 	"fmt"
-	"io/ioutil"
 	"os"
 	"path/filepath"
 	"testing"
@@ -214,7 +213,7 @@ func (en *EncryptionConfigTestSuite) encryptFile(configs *parquet.FileEncryption
 
 func (en *EncryptionConfigTestSuite) SetupSuite() {
 	var err error
-	tempdir, err = ioutil.TempDir("", "parquet-encryption-test-*")
+	tempdir, err = os.MkdirTemp("", "parquet-encryption-test-*")
 	en.Require().NoError(err)
 	fmt.Println(tempdir)
 
diff --git a/go/parquet/file/file_reader_test.go b/go/parquet/file/file_reader_test.go
index 17402af472..8faf2ad241 100644
--- a/go/parquet/file/file_reader_test.go
+++ b/go/parquet/file/file_reader_test.go
@@ -18,9 +18,9 @@ package file_test
 
 import (
 	"bytes"
+	"crypto/rand"
 	"encoding/binary"
 	"io"
-	"math/rand"
 	"testing"
 
 	"github.com/apache/arrow/go/v12/arrow/memory"