zeroshade commented on code in PR #37788:
URL: https://github.com/apache/arrow/pull/37788#discussion_r1334772355


##########
dev/archery/archery/integration/tester_go.py:
##########
@@ -119,3 +131,124 @@ def flight_request(self, port, json_path=None, 
scenario_name=None):
         if self.debug:
             log(' '.join(cmd))
         run_cmd(cmd)
+
+    def make_c_data_exporter(self):
+        return GoCDataExporter(self.debug, self.args)
+
+    def make_c_data_importer(self):
+        return GoCDataImporter(self.debug, self.args)
+
+
+_go_c_data_entrypoints = """
+    const char* ArrowGo_ExportSchemaFromJson(const char* json_path,
+                                             uintptr_t out);
+    const char* ArrowGo_ImportSchemaAndCompareToJson(
+        const char* json_path, uintptr_t c_schema);
+
+    const char* ArrowGo_ExportBatchFromJson(const char* json_path,
+                                            int num_batch,
+                                            uintptr_t out);
+    const char* ArrowGo_ImportBatchAndCompareToJson(
+        const char* json_path, int num_batch, uintptr_t c_array);
+
+    int64_t ArrowGo_BytesAllocated();
+    void ArrowGo_RunGC();
+    void ArrowGo_FreeError(const char*);
+    """
+
+
[email protected]_cache
+def _load_ffi(ffi, lib_path=_INTEGRATION_DLL):
+    ffi.cdef(_go_c_data_entrypoints)
+    dll = ffi.dlopen(lib_path)
+    dll.ArrowGo_ExportSchemaFromJson
+    return dll
+
+
+class _CDataBase:
+
+    def __init__(self, debug, args):
+        self.debug = debug
+        self.args = args
+        self.ffi = cdata.ffi()
+        self.dll = _load_ffi(self.ffi)
+
+    def _pointer_to_int(self, c_ptr):
+        return self.ffi.cast('uintptr_t', c_ptr)
+
+    def _check_go_error(self, go_error):
+        """
+        Check a `const char*` error return from an integration entrypoint.
+
+        A null means success, a non-empty string is an error message.
+        The string is dynamically allocated on the Go side.
+        """
+        assert self.ffi.typeof(go_error) is self.ffi.typeof("const char*")
+        if go_error != self.ffi.NULL:
+            try:
+                error = self.ffi.string(go_error).decode('utf8',
+                                                         errors='replace')
+                raise RuntimeError(
+                    f"Go C Data Integration call failed: {error}")
+            finally:
+                self.dll.ArrowGo_FreeError(go_error)
+
+    def _run_gc(self):
+        self.dll.ArrowGo_RunGC()
+
+
+class GoCDataExporter(CDataExporter, _CDataBase):
+    # Note: the Arrow Go C Data export functions expect their output
+    # ArrowStream or ArrowArray argument to be zero-initialized.
+    # This is currently ensured through the use of `ffi.new`.

Review Comment:
   huh, i thought `trampoline.c` zero-initialized everything but you're right, 
we're only trampolining to zero-initialize for `streamGetSchema` and 
`streamGetNext`, darn. 



##########
go/arrow/cdata/cdata_exports.go:
##########
@@ -368,34 +368,37 @@ func exportArray(arr arrow.Array, out *CArrowArray, 
outSchema *CArrowSchema) {
                exportField(arrow.Field{Type: arr.DataType()}, outSchema)
        }
 
+       nbuffers := len(arr.Data().Buffers())
+       buf_offset := 0
+       // Some types don't have validity bitmaps, but we keep them shifted
+       // to make processing easier in other contexts. This means that
+       // we have to adjust when exporting.
+       has_validity_bitmap := 
internal.DefaultHasValidityBitmap(arr.DataType().ID())
+       if nbuffers > 0 && !has_validity_bitmap {
+               nbuffers--
+               buf_offset++
+       }
+
        out.dictionary = nil
        out.null_count = C.int64_t(arr.NullN())
        out.length = C.int64_t(arr.Len())
        out.offset = C.int64_t(arr.Data().Offset())
-       out.n_buffers = C.int64_t(len(arr.Data().Buffers()))
-
-       if out.n_buffers > 0 {
-               var (
-                       nbuffers = len(arr.Data().Buffers())
-                       bufs     = arr.Data().Buffers()
-               )
-               // unions don't have validity bitmaps, but we keep them shifted
-               // to make processing easier in other contexts. This means that
-               // we have to adjust for union arrays
-               if !internal.DefaultHasValidityBitmap(arr.DataType().ID()) {
-                       out.n_buffers--
-                       nbuffers--
-                       bufs = bufs[1:]
-               }
+       out.n_buffers = C.int64_t(nbuffers)
+       out.buffers = nil
+
+       if nbuffers > 0 {
+               bufs := arr.Data().Buffers()
                buffers := allocateBufferPtrArr(nbuffers)
-               for i := range bufs {
-                       buf := bufs[i]
+               for i := 0; i < nbuffers; i++ {

Review Comment:
   You could do:
   
   ```go
   for i, buf := range buf[buf_offset:] {
   ```
   
   `i` should still go from 0 -> `nbuffers` since `nbuffers` should be equal to 
`len(buf[buf_offset:])`
   
   Alternately if we want to make sure we're being more explicit you could do:
   
   ```go
   for i, buf := range bufs[buf_offset:buf_offset+nbuffers] {
   ```
   But i don't think it's necessary.



##########
go/arrow/internal/cdata_integration/entrypoints.go:
##########
@@ -0,0 +1,193 @@
+// 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 cdata_integration
+// +build cdata_integration
+
+package main
+
+import (
+       "C"
+       "fmt"
+       "os"
+       "runtime"
+       "unsafe"
+
+       "github.com/apache/arrow/go/v14/arrow/array"
+       "github.com/apache/arrow/go/v14/arrow/cdata"
+       "github.com/apache/arrow/go/v14/arrow/internal/arrjson"
+       "github.com/apache/arrow/go/v14/arrow/memory"
+)
+
+// #include <stdint.h>
+// #include <stdlib.h>
+import "C"
+
+var alloc = memory.NewCheckedAllocator(memory.NewGoAllocator())
+
+//export ArrowGo_BytesAllocated
+func ArrowGo_BytesAllocated() int64 {
+       return int64(alloc.CurrentAlloc())
+}
+
+//export ArrowGo_RunGC
+func ArrowGo_RunGC() {
+       runtime.GC()
+}
+
+//export ArrowGo_FreeError
+func ArrowGo_FreeError(cError *C.char) {
+       C.free(unsafe.Pointer(cError))
+}
+
+// When used in a defer() statement, this functions catches an incoming
+// panic and converts it into a regular error. This avoids crashing the
+// archery integration process and lets other tests proceed.
+// Not all panics may be caught and some will still crash the process, though.
+func handlePanic(err *error) {
+       if e := recover(); e != nil {
+               // Add a prefix while wrapping the panic-error
+               *err = fmt.Errorf("panic: %w", e.(error))
+       }
+}
+
+func newJsonReader(cJsonPath *C.char) (*arrjson.Reader, error) {
+       jsonPath := C.GoString(cJsonPath)
+
+       f, err := os.Open(jsonPath)
+       if err != nil {
+               return nil, fmt.Errorf("could not open JSON file %q: %w", 
jsonPath, err)
+       }
+       defer f.Close()
+
+       jsonReader, err := arrjson.NewReader(f, arrjson.WithAllocator(alloc))
+       if err != nil {
+               return nil, fmt.Errorf("could not open JSON file reader from 
file %q: %w", jsonPath, err)
+       }
+       return jsonReader, nil
+}
+
+func exportSchemaFromJson(cJsonPath *C.char, out *cdata.CArrowSchema) (err 
error) {
+       jsonReader, err := newJsonReader(cJsonPath)
+       if err != nil {
+               return err
+       }
+       defer jsonReader.Release()
+       schema := jsonReader.Schema()
+       defer handlePanic(&err)
+       cdata.ExportArrowSchema(schema, out)
+       return err
+}
+
+func importSchemaAndCompareToJson(cJsonPath *C.char, cSchema 
*cdata.CArrowSchema) (err error) {
+       jsonReader, err := newJsonReader(cJsonPath)
+       if err != nil {
+               return err
+       }
+       defer jsonReader.Release()
+       schema := jsonReader.Schema()
+       importedSchema, err := cdata.ImportCArrowSchema(cSchema)
+       if err != nil {
+               return err
+       }
+       if !schema.Equal(importedSchema) || 
!schema.Metadata().Equal(importedSchema.Metadata()) {
+               return fmt.Errorf(
+                       "Schemas are different:\n- Json Schema: %s\n- Imported 
Schema: %s",
+                       schema.String(),
+                       importedSchema.String())
+       }
+       return err

Review Comment:
   I'd probably  be explicit and just return `nil` here rather than `return 
err` since it *should* be nil (otherwise we would have returned it at line 104)



##########
go/arrow/internal/cdata_integration/entrypoints.go:
##########
@@ -0,0 +1,193 @@
+// 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 cdata_integration
+// +build cdata_integration
+
+package main
+
+import (
+       "C"
+       "fmt"
+       "os"
+       "runtime"
+       "unsafe"
+
+       "github.com/apache/arrow/go/v14/arrow/array"
+       "github.com/apache/arrow/go/v14/arrow/cdata"
+       "github.com/apache/arrow/go/v14/arrow/internal/arrjson"
+       "github.com/apache/arrow/go/v14/arrow/memory"
+)
+
+// #include <stdint.h>
+// #include <stdlib.h>
+import "C"
+
+var alloc = memory.NewCheckedAllocator(memory.NewGoAllocator())
+
+//export ArrowGo_BytesAllocated
+func ArrowGo_BytesAllocated() int64 {
+       return int64(alloc.CurrentAlloc())
+}
+
+//export ArrowGo_RunGC
+func ArrowGo_RunGC() {
+       runtime.GC()
+}
+
+//export ArrowGo_FreeError
+func ArrowGo_FreeError(cError *C.char) {
+       C.free(unsafe.Pointer(cError))
+}
+
+// When used in a defer() statement, this functions catches an incoming
+// panic and converts it into a regular error. This avoids crashing the
+// archery integration process and lets other tests proceed.
+// Not all panics may be caught and some will still crash the process, though.
+func handlePanic(err *error) {
+       if e := recover(); e != nil {
+               // Add a prefix while wrapping the panic-error
+               *err = fmt.Errorf("panic: %w", e.(error))
+       }
+}
+
+func newJsonReader(cJsonPath *C.char) (*arrjson.Reader, error) {
+       jsonPath := C.GoString(cJsonPath)
+
+       f, err := os.Open(jsonPath)
+       if err != nil {
+               return nil, fmt.Errorf("could not open JSON file %q: %w", 
jsonPath, err)
+       }
+       defer f.Close()
+
+       jsonReader, err := arrjson.NewReader(f, arrjson.WithAllocator(alloc))
+       if err != nil {
+               return nil, fmt.Errorf("could not open JSON file reader from 
file %q: %w", jsonPath, err)
+       }
+       return jsonReader, nil
+}
+
+func exportSchemaFromJson(cJsonPath *C.char, out *cdata.CArrowSchema) (err 
error) {

Review Comment:
   since you're immediately creating a new `err` in the scope of the function 
via the `:=` on the first line, i wouldn't use a named return parameter here as 
it could cause confusion in the future. (i.e. someone modifying this might 
change the last line to be just `return` thinking it would be equivalent when 
it is *technically* not.
   
   I'd just have this function signature be `func 
exportSchemaFromJson(cJsonPath *C.char, out *cdata.CArrowSchema) error {`



##########
go/arrow/internal/cdata_integration/entrypoints.go:
##########
@@ -0,0 +1,193 @@
+// 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 cdata_integration
+// +build cdata_integration
+
+package main
+
+import (
+       "C"
+       "fmt"
+       "os"
+       "runtime"
+       "unsafe"
+
+       "github.com/apache/arrow/go/v14/arrow/array"
+       "github.com/apache/arrow/go/v14/arrow/cdata"
+       "github.com/apache/arrow/go/v14/arrow/internal/arrjson"
+       "github.com/apache/arrow/go/v14/arrow/memory"
+)
+
+// #include <stdint.h>
+// #include <stdlib.h>
+import "C"
+
+var alloc = memory.NewCheckedAllocator(memory.NewGoAllocator())
+
+//export ArrowGo_BytesAllocated
+func ArrowGo_BytesAllocated() int64 {
+       return int64(alloc.CurrentAlloc())
+}
+
+//export ArrowGo_RunGC
+func ArrowGo_RunGC() {
+       runtime.GC()
+}
+
+//export ArrowGo_FreeError
+func ArrowGo_FreeError(cError *C.char) {
+       C.free(unsafe.Pointer(cError))
+}
+
+// When used in a defer() statement, this functions catches an incoming
+// panic and converts it into a regular error. This avoids crashing the
+// archery integration process and lets other tests proceed.
+// Not all panics may be caught and some will still crash the process, though.
+func handlePanic(err *error) {
+       if e := recover(); e != nil {
+               // Add a prefix while wrapping the panic-error
+               *err = fmt.Errorf("panic: %w", e.(error))
+       }
+}
+
+func newJsonReader(cJsonPath *C.char) (*arrjson.Reader, error) {
+       jsonPath := C.GoString(cJsonPath)
+
+       f, err := os.Open(jsonPath)
+       if err != nil {
+               return nil, fmt.Errorf("could not open JSON file %q: %w", 
jsonPath, err)
+       }
+       defer f.Close()
+
+       jsonReader, err := arrjson.NewReader(f, arrjson.WithAllocator(alloc))
+       if err != nil {
+               return nil, fmt.Errorf("could not open JSON file reader from 
file %q: %w", jsonPath, err)
+       }
+       return jsonReader, nil
+}
+
+func exportSchemaFromJson(cJsonPath *C.char, out *cdata.CArrowSchema) (err 
error) {
+       jsonReader, err := newJsonReader(cJsonPath)
+       if err != nil {
+               return err
+       }
+       defer jsonReader.Release()
+       schema := jsonReader.Schema()
+       defer handlePanic(&err)
+       cdata.ExportArrowSchema(schema, out)
+       return err
+}
+
+func importSchemaAndCompareToJson(cJsonPath *C.char, cSchema 
*cdata.CArrowSchema) (err error) {
+       jsonReader, err := newJsonReader(cJsonPath)
+       if err != nil {
+               return err
+       }
+       defer jsonReader.Release()
+       schema := jsonReader.Schema()
+       importedSchema, err := cdata.ImportCArrowSchema(cSchema)
+       if err != nil {
+               return err
+       }
+       if !schema.Equal(importedSchema) || 
!schema.Metadata().Equal(importedSchema.Metadata()) {
+               return fmt.Errorf(
+                       "Schemas are different:\n- Json Schema: %s\n- Imported 
Schema: %s",
+                       schema.String(),
+                       importedSchema.String())
+       }
+       return err
+}
+
+func exportBatchFromJson(cJsonPath *C.char, num_batch int, out 
*cdata.CArrowArray) (err error) {
+       // XXX this function exports a single batch at a time, but the JSON 
reader
+       // reads all batches at construction.
+       jsonReader, err := newJsonReader(cJsonPath)
+       if err != nil {
+               return err
+       }
+       defer jsonReader.Release()
+       batch, err := jsonReader.ReadAt(num_batch)
+       if err != nil {
+               return err
+       }
+       defer handlePanic(&err)
+       cdata.ExportArrowRecordBatch(batch, out, nil)
+       return err
+}
+
+func importBatchAndCompareToJson(cJsonPath *C.char, num_batch int, cArray 
*cdata.CArrowArray) (err error) {
+       jsonReader, err := newJsonReader(cJsonPath)

Review Comment:
   same comment on the function signature as above



##########
dev/archery/archery/integration/tester_go.py:
##########
@@ -119,3 +131,124 @@ def flight_request(self, port, json_path=None, 
scenario_name=None):
         if self.debug:
             log(' '.join(cmd))
         run_cmd(cmd)
+
+    def make_c_data_exporter(self):
+        return GoCDataExporter(self.debug, self.args)
+
+    def make_c_data_importer(self):
+        return GoCDataImporter(self.debug, self.args)
+
+
+_go_c_data_entrypoints = """
+    const char* ArrowGo_ExportSchemaFromJson(const char* json_path,
+                                             uintptr_t out);
+    const char* ArrowGo_ImportSchemaAndCompareToJson(
+        const char* json_path, uintptr_t c_schema);
+
+    const char* ArrowGo_ExportBatchFromJson(const char* json_path,
+                                            int num_batch,
+                                            uintptr_t out);
+    const char* ArrowGo_ImportBatchAndCompareToJson(
+        const char* json_path, int num_batch, uintptr_t c_array);
+
+    int64_t ArrowGo_BytesAllocated();
+    void ArrowGo_RunGC();
+    void ArrowGo_FreeError(const char*);
+    """
+
+
[email protected]_cache
+def _load_ffi(ffi, lib_path=_INTEGRATION_DLL):
+    ffi.cdef(_go_c_data_entrypoints)
+    dll = ffi.dlopen(lib_path)
+    dll.ArrowGo_ExportSchemaFromJson

Review Comment:
   what's the purpose of this line? isn't `dll.ArrowGo_ExportSchemaFromJson` 
just a function pointer? or am i missing something? are we just validating it 
loaded?



##########
go/arrow/cdata/cdata.go:
##########
@@ -197,7 +197,7 @@ func importSchema(schema *CArrowSchema) (ret arrow.Field, 
err error) {
 
        // handle types with params via colon
        typs := strings.Split(f, ":")
-       defaulttz := "UTC"
+       defaulttz := ""

Review Comment:
   was this incorrect by the spec? I thought I pulled this from the c bridge 
which had the same default? (I could be misremembering though)



##########
go/arrow/internal/cdata_integration/entrypoints.go:
##########
@@ -0,0 +1,193 @@
+// 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 cdata_integration
+// +build cdata_integration
+
+package main
+
+import (
+       "C"
+       "fmt"
+       "os"
+       "runtime"
+       "unsafe"
+
+       "github.com/apache/arrow/go/v14/arrow/array"
+       "github.com/apache/arrow/go/v14/arrow/cdata"
+       "github.com/apache/arrow/go/v14/arrow/internal/arrjson"
+       "github.com/apache/arrow/go/v14/arrow/memory"
+)
+
+// #include <stdint.h>
+// #include <stdlib.h>
+import "C"
+
+var alloc = memory.NewCheckedAllocator(memory.NewGoAllocator())
+
+//export ArrowGo_BytesAllocated
+func ArrowGo_BytesAllocated() int64 {
+       return int64(alloc.CurrentAlloc())
+}
+
+//export ArrowGo_RunGC
+func ArrowGo_RunGC() {
+       runtime.GC()
+}
+
+//export ArrowGo_FreeError
+func ArrowGo_FreeError(cError *C.char) {
+       C.free(unsafe.Pointer(cError))
+}
+
+// When used in a defer() statement, this functions catches an incoming
+// panic and converts it into a regular error. This avoids crashing the
+// archery integration process and lets other tests proceed.
+// Not all panics may be caught and some will still crash the process, though.
+func handlePanic(err *error) {
+       if e := recover(); e != nil {
+               // Add a prefix while wrapping the panic-error
+               *err = fmt.Errorf("panic: %w", e.(error))
+       }
+}
+
+func newJsonReader(cJsonPath *C.char) (*arrjson.Reader, error) {
+       jsonPath := C.GoString(cJsonPath)
+
+       f, err := os.Open(jsonPath)
+       if err != nil {
+               return nil, fmt.Errorf("could not open JSON file %q: %w", 
jsonPath, err)
+       }
+       defer f.Close()
+
+       jsonReader, err := arrjson.NewReader(f, arrjson.WithAllocator(alloc))
+       if err != nil {
+               return nil, fmt.Errorf("could not open JSON file reader from 
file %q: %w", jsonPath, err)
+       }
+       return jsonReader, nil
+}
+
+func exportSchemaFromJson(cJsonPath *C.char, out *cdata.CArrowSchema) (err 
error) {
+       jsonReader, err := newJsonReader(cJsonPath)
+       if err != nil {
+               return err
+       }
+       defer jsonReader.Release()
+       schema := jsonReader.Schema()
+       defer handlePanic(&err)
+       cdata.ExportArrowSchema(schema, out)
+       return err
+}
+
+func importSchemaAndCompareToJson(cJsonPath *C.char, cSchema 
*cdata.CArrowSchema) (err error) {
+       jsonReader, err := newJsonReader(cJsonPath)
+       if err != nil {
+               return err
+       }
+       defer jsonReader.Release()
+       schema := jsonReader.Schema()
+       importedSchema, err := cdata.ImportCArrowSchema(cSchema)
+       if err != nil {
+               return err
+       }
+       if !schema.Equal(importedSchema) || 
!schema.Metadata().Equal(importedSchema.Metadata()) {
+               return fmt.Errorf(
+                       "Schemas are different:\n- Json Schema: %s\n- Imported 
Schema: %s",
+                       schema.String(),
+                       importedSchema.String())
+       }
+       return err
+}
+
+func exportBatchFromJson(cJsonPath *C.char, num_batch int, out 
*cdata.CArrowArray) (err error) {
+       // XXX this function exports a single batch at a time, but the JSON 
reader
+       // reads all batches at construction.
+       jsonReader, err := newJsonReader(cJsonPath)

Review Comment:
   same comment as above regarding the function signature. the defer 
`handlePanic` only works because you're passing the address of the locally 
scoped `err` variable that you shadowed the return var with.



##########
go/arrow/internal/cdata_integration/entrypoints.go:
##########
@@ -0,0 +1,193 @@
+// 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 cdata_integration
+// +build cdata_integration
+
+package main
+
+import (
+       "C"

Review Comment:
   don't need this since you already have the `import "C"` below



##########
ci/scripts/go_build.sh:
##########
@@ -41,3 +41,22 @@ pushd ${source_dir}/parquet
 go install -v ./...
 
 popd
+
+if [[ -n "${ARROW_GO_INTEGRATION}" ]]; then
+    pushd ${source_dir}/arrow/internal/cdata_integration
+
+    case "$(uname)" in
+        Linux)
+            go_lib="arrow_go_integration.so"
+            ;;
+        Darwin)
+            go_lib="arrow_go_integration.so"

Review Comment:
   we don't use `.dylib` for mac?



##########
go/arrow/internal/cdata_integration/entrypoints.go:
##########
@@ -0,0 +1,193 @@
+// 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 cdata_integration
+// +build cdata_integration
+
+package main
+
+import (
+       "C"
+       "fmt"
+       "os"
+       "runtime"
+       "unsafe"
+
+       "github.com/apache/arrow/go/v14/arrow/array"
+       "github.com/apache/arrow/go/v14/arrow/cdata"
+       "github.com/apache/arrow/go/v14/arrow/internal/arrjson"
+       "github.com/apache/arrow/go/v14/arrow/memory"
+)
+
+// #include <stdint.h>
+// #include <stdlib.h>
+import "C"
+
+var alloc = memory.NewCheckedAllocator(memory.NewGoAllocator())
+
+//export ArrowGo_BytesAllocated
+func ArrowGo_BytesAllocated() int64 {
+       return int64(alloc.CurrentAlloc())
+}
+
+//export ArrowGo_RunGC
+func ArrowGo_RunGC() {
+       runtime.GC()
+}
+
+//export ArrowGo_FreeError
+func ArrowGo_FreeError(cError *C.char) {
+       C.free(unsafe.Pointer(cError))
+}
+
+// When used in a defer() statement, this functions catches an incoming
+// panic and converts it into a regular error. This avoids crashing the
+// archery integration process and lets other tests proceed.
+// Not all panics may be caught and some will still crash the process, though.
+func handlePanic(err *error) {
+       if e := recover(); e != nil {
+               // Add a prefix while wrapping the panic-error
+               *err = fmt.Errorf("panic: %w", e.(error))
+       }
+}
+
+func newJsonReader(cJsonPath *C.char) (*arrjson.Reader, error) {
+       jsonPath := C.GoString(cJsonPath)
+
+       f, err := os.Open(jsonPath)
+       if err != nil {
+               return nil, fmt.Errorf("could not open JSON file %q: %w", 
jsonPath, err)
+       }
+       defer f.Close()
+
+       jsonReader, err := arrjson.NewReader(f, arrjson.WithAllocator(alloc))
+       if err != nil {
+               return nil, fmt.Errorf("could not open JSON file reader from 
file %q: %w", jsonPath, err)
+       }
+       return jsonReader, nil
+}
+
+func exportSchemaFromJson(cJsonPath *C.char, out *cdata.CArrowSchema) (err 
error) {
+       jsonReader, err := newJsonReader(cJsonPath)
+       if err != nil {
+               return err
+       }
+       defer jsonReader.Release()
+       schema := jsonReader.Schema()
+       defer handlePanic(&err)
+       cdata.ExportArrowSchema(schema, out)
+       return err
+}
+
+func importSchemaAndCompareToJson(cJsonPath *C.char, cSchema 
*cdata.CArrowSchema) (err error) {
+       jsonReader, err := newJsonReader(cJsonPath)

Review Comment:
   same comment as above



##########
go/arrow/internal/cdata_integration/entrypoints.go:
##########
@@ -0,0 +1,193 @@
+// 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 cdata_integration
+// +build cdata_integration
+
+package main
+
+import (
+       "C"
+       "fmt"
+       "os"
+       "runtime"
+       "unsafe"
+
+       "github.com/apache/arrow/go/v14/arrow/array"
+       "github.com/apache/arrow/go/v14/arrow/cdata"
+       "github.com/apache/arrow/go/v14/arrow/internal/arrjson"
+       "github.com/apache/arrow/go/v14/arrow/memory"
+)
+
+// #include <stdint.h>
+// #include <stdlib.h>
+import "C"
+
+var alloc = memory.NewCheckedAllocator(memory.NewGoAllocator())
+
+//export ArrowGo_BytesAllocated
+func ArrowGo_BytesAllocated() int64 {
+       return int64(alloc.CurrentAlloc())
+}
+
+//export ArrowGo_RunGC
+func ArrowGo_RunGC() {
+       runtime.GC()
+}
+
+//export ArrowGo_FreeError
+func ArrowGo_FreeError(cError *C.char) {
+       C.free(unsafe.Pointer(cError))
+}
+
+// When used in a defer() statement, this functions catches an incoming
+// panic and converts it into a regular error. This avoids crashing the
+// archery integration process and lets other tests proceed.
+// Not all panics may be caught and some will still crash the process, though.
+func handlePanic(err *error) {
+    if e := recover(); e != nil {
+        *err = e.(error)
+    }
+}
+
+func newJsonReader(cJsonPath *C.char) (*arrjson.Reader, error) {
+       jsonPath := C.GoString(cJsonPath)

Review Comment:
   I'll see if there's an option i'm missing in the linter workflow that would 
enforce it



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to