You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "chriscasola (via GitHub)" <gi...@apache.org> on 2023/05/23 12:48:15 UTC

[GitHub] [arrow] chriscasola opened a new pull request, #35723: GH-32832: [Go] support building with tinygo

chriscasola opened a new pull request, #35723:
URL: https://github.com/apache/arrow/pull/35723

   <!--
   Thanks for opening a pull request!
   If this is your first pull request you can find detailed information on how 
   to contribute here:
     * [New Contributor's Guide](https://arrow.apache.org/docs/dev/developers/guide/step_by_step/pr_lifecycle.html#reviews-and-merge-of-the-pull-request)
     * [Contributing Overview](https://arrow.apache.org/docs/dev/developers/overview.html)
   
   
   If this is not a [minor PR](https://github.com/apache/arrow/blob/main/CONTRIBUTING.md#Minor-Fixes). Could you open an issue for this pull request on GitHub? https://github.com/apache/arrow/issues/new/choose
   
   Opening GitHub issues ahead of time contributes to the [Openness](http://theapacheway.com/open/#:~:text=Openness%20allows%20new%20users%20the,must%20happen%20in%20the%20open.) of the Apache Arrow project.
   
   Then could you also rename the pull request title in the following format?
   
       GH-${GITHUB_ISSUE_ID}: [${COMPONENT}] ${SUMMARY}
   
   or
   
       MINOR: [${COMPONENT}] ${SUMMARY}
   
   In the case of PARQUET issues on JIRA the title also supports:
   
       PARQUET-${JIRA_ISSUE_ID}: [${COMPONENT}] ${SUMMARY}
   
   -->
   
   ### Rationale for this change
   
   To support compiling with tinygo which enables use of arrow in environments where binary size is important, like web assembly.
   <!--
    Why are you proposing this change? If this is already explained clearly in the issue then this section is not needed.
    Explaining clearly why changes are proposed helps reviewers understand your changes and offer better suggestions for fixes.  
   -->
   
   ### What changes are included in this PR?
   
   Using an internal JSON package that uses `goccy/go-json` for regular builds as it does currently, but uses the native `encoding/json` for tinygo builds. This is necessary because go-json has a lot of code that is incompatible with tinygo.
   
   Remove dependency on `parquet` package from non-parquet code since it is also incompatible with tinygo.
   
   Other minor tweaks for compatibility with tinygo.
   <!--
   There is no need to duplicate the description in the issue here but it is sometimes worth providing a summary of the individual changes in this PR.
   -->
   
   ### Are these changes tested?
   
   <!--
   We typically require tests for all PRs in order to:
   1. Prevent the code from being accidentally broken by subsequent changes
   2. Serve as another way to document the expected behavior of the code
   
   If tests are not included in your PR, please explain why (for example, are they covered by existing tests)?
   -->
   Should we add a build step that compiles the example with tinygo?
   
   ### Are there any user-facing changes?
   
   None.
   <!--
   If there are user-facing changes then we may require documentation to be updated before approving the PR.
   -->
   
   <!--
   If there are any breaking changes to public APIs, please uncomment the line below and explain which changes are breaking.
   -->
   <!-- **This PR includes breaking changes to public APIs.** -->
   
   <!--
   Please uncomment the line below (and provide explanation) if the changes fix either (a) a security vulnerability, (b) a bug that caused incorrect or invalid data to be produced, or (c) a bug that causes a crash (even when the API contract is upheld). We use this to highlight fixes to issues that may affect users without their knowledge. For this reason, fixing bugs that cause errors don't count, since those are usually obvious.
   -->
   <!-- **This PR contains a "Critical Fix".** -->


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] chriscasola commented on pull request #35723: GH-32832: [Go] support building with tinygo

Posted by "chriscasola (via GitHub)" <gi...@apache.org>.
chriscasola commented on PR #35723:
URL: https://github.com/apache/arrow/pull/35723#issuecomment-1568943243

   @zeroshade thanks for the review, I think I addressed everything.
   
   What are you thoughts on building with tinygo in CI? The CI setup on this repo seems complex so I'm not sure what that would entail, but it would be nice to avoid new code being introduced that does not build.


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] zeroshade commented on a diff in pull request #35723: GH-32832: [Go] support building with tinygo

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #35723:
URL: https://github.com/apache/arrow/pull/35723#discussion_r1219972602


##########
ci/scripts/go_tinygo_example.sh:
##########
@@ -0,0 +1,27 @@
+#!/usr/bin/env bash
+#
+# 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.
+
+set -ex
+
+cd ~
+pushd /src
+tinygo build -tags noasm -o ~/example_tinygo arrow/_examples/helloworld/main.go

Review Comment:
   should we update all of the go build constraints for `noasm` to be `noasm || tinygo` or whatever the equivalent is?



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] zeroshade commented on a diff in pull request #35723: GH-32832: [Go] support building with tinygo

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #35723:
URL: https://github.com/apache/arrow/pull/35723#discussion_r1226941598


##########
ci/scripts/go_tinygo_example.sh:
##########
@@ -0,0 +1,27 @@
+#!/usr/bin/env bash
+#
+# 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.
+
+set -ex
+
+cd ~
+pushd /src
+tinygo build -tags noasm -o ~/example_tinygo arrow/_examples/helloworld/main.go

Review Comment:
   yea that's fine for now. We should probably figure out which package it is that has the `noasm` condition etc.
   
   



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] chriscasola commented on a diff in pull request #35723: GH-32832: [Go] support building with tinygo

Posted by "chriscasola (via GitHub)" <gi...@apache.org>.
chriscasola commented on code in PR #35723:
URL: https://github.com/apache/arrow/pull/35723#discussion_r1210672435


##########
go/arrow/array/string.go:
##########
@@ -79,16 +79,12 @@ func (a *String) ValueOffsets() []int32 {
 	return a.offsets[beg:end]
 }
 
-func (a *String) ValueBytes() (ret []byte) {
+func (a *String) ValueBytes() []byte {
 	beg := a.array.data.offset
 	end := beg + a.array.data.length
 	data := a.values[a.offsets[beg]:a.offsets[end]]
 
-	s := (*reflect.SliceHeader)(unsafe.Pointer(&ret))
-	s.Data = (*reflect.StringHeader)(unsafe.Pointer(&data)).Data
-	s.Len = len(data)
-	s.Cap = len(data)
-	return
+	return []byte(data)

Review Comment:
   Yes we changed this because of the `SliceHeader` issue.
   
   When would this cause a copy? Aren't we slicing the string which is referencing the same underlying data?



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] chriscasola commented on a diff in pull request #35723: GH-32832: [Go] support building with tinygo

Posted by "chriscasola (via GitHub)" <gi...@apache.org>.
chriscasola commented on code in PR #35723:
URL: https://github.com/apache/arrow/pull/35723#discussion_r1210677046


##########
go/arrow/compute/arithmetic_test.go:
##########
@@ -2621,7 +2621,7 @@ func (us *UnaryArithmeticSigned[T]) TestAbsoluteValue() {
 				fn(`[]`, `[]`)
 				// scalar/arrays with nulls
 				fn(`[null]`, `[null]`)
-				fn(`[1, null -10]`, `[1, null, 10]`)
+				fn(`[1, null, -10]`, `[1, null, 10]`)

Review Comment:
   The goccy JSON parser seems to be more forgiving than the built-in one. So this fails if you run the tests using the native JSON package.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] github-actions[bot] commented on pull request #35723: GH-32832: [Go] support building with tinygo

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #35723:
URL: https://github.com/apache/arrow/pull/35723#issuecomment-1559249101

   * Closes: #32832


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] chriscasola commented on a diff in pull request #35723: GH-32832: [Go] support building with tinygo

Posted by "chriscasola (via GitHub)" <gi...@apache.org>.
chriscasola commented on code in PR #35723:
URL: https://github.com/apache/arrow/pull/35723#discussion_r1210677046


##########
go/arrow/compute/arithmetic_test.go:
##########
@@ -2621,7 +2621,7 @@ func (us *UnaryArithmeticSigned[T]) TestAbsoluteValue() {
 				fn(`[]`, `[]`)
 				// scalar/arrays with nulls
 				fn(`[null]`, `[null]`)
-				fn(`[1, null -10]`, `[1, null, 10]`)
+				fn(`[1, null, -10]`, `[1, null, 10]`)

Review Comment:
   The goccy JSON parser seems to be more forgiving than the built-in one.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] zeroshade commented on a diff in pull request #35723: GH-32832: [Go] support building with tinygo

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #35723:
URL: https://github.com/apache/arrow/pull/35723#discussion_r1213378708


##########
go/arrow/array/string.go:
##########
@@ -221,16 +217,12 @@ func (a *LargeString) ValueOffsets() []int64 {
 	return a.offsets[beg:end]
 }
 
-func (a *LargeString) ValueBytes() (ret []byte) {
+func (a *LargeString) ValueBytes() []byte {
 	beg := a.array.data.offset
 	end := beg + a.array.data.length
 	data := a.values[a.offsets[beg]:a.offsets[end]]
 
-	s := (*reflect.SliceHeader)(unsafe.Pointer(&ret))
-	s.Data = (*reflect.StringHeader)(unsafe.Pointer(&data)).Data
-	s.Len = len(data)
-	s.Cap = len(data)
-	return
+	return []byte(data)

Review Comment:
   This still needs to get updated to no longer be copying the string (like the equivalent function for `(*String)`



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] zeroshade commented on a diff in pull request #35723: GH-32832: [Go] support building with tinygo

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #35723:
URL: https://github.com/apache/arrow/pull/35723#discussion_r1219970876


##########
.github/workflows/go.yml:
##########
@@ -394,3 +394,20 @@ jobs:
       - name: Test
         shell: bash
         run: ci/scripts/go_test.sh $(pwd)
+
+  tinygo:
+    name: TinyGo
+    runs-on: ubuntu-latest
+    if: ${{ !contains(github.event.pull_request.title, 'WIP') }}
+    env:
+      TINYGO_VERSION: 0.27.0

Review Comment:
   i would suggest adding an image to the docker-compose.yml and a dockerfile to `ci/docker/` which is an image with tinygo installed and then use `archery docker run` like other examples.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] zeroshade commented on a diff in pull request #35723: GH-32832: [Go] support building with tinygo

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #35723:
URL: https://github.com/apache/arrow/pull/35723#discussion_r1205597091


##########
dev/release/rat_exclude_files.txt:
##########
@@ -82,6 +82,8 @@ go/parquet/internal/gen-go/parquet/GoUnusedProtection__.go
 go/parquet/internal/gen-go/parquet/parquet-consts.go
 go/parquet/internal/gen-go/parquet/parquet.go
 go/parquet/version_string.go
+go/internal/json/json.go
+go/internal/json/json_tinygo.go

Review Comment:
   instead of excluding these,  can you please add the Apache license heading to them? Typically this exclusion is reserved for generated files.



##########
go/arrow/array/string.go:
##########
@@ -221,16 +217,12 @@ func (a *LargeString) ValueOffsets() []int64 {
 	return a.offsets[beg:end]
 }
 
-func (a *LargeString) ValueBytes() (ret []byte) {
+func (a *LargeString) ValueBytes() []byte {
 	beg := a.array.data.offset
 	end := beg + a.array.data.length
 	data := a.values[a.offsets[beg]:a.offsets[end]]
 
-	s := (*reflect.SliceHeader)(unsafe.Pointer(&ret))
-	s.Data = (*reflect.StringHeader)(unsafe.Pointer(&data)).Data
-	s.Len = len(data)
-	s.Cap = len(data)
-	return
+	return []byte(data)

Review Comment:
   same comment as above



##########
go/arrow/array/string.go:
##########
@@ -79,16 +79,12 @@ func (a *String) ValueOffsets() []int32 {
 	return a.offsets[beg:end]
 }
 
-func (a *String) ValueBytes() (ret []byte) {
+func (a *String) ValueBytes() []byte {
 	beg := a.array.data.offset
 	end := beg + a.array.data.length
 	data := a.values[a.offsets[beg]:a.offsets[end]]
 
-	s := (*reflect.SliceHeader)(unsafe.Pointer(&ret))
-	s.Data = (*reflect.StringHeader)(unsafe.Pointer(&data)).Data
-	s.Len = len(data)
-	s.Cap = len(data)
-	return
+	return []byte(data)

Review Comment:
   this causes a copy which isn't what we want. This could end up creating a copy of the entire array essentially which would be pretty bad. I assume this change is because of the issues with `SliceHeader`/`StringHeader` in tinygo?
   
   An alternative solution here would be to just pull the bytes from the buffer directly:
   
   ```go
   if a.array.data.buffers[2] != nil {
       return a.array.data.buffers[2].Bytes()[a.offsets[beg]:a.offsets[end]]
   }
   return nil
   ```
   
   This way we preserve the fact that we're not copying the data.



##########
go/arrow/bitutil/bitutil.go:
##########
@@ -150,15 +150,12 @@ const (
 )
 
 func bytesToUint64(b []byte) []uint64 {
-	h := (*reflect.SliceHeader)(unsafe.Pointer(&b))
-
-	var res []uint64
-	s := (*reflect.SliceHeader)(unsafe.Pointer(&res))
-	s.Data = h.Data
-	s.Len = h.Len / uint64SizeBytes
-	s.Cap = h.Cap / uint64SizeBytes
+	if cap(b) < uint64SizeBytes {
+		return nil
+	}
 
-	return res
+	h := (*reflect.SliceHeader)(unsafe.Pointer(&b))
+	return unsafe.Slice((*uint64)(unsafe.Pointer(h.Data)), cap(b)/uint64SizeBytes)[:len(b)/uint64SizeBytes]

Review Comment:
   if we're sure this is only ever called with a slice with a non-zero length, this could be simplified to 
   
   ```go
   return unsafe.Slice((*uint64)(unsafe.Pointer(&b[0])), cap(b)/uint64SizeBytes)[:len(b)/uint64SizeBytes]
   ```



##########
go/arrow/compute/cast_test.go:
##########
@@ -2589,7 +2589,7 @@ func (c *CastSuite) TestStructToDifferentNullabilityStruct() {
 		}
 		srcNonNull, _, err := array.FromJSON(c.mem, arrow.StructOf(fieldsSrcNonNullable...),
 			strings.NewReader(`[
-				{"a": 11, "b": 32, "c", 95},
+				{"a": 11, "b": 32, "c": 95},

Review Comment:
   same comment as before, how is this not already failing?



##########
go/arrow/compute/arithmetic_test.go:
##########
@@ -2621,7 +2621,7 @@ func (us *UnaryArithmeticSigned[T]) TestAbsoluteValue() {
 				fn(`[]`, `[]`)
 				// scalar/arrays with nulls
 				fn(`[null]`, `[null]`)
-				fn(`[1, null -10]`, `[1, null, 10]`)
+				fn(`[1, null, -10]`, `[1, null, 10]`)

Review Comment:
   so this is interesting, how was this not failing already?



##########
go/internal/hashing/xxh3_memo_table.go:
##########
@@ -166,16 +168,13 @@ func (s *BinaryMemoTable) Size() int {
 }
 
 // helper function to easily return a byte slice for any given value
-// regardless of the type if it's a []byte, parquet.ByteArray,
-// parquet.FixedLenByteArray or string.
+// regardless of the type.

Review Comment:
   should probably clarify that this only works for `[]byte`, `string` or something which fulfills the `ByteSlice` interface



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] zeroshade commented on pull request #35723: GH-32832: [Go] support building with tinygo

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on PR #35723:
URL: https://github.com/apache/arrow/pull/35723#issuecomment-1572338180

   @chriscasola Do you think it's sufficient to just have a single x86-64/linux CI run with tinygo? Or do you think we'd need for arm and other cases too?


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] chriscasola commented on a diff in pull request #35723: GH-32832: [Go] support building with tinygo

Posted by "chriscasola (via GitHub)" <gi...@apache.org>.
chriscasola commented on code in PR #35723:
URL: https://github.com/apache/arrow/pull/35723#discussion_r1222042549


##########
ci/scripts/go_tinygo_example.sh:
##########
@@ -0,0 +1,27 @@
+#!/usr/bin/env bash
+#
+# 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.
+
+set -ex
+
+cd ~
+pushd /src
+tinygo build -tags noasm -o ~/example_tinygo arrow/_examples/helloworld/main.go
+popd
+
+./example_tinygo

Review Comment:
   Unfortunately tinygo can't compile/run unit tests right now. At least it can't handle the ones in this project. See [this issue](https://github.com/tinygo-org/tinygo/issues/1359#issuecomment-687668399).



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] zeroshade commented on a diff in pull request #35723: GH-32832: [Go] support building with tinygo

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #35723:
URL: https://github.com/apache/arrow/pull/35723#discussion_r1219971334


##########
ci/scripts/go_tinygo_example.sh:
##########
@@ -0,0 +1,27 @@
+#!/usr/bin/env bash
+#
+# 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.
+
+set -ex
+
+cd ~
+pushd /src
+tinygo build -tags noasm -o ~/example_tinygo arrow/_examples/helloworld/main.go
+popd
+
+./example_tinygo

Review Comment:
   any reason to not just have it run the unit tests?



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] ursabot commented on pull request #35723: GH-32832: [Go] support building with tinygo

Posted by "ursabot (via GitHub)" <gi...@apache.org>.
ursabot commented on PR #35723:
URL: https://github.com/apache/arrow/pull/35723#issuecomment-1588348269

   Benchmark runs are scheduled for baseline = b642707f6de72ec12c1d4a802680b0a3fdc6a216 and contender = 245404e2d462f0eb214f42787cdfc4bf5956cd9f. 245404e2d462f0eb214f42787cdfc4bf5956cd9f is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Finished :arrow_down:25.0% :arrow_up:0.0%] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/87e6988b0e464c1cb1759306052c0974...171d0d65ada24e0d96bf53d6d4b78b12/)
   [Finished :arrow_down:1.24% :arrow_up:0.12%] [test-mac-arm](https://conbench.ursa.dev/compare/runs/9ead9702810647118c7609be590b6272...edeb150a770a4da19fd8903956ddf1c8/)
   [Finished :arrow_down:0.0% :arrow_up:0.65%] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/4842a9c0c97f47c896967a1390bc9074...227826e2283142e7ade5c8fe1d546881/)
   [Finished :arrow_down:0.06% :arrow_up:0.48%] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/cc6d8dcc1d2340a3a285f269c6f8422e...592104b7cb284aae81daefc56b1b6a5c/)
   Buildkite builds:
   [Finished] [`245404e2` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/3026)
   [Finished] [`245404e2` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/3062)
   [Finished] [`245404e2` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/3027)
   [Finished] [`245404e2` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/3052)
   [Finished] [`b642707f` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/3025)
   [Finished] [`b642707f` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/3061)
   [Finished] [`b642707f` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/3026)
   [Finished] [`b642707f` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/3051)
   Supported benchmarks:
   ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python, R. Runs only benchmarks with cloud = True
   test-mac-arm: Supported benchmark langs: C++, Python, R
   ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
   ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java
   


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] chriscasola commented on a diff in pull request #35723: GH-32832: [Go] support building with tinygo

Posted by "chriscasola (via GitHub)" <gi...@apache.org>.
chriscasola commented on code in PR #35723:
URL: https://github.com/apache/arrow/pull/35723#discussion_r1210676066


##########
go/arrow/bitutil/bitutil.go:
##########
@@ -150,15 +150,12 @@ const (
 )
 
 func bytesToUint64(b []byte) []uint64 {
-	h := (*reflect.SliceHeader)(unsafe.Pointer(&b))
-
-	var res []uint64
-	s := (*reflect.SliceHeader)(unsafe.Pointer(&res))
-	s.Data = h.Data
-	s.Len = h.Len / uint64SizeBytes
-	s.Cap = h.Cap / uint64SizeBytes
+	if cap(b) < uint64SizeBytes {
+		return nil
+	}
 
-	return res
+	h := (*reflect.SliceHeader)(unsafe.Pointer(&b))
+	return unsafe.Slice((*uint64)(unsafe.Pointer(h.Data)), cap(b)/uint64SizeBytes)[:len(b)/uint64SizeBytes]

Review Comment:
   I'm not sure I can guarantee that. Are you suggesting we add a check just above and `return nil` if the length is `< 1`?



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] zeroshade merged pull request #35723: GH-32832: [Go] support building with tinygo

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade merged PR #35723:
URL: https://github.com/apache/arrow/pull/35723


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] chriscasola commented on a diff in pull request #35723: GH-32832: [Go] support building with tinygo

Posted by "chriscasola (via GitHub)" <gi...@apache.org>.
chriscasola commented on code in PR #35723:
URL: https://github.com/apache/arrow/pull/35723#discussion_r1226966420


##########
ci/scripts/go_tinygo_example.sh:
##########
@@ -0,0 +1,27 @@
+#!/usr/bin/env bash
+#
+# 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.
+
+set -ex
+
+cd ~
+pushd /src
+tinygo build -tags noasm -o ~/example_tinygo arrow/_examples/helloworld/main.go

Review Comment:
   Unfortunately tinygo is just erroring without enough context to find the file at fault. I'm not sure if there's a way to find it with the standard go toolchain.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] tschaub commented on pull request #35723: GH-32832: [Go] support building with tinygo

Posted by "tschaub (via GitHub)" <gi...@apache.org>.
tschaub commented on PR #35723:
URL: https://github.com/apache/arrow/pull/35723#issuecomment-1728663949

   Cool to see this got in.  I wonder if anybody looked into what it would take to get the Parquet packages working as well.  I may give it another try at some point.


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] github-actions[bot] commented on pull request #35723: GH-32832: [Go] support building with tinygo

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #35723:
URL: https://github.com/apache/arrow/pull/35723#issuecomment-1559249224

   :warning: GitHub issue #32832 **has been automatically assigned in GitHub** to PR creator.


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] zeroshade commented on pull request #35723: GH-32832: [Go] support building with tinygo

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on PR #35723:
URL: https://github.com/apache/arrow/pull/35723#issuecomment-1729712999

   I believe the current blocker on the parquet packages is that something in the Thrift dependency is not compatible with tinygo


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] zeroshade commented on a diff in pull request #35723: GH-32832: [Go] support building with tinygo

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #35723:
URL: https://github.com/apache/arrow/pull/35723#discussion_r1210696225


##########
go/arrow/array/string.go:
##########
@@ -79,16 +79,12 @@ func (a *String) ValueOffsets() []int32 {
 	return a.offsets[beg:end]
 }
 
-func (a *String) ValueBytes() (ret []byte) {
+func (a *String) ValueBytes() []byte {
 	beg := a.array.data.offset
 	end := beg + a.array.data.length
 	data := a.values[a.offsets[beg]:a.offsets[end]]
 
-	s := (*reflect.SliceHeader)(unsafe.Pointer(&ret))
-	s.Data = (*reflect.StringHeader)(unsafe.Pointer(&data)).Data
-	s.Len = len(data)
-	s.Cap = len(data)
-	return
+	return []byte(data)

Review Comment:
   Strings in Go are immutable, so unless the compiler can confirm that `[]byte(str)` is only used as a temporary it will do a copy of the bytes to construct the byte slice from the string. 



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] chriscasola commented on pull request #35723: GH-32832: [Go] support building with tinygo

Posted by "chriscasola (via GitHub)" <gi...@apache.org>.
chriscasola commented on PR #35723:
URL: https://github.com/apache/arrow/pull/35723#issuecomment-1572407525

   > @chriscasola Do you think it's sufficient to just have a single x86-64/linux CI run with tinygo? Or do you think we'd need for arm and other cases too?
   
   I think a single architecture is fine.


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] chriscasola commented on pull request #35723: GH-32832: [Go] support building with tinygo

Posted by "chriscasola (via GitHub)" <gi...@apache.org>.
chriscasola commented on PR #35723:
URL: https://github.com/apache/arrow/pull/35723#issuecomment-1562916363

   @zeroshade we're still testing this on our end, but it does work. I'm curious what your thoughts are.


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] chriscasola commented on a diff in pull request #35723: GH-32832: [Go] support building with tinygo

Posted by "chriscasola (via GitHub)" <gi...@apache.org>.
chriscasola commented on code in PR #35723:
URL: https://github.com/apache/arrow/pull/35723#discussion_r1223015929


##########
ci/scripts/go_tinygo_example.sh:
##########
@@ -0,0 +1,27 @@
+#!/usr/bin/env bash
+#
+# 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.
+
+set -ex
+
+cd ~
+pushd /src
+tinygo build -tags noasm -o ~/example_tinygo arrow/_examples/helloworld/main.go

Review Comment:
   @zeroshade I attempted to do this, but it seems like package outside arrow might also have a `noasm` flag. Even after adding the `tinygo` condition everywhere there was a `noasm` condition, I still couldn't build with out the `noasm` tag. Are you okay omitting this for now?



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] chriscasola commented on pull request #35723: GH-32832: [Go] support building with tinygo

Posted by "chriscasola (via GitHub)" <gi...@apache.org>.
chriscasola commented on PR #35723:
URL: https://github.com/apache/arrow/pull/35723#issuecomment-1587604611

   @zeroshade this is ready for review and has been fully tested to work with tinygo in our app's environment.


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] zeroshade commented on a diff in pull request #35723: GH-32832: [Go] support building with tinygo

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #35723:
URL: https://github.com/apache/arrow/pull/35723#discussion_r1213376339


##########
go/internal/types/extension_types.go:
##########
@@ -24,7 +24,7 @@ import (
 	"reflect"
 	"strings"
 
-	"github.com/goccy/go-json"
+	"github.com/apache/arrow/go/v13/internal/json"
 
 	"github.com/apache/arrow/go/v13/arrow"
 	"github.com/apache/arrow/go/v13/arrow/array"

Review Comment:
   the failed dev scripts test is because when we run the auto update scripts this gets updated as a separate block. Can you remove the empty space below the updated import and just group all the arrow imports together (alphabetically)?
   
   i.e.:
   
   ```
   ...
   "strings"
   
   "github.com/apache/arrow/go/v13/arrow"
   "github.com/apache/arrow/go/v13/arrow/array"
   "github.com/apache/arrow/go/v13/internal/json"
   "github.com/google/uuid"
   "golang.org/x/xerrors"
   ```
   
   thanks.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] zeroshade commented on a diff in pull request #35723: GH-32832: [Go] support building with tinygo

Posted by "zeroshade (via GitHub)" <gi...@apache.org>.
zeroshade commented on code in PR #35723:
URL: https://github.com/apache/arrow/pull/35723#discussion_r1210697021


##########
go/arrow/bitutil/bitutil.go:
##########
@@ -150,15 +150,12 @@ const (
 )
 
 func bytesToUint64(b []byte) []uint64 {
-	h := (*reflect.SliceHeader)(unsafe.Pointer(&b))
-
-	var res []uint64
-	s := (*reflect.SliceHeader)(unsafe.Pointer(&res))
-	s.Data = h.Data
-	s.Len = h.Len / uint64SizeBytes
-	s.Cap = h.Cap / uint64SizeBytes
+	if cap(b) < uint64SizeBytes {
+		return nil
+	}
 
-	return res
+	h := (*reflect.SliceHeader)(unsafe.Pointer(&b))
+	return unsafe.Slice((*uint64)(unsafe.Pointer(h.Data)), cap(b)/uint64SizeBytes)[:len(b)/uint64SizeBytes]

Review Comment:
   nah, if we can't guarantee that it's a non-zero length then this is likely fine as is.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org