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

[GitHub] [arrow] zeroshade commented on a diff in pull request #35457: GH-35421: [Go] Ensure interface contract between `array.X.ValueStr` & `array.XBuilder.AppendValueFromString`

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


##########
go/arrow/array/dictionary.go:
##########
@@ -742,11 +745,29 @@ func (b *dictionaryBuilder) Unmarshal(dec *json.Decoder) error {
 }
 
 func (b *dictionaryBuilder) AppendValueFromString(s string) error {

Review Comment:
   you should update the comment on the `UnmarshalJSON` method to indicate that it will no longer error.



##########
go/arrow/array/encoded.go:
##########
@@ -280,7 +280,10 @@ type RunEndEncodedBuilder struct {
 	values    Builder
 	maxRunEnd uint64
 
+	// currently, mixing AppendValueFromString & UnmarshalOne is unsupported

Review Comment:
   add this to the docstring of both AppendValueFromString and UnmarshalOne please.



##########
go/arrow/array/numericbuilder.gen.go.tmpl:
##########
@@ -203,6 +203,21 @@ func (b *{{.Name}}Builder) AppendValueFromString(s string) error {
       return err
     }
     b.Append(arrow.Date64FromTime(tm))  
+  {{else if or (eq .Name "Dictionary") -}}
+  	if d.IsNull(i) {
+  		return NullValueStr
+  	}
+  	return d.Dictionary().ValueStr(d.GetValueIndex(i))

Review Comment:
   this return can't be in the right spot... either that or everything below this is wrong



##########
go/arrow/array/numericbuilder.gen.go:
##########
@@ -3534,7 +3534,29 @@ func (b *DurationBuilder) AppendValueFromString(s string) error {
 		b.AppendNull()
 		return nil
 	}
-	return fmt.Errorf("%w: AppendValueFromString not implemented for Duration", arrow.ErrNotImplemented)
+	var value arrow.Duration
+	var unitStr string
+	n, err := fmt.Sscanf(s, "%d%s", &value, &unitStr)
+	if err != nil {
+		b.AppendNull()
+		return err
+	}
+	if n != 2 {
+		b.AppendNull()
+		return fmt.Errorf("failed to parse Duration from %q", s)
+	}
+
+	// to ns & back
+	unit, err := arrow.TimeUnitFromString(unitStr)
+	if err != nil {
+		b.AppendNull()
+		return err
+	}
+
+	value *= arrow.Duration(unit.Multiplier())
+	value /= arrow.Duration(b.dtype.Unit.Multiplier())
+
+	b.Append(value)

Review Comment:
   can you just use `https://pkg.go.dev/time#ParseDuration` ?



##########
go/arrow/array/union.go:
##########
@@ -1110,10 +1133,15 @@ func NewEmptyDenseUnionBuilder(mem memory.Allocator) *DenseUnionBuilder {
 // children and type codes. Builders will be constructed for each child
 // using the fields in typ
 func NewDenseUnionBuilder(mem memory.Allocator, typ *arrow.DenseUnionType) *DenseUnionBuilder {
-	children := make([]Builder, len(typ.Fields()))
-	for i, f := range typ.Fields() {
-		children[i] = NewBuilder(mem, f.Type)
-		defer children[i].Release()
+	children := make([]Builder, 0, len(typ.Fields()))
+	defer func() {
+		for _, child := range children {
+			child.Release()
+		}
+	}()

Review Comment:
   is this just to condense everything into a single defer rather than multiple defers?



##########
go/internal/types/extension_types.go:
##########
@@ -82,14 +105,6 @@ func (b *UUIDBuilder) UnmarshalOne(dec *json.Decoder) error {
 		}
 	}
 
-	if len(val) != 16 {
-		return &json.UnmarshalTypeError{
-			Value:  fmt.Sprint(val),
-			Type:   reflect.TypeOf([]byte{}),
-			Offset: dec.InputOffset(),
-			Struct: fmt.Sprintf("FixedSizeBinary[%d]", 16),
-		}
-	}

Review Comment:
   Why remove this?



##########
go/arrow/array/array.go:
##########
@@ -27,7 +27,7 @@ import (
 const (
 	// UnknownNullCount specifies the NullN should be calculated from the null bitmap buffer.
 	UnknownNullCount = -1
-	NullValueStr = "(null)"
+	NullValueStr     = "(null)"

Review Comment:
   Can you add a docstring here since now this is getting used as the official string expected for appending a null if you are using `AppendValueFromString`, so we should give this a docstring for the documentation to explain this.



##########
go/internal/types/extension_types.go:
##########
@@ -208,37 +199,49 @@ type UUIDType struct {
 // NewUUIDType is a convenience function to create an instance of UuidType
 // with the correct storage type
 func NewUUIDType() *UUIDType {
-	return &UUIDType{
-		ExtensionBase: arrow.ExtensionBase{
-			Storage: &arrow.FixedSizeBinaryType{ByteWidth: 16}}}
+	return &UUIDType{ExtensionBase: arrow.ExtensionBase{Storage: &arrow.FixedSizeBinaryType{ByteWidth: 16}}}
 }
 
-// ArrayType returns TypeOf(UuidArray) for constructing uuid arrays
-func (UUIDType) ArrayType() reflect.Type { return reflect.TypeOf(UUIDArray{}) }
+// ArrayType returns TypeOf(UUIDArray{}) for constructing UUID arrays
+func (*UUIDType) ArrayType() reflect.Type {
+	return reflect.TypeOf(UUIDArray{})
+}
+
+func (*UUIDType) ExtensionName() string {
+	return "uuid"
+}
+
+func (e *UUIDType) String() string {
+	return fmt.Sprintf("extension_type<storage=%s>", e.Storage)
+}
 
-func (UUIDType) ExtensionName() string { return "uuid" }
+func (e *UUIDType) MarshalJSON() ([]byte, error) {
+	return []byte(fmt.Sprintf(`{"name":"%s","metadata":%s}`, e.ExtensionName(), e.Serialize())), nil
+}
 
 // Serialize returns "uuid-serialized" for testing proper metadata passing
-func (UUIDType) Serialize() string { return "uuid-serialized" }
+func (*UUIDType) Serialize() string {
+	return "uuid-serialized"
+}
 
 // Deserialize expects storageType to be FixedSizeBinaryType{ByteWidth: 16} and the data to be
-// "uuid-serialized" in order to correctly create a UUIDType for testing deserialize.
-func (UUIDType) Deserialize(storageType arrow.DataType, data string) (arrow.ExtensionType, error) {
-	if string(data) != "uuid-serialized" {
-		return nil, fmt.Errorf("type identifier did not match: '%s'", string(data))
+// "uuid-serialized" in order to correctly create a UuidType for testing deserialize.
+func (*UUIDType) Deserialize(storageType arrow.DataType, data string) (arrow.ExtensionType, error) {
+	if data != "uuid-serialized" {
+		return nil, fmt.Errorf("type identifier did not match: '%s'", data)
 	}
 	if !arrow.TypeEqual(storageType, &arrow.FixedSizeBinaryType{ByteWidth: 16}) {
-		return nil, fmt.Errorf("invalid storage type for UUIDType: %s", storageType.Name())
+		return nil, fmt.Errorf("invalid storage type for UuidType: %s", storageType.Name())
 	}
 	return NewUUIDType(), nil
 }
 
-// UuidTypes are equal if both are named "uuid"
-func (u UUIDType) ExtensionEquals(other arrow.ExtensionType) bool {
-	return u.ExtensionName() == other.ExtensionName()
+// ExtensionEquals returns true if both extensions have the same name
+func (e *UUIDType) ExtensionEquals(other arrow.ExtensionType) bool {

Review Comment:
   again, is this change in the receiver just for consistency? or was there actually an issue?



##########
go/arrow/array/numericbuilder.gen.go.tmpl:
##########
@@ -203,6 +203,21 @@ func (b *{{.Name}}Builder) AppendValueFromString(s string) error {
       return err
     }
     b.Append(arrow.Date64FromTime(tm))  
+  {{else if or (eq .Name "Dictionary") -}}

Review Comment:
   no need for the `or` here



##########
go/arrow/array/numericbuilder.gen.go.tmpl:
##########
@@ -203,6 +203,21 @@ func (b *{{.Name}}Builder) AppendValueFromString(s string) error {
       return err
     }
     b.Append(arrow.Date64FromTime(tm))  
+  {{else if or (eq .Name "Dictionary") -}}

Review Comment:
   why don't I see these changes in the `numericbuilder.gen.go` file? do you need to re-run go generate? Also, I don't think Dictionary goes through this code path at all? Or am I wrong?



##########
.gitignore:
##########
@@ -91,4 +91,6 @@ java-native-cpp/
 # archery files
 dev/archery/build
 
-swift/Arrow/.build
\ No newline at end of file
+swift/Arrow/.build
+
+go/vendor

Review Comment:
   why the addition here?



##########
go/internal/types/extension_types.go:
##########
@@ -32,26 +32,45 @@ import (
 	"golang.org/x/xerrors"
 )
 
+var UUID = NewUUIDType()
+
 type UUIDBuilder struct {
 	*array.ExtensionBuilder
 }
 
-func NewUUIDBuilder(bldr *array.ExtensionBuilder) *UUIDBuilder {
-	b := &UUIDBuilder{
-		ExtensionBuilder: bldr,
-	}
-	return b
+func NewUUIDBuilder(builder *array.ExtensionBuilder) *UUIDBuilder {
+	return &UUIDBuilder{ExtensionBuilder: builder}
 }
 
 func (b *UUIDBuilder) Append(v uuid.UUID) {
 	b.ExtensionBuilder.Builder.(*array.FixedSizeBinaryBuilder).Append(v[:])
 }
 
+func (b *UUIDBuilder) UnsafeAppend(v uuid.UUID) {
+	b.ExtensionBuilder.Builder.(*array.FixedSizeBinaryBuilder).UnsafeAppend(v[:])
+}
+
+func (b *UUIDBuilder) AppendValueFromString(s string) error {
+	if s == array.NullValueStr {
+		b.AppendNull()
+		return nil
+	}
+
+	b.Append(uuid.MustParse(s))

Review Comment:
   this should use `uuid.Parse` and return the `err` from it rather than panic'ing by using `MustParse`



##########
go/arrow/example_test.go:
##########
@@ -252,7 +252,7 @@ func Example_fixedSizeListArray() {
 	// NullN()   = 2
 	// Len()     = 5
 	// Type()    = fixed_size_list<item: int64>[3]
-	// List      = [[0 1 2] (null) [3 4 5] [6 7 8] (null)]
+	// List      = [[0 1 2] (null) [-1 -1 -1] [3 4 5] (null)]

Review Comment:
   Ah, I see what's going on here. The test should be modified and have the appending of -1's removed to maintain the same output. But should be shifted into a separate change/PR with the change to FixedSizeListBuilder.AppendNull which we can mark as a breaking change.



##########
go/arrow/array/fixed_size_list.go:
##########
@@ -207,6 +207,10 @@ func (b *FixedSizeListBuilder) Append(v bool) {
 func (b *FixedSizeListBuilder) AppendNull() {
 	b.Reserve(1)
 	b.unsafeAppendBoolToBitmap(false)
+	// require to append this due to value indexes
+	for i := int32(0); i < b.n; i++ {
+		b.values.AppendNull()
+	}

Review Comment:
   This is a breaking change, though I can agree with the sentiment in doing this since it'll make it easier to use FixedSizeList builders to avoid this gotcha. 
   
   Can this be done as a separate PR/change which we can mark as a breaking change?



##########
go/internal/types/extension_types.go:
##########
@@ -111,43 +126,18 @@ func (b *UUIDBuilder) UnmarshalJSON(data []byte) error {
 	}
 
 	if delim, ok := t.(json.Delim); !ok || delim != '[' {
-		return fmt.Errorf("fixed size binary builder must unpack from json array, found %s", delim)
+		return fmt.Errorf("uuid builder must unpack from json array, found %s", delim)
 	}
 
 	return b.Unmarshal(dec)
 }
 
-func (b *UUIDBuilder) AppendValueFromString(s string) error {
-	if s == array.NullValueStr {
-		b.AppendNull()
-		return nil
-	}
-	u, err := uuid.Parse(s)
-	if err != nil {
-		return fmt.Errorf("%w: invalid uuid: %v", arrow.ErrInvalid, err)
-	}
-	b.Append(u)
-	return nil
-}

Review Comment:
   why are we shifting this around and replacing it with something that doesn't return the error?



##########
go/internal/types/extension_types.go:
##########
@@ -208,37 +199,49 @@ type UUIDType struct {
 // NewUUIDType is a convenience function to create an instance of UuidType
 // with the correct storage type
 func NewUUIDType() *UUIDType {
-	return &UUIDType{
-		ExtensionBase: arrow.ExtensionBase{
-			Storage: &arrow.FixedSizeBinaryType{ByteWidth: 16}}}
+	return &UUIDType{ExtensionBase: arrow.ExtensionBase{Storage: &arrow.FixedSizeBinaryType{ByteWidth: 16}}}
 }
 
-// ArrayType returns TypeOf(UuidArray) for constructing uuid arrays
-func (UUIDType) ArrayType() reflect.Type { return reflect.TypeOf(UUIDArray{}) }
+// ArrayType returns TypeOf(UUIDArray{}) for constructing UUID arrays
+func (*UUIDType) ArrayType() reflect.Type {
+	return reflect.TypeOf(UUIDArray{})
+}
+
+func (*UUIDType) ExtensionName() string {
+	return "uuid"
+}
+
+func (e *UUIDType) String() string {
+	return fmt.Sprintf("extension_type<storage=%s>", e.Storage)
+}
 
-func (UUIDType) ExtensionName() string { return "uuid" }
+func (e *UUIDType) MarshalJSON() ([]byte, error) {
+	return []byte(fmt.Sprintf(`{"name":"%s","metadata":%s}`, e.ExtensionName(), e.Serialize())), nil
+}
 
 // Serialize returns "uuid-serialized" for testing proper metadata passing
-func (UUIDType) Serialize() string { return "uuid-serialized" }
+func (*UUIDType) Serialize() string {

Review Comment:
   is this change just for consistency of the receiver? or was there actually an issue?



##########
go/internal/types/extension_types.go:
##########
@@ -157,46 +147,47 @@ func (a UUIDArray) String() string {
 		}
 		switch {
 		case a.IsNull(i):
-			o.WriteString("(null)")
+			o.WriteString(array.NullValueStr)
 		default:
-			uuidStr, err := uuid.FromBytes(arr.Value(i))
-			if err != nil {
-				panic(fmt.Errorf("invalid uuid: %w", err))
-			}
-			fmt.Fprintf(o, "%q", uuidStr)
+			fmt.Fprintf(o, "%q", a.Value(i))
 		}
 	}
 	o.WriteString("]")
 	return o.String()
 }
 
+func (a *UUIDArray) Value(i int) uuid.UUID {
+	if a.IsNull(i) {
+		return uuid.Nil
+	}
+	return uuid.Must(uuid.FromBytes(a.Storage().(*array.FixedSizeBinary).Value(i)))
+}
+
+func (a *UUIDArray) ValueStr(i int) string {
+	switch {
+	case a.IsNull(i):
+		return array.NullValueStr
+	default:
+		return a.Value(i).String()
+	}
+}
+
 func (a *UUIDArray) MarshalJSON() ([]byte, error) {
 	arr := a.Storage().(*array.FixedSizeBinary)
-	vals := make([]interface{}, a.Len())
+	values := make([]interface{}, a.Len())
 	for i := 0; i < a.Len(); i++ {
 		if a.IsValid(i) {
-			uuidStr, err := uuid.FromBytes(arr.Value(i))
-			if err != nil {
-				panic(fmt.Errorf("invalid uuid: %w", err))
-			}
-			vals[i] = uuidStr
-		} else {
-			vals[i] = nil
+			values[i] = uuid.Must(uuid.FromBytes(arr.Value(i))).String()
 		}
 	}
-	return json.Marshal(vals)
+	return json.Marshal(values)
 }
 
 func (a *UUIDArray) GetOneForMarshal(i int) interface{} {
-	arr := a.Storage().(*array.FixedSizeBinary)
-	if a.IsValid(i) {
-		uuidObj, err := uuid.FromBytes(arr.Value(i))
-		if err != nil {
-			panic(fmt.Errorf("invalid uuid: %w", err))
-		}
-		return uuidObj
+	if a.IsNull(i) {
+		return nil
 	}
-	return nil
+	return uuid.Must(uuid.FromBytes(a.Storage().(*array.FixedSizeBinary).Value(i)))

Review Comment:
   couldn't this just be `a.Value(i)`?



##########
go/internal/types/extension_types.go:
##########
@@ -208,37 +199,49 @@ type UUIDType struct {
 // NewUUIDType is a convenience function to create an instance of UuidType
 // with the correct storage type
 func NewUUIDType() *UUIDType {
-	return &UUIDType{
-		ExtensionBase: arrow.ExtensionBase{
-			Storage: &arrow.FixedSizeBinaryType{ByteWidth: 16}}}
+	return &UUIDType{ExtensionBase: arrow.ExtensionBase{Storage: &arrow.FixedSizeBinaryType{ByteWidth: 16}}}
 }
 
-// ArrayType returns TypeOf(UuidArray) for constructing uuid arrays
-func (UUIDType) ArrayType() reflect.Type { return reflect.TypeOf(UUIDArray{}) }
+// ArrayType returns TypeOf(UUIDArray{}) for constructing UUID arrays
+func (*UUIDType) ArrayType() reflect.Type {
+	return reflect.TypeOf(UUIDArray{})
+}
+
+func (*UUIDType) ExtensionName() string {
+	return "uuid"
+}
+
+func (e *UUIDType) String() string {
+	return fmt.Sprintf("extension_type<storage=%s>", e.Storage)
+}
 
-func (UUIDType) ExtensionName() string { return "uuid" }
+func (e *UUIDType) MarshalJSON() ([]byte, error) {
+	return []byte(fmt.Sprintf(`{"name":"%s","metadata":%s}`, e.ExtensionName(), e.Serialize())), nil
+}
 
 // Serialize returns "uuid-serialized" for testing proper metadata passing
-func (UUIDType) Serialize() string { return "uuid-serialized" }
+func (*UUIDType) Serialize() string {
+	return "uuid-serialized"
+}
 
 // Deserialize expects storageType to be FixedSizeBinaryType{ByteWidth: 16} and the data to be
-// "uuid-serialized" in order to correctly create a UUIDType for testing deserialize.
-func (UUIDType) Deserialize(storageType arrow.DataType, data string) (arrow.ExtensionType, error) {
-	if string(data) != "uuid-serialized" {
-		return nil, fmt.Errorf("type identifier did not match: '%s'", string(data))
+// "uuid-serialized" in order to correctly create a UuidType for testing deserialize.
+func (*UUIDType) Deserialize(storageType arrow.DataType, data string) (arrow.ExtensionType, error) {
+	if data != "uuid-serialized" {
+		return nil, fmt.Errorf("type identifier did not match: '%s'", data)
 	}
 	if !arrow.TypeEqual(storageType, &arrow.FixedSizeBinaryType{ByteWidth: 16}) {
-		return nil, fmt.Errorf("invalid storage type for UUIDType: %s", storageType.Name())
+		return nil, fmt.Errorf("invalid storage type for UuidType: %s", storageType.Name())

Review Comment:
   this change is incorrect.



##########
go/internal/types/extension_types.go:
##########
@@ -208,37 +199,49 @@ type UUIDType struct {
 // NewUUIDType is a convenience function to create an instance of UuidType
 // with the correct storage type
 func NewUUIDType() *UUIDType {
-	return &UUIDType{
-		ExtensionBase: arrow.ExtensionBase{
-			Storage: &arrow.FixedSizeBinaryType{ByteWidth: 16}}}
+	return &UUIDType{ExtensionBase: arrow.ExtensionBase{Storage: &arrow.FixedSizeBinaryType{ByteWidth: 16}}}
 }
 
-// ArrayType returns TypeOf(UuidArray) for constructing uuid arrays
-func (UUIDType) ArrayType() reflect.Type { return reflect.TypeOf(UUIDArray{}) }
+// ArrayType returns TypeOf(UUIDArray{}) for constructing UUID arrays
+func (*UUIDType) ArrayType() reflect.Type {
+	return reflect.TypeOf(UUIDArray{})
+}
+
+func (*UUIDType) ExtensionName() string {
+	return "uuid"
+}
+
+func (e *UUIDType) String() string {
+	return fmt.Sprintf("extension_type<storage=%s>", e.Storage)
+}
 
-func (UUIDType) ExtensionName() string { return "uuid" }
+func (e *UUIDType) MarshalJSON() ([]byte, error) {
+	return []byte(fmt.Sprintf(`{"name":"%s","metadata":%s}`, e.ExtensionName(), e.Serialize())), nil
+}
 
 // Serialize returns "uuid-serialized" for testing proper metadata passing
-func (UUIDType) Serialize() string { return "uuid-serialized" }
+func (*UUIDType) Serialize() string {
+	return "uuid-serialized"
+}
 
 // Deserialize expects storageType to be FixedSizeBinaryType{ByteWidth: 16} and the data to be
-// "uuid-serialized" in order to correctly create a UUIDType for testing deserialize.
-func (UUIDType) Deserialize(storageType arrow.DataType, data string) (arrow.ExtensionType, error) {
-	if string(data) != "uuid-serialized" {
-		return nil, fmt.Errorf("type identifier did not match: '%s'", string(data))
+// "uuid-serialized" in order to correctly create a UuidType for testing deserialize.

Review Comment:
   this correction is incorrect, the type is called `UUIDType` and the comment should reflect that.



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