You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "yevgenypats (via GitHub)" <gi...@apache.org> on 2023/04/08 14:35:18 UTC

[GitHub] [arrow] yevgenypats opened a new pull request, #34986: feat: Feat/append from string

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

   <!--
   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
   
   <!--
    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?
   
   <!--
   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)?
   -->
   
   ### Are there any user-facing changes?
   
   <!--
   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] zeroshade commented on a diff in pull request #34986: GH-34657: [Go] Add ValueString(i int) string to array

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


##########
go/arrow/array/binary.go:
##########
@@ -56,9 +57,17 @@ func (a *Binary) Value(i int) []byte {
 	return a.valueBytes[a.valueOffsets[idx]:a.valueOffsets[idx+1]]
 }
 
-// ValueString returns the string at index i without performing additional allocations.
-// The string is only valid for the lifetime of the Binary array.
+// ValueString returns the string at index i
 func (a *Binary) ValueString(i int) string {
+	if a.IsNull(i) {
+		return NullValueStr
+	}
+	return base64.StdEncoding.EncodeToString(a.Value(i))
+}
+
+// ValueStr returns the string at index i without performing additional allocations.
+// The string is only valid for the lifetime of the Binary array.
+func (a *Binary) ValueStr(i int) string {

Review Comment:
   Hmm i'd prefer to avoid the breaking change. Can we choose a different name for the function so we can avoid having to rename the `*Binary.ValueString` method?



-- 
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 #34986: GH-34657: [Go] Add ValueString(i int) string to array

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


##########
go/arrow/array/binary.go:
##########
@@ -56,9 +57,17 @@ func (a *Binary) Value(i int) []byte {
 	return a.valueBytes[a.valueOffsets[idx]:a.valueOffsets[idx+1]]
 }
 
-// ValueString returns the string at index i without performing additional allocations.
-// The string is only valid for the lifetime of the Binary array.
+// ValueString returns the string at index i
 func (a *Binary) ValueString(i int) string {
+	if a.IsNull(i) {
+		return NullValueStr
+	}
+	return base64.StdEncoding.EncodeToString(a.Value(i))

Review Comment:
   same comment as for LargeBinary, this seems backwards. `ValueString` should be returning the `*(*string)(unsafe.Pointer(&b))` and `ValueStr` should return the base64 encoded version



##########
go/arrow/array/binary.go:
##########
@@ -192,6 +201,12 @@ func (a *LargeBinary) Value(i int) []byte {
 }
 
 func (a *LargeBinary) ValueString(i int) string {
+	if a.IsNull(i) {
+		return NullValueStr
+	}
+	return base64.StdEncoding.EncodeToString(a.Value(i))

Review Comment:
   I think this is backwards. Shouldn't `ValueString` be implemented as `return *(*string)(unsafe.Pointer(&b))` while `ValueStr` returns the base64 encoded version?



##########
go/arrow/array/boolean.go:
##########
@@ -55,6 +55,14 @@ func (a *Boolean) Value(i int) bool {
 	return bitutil.BitIsSet(a.values, a.array.data.offset+i)
 }
 
+func (a *Boolean) ValueStr(i int) string {
+	if a.IsNull(i) {
+		return NullValueStr
+	} else {
+		return fmt.Sprintf("%v", a.Value(i))

Review Comment:
   `strconv.FormatBool` ?



##########
go/arrow/array/extension.go:
##########
@@ -187,7 +185,7 @@ func (e *ExtensionArrayBase) setData(data *Data) {
 
 // ValueString returns the value at index i as a string.

Review Comment:
   update the docstring comment to be `ValueStr`



##########
go/arrow/array/fixed_size_list.go:
##########
@@ -46,6 +46,13 @@ func NewFixedSizeListData(data arrow.ArrayData) *FixedSizeList {
 
 func (a *FixedSizeList) ListValues() arrow.Array { return a.values }
 
+func (a *FixedSizeList) ValueStr(i int) string {
+	if !a.IsValid(i) {
+		return NullValueStr
+	}
+	sub := a.newListValue(i)

Review Comment:
   this should have `defer sub.Release()`. can you add a test that uses `memory.CheckedAllocator` to construct this and then calls `ValueStr` to verify we don't have a memory leak?



##########
go/arrow/array/list.go:
##########
@@ -52,6 +52,13 @@ func NewListData(data arrow.ArrayData) *List {
 
 func (a *List) ListValues() arrow.Array { return a.values }
 
+func (a *List) ValueStr(i int) string {
+	if !a.IsValid(i) {
+		return NullValueStr
+	}
+	return fmt.Sprintf("%v", a.newListValue(i))

Review Comment:
   same comment as for FixedSizeList. needs to defer call release on the return from `newListValue` and please add a test that verifies no leaked memory



##########
go/arrow/array/numeric.gen.go.tmpl:
##########
@@ -82,6 +82,32 @@ func (a *{{.Name}}) setData(data *Data) {
 	}
 }
 
+func (a *{{.Name}}) ValueStr(i int) string {
+	if a.IsNull(i) {
+		return NullValueStr
+	}
+{{if or (eq .Name "Date32") (eq .Name "Date64") -}}
+	return a.values[i].ToTime().Format("2006-01-02")

Review Comment:
   these types have a `FormattedString()` method that you can call instead. just do `a.values[i].FormattedString()`



##########
go/arrow/array/numeric.gen.go.tmpl:
##########
@@ -82,6 +82,32 @@ func (a *{{.Name}}) setData(data *Data) {
 	}
 }
 
+func (a *{{.Name}}) ValueStr(i int) string {
+	if a.IsNull(i) {
+		return NullValueStr
+	}
+{{if or (eq .Name "Date32") (eq .Name "Date64") -}}
+	return a.values[i].ToTime().Format("2006-01-02")
+{{else if or (eq .Name "Time32") (eq .Name "Time64") -}}
+	return a.values[i].ToTime(a.DataType().(*{{.QualifiedType}}Type).Unit).Format("15:04:05.999999999")
+{{else if or (eq .Name "Timestamp") -}}
+	return a.values[i].ToTime(a.DataType().(*{{.QualifiedType}}Type).Unit).Format("2006-01-02 15:04:05.999999999")
+{{else if (eq .Name "Duration") -}}	
+	// return value and suffix as a string such as "12345ms"
+	return fmt.Sprintf("%d%s", a.values[i], a.DataType().(*{{.QualifiedType}}Type).Unit.String())	

Review Comment:
   you don't need the `.String()` since you're already using `%s` in the format



##########
go/arrow/array/numeric.gen.go.tmpl:
##########
@@ -82,6 +82,32 @@ func (a *{{.Name}}) setData(data *Data) {
 	}
 }
 
+func (a *{{.Name}}) ValueStr(i int) string {
+	if a.IsNull(i) {
+		return NullValueStr
+	}
+{{if or (eq .Name "Date32") (eq .Name "Date64") -}}
+	return a.values[i].ToTime().Format("2006-01-02")
+{{else if or (eq .Name "Time32") (eq .Name "Time64") -}}
+	return a.values[i].ToTime(a.DataType().(*{{.QualifiedType}}Type).Unit).Format("15:04:05.999999999")

Review Comment:
   `Time32` and `Time64` also have `FormattedString` methods which take the unit. so this should be:
   
   `return a.values[i].FormattedString(a.DataType().(*{{.QualifiedType}}Type).Unit)`



##########
go/arrow/array/numericbuilder.gen.go.tmpl:
##########
@@ -184,6 +184,124 @@ func (b *{{.Name}}Builder) newData() (data *Data) {
 	return
 }
 
+func (b *{{.Name}}Builder) AppendValueFromString(s string) error {
+	if s == NullValueStr {
+		b.AppendNull()
+		return nil
+	}
+  {{if or (eq .Name "Date32") -}}
+  	tm, err := time.Parse("2006-01-02", s)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(arrow.Date32FromTime(tm))
+  {{else if or (eq .Name "Date64") -}}
+  	tm, err := time.Parse("2006-01-02", s)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(arrow.Date64FromTime(tm))  
+  {{else if or (eq .Name "Time32") -}}
+  	val, err := arrow.Time32FromString(s, b.dtype.Unit)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(val)
+  {{else if or (eq .Name "Time64") -}}
+    val, err := arrow.Time64FromString(s, b.dtype.Unit)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(val)
+  {{else if or (eq .Name "Timestamp") -}}
+  	v, err := arrow.TimestampFromString(s, b.dtype.Unit)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(v)
+  {{else if (eq .Name "Duration") -}}	
+    return fmt.Errorf("AppendValueFromString not implemented for Duration")

Review Comment:
   please prefix the error with `%w` and use `arrow.ErrNotImplemented` so that we consistently wrap that error for this scenario.



##########
go/arrow/array/numericbuilder.gen.go.tmpl:
##########
@@ -184,6 +184,124 @@ func (b *{{.Name}}Builder) newData() (data *Data) {
 	return
 }
 
+func (b *{{.Name}}Builder) AppendValueFromString(s string) error {
+	if s == NullValueStr {
+		b.AppendNull()
+		return nil
+	}
+  {{if or (eq .Name "Date32") -}}
+  	tm, err := time.Parse("2006-01-02", s)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(arrow.Date32FromTime(tm))
+  {{else if or (eq .Name "Date64") -}}
+  	tm, err := time.Parse("2006-01-02", s)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(arrow.Date64FromTime(tm))  
+  {{else if or (eq .Name "Time32") -}}
+  	val, err := arrow.Time32FromString(s, b.dtype.Unit)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(val)
+  {{else if or (eq .Name "Time64") -}}
+    val, err := arrow.Time64FromString(s, b.dtype.Unit)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(val)
+  {{else if or (eq .Name "Timestamp") -}}
+  	v, err := arrow.TimestampFromString(s, b.dtype.Unit)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(v)
+  {{else if (eq .Name "Duration") -}}	
+    return fmt.Errorf("AppendValueFromString not implemented for Duration")
+  {{else if or (eq .Name "Int8") -}}
+    v, err := strconv.ParseInt(s, 10, 8)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(int8(v))

Review Comment:
   You can condense all of these to:
   
   ```
   v, err := strconv.ParseInt(s, 10, {{.Size}})
   if err != nil {
       b.AppendNull()
       return err
   }
   b.Append({{.name}}(v))
   ```
   
   instead of repeating it



##########
go/arrow/array/binary.go:
##########
@@ -105,7 +114,7 @@ func (a *Binary) String() string {
 		case a.IsNull(i):
 			o.WriteString("(null)")
 		default:
-			fmt.Fprintf(o, "%q", a.ValueString(i))
+			fmt.Fprintf(o, "%q", a.ValueStr(i))

Review Comment:
   this entire switch can probably be replaced with just calling `a.ValueStr(i)` right? since it already does the `a.IsNull(i)` check



##########
go/arrow/array/binarybuilder_test.go:
##########
@@ -50,11 +51,16 @@ func TestBinaryBuilder(t *testing.T) {
 		}
 		assert.Equal(t, v, ab.Value(i), "unexpected BinaryArrayBuilder.Value(%d)", i)
 	}
+	// Zm9v is foo in base64
+	assert.NoError(t, ab.AppendValueFromString("Zm9v"))
 
 	ar := ab.NewBinaryArray()
+	assert.Equal(t, "Zm9v", ar.ValueString(5))
+
 	ab.Release()
 	ar.Release()
 
+

Review Comment:
   just running `go fmt` would clear up the extraneous extra lines



##########
go/arrow/array/float16.go:
##########
@@ -39,6 +39,7 @@ func NewFloat16Data(data arrow.ArrayData) *Float16 {
 }
 
 func (a *Float16) Value(i int) float16.Num { return a.values[i] }
+func (a *Float16) ValueStr(i int) string { return fmt.Sprintf("%v", a.Value(i).Float32())}

Review Comment:
   `float16.Num` has a `String()` method. You can just return `a.values[i].String()`



##########
go/arrow/array/encoded.go:
##########
@@ -192,6 +193,15 @@ func (r *RunEndEncoded) GetPhysicalLength() int {
 	return encoded.GetPhysicalLength(r.data)
 }
 
+func (r *RunEndEncoded) ValueStr(i int) string {

Review Comment:
   Should this be taking a physical index or a logical index? I personally think this should be a logical index and should use `FindPhysicalIndex` to return `r.values.ValueStr(encoded.FindPhysicalIndex(r.Data(), i))` thoughts?



##########
go/arrow/array/string.go:
##########
@@ -188,6 +195,7 @@ func (a *LargeString) Value(i int) string {
 	i = i + a.array.data.offset
 	return a.values[a.offsets[i]:a.offsets[i+1]]
 }
+func (a *LargeString) ValueStr(i int) string { return fmt.Sprintf("%q", a.Value(i)) }

Review Comment:
   why not just return `a.Value(i)` ?



##########
go/arrow/array/binarybuilder_test.go:
##########
@@ -38,6 +38,7 @@ func TestBinaryBuilder(t *testing.T) {
 			ab.AppendNull()
 		} else {
 			ab.Append(v)
+			

Review Comment:
   unnecessary empty line



##########
go/arrow/array/list.go:
##########
@@ -175,6 +182,12 @@ func NewLargeListData(data arrow.ArrayData) *LargeList {
 
 func (a *LargeList) ListValues() arrow.Array { return a.values }
 
+func (a *LargeList) ValueStr(i int) string {
+	if !a.IsValid(i) {
+		return NullValueStr
+	}
+	return fmt.Sprintf("%v", a.newListValue(i))

Review Comment:
   same comment as above



##########
go/arrow/array/struct.go:
##########
@@ -351,6 +360,14 @@ func (b *StructBuilder) newData() (data *Data) {
 	return
 }
 
+func (b *StructBuilder) AppendValueFromString(s string) error {
+	if !strings.HasPrefix(s, "{") && !strings.HasSuffix(s, "}") {
+		return fmt.Errorf("invalid string for struct")

Review Comment:
   add `%w` and wrap `arrow.ErrInvalid` please



##########
go/arrow/array/list.go:
##########
@@ -416,6 +429,9 @@ func (b *baseListBuilder) AppendEmptyValue() {
 	b.Append(true)
 }
 
+// func (b *ListBuilder) AppendFromString(s string) {
+// 	b.AppendFromStringArray([]string{s})
+// }

Review Comment:
   remove commented code?



##########
go/arrow/array/map.go:
##########
@@ -300,6 +300,10 @@ func (b *MapBuilder) ValueBuilder() Builder {
 	return b.listBuilder.ValueBuilder()
 }
 
+func (b *MapBuilder) AppendValueFromString(s string) error {
+	panic("not implemented")	

Review Comment:
   can we just return `arrow.ErrNotImplemented` instead of panicing?



##########
go/arrow/array/numeric.gen.go.tmpl:
##########
@@ -82,6 +82,32 @@ func (a *{{.Name}}) setData(data *Data) {
 	}
 }
 
+func (a *{{.Name}}) ValueStr(i int) string {
+	if a.IsNull(i) {
+		return NullValueStr
+	}
+{{if or (eq .Name "Date32") (eq .Name "Date64") -}}
+	return a.values[i].ToTime().Format("2006-01-02")
+{{else if or (eq .Name "Time32") (eq .Name "Time64") -}}
+	return a.values[i].ToTime(a.DataType().(*{{.QualifiedType}}Type).Unit).Format("15:04:05.999999999")
+{{else if or (eq .Name "Timestamp") -}}
+	return a.values[i].ToTime(a.DataType().(*{{.QualifiedType}}Type).Unit).Format("2006-01-02 15:04:05.999999999")
+{{else if (eq .Name "Duration") -}}	
+	// return value and suffix as a string such as "12345ms"
+	return fmt.Sprintf("%d%s", a.values[i], a.DataType().(*{{.QualifiedType}}Type).Unit.String())	
+{{else if or (eq .Name "Int8") (eq .Name "Int16") (eq .Name "Int32") (eq .Name "Int64") -}}
+  return strconv.FormatInt(int64(a.Value(i)), 10)
+{{else if or (eq .Name "Uint8") (eq .Name "Uint16") (eq .Name "Uint32") (eq .Name "Uint64") -}}
+  return strconv.FormatUint(uint64(a.Value(i)), 10)

Review Comment:
   any reason not to just use `fmt.Sprintf("%d", a.values[i])`



##########
go/arrow/array/numericbuilder.gen.go.tmpl:
##########
@@ -184,6 +184,124 @@ func (b *{{.Name}}Builder) newData() (data *Data) {
 	return
 }
 
+func (b *{{.Name}}Builder) AppendValueFromString(s string) error {
+	if s == NullValueStr {
+		b.AppendNull()
+		return nil
+	}
+  {{if or (eq .Name "Date32") -}}
+  	tm, err := time.Parse("2006-01-02", s)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(arrow.Date32FromTime(tm))

Review Comment:
   fix the indentation formatting please



##########
go/arrow/array/numericbuilder.gen_test.go.tmpl:
##########
@@ -51,9 +51,10 @@ func TestNew{{.Name}}Builder(t *testing.T) {
 	ab.Append(8)
 	ab.Append(9)
 	ab.Append(10)
+  ab.AppendValueFromString(11)

Review Comment:
   indentation



##########
go/arrow/array/struct.go:
##########
@@ -81,6 +81,15 @@ func NewStructData(data arrow.ArrayData) *Struct {
 func (a *Struct) NumField() int           { return len(a.fields) }
 func (a *Struct) Field(i int) arrow.Array { return a.fields[i] }
 
+func (a *Struct) ValueStr(i int) string {
+	var buf bytes.Buffer
+	enc := json.NewEncoder(&buf)
+	if err := enc.Encode(a.GetOneForMarshal(i)); err != nil {
+		return ""

Review Comment:
   we shouldn't swallow the error. either return `err.String()` or `panic(err)` please



##########
go/arrow/array/numericbuilder.gen.go.tmpl:
##########
@@ -184,6 +184,124 @@ func (b *{{.Name}}Builder) newData() (data *Data) {
 	return
 }
 
+func (b *{{.Name}}Builder) AppendValueFromString(s string) error {
+	if s == NullValueStr {
+		b.AppendNull()
+		return nil
+	}
+  {{if or (eq .Name "Date32") -}}
+  	tm, err := time.Parse("2006-01-02", s)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(arrow.Date32FromTime(tm))
+  {{else if or (eq .Name "Date64") -}}
+  	tm, err := time.Parse("2006-01-02", s)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(arrow.Date64FromTime(tm))  
+  {{else if or (eq .Name "Time32") -}}
+  	val, err := arrow.Time32FromString(s, b.dtype.Unit)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(val)
+  {{else if or (eq .Name "Time64") -}}
+    val, err := arrow.Time64FromString(s, b.dtype.Unit)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(val)
+  {{else if or (eq .Name "Timestamp") -}}
+  	v, err := arrow.TimestampFromString(s, b.dtype.Unit)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(v)
+  {{else if (eq .Name "Duration") -}}	
+    return fmt.Errorf("AppendValueFromString not implemented for Duration")
+  {{else if or (eq .Name "Int8") -}}
+    v, err := strconv.ParseInt(s, 10, 8)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(int8(v))
+  {{else if or (eq .Name "Int16") -}}
+    v, err := strconv.ParseInt(s, 10, 16)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(int16(v))
+  {{else if or (eq .Name "Int32") -}}
+    v, err := strconv.ParseInt(s, 10, 32)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(int32(v))
+  {{else if or (eq .Name "Int64") -}}
+    v, err := strconv.ParseInt(s, 10, 64)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(int64(v))
+  {{else if or (eq .Name "Uint8") -}}
+  	v, err := strconv.ParseUint(s, 10, 8)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(uint8(v))
+  {{else if or (eq .Name "Uint16") -}}
+  	v, err := strconv.ParseUint(s, 10, 16)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(uint16(v))
+  {{else if or (eq .Name "Uint32") -}}
+  	v, err := strconv.ParseUint(s, 10, 32)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(uint32(v))
+  {{else if or (eq .Name "Uint64") -}}
+  	v, err := strconv.ParseUint(s, 10, 64)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(uint64(v))
+  {{else if or (eq .Name "Float32") -}}
+    v, err := strconv.ParseFloat(s, 32)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(float32(v))
+  {{else if or (eq .Name "Float64") -}}
+    v, err := strconv.ParseFloat(s, 64)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(float64(v))

Review Comment:
   same thing as with the integral types, you can condense these by using `{{.Size}}` and `{{.name}}`



##########
go/arrow/array/numericbuilder.gen.go.tmpl:
##########
@@ -184,6 +184,124 @@ func (b *{{.Name}}Builder) newData() (data *Data) {
 	return
 }
 
+func (b *{{.Name}}Builder) AppendValueFromString(s string) error {
+	if s == NullValueStr {
+		b.AppendNull()
+		return nil
+	}
+  {{if or (eq .Name "Date32") -}}
+  	tm, err := time.Parse("2006-01-02", s)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(arrow.Date32FromTime(tm))
+  {{else if or (eq .Name "Date64") -}}
+  	tm, err := time.Parse("2006-01-02", s)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(arrow.Date64FromTime(tm))  
+  {{else if or (eq .Name "Time32") -}}
+  	val, err := arrow.Time32FromString(s, b.dtype.Unit)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(val)
+  {{else if or (eq .Name "Time64") -}}
+    val, err := arrow.Time64FromString(s, b.dtype.Unit)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(val)
+  {{else if or (eq .Name "Timestamp") -}}
+  	v, err := arrow.TimestampFromString(s, b.dtype.Unit)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(v)
+  {{else if (eq .Name "Duration") -}}	
+    return fmt.Errorf("AppendValueFromString not implemented for Duration")
+  {{else if or (eq .Name "Int8") -}}
+    v, err := strconv.ParseInt(s, 10, 8)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(int8(v))
+  {{else if or (eq .Name "Int16") -}}
+    v, err := strconv.ParseInt(s, 10, 16)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(int16(v))
+  {{else if or (eq .Name "Int32") -}}
+    v, err := strconv.ParseInt(s, 10, 32)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(int32(v))
+  {{else if or (eq .Name "Int64") -}}
+    v, err := strconv.ParseInt(s, 10, 64)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(int64(v))
+  {{else if or (eq .Name "Uint8") -}}
+  	v, err := strconv.ParseUint(s, 10, 8)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(uint8(v))
+  {{else if or (eq .Name "Uint16") -}}
+  	v, err := strconv.ParseUint(s, 10, 16)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(uint16(v))
+  {{else if or (eq .Name "Uint32") -}}
+  	v, err := strconv.ParseUint(s, 10, 32)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(uint32(v))
+  {{else if or (eq .Name "Uint64") -}}
+  	v, err := strconv.ParseUint(s, 10, 64)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(uint64(v))
+  {{else if or (eq .Name "Float32") -}}
+    v, err := strconv.ParseFloat(s, 32)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(float32(v))
+  {{else if or (eq .Name "Float64") -}}
+    v, err := strconv.ParseFloat(s, 64)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(float64(v))
+  {{else}}
+    return fmt.Errorf("AppendValueFromString not implemented for {{.Name}}")

Review Comment:
   same comment as above, use `%w` and wrap the `arrow.ErrNotImplemented`



##########
go/arrow/array/struct.go:
##########
@@ -81,6 +81,15 @@ func NewStructData(data arrow.ArrayData) *Struct {
 func (a *Struct) NumField() int           { return len(a.fields) }
 func (a *Struct) Field(i int) arrow.Array { return a.fields[i] }
 
+func (a *Struct) ValueStr(i int) string {

Review Comment:
   add a docstring comment on here to indicate that the value is returned as a JSON string please.



##########
go/arrow/array/union.go:
##########
@@ -343,6 +343,21 @@ func (a *SparseUnion) MarshalJSON() ([]byte, error) {
 	return buf.Bytes(), nil
 }
 
+func (a *SparseUnion) Value(i int) string {
+	fieldList := a.unionType.Fields()
+	field := fieldList[a.ChildID(i)]
+	f := a.Field(a.ChildID(i))
+	return fmt.Sprintf("{%s=%v}", field.Name, f.GetOneForMarshal(i))
+}

Review Comment:
   I think this is unnecesary. having the `ValueStr` 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] yevgenypats commented on a diff in pull request #34986: GH-34657: [Go] Add ValueString(i int) string to array

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


##########
go/arrow/array/numeric.gen.go.tmpl:
##########
@@ -82,6 +82,32 @@ func (a *{{.Name}}) setData(data *Data) {
 	}
 }
 
+func (a *{{.Name}}) ValueStr(i int) string {
+	if a.IsNull(i) {
+		return NullValueStr
+	}
+{{if or (eq .Name "Date32") (eq .Name "Date64") -}}
+	return a.values[i].ToTime().Format("2006-01-02")
+{{else if or (eq .Name "Time32") (eq .Name "Time64") -}}
+	return a.values[i].ToTime(a.DataType().(*{{.QualifiedType}}Type).Unit).Format("15:04:05.999999999")
+{{else if or (eq .Name "Timestamp") -}}
+	return a.values[i].ToTime(a.DataType().(*{{.QualifiedType}}Type).Unit).Format("2006-01-02 15:04:05.999999999")
+{{else if (eq .Name "Duration") -}}	
+	// return value and suffix as a string such as "12345ms"
+	return fmt.Sprintf("%d%s", a.values[i], a.DataType().(*{{.QualifiedType}}Type).Unit.String())	
+{{else if or (eq .Name "Int8") (eq .Name "Int16") (eq .Name "Int32") (eq .Name "Int64") -}}
+  return strconv.FormatInt(int64(a.Value(i)), 10)
+{{else if or (eq .Name "Uint8") (eq .Name "Uint16") (eq .Name "Uint32") (eq .Name "Uint64") -}}
+  return strconv.FormatUint(uint64(a.Value(i)), 10)

Review Comment:
   not sure. I think it's more efficient and we also do the same in CSV reader/writer but I don't mind changing 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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] zeroshade merged pull request #34986: GH-34657: [Go] Add ValueString(i int) string to array

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


-- 
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] yevgenypats commented on a diff in pull request #34986: GH-34657: [Go] Add ValueString(i int) string to array

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


##########
go/arrow/array/binary.go:
##########
@@ -105,7 +114,7 @@ func (a *Binary) String() string {
 		case a.IsNull(i):
 			o.WriteString("(null)")
 		default:
-			fmt.Fprintf(o, "%q", a.ValueString(i))
+			fmt.Fprintf(o, "%q", a.ValueStr(i))

Review Comment:
   I just kept it the same. as it was. also returned it to be `ValueString` as it doesn't need any change



-- 
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 #34986: GH-34657: [Go] Add ValueString(i int) string to array

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


##########
go/arrow/array/binarybuilder.go:
##########
@@ -289,6 +289,26 @@ func (b *BinaryBuilder) appendNextOffset() {
 	b.appendOffsetVal(numBytes)
 }
 
+func (b *BinaryBuilder) AppendValueFromString(s string) error {
+	if s == NullValueStr {
+		b.AppendNull()
+		return nil
+	}
+	switch b.dtype.ID() {
+	case arrow.BINARY, arrow.LARGE_BINARY:
+		decodedVal, err := base64.StdEncoding.DecodeString(s)
+		if err != nil {
+			return fmt.Errorf("could not decode base64 string: %w", err)
+		}

Review Comment:
   that's fair. I suppose if we run into an issue for individuals that are using non-padded base64 for this they'll file an issue and it's fairly easy to add the support. so this 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] yevgenypats commented on a diff in pull request #34986: GH-34657: [Go] Add ValueString(i int) string to array

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


##########
go/arrow/array/binary.go:
##########
@@ -56,9 +57,17 @@ func (a *Binary) Value(i int) []byte {
 	return a.valueBytes[a.valueOffsets[idx]:a.valueOffsets[idx+1]]
 }
 
-// ValueString returns the string at index i without performing additional allocations.
-// The string is only valid for the lifetime of the Binary array.
+// ValueString returns the string at index i
 func (a *Binary) ValueString(i int) string {
+	if a.IsNull(i) {
+		return NullValueStr
+	}
+	return base64.StdEncoding.EncodeToString(a.Value(i))
+}
+
+// ValueStr returns the string at index i without performing additional allocations.
+// The string is only valid for the lifetime of the Binary array.
+func (a *Binary) ValueStr(i int) string {

Review Comment:
   Interesting. I prefer though to make it a public function `ValueStr` and the reverse `AppendFromValueStr` so it's easy to test it. I did call `GetOneForMarshal` in some of the `ValueStr` implementation where it made sense but I think in any case better to expose a proper function. 



-- 
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 #34986: GH-34657: [Go] Add ValueString(i int) string to array

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


##########
go/arrow/array/numericbuilder.gen.go.tmpl:
##########
@@ -184,6 +184,124 @@ func (b *{{.Name}}Builder) newData() (data *Data) {
 	return
 }
 
+func (b *{{.Name}}Builder) AppendValueFromString(s string) error {
+	if s == NullValueStr {
+		b.AppendNull()
+		return nil
+	}
+  {{if or (eq .Name "Date32") -}}
+  	tm, err := time.Parse("2006-01-02", s)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(arrow.Date32FromTime(tm))

Review Comment:
   I'm fine with converting the entire file to spaces. Go for 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: 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 #34986: GH-34657: [Go] Add ValueString(i int) string to array

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

   Benchmark runs are scheduled for baseline = ff94702eee6ee419329d32d86c14bf0bf0c244f3 and contender = f0c8229f5a09fe53186df171d518430243ddf112. f0c8229f5a09fe53186df171d518430243ddf112 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/42d5864f31694dd9acdee18949d31cc4...834452df7b204ae8968b5618ec22bbfd/)
   [Failed] [test-mac-arm](https://conbench.ursa.dev/compare/runs/1af69ab61ad145b78d4fe38f07be03dc...ddee5ff5510b434eafadeeb8e992e7cf/)
   [Finished :arrow_down:7.14% :arrow_up:0.0%] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/acf048d615a44a3e86cc7cbc3873e23c...1be141034b5649728420c12ada7c87ef/)
   [Finished :arrow_down:0.49% :arrow_up:0.09%] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/bf31db6dc3bb46cab087f7298a602870...bf771adda7f044faafbc6d7a03ea3aaa/)
   Buildkite builds:
   [Finished] [`f0c8229f` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/2711)
   [Failed] [`f0c8229f` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/2745)
   [Finished] [`f0c8229f` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/2709)
   [Finished] [`f0c8229f` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/2736)
   [Finished] [`ff94702e` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/2710)
   [Failed] [`ff94702e` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/2744)
   [Finished] [`ff94702e` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/2708)
   [Finished] [`ff94702e` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/2735)
   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] github-actions[bot] commented on pull request #34986: GH-34657: [Go] Add ValueString(i int) string to array

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

   * Closes: #34657


-- 
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] yevgenypats commented on a diff in pull request #34986: GH-34657: [Go] Add ValueString(i int) string to array

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


##########
go/arrow/array/binarybuilder.go:
##########
@@ -289,6 +289,26 @@ func (b *BinaryBuilder) appendNextOffset() {
 	b.appendOffsetVal(numBytes)
 }
 
+func (b *BinaryBuilder) AppendValueFromString(s string) error {
+	if s == NullValueStr {
+		b.AppendNull()
+		return nil
+	}
+	switch b.dtype.ID() {
+	case arrow.BINARY, arrow.LARGE_BINARY:
+		decodedVal, err := base64.StdEncoding.DecodeString(s)
+		if err != nil {
+			return fmt.Errorf("could not decode base64 string: %w", err)
+		}

Review Comment:
   hmm. I think we should try to always use `std` otherwise this fallback will propagate everywhere because we want catch if someone encoded something with `RawStdEncoding` instead of `StdEncoding`. 



-- 
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 #34986: GH-34657: [Go] Add ValueString(i int) string to array

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


##########
go/arrow/array/binarybuilder.go:
##########
@@ -289,6 +289,26 @@ func (b *BinaryBuilder) appendNextOffset() {
 	b.appendOffsetVal(numBytes)
 }
 
+func (b *BinaryBuilder) AppendValueFromString(s string) error {
+	if s == NullValueStr {
+		b.AppendNull()
+		return nil
+	}
+	switch b.dtype.ID() {
+	case arrow.BINARY, arrow.LARGE_BINARY:
+		decodedVal, err := base64.StdEncoding.DecodeString(s)
+		if err != nil {
+			return fmt.Errorf("could not decode base64 string: %w", err)
+		}

Review Comment:
   do we want to try falling back to `RawStdEncoding` if this errors so we can handle both situations of padding or no padding?



-- 
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] yevgenypats commented on a diff in pull request #34986: GH-34657: [Go] Add ValueString(i int) string to array

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


##########
go/arrow/array/encoded.go:
##########
@@ -192,6 +193,15 @@ func (r *RunEndEncoded) GetPhysicalLength() int {
 	return encoded.GetPhysicalLength(r.data)
 }
 
+func (r *RunEndEncoded) ValueStr(i int) string {

Review Comment:
   hmm. maybe I should just use then `json.Marshal(r.GetOneForMarshal(i))` for consistency in this case?



-- 
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 #34986: GH-34657: [Go] Add ValueString(i int) string to array

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


##########
go/arrow/array/binary.go:
##########
@@ -56,9 +57,17 @@ func (a *Binary) Value(i int) []byte {
 	return a.valueBytes[a.valueOffsets[idx]:a.valueOffsets[idx+1]]
 }
 
-// ValueString returns the string at index i without performing additional allocations.
-// The string is only valid for the lifetime of the Binary array.
+// ValueString returns the string at index i
 func (a *Binary) ValueString(i int) string {
+	if a.IsNull(i) {
+		return NullValueStr
+	}
+	return base64.StdEncoding.EncodeToString(a.Value(i))
+}
+
+// ValueStr returns the string at index i without performing additional allocations.
+// The string is only valid for the lifetime of the Binary array.
+func (a *Binary) ValueStr(i int) string {

Review Comment:
   You missed a few spots for `arraymarshal`:
   ![image](https://user-images.githubusercontent.com/555095/231020740-14e46721-30ce-4419-992e-07362e607771.png)
   



-- 
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 #34986: GH-34657: [Go] Add ValueString(i int) string to array

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


##########
go/arrow/array/dictionary.go:
##########
@@ -737,6 +741,10 @@ func (b *dictionaryBuilder) Unmarshal(dec *json.Decoder) error {
 	return b.AppendArray(arr)
 }
 
+func (b *dictionaryBuilder) AppendValueFromString(s string) error {
+	return errors.New("AppendValueFromString to dictionary not yet implemented")

Review Comment:
   add "%w" and `arrow.ErrNotImplemented` please using `fmt.Errorf`



-- 
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 #34986: GH-34657: [Go] Add ValueString(i int) string to array

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


##########
go/arrow/array/numericbuilder.gen.go.tmpl:
##########
@@ -225,79 +225,30 @@ func (b *{{.Name}}Builder) AppendValueFromString(s string) error {
     }
     b.Append(v)
   {{else if (eq .Name "Duration") -}}	
-    return fmt.Errorf("AppendValueFromString not implemented for Duration")
-  {{else if or (eq .Name "Int8") -}}
-    v, err := strconv.ParseInt(s, 10, 8)
+    return fmt.Errorf("%w: AppendValueFromString not implemented for Duration", ErrNotImplemented)
+  {{else if or (eq .Name "Int8") (eq .Name "Int16") (eq .Name "Int32") (eq .Name "Int64") -}}
+    v, err := strconv.ParseInt(s, 10, {{.Size}})
     if err != nil {
-      b.AppendNull()
-      return err
-    }
-    b.Append(int8(v))
-  {{else if or (eq .Name "Int16") -}}
-    v, err := strconv.ParseInt(s, 10, 16)
-    if err != nil {
-      b.AppendNull()
-      return err
-    }
-    b.Append(int16(v))
-  {{else if or (eq .Name "Int32") -}}
-    v, err := strconv.ParseInt(s, 10, 32)
-    if err != nil {
-      b.AppendNull()
-      return err
-    }
-    b.Append(int32(v))
-  {{else if or (eq .Name "Int64") -}}
-    v, err := strconv.ParseInt(s, 10, 64)
-    if err != nil {
-      b.AppendNull()
-      return err
-    }
-    b.Append(int64(v))
-  {{else if or (eq .Name "Uint8") -}}
-  	v, err := strconv.ParseUint(s, 10, 8)
-    if err != nil {
-      b.AppendNull()
-      return err
-    }
-    b.Append(uint8(v))
-  {{else if or (eq .Name "Uint16") -}}
-  	v, err := strconv.ParseUint(s, 10, 16)
-    if err != nil {
-      b.AppendNull()
-      return err
-    }
-    b.Append(uint16(v))
-  {{else if or (eq .Name "Uint32") -}}
-  	v, err := strconv.ParseUint(s, 10, 32)
-    if err != nil {
-      b.AppendNull()
-      return err
-    }
-    b.Append(uint32(v))
-  {{else if or (eq .Name "Uint64") -}}
-  	v, err := strconv.ParseUint(s, 10, 64)
-    if err != nil {
-      b.AppendNull()
-      return err
+        b.AppendNull()
+        return err
     }
-    b.Append(uint64(v))
-  {{else if or (eq .Name "Float32") -}}
-    v, err := strconv.ParseFloat(s, 32)
+    b.Append({{.name}}(v))
+  {{else if or (eq .Name "Uint8") (eq .Name "Uint16") (eq .Name "Uint32") (eq .Name "Uint64") -}}
+    v, err := strconv.ParseInt(s, 10, {{.Size}})

Review Comment:
   `ParseUint` ?



-- 
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 #34986: GH-34657: [Go] Add ValueString(i int) string to array

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

   @yevgenypats I've resolved the outdated issues you've fixed but there's still outstanding changes that need to be made such as https://github.com/apache/arrow/pull/34986#discussion_r1162987868 and the other related comments.


-- 
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 #34986: GH-34657: [Go] Add ValueString(i int) string to array

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


##########
go/arrow/array/builder.go:
##########
@@ -82,6 +85,7 @@ type Builder interface {
 	UnmarshalOne(*json.Decoder) error
 	Unmarshal(*json.Decoder) error
 
+

Review Comment:
   extraneous line



-- 
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] yevgenypats commented on pull request #34986: GH-34657: [Go] Add ValueString(i int) string to array

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

   Thanks @zeroshade ! I addressed other comments. I only have one outstanding question for the template indentation and yeah if your or someone would be able to set the CI step that that does that automatically would be awesome as Im not super familiar with the arrow CI (so someone knowledgable would be able to do that faster I think - Opened issue - https://github.com/apache/arrow/issues/35079 )


-- 
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 #34986: GH-34657: [Go] Add ValueString(i int) string to array

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


##########
go/arrow/array/binarybuilder_test.go:
##########
@@ -50,11 +51,16 @@ func TestBinaryBuilder(t *testing.T) {
 		}
 		assert.Equal(t, v, ab.Value(i), "unexpected BinaryArrayBuilder.Value(%d)", i)
 	}
+	// Zm9v is foo in base64
+	assert.NoError(t, ab.AppendValueFromString("Zm9v"))
 
 	ar := ab.NewBinaryArray()
+	assert.Equal(t, "Zm9v", ar.ValueString(5))
+
 	ab.Release()
 	ar.Release()
 
+

Review Comment:
   I think that it would definitely be a good idea to add that to the existing linting. Right now the linter just runs `staticcheck`, but doesn't check the formatting.



-- 
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] yevgenypats commented on a diff in pull request #34986: GH-34657: [Go] Add ValueString(i int) string to array

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


##########
go/arrow/array/numericbuilder.gen.go.tmpl:
##########
@@ -184,6 +184,124 @@ func (b *{{.Name}}Builder) newData() (data *Data) {
 	return
 }
 
+func (b *{{.Name}}Builder) AppendValueFromString(s string) error {
+	if s == NullValueStr {
+		b.AppendNull()
+		return nil
+	}
+  {{if or (eq .Name "Date32") -}}
+  	tm, err := time.Parse("2006-01-02", s)
+    if err != nil {
+      b.AppendNull()
+      return err
+    }
+    b.Append(arrow.Date32FromTime(tm))

Review Comment:
   The problem with the indentation here is because in this file we use tabs instead of spaces. should we all turn it into spaces or should I keep using tabs in this file while in all `.go` files we do use spaces I believe. wdyt?



-- 
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] yevgenypats commented on a diff in pull request #34986: GH-34657: [Go] Add ValueString(i int) string to array

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


##########
go/arrow/array/binary.go:
##########
@@ -56,9 +57,17 @@ func (a *Binary) Value(i int) []byte {
 	return a.valueBytes[a.valueOffsets[idx]:a.valueOffsets[idx+1]]
 }
 
-// ValueString returns the string at index i without performing additional allocations.
-// The string is only valid for the lifetime of the Binary array.
+// ValueString returns the string at index i
 func (a *Binary) ValueString(i int) string {
+	if a.IsNull(i) {
+		return NullValueStr
+	}
+	return base64.StdEncoding.EncodeToString(a.Value(i))
+}
+
+// ValueStr returns the string at index i without performing additional allocations.
+// The string is only valid for the lifetime of the Binary array.
+func (a *Binary) ValueStr(i int) string {

Review Comment:
   @zeroshade  I can do the other way around. `ValueStr` for the new function? WDYT?



-- 
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 #34986: GH-34657: [Go] Add ValueString(i int) string to array

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


##########
go/arrow/array/binary.go:
##########
@@ -56,9 +57,17 @@ func (a *Binary) Value(i int) []byte {
 	return a.valueBytes[a.valueOffsets[idx]:a.valueOffsets[idx+1]]
 }
 
-// ValueString returns the string at index i without performing additional allocations.
-// The string is only valid for the lifetime of the Binary array.
+// ValueString returns the string at index i
 func (a *Binary) ValueString(i int) string {
+	if a.IsNull(i) {
+		return NullValueStr
+	}
+	return base64.StdEncoding.EncodeToString(a.Value(i))
+}
+
+// ValueStr returns the string at index i without performing additional allocations.
+// The string is only valid for the lifetime of the Binary array.
+func (a *Binary) ValueStr(i int) string {

Review Comment:
   That's fine for me. Alternately, now that we have `GetOneForMarshal` publicly available, in theory with some trickery it should be pretty easy to leverage that and some strong typing with `String` methods we might not even need this `ValueStr` method but instead could simply just be able to `fmt.Sprint(GetOneForMarshal(...))` to get text. what do you think?



-- 
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] yevgenypats commented on a diff in pull request #34986: GH-34657: [Go] Add ValueString(i int) string to array

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


##########
go/arrow/array/struct_test.go:
##########
@@ -290,15 +291,17 @@ func TestStructArrayStringer(t *testing.T) {
 			f2b.Append(f2s[i])
 		}
 	}
-
+	assert.NoError(t, sb.AppendValueFromString(`{"f1": 1.1, "f2": 1}`))
 	arr := sb.NewArray().(*array.Struct)
 	defer arr.Release()
 
-	want := "{[1.1 (null) 1.3 1.4] [1 2 (null) 4]}"
+	assert.Equal(t, "{\"f1\":1.1,\"f2\":1}\n", arr.ValueStr(4))
+	want := "{[1.1 (null) 1.3 1.4 1.1] [1 2 (null) 4 1]}"
 	got := arr.String()
 	if got != want {
 		t.Fatalf("invalid string representation:\ngot = %q\nwant= %q", got, want)
 	}
+	// assert.Equal(t, "[1.1 (null) 1.3 1.4]", arr.ValueStr(0))

Review Comment:
   Removed. I already test the method on line `298`



-- 
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 #34986: GH-34657: [Go] Add ValueString(i int) string to array

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

   ['Python', 'R'] benchmarks have high level of regressions.
   [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/acf048d615a44a3e86cc7cbc3873e23c...1be141034b5649728420c12ada7c87ef/)
   


-- 
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] yevgenypats commented on a diff in pull request #34986: GH-34657: [Go] Add ValueString(i int) string to array

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


##########
go/arrow/array/binary.go:
##########
@@ -56,9 +57,17 @@ func (a *Binary) Value(i int) []byte {
 	return a.valueBytes[a.valueOffsets[idx]:a.valueOffsets[idx+1]]
 }
 
-// ValueString returns the string at index i without performing additional allocations.
-// The string is only valid for the lifetime of the Binary array.
+// ValueString returns the string at index i
 func (a *Binary) ValueString(i int) string {
+	if a.IsNull(i) {
+		return NullValueStr
+	}
+	return base64.StdEncoding.EncodeToString(a.Value(i))
+}
+
+// ValueStr returns the string at index i without performing additional allocations.
+// The string is only valid for the lifetime of the Binary array.
+func (a *Binary) ValueStr(i int) string {

Review Comment:
   Ok seems I was missing a rebase. fixed 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] yevgenypats commented on a diff in pull request #34986: GH-34657: [Go] Add ValueString(i int) string to array

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


##########
go/arrow/array/binary.go:
##########
@@ -56,9 +57,17 @@ func (a *Binary) Value(i int) []byte {
 	return a.valueBytes[a.valueOffsets[idx]:a.valueOffsets[idx+1]]
 }
 
-// ValueString returns the string at index i without performing additional allocations.
-// The string is only valid for the lifetime of the Binary array.
+// ValueString returns the string at index i
 func (a *Binary) ValueString(i int) string {
+	if a.IsNull(i) {
+		return NullValueStr
+	}
+	return base64.StdEncoding.EncodeToString(a.Value(i))

Review Comment:
   Yeah good catch. fixed.



-- 
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 #34986: GH-34657: [Go] Add ValueString(i int) string to array

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


##########
go/arrow/array/numeric.gen.go.tmpl:
##########
@@ -82,6 +82,32 @@ func (a *{{.Name}}) setData(data *Data) {
 	}
 }
 
+func (a *{{.Name}}) ValueStr(i int) string {
+	if a.IsNull(i) {
+		return NullValueStr
+	}
+{{if or (eq .Name "Date32") (eq .Name "Date64") -}}
+	return a.values[i].ToTime().Format("2006-01-02")
+{{else if or (eq .Name "Time32") (eq .Name "Time64") -}}
+	return a.values[i].ToTime(a.DataType().(*{{.QualifiedType}}Type).Unit).Format("15:04:05.999999999")
+{{else if or (eq .Name "Timestamp") -}}
+	return a.values[i].ToTime(a.DataType().(*{{.QualifiedType}}Type).Unit).Format("2006-01-02 15:04:05.999999999")
+{{else if (eq .Name "Duration") -}}	
+	// return value and suffix as a string such as "12345ms"
+	return fmt.Sprintf("%d%s", a.values[i], a.DataType().(*{{.QualifiedType}}Type).Unit.String())	
+{{else if or (eq .Name "Int8") (eq .Name "Int16") (eq .Name "Int32") (eq .Name "Int64") -}}
+  return strconv.FormatInt(int64(a.Value(i)), 10)
+{{else if or (eq .Name "Uint8") (eq .Name "Uint16") (eq .Name "Uint32") (eq .Name "Uint64") -}}
+  return strconv.FormatUint(uint64(a.Value(i)), 10)

Review Comment:
   i just like condensing this where possible :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 #34986: GH-34657: [Go] Add ValueString(i int) string to array

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


##########
go/arrow/array/struct_test.go:
##########
@@ -290,15 +291,17 @@ func TestStructArrayStringer(t *testing.T) {
 			f2b.Append(f2s[i])
 		}
 	}
-
+	assert.NoError(t, sb.AppendValueFromString(`{"f1": 1.1, "f2": 1}`))
 	arr := sb.NewArray().(*array.Struct)
 	defer arr.Release()
 
-	want := "{[1.1 (null) 1.3 1.4] [1 2 (null) 4]}"
+	assert.Equal(t, "{\"f1\":1.1,\"f2\":1}\n", arr.ValueStr(4))
+	want := "{[1.1 (null) 1.3 1.4 1.1] [1 2 (null) 4 1]}"
 	got := arr.String()
 	if got != want {
 		t.Fatalf("invalid string representation:\ngot = %q\nwant= %q", got, want)
 	}
+	// assert.Equal(t, "[1.1 (null) 1.3 1.4]", arr.ValueStr(0))

Review Comment:
   why is this commented out?



-- 
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 #34986: feat: Feat/append from string

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

   <!--
     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.
   -->
   
   Thanks for opening a pull request!
   
   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}
   
   See also:
   
     * [Other pull requests](https://github.com/apache/arrow/pulls/)
     * [Contribution Guidelines - How to contribute patches](https://arrow.apache.org/docs/developers/contributing.html#how-to-contribute-patches)
   


-- 
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] yevgenypats commented on a diff in pull request #34986: GH-34657: [Go] Add ValueString(i int) string to array

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


##########
go/arrow/array/binary.go:
##########
@@ -56,9 +57,17 @@ func (a *Binary) Value(i int) []byte {
 	return a.valueBytes[a.valueOffsets[idx]:a.valueOffsets[idx+1]]
 }
 
-// ValueString returns the string at index i without performing additional allocations.
-// The string is only valid for the lifetime of the Binary array.
+// ValueString returns the string at index i
 func (a *Binary) ValueString(i int) string {
+	if a.IsNull(i) {
+		return NullValueStr
+	}
+	return base64.StdEncoding.EncodeToString(a.Value(i))
+}
+
+// ValueStr returns the string at index i without performing additional allocations.
+// The string is only valid for the lifetime of the Binary array.
+func (a *Binary) ValueStr(i int) string {

Review Comment:
   I've updated. Not sure why the tests now fail though in the CI



-- 
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] yevgenypats commented on a diff in pull request #34986: GH-34657: [Go] Add ValueString(i int) string to array

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


##########
go/arrow/array/binary.go:
##########
@@ -192,6 +201,12 @@ func (a *LargeBinary) Value(i int) []byte {
 }
 
 func (a *LargeBinary) ValueString(i int) string {
+	if a.IsNull(i) {
+		return NullValueStr
+	}
+	return base64.StdEncoding.EncodeToString(a.Value(i))

Review Comment:
   Yeah, seems I was missing one commit from the previous review. fixed 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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] yevgenypats commented on a diff in pull request #34986: GH-34657: [Go] Add ValueString(i int) string to array

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


##########
go/arrow/array/binarybuilder_test.go:
##########
@@ -50,11 +51,16 @@ func TestBinaryBuilder(t *testing.T) {
 		}
 		assert.Equal(t, v, ab.Value(i), "unexpected BinaryArrayBuilder.Value(%d)", i)
 	}
+	// Zm9v is foo in base64
+	assert.NoError(t, ab.AppendValueFromString("Zm9v"))
 
 	ar := ab.NewBinaryArray()
+	assert.Equal(t, "Zm9v", ar.ValueString(5))
+
 	ab.Release()
 	ar.Release()
 
+

Review Comment:
   Sure. BTW - maybe we should open an issue on adding a linter that runs fmt on all files as a pre-set ? this way it wont need review as I'll be able to just see in the CI. WDYT ? :) 



-- 
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 #34986: GH-34657: [Go] Add ValueString(i int) string to array

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


##########
go/arrow/array/encoded.go:
##########
@@ -192,6 +193,15 @@ func (r *RunEndEncoded) GetPhysicalLength() int {
 	return encoded.GetPhysicalLength(r.data)
 }
 
+func (r *RunEndEncoded) ValueStr(i int) string {

Review Comment:
   I think that's actually the difference here between `GetOneForMarshal` and `ValueStr`. `GetOneForMarshal` would return the whole run-mapping etc. While I think `ValueStr` should just return the actual "value" for the given "logical" index, so that a consumer doesn't need to do that themselves if they are actually trying to retrieve the values from the run-end-encoded array. It also maintains consistency with the Dictionary encoded arrays, where `ValueStr` returns the actual dictionary value, not the dictionary index. Does that make sense?
   
   We should also explicitly document this in a doc string on the function.



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