You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "candiduslynx (via GitHub)" <gi...@apache.org> on 2023/06/07 16:52:09 UTC

[GitHub] [arrow] candiduslynx opened a new pull request, #35973: GH-35015: [Go] Fix parquet memleak

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

   ### Rationale for this change
   
   Some memory leaks resulted in partially skipped memory checks in pqarrow package.
   This PR brings the checks back.
   
   ### What changes are included in this PR?
   
   Releases in proper places.
   
   ### Are these changes tested?
   
   Yes, the tests from #35015 are fully enabled now.
   
   ### Are there any user-facing changes?
   
   No.
   
   <!--
   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] candiduslynx commented on a diff in pull request #35973: GH-35015: [Go] Fix parquet memleak

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


##########
go/arrow/memory/checked_allocator.go:
##########
@@ -160,14 +160,24 @@ func (a *CheckedAllocator) AssertSize(t TestingT, sz int) {
 				break
 			}
 			callersMsg.WriteString("\t")
-			callersMsg.WriteString(frame.Function)
-			callersMsg.WriteString(fmt.Sprintf(" line %d", frame.Line))
+			if fn := frame.Func; fn != nil {
+				callersMsg.WriteString(fmt.Sprintf("%s+%x", fn.Name(), frame.PC-fn.Entry()))

Review Comment:
   Done in c95516aaef66bf4de3d925fde030eddbc5a78af7



-- 
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] candiduslynx commented on pull request #35973: GH-35015: [Go] Fix parquet memleak

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

   > this looks good to me, thanks for tracking this down! Must have been quite annoying, but the changes to the checked allocator should make it much easier to track down these in the future.
   
   Yeah, I don't want to have this rodeo ride once more 😅 
   
   > Just a nit pick, and then waiting for the rebase now that I merged #35976
   
   Done!


-- 
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] candiduslynx commented on pull request #35973: GH-35015: [Go] Fix parquet memleak

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

   > @zeroshade I guess the value type [issue](https://github.com/apache/arrow/actions/runs/5202584206/jobs/9384384500?pr=35973#step:5:797) is caused by #35899. Is there some place I can fix the (generated?) code to make it pass?
   
   @zeroshade the fix for the code: #35976 in compute. I'll leave creation of the issue & filling in the PR up to you.


-- 
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 #35973: GH-35015: [Go] Fix parquet memleak

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


##########
go/arrow/memory/checked_allocator.go:
##########
@@ -160,22 +160,33 @@ func (a *CheckedAllocator) AssertSize(t TestingT, sz int) {
 				break
 			}
 			callersMsg.WriteString("\t")
+			// frame.Func is a useful source of information if it's present.
+			// Tt may be nil for non-Go code or fully inlined functions.
 			if fn := frame.Func; fn != nil {
+				// format as func name + the offset in bytes from func entrypoint
 				callersMsg.WriteString(fmt.Sprintf("%s+%x", fn.Name(), frame.PC-fn.Entry()))
 			} else {
+				// fallback to outer func name + file line
 				callersMsg.WriteString(fmt.Sprintf("%s, line %d", frame.Function, frame.Line))
 			}
+
+			// Write a proper file name + line, so it's really easy to find the leak
 			callersMsg.WriteString("\n\t\t")
 			callersMsg.WriteString(frame.File + ":" + strconv.Itoa(frame.Line))
 			callersMsg.WriteString("\n")
 			if !more {
 				break
 			}
 		}
+
 		file, line := f.FileLine(info.pc)
-		t.Errorf("LEAK of %d bytes FROM\n\t%s+%x\n\t\t%s:%d\n%v",
-			info.sz, f.Name(), info.pc-f.Entry(),
-			file, line,
+		t.Errorf(`LEAK of %d bytes FROM
+	%s+%x
+		%s:%d
+%v`,

Review Comment:
   what's with the spacing here?



-- 
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 #35973: GH-35015: [Go] Fix parquet memleak

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


##########
go/arrow/memory/checked_allocator.go:
##########
@@ -160,22 +160,33 @@ func (a *CheckedAllocator) AssertSize(t TestingT, sz int) {
 				break
 			}
 			callersMsg.WriteString("\t")
+			// frame.Func is a useful source of information if it's present.
+			// Tt may be nil for non-Go code or fully inlined functions.

Review Comment:
   ```suggestion
   			// It may be nil for non-Go code or fully inlined functions.
   ```



-- 
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] candiduslynx commented on pull request #35973: GH-35015: [Go] Fix parquet memleak

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

   @zeroshade just a mention that everything has passed & it only waits for you to merge


-- 
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] candiduslynx commented on a diff in pull request #35973: GH-35015: [Go] Fix parquet memleak

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


##########
go/arrow/memory/buffer.go:
##########
@@ -35,7 +35,7 @@ type Buffer struct {
 
 // NewBufferBytes creates a fixed-size buffer from the specified data.
 func NewBufferBytes(data []byte) *Buffer {
-	return &Buffer{refCount: 0, buf: data, length: len(data)}
+	return &Buffer{refCount: 1, buf: data, length: len(data)}

Review Comment:
   removed in 6d8a6b715b1611fbeeded130b3869345902c5ca8



-- 
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] candiduslynx commented on a diff in pull request #35973: GH-35015: [Go] Fix parquet memleak

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


##########
go/arrow/memory/checked_allocator.go:
##########
@@ -160,14 +160,24 @@ func (a *CheckedAllocator) AssertSize(t TestingT, sz int) {
 				break
 			}
 			callersMsg.WriteString("\t")
-			callersMsg.WriteString(frame.Function)
-			callersMsg.WriteString(fmt.Sprintf(" line %d", frame.Line))
+			if fn := frame.Func; fn != nil {
+				callersMsg.WriteString(fmt.Sprintf("%s+%x", fn.Name(), frame.PC-fn.Entry()))
+			} else {
+				callersMsg.WriteString(fmt.Sprintf("%s, line %d", frame.Function, frame.Line))

Review Comment:
   sad option for telling the line only



##########
go/arrow/array/extension.go:
##########
@@ -74,28 +74,12 @@ func NewExtensionArrayWithStorage(dt arrow.ExtensionType, storage arrow.Array) a
 		panic(fmt.Errorf("arrow/array: storage type %s for extension type %s, does not match expected type %s", storage.DataType(), dt.ExtensionName(), dt.StorageType()))
 	}
 
-	base := ExtensionArrayBase{}

Review Comment:
   basically, it was a copied code



##########
go/arrow/memory/buffer.go:
##########
@@ -35,7 +35,7 @@ type Buffer struct {
 
 // NewBufferBytes creates a fixed-size buffer from the specified data.
 func NewBufferBytes(data []byte) *Buffer {
-	return &Buffer{refCount: 0, buf: data, length: len(data)}
+	return &Buffer{refCount: 1, buf: data, length: len(data)}

Review Comment:
   for housekeeping



##########
go/parquet/file/record_reader.go:
##########
@@ -783,19 +783,19 @@ func newByteArrayRecordReader(descr *schema.Column, info LevelInfo, dtype arrow.
 	}}
 }
 
-func (fr *byteArrayRecordReader) ReserveValues(extra int64, hasNullable bool) error {
-	fr.bldr.Reserve(int(extra))
-	return fr.primitiveRecordReader.ReserveValues(extra, hasNullable)
+func (br *byteArrayRecordReader) ReserveValues(extra int64, hasNullable bool) error {

Review Comment:
   just renamed the receiver to have the same name across all methods



##########
go/arrow/memory/checked_allocator.go:
##########
@@ -160,14 +160,24 @@ func (a *CheckedAllocator) AssertSize(t TestingT, sz int) {
 				break
 			}
 			callersMsg.WriteString("\t")
-			callersMsg.WriteString(frame.Function)
-			callersMsg.WriteString(fmt.Sprintf(" line %d", frame.Line))
+			if fn := frame.Func; fn != nil {
+				callersMsg.WriteString(fmt.Sprintf("%s+%x", fn.Name(), frame.PC-fn.Entry()))
+			} else {
+				callersMsg.WriteString(fmt.Sprintf("%s, line %d", frame.Function, frame.Line))
+			}
+			callersMsg.WriteString("\n\t\t")
+			callersMsg.WriteString(frame.File + ":" + strconv.Itoa(frame.Line))
 			callersMsg.WriteString("\n")
 			if !more {
 				break
 			}
 		}
-		t.Errorf("LEAK of %d bytes FROM %s line %d\n%v", info.sz, f.Name(), info.line, callersMsg.String())
+		file, line := f.FileLine(info.pc)
+		t.Errorf("LEAK of %d bytes FROM\n\t%s+%x\n\t\t%s:%d\n%v",

Review Comment:
   this gives a proper fill filename, so the leak is easier to find (IntelliJ IDEA even allows clicking on a link to line)



##########
go/parquet/pqarrow/column_readers.go:
##########
@@ -117,12 +111,25 @@ func (lr *leafReader) LoadBatch(nrecords int64) (err error) {
 			}
 		}
 	}
-	lr.out, err = transferColumnData(lr.recordRdr, lr.field.Type, lr.descr, lr.rctx.mem)
+	lr.out, err = transferColumnData(lr.recordRdr, lr.field.Type, lr.descr)

Review Comment:
   mem param was unused



##########
go/parquet/pqarrow/column_readers.go:
##########
@@ -443,14 +454,16 @@ func chunksToSingle(chunked *arrow.Chunked) (arrow.ArrayData, error) {
 	case 0:
 		return array.NewData(chunked.DataType(), 0, []*memory.Buffer{nil, nil}, nil, 0, 0), nil
 	case 1:
-		return chunked.Chunk(0).Data(), nil
+		data := chunked.Chunk(0).Data()
+		data.Retain() // we pass control to the caller

Review Comment:
   we want the caller to have the full control of the data (so, releasing the array after the chunksToSingle should have no effect on the returned data.
   For 0 len it was already true (we constructed a new data), so this retain comes in naturally



##########
go/parquet/pqarrow/column_readers.go:
##########
@@ -525,8 +538,9 @@ func transferZeroCopy(rdr file.RecordReader, dt arrow.DataType) arrow.ArrayData
 		}
 	}()
 
-	return array.NewData(dt, rdr.ValuesWritten(), []*memory.Buffer{
-		bitmap, values}, nil, int(rdr.NullCount()), 0)
+	return array.NewData(dt, rdr.ValuesWritten(),
+		[]*memory.Buffer{bitmap, values},

Review Comment:
   put `[]*memory.Buffer{bitmap, values},` on a separate line



##########
go/parquet/pqarrow/encode_arrow_test.go:
##########
@@ -320,6 +320,7 @@ func writeTableToBuffer(t *testing.T, mem memory.Allocator, tbl arrow.Table, row
 }
 
 func simpleRoundTrip(t *testing.T, tbl arrow.Table, rowGroupSize int64) {
+	t.Helper()

Review Comment:
   so that we see the real failing test



##########
go/arrow/memory/checked_allocator.go:
##########
@@ -160,14 +160,24 @@ func (a *CheckedAllocator) AssertSize(t TestingT, sz int) {
 				break
 			}
 			callersMsg.WriteString("\t")
-			callersMsg.WriteString(frame.Function)
-			callersMsg.WriteString(fmt.Sprintf(" line %d", frame.Line))
+			if fn := frame.Func; fn != nil {
+				callersMsg.WriteString(fmt.Sprintf("%s+%x", fn.Name(), frame.PC-fn.Entry()))

Review Comment:
   offset in bytes from func start, if we have fn (we don't for inlined funcs



##########
go/parquet/pqarrow/encode_arrow_test.go:
##########
@@ -338,6 +339,7 @@ func simpleRoundTrip(t *testing.T, tbl arrow.Table, rowGroupSize int64) {
 
 		chunked, err := crdr.NextBatch(tbl.NumRows())
 		require.NoError(t, err)
+		defer chunked.Release()

Review Comment:
   lazy, but only in test



##########
go/parquet/pqarrow/helpers.go:
##########
@@ -0,0 +1,43 @@
+// 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 pqarrow
+
+import (
+	"github.com/apache/arrow/go/v13/arrow"
+)
+
+func releaseArrays(arrays []arrow.Array) {

Review Comment:
   Helpers that allowed not to define a lot of the same funcs.
   This might be useful elsewhere, but for now it's here



##########
go/parquet/pqarrow/encode_dictionary_test.go:
##########
@@ -543,6 +543,7 @@ func (ar *ArrowReadDictSuite) TestIncrementalReads() {
 	for i := 0; i < numReads; i++ {
 		chunk, err := col.NextBatch(int64(batchSize))
 		ar.Require().NoError(err)
+		defer chunk.Release()

Review Comment:
   lazy, but only in tests



##########
go/arrow/memory/checked_allocator.go:
##########
@@ -160,14 +160,24 @@ func (a *CheckedAllocator) AssertSize(t TestingT, sz int) {
 				break
 			}
 			callersMsg.WriteString("\t")
-			callersMsg.WriteString(frame.Function)
-			callersMsg.WriteString(fmt.Sprintf(" line %d", frame.Line))
+			if fn := frame.Func; fn != nil {
+				callersMsg.WriteString(fmt.Sprintf("%s+%x", fn.Name(), frame.PC-fn.Entry()))
+			} else {
+				callersMsg.WriteString(fmt.Sprintf("%s, line %d", frame.Function, frame.Line))
+			}
+			callersMsg.WriteString("\n\t\t")
+			callersMsg.WriteString(frame.File + ":" + strconv.Itoa(frame.Line))

Review Comment:
   this gives a proper fill filename, so the leak is easier to find (IntelliJ IDEA even allows clicking on a link to line)



##########
go/parquet/pqarrow/column_readers.go:
##########
@@ -296,11 +306,11 @@ func (sr *structReader) BuildArray(lenBound int64) (*arrow.Chunked, error) {
 	buffers := make([]*memory.Buffer, 1)
 	if validityIO.NullCount > 0 {
 		buffers[0] = nullBitmap
+		defer nullBitmap.Release()
 	}
-
 	data := array.NewData(sr.filtered.Type, int(validityIO.Read), buffers, childArrData, int(validityIO.NullCount), 0)
 	defer data.Release()
-	arr := array.MakeFromData(data)
+	arr := array.NewStructData(data)

Review Comment:
   call directly



##########
go/parquet/pqarrow/column_readers.go:
##########
@@ -443,14 +454,16 @@ func chunksToSingle(chunked *arrow.Chunked) (arrow.ArrayData, error) {
 	case 0:
 		return array.NewData(chunked.DataType(), 0, []*memory.Buffer{nil, nil}, nil, 0, 0), nil
 	case 1:
-		return chunked.Chunk(0).Data(), nil
+		data := chunked.Chunk(0).Data()
+		data.Retain() // we pass control to the caller
+		return data, nil
 	default: // if an item reader yields a chunked array, this is not yet implemented
 		return nil, arrow.ErrNotImplemented
 	}
 }
 
 // create a chunked arrow array from the raw record data
-func transferColumnData(rdr file.RecordReader, valueType arrow.DataType, descr *schema.Column, mem memory.Allocator) (*arrow.Chunked, error) {
+func transferColumnData(rdr file.RecordReader, valueType arrow.DataType, descr *schema.Column) (*arrow.Chunked, error) {

Review Comment:
   mem param was unused



##########
go/parquet/pqarrow/encode_arrow_test.go:
##########
@@ -1180,7 +1181,7 @@ func prepareListOfListTable(dt arrow.DataType, size, nullCount int, nullablePare
 
 func (ps *ParquetIOTestSuite) TestSingleEmptyListsColumnReadWrite() {
 	mem := memory.NewCheckedAllocator(memory.DefaultAllocator)
-	//defer mem.AssertSize(ps.T(), 0) // FIXME: known leak
+	defer mem.AssertSize(ps.T(), 0)

Review Comment:
   return of the original mem check



##########
go/parquet/pqarrow/encode_arrow_test.go:
##########
@@ -1040,10 +1044,7 @@ func (ps *ParquetIOTestSuite) TestSingleColumnRequiredWrite() {
 	}
 }
 
-func (ps *ParquetIOTestSuite) roundTripTable(_ memory.Allocator, expected arrow.Table, storeSchema bool) {
-	mem := memory.NewCheckedAllocator(memory.DefaultAllocator) // FIXME: currently overriding allocator to isolate leaks between roundTripTable and caller
-	//defer mem.AssertSize(ps.T(), 0)                            // FIXME: known leak
-
+func (ps *ParquetIOTestSuite) roundTripTable(mem memory.Allocator, expected arrow.Table, storeSchema bool) {

Review Comment:
   return of the original `mem`



##########
go/parquet/pqarrow/encode_dictionary_test.go:
##########
@@ -400,7 +400,7 @@ func (ar *ArrowReadDictSuite) writeSimple() {
 		pqarrow.DefaultWriterProps()))
 }
 
-func (ArrowReadDictSuite) NullProbabilities() []float64 {
+func (*ArrowReadDictSuite) NullProbabilities() []float64 {

Review Comment:
   Go asks to stick to one of the receivers type: either value or pointer (no mixing)



-- 
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] candiduslynx commented on pull request #35973: GH-35015: [Go] Fix parquet memleak

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

   @zeroshade I've fixed the extra release in bf0e4a9196ef72748c384ee2c1f4a8d5966390f5
   I see that one release script test was cancelled (maybe it needs more time?), but the tests are passing


-- 
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] candiduslynx commented on a diff in pull request #35973: GH-35015: [Go] Fix parquet memleak

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


##########
go/arrow/memory/buffer.go:
##########
@@ -35,7 +35,7 @@ type Buffer struct {
 
 // NewBufferBytes creates a fixed-size buffer from the specified data.
 func NewBufferBytes(data []byte) *Buffer {
-	return &Buffer{refCount: 0, buf: data, length: len(data)}
+	return &Buffer{refCount: 1, buf: data, length: len(data)}

Review Comment:
   This one doesn't affect anything, so I can revert it with no issues



-- 
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 #35973: GH-35015: [Go] Fix parquet memleak

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

   Looks like you've got some tests failing with `Too many releases`


-- 
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] candiduslynx commented on a diff in pull request #35973: GH-35015: [Go] Fix parquet memleak

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


##########
go/arrow/memory/checked_allocator.go:
##########
@@ -160,14 +160,24 @@ func (a *CheckedAllocator) AssertSize(t TestingT, sz int) {
 				break
 			}
 			callersMsg.WriteString("\t")
-			callersMsg.WriteString(frame.Function)
-			callersMsg.WriteString(fmt.Sprintf(" line %d", frame.Line))
+			if fn := frame.Func; fn != nil {
+				callersMsg.WriteString(fmt.Sprintf("%s+%x", fn.Name(), frame.PC-fn.Entry()))
+			} else {
+				callersMsg.WriteString(fmt.Sprintf("%s, line %d", frame.Function, frame.Line))
+			}
+			callersMsg.WriteString("\n\t\t")
+			callersMsg.WriteString(frame.File + ":" + strconv.Itoa(frame.Line))
 			callersMsg.WriteString("\n")
 			if !more {
 				break
 			}
 		}
-		t.Errorf("LEAK of %d bytes FROM %s line %d\n%v", info.sz, f.Name(), info.line, callersMsg.String())
+		file, line := f.FileLine(info.pc)
+		t.Errorf("LEAK of %d bytes FROM\n\t%s+%x\n\t\t%s:%d\n%v",

Review Comment:
   commented in c95516aaef66bf4de3d925fde030eddbc5a78af7



-- 
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] candiduslynx commented on pull request #35973: GH-35015: [Go] Fix parquet memleak

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

   > LGTM just one nitpick [here](https://github.com/apache/arrow/pull/35973/files#r1222095085)
   
   Done in 95a0333f9178e02ffe2a017c3cf83491cb313bf8


-- 
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] candiduslynx commented on a diff in pull request #35973: GH-35015: [Go] Fix parquet memleak

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


##########
go/arrow/memory/checked_allocator.go:
##########
@@ -160,14 +160,35 @@ func (a *CheckedAllocator) AssertSize(t TestingT, sz int) {
 				break
 			}
 			callersMsg.WriteString("\t")
-			callersMsg.WriteString(frame.Function)
-			callersMsg.WriteString(fmt.Sprintf(" line %d", frame.Line))
+			// frame.Func is a useful source of information if it's present.
+			// It may be nil for non-Go code or fully inlined functions.
+			if fn := frame.Func; fn != nil {
+				// format as func name + the offset in bytes from func entrypoint
+				callersMsg.WriteString(fmt.Sprintf("%s+%x", fn.Name(), frame.PC-fn.Entry()))
+			} else {
+				// fallback to outer func name + file line
+				callersMsg.WriteString(fmt.Sprintf("%s, line %d", frame.Function, frame.Line))
+			}
+
+			// Write a proper file name + line, so it's really easy to find the leak
+			callersMsg.WriteString("\n\t\t")
+			callersMsg.WriteString(frame.File + ":" + strconv.Itoa(frame.Line))
 			callersMsg.WriteString("\n")
 			if !more {
 				break
 			}
 		}
-		t.Errorf("LEAK of %d bytes FROM %s line %d\n%v", info.sz, f.Name(), info.line, callersMsg.String())
+
+		file, line := f.FileLine(info.pc)
+		t.Errorf(`LEAK of %d bytes FROM
+	%s+%x
+		%s:%d
+%v`,

Review Comment:
   ```suggestion
   ```



-- 
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 #35973: GH-35015: [Go] Fix parquet memleak

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

   :warning: GitHub issue #35015 **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] candiduslynx commented on a diff in pull request #35973: GH-35015: [Go] Fix parquet memleak

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


##########
go/arrow/memory/checked_allocator.go:
##########
@@ -160,14 +160,35 @@ func (a *CheckedAllocator) AssertSize(t TestingT, sz int) {
 				break
 			}
 			callersMsg.WriteString("\t")
-			callersMsg.WriteString(frame.Function)
-			callersMsg.WriteString(fmt.Sprintf(" line %d", frame.Line))
+			// frame.Func is a useful source of information if it's present.
+			// It may be nil for non-Go code or fully inlined functions.
+			if fn := frame.Func; fn != nil {
+				// format as func name + the offset in bytes from func entrypoint
+				callersMsg.WriteString(fmt.Sprintf("%s+%x", fn.Name(), frame.PC-fn.Entry()))
+			} else {
+				// fallback to outer func name + file line
+				callersMsg.WriteString(fmt.Sprintf("%s, line %d", frame.Function, frame.Line))
+			}
+
+			// Write a proper file name + line, so it's really easy to find the leak
+			callersMsg.WriteString("\n\t\t")
+			callersMsg.WriteString(frame.File + ":" + strconv.Itoa(frame.Line))
 			callersMsg.WriteString("\n")
 			if !more {
 				break
 			}
 		}
-		t.Errorf("LEAK of %d bytes FROM %s line %d\n%v", info.sz, f.Name(), info.line, callersMsg.String())
+
+		file, line := f.FileLine(info.pc)
+		t.Errorf(`LEAK of %d bytes FROM

Review Comment:
   ```suggestion
   		t.Errorf("LEAK of %d bytes FROM\n\t%s+%x\n\t\t%s:%d\n%v",
   ```



-- 
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 #35973: GH-35015: [Go] Fix parquet memleak

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

   Benchmark runs are scheduled for baseline = 8b5919d886125c3dae9dd5484f7e9e45ae8580d3 and contender = 17311b6a14dedf81b552434f92775cdd2ecd74d8. 17311b6a14dedf81b552434f92775cdd2ecd74d8 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:0.0% :arrow_up:0.0%] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/78cfd14b6483415d924b194c70205192...a7403a93867242aaa6c180a60120b5c0/)
   [Finished :arrow_down:1.03% :arrow_up:0.03%] [test-mac-arm](https://conbench.ursa.dev/compare/runs/0538fdf29d2f4d0a81827f4c43661cea...5d50c9e06e244a918b4c3d3d03c993eb/)
   [Finished :arrow_down:9.8% :arrow_up:5.56%] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/1f83bcc558234a1ab1c5bfd8134fa327...b0bd911a1ac84173a8f8f2bd639a55f9/)
   [Finished :arrow_down:0.21% :arrow_up:0.06%] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/06416611c0e64450ba3a9f50a26d707b...d5eb106307b14020a51bbf169dacd8a6/)
   Buildkite builds:
   [Finished] [`17311b6a` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/3011)
   [Finished] [`17311b6a` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/3047)
   [Finished] [`17311b6a` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/3012)
   [Finished] [`17311b6a` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/3037)
   [Finished] [`8b5919d8` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/3010)
   [Finished] [`8b5919d8` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/3046)
   [Finished] [`8b5919d8` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/3011)
   [Finished] [`8b5919d8` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/3036)
   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] candiduslynx commented on pull request #35973: GH-35015: [Go] Fix parquet memleak

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

   @zeroshade I guess the value type [issue](https://github.com/apache/arrow/actions/runs/5202584206/jobs/9384384500?pr=35973#step:5:797) is caused by #35899. Is there some place I can fix the (generated?) code to make it pass?


-- 
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 #35973: GH-35015: [Go] Fix parquet memleak

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


##########
go/arrow/memory/checked_allocator.go:
##########
@@ -160,14 +160,24 @@ func (a *CheckedAllocator) AssertSize(t TestingT, sz int) {
 				break
 			}
 			callersMsg.WriteString("\t")
-			callersMsg.WriteString(frame.Function)
-			callersMsg.WriteString(fmt.Sprintf(" line %d", frame.Line))
+			if fn := frame.Func; fn != nil {
+				callersMsg.WriteString(fmt.Sprintf("%s+%x", fn.Name(), frame.PC-fn.Entry()))

Review Comment:
   add this as a comment here so that we don't lose the information? This is good to know so we don't look at this in the future and go "wtf is this doing!"



##########
go/arrow/memory/buffer.go:
##########
@@ -35,7 +35,7 @@ type Buffer struct {
 
 // NewBufferBytes creates a fixed-size buffer from the specified data.
 func NewBufferBytes(data []byte) *Buffer {
-	return &Buffer{refCount: 0, buf: data, length: len(data)}
+	return &Buffer{refCount: 1, buf: data, length: len(data)}

Review Comment:
   So the interesting aspect about this is that the `refCount` on this will never get reduced to 0 because in this situation `Release` and `Retain` are no-ops (because `b.mem` and `b.parent` are both `nil`). So we should probably leave this as 0, not 1.



##########
go/arrow/memory/checked_allocator.go:
##########
@@ -160,14 +160,24 @@ func (a *CheckedAllocator) AssertSize(t TestingT, sz int) {
 				break
 			}
 			callersMsg.WriteString("\t")
-			callersMsg.WriteString(frame.Function)
-			callersMsg.WriteString(fmt.Sprintf(" line %d", frame.Line))
+			if fn := frame.Func; fn != nil {
+				callersMsg.WriteString(fmt.Sprintf("%s+%x", fn.Name(), frame.PC-fn.Entry()))
+			} else {
+				callersMsg.WriteString(fmt.Sprintf("%s, line %d", frame.Function, frame.Line))
+			}
+			callersMsg.WriteString("\n\t\t")
+			callersMsg.WriteString(frame.File + ":" + strconv.Itoa(frame.Line))

Review Comment:
   same as my previous comment, make this a comment in the code so that we know what this is and why it's there :smile:



-- 
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 #35973: GH-35015: [Go] Fix parquet memleak

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


##########
go/arrow/memory/checked_allocator.go:
##########
@@ -160,22 +160,33 @@ func (a *CheckedAllocator) AssertSize(t TestingT, sz int) {
 				break
 			}
 			callersMsg.WriteString("\t")
+			// frame.Func is a useful source of information if it's present.
+			// Tt may be nil for non-Go code or fully inlined functions.
 			if fn := frame.Func; fn != nil {
+				// format as func name + the offset in bytes from func entrypoint
 				callersMsg.WriteString(fmt.Sprintf("%s+%x", fn.Name(), frame.PC-fn.Entry()))
 			} else {
+				// fallback to outer func name + file line
 				callersMsg.WriteString(fmt.Sprintf("%s, line %d", frame.Function, frame.Line))
 			}
+
+			// Write a proper file name + line, so it's really easy to find the leak
 			callersMsg.WriteString("\n\t\t")
 			callersMsg.WriteString(frame.File + ":" + strconv.Itoa(frame.Line))
 			callersMsg.WriteString("\n")
 			if !more {
 				break
 			}
 		}
+
 		file, line := f.FileLine(info.pc)
-		t.Errorf("LEAK of %d bytes FROM\n\t%s+%x\n\t\t%s:%d\n%v",
-			info.sz, f.Name(), info.pc-f.Entry(),
-			file, line,
+		t.Errorf(`LEAK of %d bytes FROM
+	%s+%x
+		%s:%d
+%v`,

Review Comment:
   could you use double quotes instead and use `\t` and `\n`  so it's more explicit and not subject to looking weird based on your UI settings?



-- 
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] candiduslynx commented on a diff in pull request #35973: GH-35015: [Go] Fix parquet memleak

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


##########
go/arrow/memory/checked_allocator.go:
##########
@@ -160,14 +160,35 @@ func (a *CheckedAllocator) AssertSize(t TestingT, sz int) {
 				break
 			}
 			callersMsg.WriteString("\t")
-			callersMsg.WriteString(frame.Function)
-			callersMsg.WriteString(fmt.Sprintf(" line %d", frame.Line))
+			// frame.Func is a useful source of information if it's present.
+			// It may be nil for non-Go code or fully inlined functions.
+			if fn := frame.Func; fn != nil {
+				// format as func name + the offset in bytes from func entrypoint
+				callersMsg.WriteString(fmt.Sprintf("%s+%x", fn.Name(), frame.PC-fn.Entry()))
+			} else {
+				// fallback to outer func name + file line
+				callersMsg.WriteString(fmt.Sprintf("%s, line %d", frame.Function, frame.Line))
+			}
+
+			// Write a proper file name + line, so it's really easy to find the leak
+			callersMsg.WriteString("\n\t\t")
+			callersMsg.WriteString(frame.File + ":" + strconv.Itoa(frame.Line))
 			callersMsg.WriteString("\n")
 			if !more {
 				break
 			}
 		}
-		t.Errorf("LEAK of %d bytes FROM %s line %d\n%v", info.sz, f.Name(), info.line, callersMsg.String())
+
+		file, line := f.FileLine(info.pc)
+		t.Errorf(`LEAK of %d bytes FROM
+	%s+%x

Review Comment:
   ```suggestion
   ```



-- 
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] candiduslynx commented on a diff in pull request #35973: GH-35015: [Go] Fix parquet memleak

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


##########
go/arrow/memory/checked_allocator.go:
##########
@@ -160,14 +160,35 @@ func (a *CheckedAllocator) AssertSize(t TestingT, sz int) {
 				break
 			}
 			callersMsg.WriteString("\t")
-			callersMsg.WriteString(frame.Function)
-			callersMsg.WriteString(fmt.Sprintf(" line %d", frame.Line))
+			// frame.Func is a useful source of information if it's present.
+			// It may be nil for non-Go code or fully inlined functions.
+			if fn := frame.Func; fn != nil {
+				// format as func name + the offset in bytes from func entrypoint
+				callersMsg.WriteString(fmt.Sprintf("%s+%x", fn.Name(), frame.PC-fn.Entry()))
+			} else {
+				// fallback to outer func name + file line
+				callersMsg.WriteString(fmt.Sprintf("%s, line %d", frame.Function, frame.Line))
+			}
+
+			// Write a proper file name + line, so it's really easy to find the leak
+			callersMsg.WriteString("\n\t\t")
+			callersMsg.WriteString(frame.File + ":" + strconv.Itoa(frame.Line))
 			callersMsg.WriteString("\n")
 			if !more {
 				break
 			}
 		}
-		t.Errorf("LEAK of %d bytes FROM %s line %d\n%v", info.sz, f.Name(), info.line, callersMsg.String())
+
+		file, line := f.FileLine(info.pc)
+		t.Errorf(`LEAK of %d bytes FROM
+	%s+%x
+		%s:%d

Review Comment:
   ```suggestion
   ```



-- 
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] candiduslynx commented on a diff in pull request #35973: GH-35015: [Go] Fix parquet memleak

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


##########
go/arrow/memory/checked_allocator.go:
##########
@@ -160,22 +160,33 @@ func (a *CheckedAllocator) AssertSize(t TestingT, sz int) {
 				break
 			}
 			callersMsg.WriteString("\t")
+			// frame.Func is a useful source of information if it's present.
+			// Tt may be nil for non-Go code or fully inlined functions.
 			if fn := frame.Func; fn != nil {
+				// format as func name + the offset in bytes from func entrypoint
 				callersMsg.WriteString(fmt.Sprintf("%s+%x", fn.Name(), frame.PC-fn.Entry()))
 			} else {
+				// fallback to outer func name + file line
 				callersMsg.WriteString(fmt.Sprintf("%s, line %d", frame.Function, frame.Line))
 			}
+
+			// Write a proper file name + line, so it's really easy to find the leak
 			callersMsg.WriteString("\n\t\t")
 			callersMsg.WriteString(frame.File + ":" + strconv.Itoa(frame.Line))
 			callersMsg.WriteString("\n")
 			if !more {
 				break
 			}
 		}
+
 		file, line := f.FileLine(info.pc)
-		t.Errorf("LEAK of %d bytes FROM\n\t%s+%x\n\t\t%s:%d\n%v",
-			info.sz, f.Name(), info.pc-f.Entry(),
-			file, line,
+		t.Errorf(`LEAK of %d bytes FROM
+	%s+%x
+		%s:%d
+%v`,

Review Comment:
   it uses tabs, github tabs vary on your viewing settings in UI



-- 
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] candiduslynx commented on pull request #35973: GH-35015: [Go] Fix parquet memleak

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

   > Looks like you've got some tests failing with `Too many releases`, you can enable that check locally by using the `-tags assert` flag when you run `go test`
   
   Yeah, I forgot to remove the extra release for mem buffer (as I moved it to the initialization place): bf0e4a9196ef72748c384ee2c1f4a8d5966390f5


-- 
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] candiduslynx commented on a diff in pull request #35973: GH-35015: [Go] Fix parquet memleak

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


##########
go/arrow/memory/checked_allocator.go:
##########
@@ -160,14 +160,24 @@ func (a *CheckedAllocator) AssertSize(t TestingT, sz int) {
 				break
 			}
 			callersMsg.WriteString("\t")
-			callersMsg.WriteString(frame.Function)
-			callersMsg.WriteString(fmt.Sprintf(" line %d", frame.Line))
+			if fn := frame.Func; fn != nil {
+				callersMsg.WriteString(fmt.Sprintf("%s+%x", fn.Name(), frame.PC-fn.Entry()))
+			} else {
+				callersMsg.WriteString(fmt.Sprintf("%s, line %d", frame.Function, frame.Line))

Review Comment:
   commented in c95516aaef66bf4de3d925fde030eddbc5a78af7



##########
go/arrow/memory/checked_allocator.go:
##########
@@ -160,14 +160,24 @@ func (a *CheckedAllocator) AssertSize(t TestingT, sz int) {
 				break
 			}
 			callersMsg.WriteString("\t")
-			callersMsg.WriteString(frame.Function)
-			callersMsg.WriteString(fmt.Sprintf(" line %d", frame.Line))
+			if fn := frame.Func; fn != nil {
+				callersMsg.WriteString(fmt.Sprintf("%s+%x", fn.Name(), frame.PC-fn.Entry()))
+			} else {
+				callersMsg.WriteString(fmt.Sprintf("%s, line %d", frame.Function, frame.Line))
+			}
+			callersMsg.WriteString("\n\t\t")
+			callersMsg.WriteString(frame.File + ":" + strconv.Itoa(frame.Line))

Review Comment:
   Done in c95516aaef66bf4de3d925fde030eddbc5a78af7



-- 
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 #35973: GH-35015: [Go] Fix parquet memleak

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


-- 
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