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

import (
	
	
	
	

	
)

type (
	NestedType interface {
		DataType

		// Fields method provides a copy of NestedType fields
		// (so it can be safely mutated and will not result in updating the NestedType).
		Fields() []Field
		// NumFields provides the number of fields without allocating.
		NumFields() int
	}

	ListLikeType interface {
		DataType
		Elem() DataType
		ElemField() Field
	}

	VarLenListLikeType interface {
		ListLikeType
	}
)

// ListType describes a nested type in which each array slot contains
// a variable-size sequence of values, all having the same relative type.
type ListType struct {
	elem Field
}

func ( Field) *ListType {
	if .Type == nil {
		panic("arrow: nil type for list field")
	}
	return &ListType{elem: }
}

// ListOf returns the list type with element type t.
// For example, if t represents int32, ListOf(t) represents []int32.
//
// ListOf panics if t is nil or invalid. NullableElem defaults to true
func ( DataType) *ListType {
	if  == nil {
		panic("arrow: nil DataType")
	}
	return &ListType{elem: Field{Name: "item", Type: , Nullable: true}}
}

// ListOfNonNullable is like ListOf but NullableElem defaults to false, indicating
// that the child type should be marked as non-nullable.
func ( DataType) *ListType {
	if  == nil {
		panic("arrow: nil DataType")
	}
	return &ListType{elem: Field{Name: "item", Type: , Nullable: false}}
}

func (*ListType) () Type     { return LIST }
func (*ListType) () string { return "list" }

func ( *ListType) () string {
	if .elem.Nullable {
		return fmt.Sprintf("list<%s: %s, nullable>", .elem.Name, .elem.Type)
	}
	return fmt.Sprintf("list<%s: %s>", .elem.Name, .elem.Type)
}

func ( *ListType) () string {
	 := .elem.Type.Fingerprint()
	if len() > 0 {
		return typeFingerprint() + "{" +  + "}"
	}
	return ""
}

func ( *ListType) ( Metadata) { .elem.Metadata =  }

func ( *ListType) ( bool) { .elem.Nullable =  }

// Elem returns the ListType's element type.
func ( *ListType) () DataType { return .elem.Type }

func ( *ListType) () Field {
	return .elem
}

func ( *ListType) () []Field { return []Field{.ElemField()} }

func ( *ListType) () int { return 1 }

func (*ListType) () DataTypeLayout {
	return DataTypeLayout{Buffers: []BufferSpec{SpecBitmap(), SpecFixedWidth(Int32SizeBytes)}}
}

func (*ListType) () OffsetTraits { return Int32Traits }

type LargeListType struct {
	ListType
}

func (LargeListType) () Type     { return LARGE_LIST }
func (LargeListType) () string { return "large_list" }
func ( *LargeListType) () string {
	return "large_" + .ListType.String()
}

func ( *LargeListType) () string {
	 := .elem.Type.Fingerprint()
	if len() > 0 {
		return typeFingerprint() + "{" +  + "}"
	}
	return ""
}

func (*LargeListType) () DataTypeLayout {
	return DataTypeLayout{Buffers: []BufferSpec{SpecBitmap(), SpecFixedWidth(Int64SizeBytes)}}
}

func (*LargeListType) () OffsetTraits { return Int64Traits }

func ( Field) *LargeListType {
	if .Type == nil {
		panic("arrow: nil type for list field")
	}
	return &LargeListType{ListType{elem: }}
}

// LargeListOf returns the list type with element type t.
// For example, if t represents int32, LargeListOf(t) represents []int32.
//
// LargeListOf panics if t is nil or invalid. NullableElem defaults to true
func ( DataType) *LargeListType {
	if  == nil {
		panic("arrow: nil DataType")
	}
	return &LargeListType{ListType{elem: Field{Name: "item", Type: , Nullable: true}}}
}

// LargeListOfNonNullable is like ListOf but NullableElem defaults to false, indicating
// that the child type should be marked as non-nullable.
func ( DataType) *LargeListType {
	if  == nil {
		panic("arrow: nil DataType")
	}
	return &LargeListType{ListType{elem: Field{Name: "item", Type: , Nullable: false}}}
}

// FixedSizeListType describes a nested type in which each array slot contains
// a fixed-size sequence of values, all having the same relative type.
type FixedSizeListType struct {
	n    int32 // number of elements in the list
	elem Field
}

func ( int32,  Field) *FixedSizeListType {
	if .Type == nil {
		panic("arrow: nil DataType")
	}
	if  <= 0 {
		panic("arrow: invalid size")
	}
	return &FixedSizeListType{n: , elem: }
}

// FixedSizeListOf returns the list type with element type t.
// For example, if t represents int32, FixedSizeListOf(10, t) represents [10]int32.
//
// FixedSizeListOf panics if t is nil or invalid.
// FixedSizeListOf panics if n is <= 0.
// NullableElem defaults to true
func ( int32,  DataType) *FixedSizeListType {
	if  == nil {
		panic("arrow: nil DataType")
	}
	if  <= 0 {
		panic("arrow: invalid size")
	}
	return &FixedSizeListType{n: , elem: Field{Name: "item", Type: , Nullable: true}}
}

// FixedSizeListOfNonNullable is like FixedSizeListOf but NullableElem defaults to false
// indicating that the child type should be marked as non-nullable.
func ( int32,  DataType) *FixedSizeListType {
	if  == nil {
		panic("arrow: nil DataType")
	}
	if  <= 0 {
		panic("arrow: invalid size")
	}
	return &FixedSizeListType{n: , elem: Field{Name: "item", Type: , Nullable: false}}
}

func (*FixedSizeListType) () Type     { return FIXED_SIZE_LIST }
func (*FixedSizeListType) () string { return "fixed_size_list" }
func ( *FixedSizeListType) () string {
	if .elem.Nullable {
		return fmt.Sprintf("fixed_size_list<%s: %s, nullable>[%d]", .elem.Name, .elem.Type, .n)
	}
	return fmt.Sprintf("fixed_size_list<%s: %s>[%d]", .elem.Name, .elem.Type, .n)
}

func ( *FixedSizeListType) ( bool) { .elem.Nullable =  }

// Elem returns the FixedSizeListType's element type.
func ( *FixedSizeListType) () DataType { return .elem.Type }

// Len returns the FixedSizeListType's size.
func ( *FixedSizeListType) () int32 { return .n }

func ( *FixedSizeListType) () Field {
	return .elem
}

func ( *FixedSizeListType) () string {
	 := .elem.Type.Fingerprint()
	if len() > 0 {
		return fmt.Sprintf("%s[%d]{%s}", typeFingerprint(), .n, )
	}
	return ""
}

func ( *FixedSizeListType) () []Field { return []Field{.ElemField()} }

func ( *FixedSizeListType) () int { return 1 }

func (*FixedSizeListType) () DataTypeLayout {
	return DataTypeLayout{Buffers: []BufferSpec{SpecBitmap()}}
}

type ListViewType struct {
	elem Field
}

func ( Field) *ListViewType {
	if .Type == nil {
		panic("arrow: nil DataType")
	}
	return &ListViewType{elem: }
}

// ListViewOf returns the list-view type with element type t.
// For example, if t represents int32, ListViewOf(t) represents []int32.
//
// ListViewOf panics if t is nil or invalid. NullableElem defaults to true
func ( DataType) *ListViewType {
	if  == nil {
		panic("arrow: nil DataType")
	}
	return &ListViewType{elem: Field{Name: "item", Type: , Nullable: true}}
}

// ListViewOfNonNullable is like ListViewOf but NullableElem defaults to false, indicating
// that the child type should be marked as non-nullable.
func ( DataType) *ListViewType {
	if  == nil {
		panic("arrow: nil DataType")
	}
	return &ListViewType{elem: Field{Name: "item", Type: , Nullable: false}}
}

func (*ListViewType) () Type     { return LIST_VIEW }
func (*ListViewType) () string { return "list_view" }

func ( *ListViewType) () string {
	if .elem.Nullable {
		return fmt.Sprintf("list_view<%s: %s, nullable>", .elem.Name, .elem.Type)
	}
	return fmt.Sprintf("list_view<%s: %s>", .elem.Name, .elem.Type)
}

func ( *ListViewType) () string {
	 := .elem.Type.Fingerprint()
	if len() > 0 {
		return typeFingerprint() + "{" +  + "}"
	}
	return ""
}

func ( *ListViewType) ( Metadata) { .elem.Metadata =  }

func ( *ListViewType) ( bool) { .elem.Nullable =  }

// Elem returns the ListViewType's element type.
func ( *ListViewType) () DataType { return .elem.Type }

func ( *ListViewType) () Field {
	return .elem
}

func ( *ListViewType) () []Field { return []Field{.ElemField()} }

func ( *ListViewType) () int { return 1 }

func (*ListViewType) () DataTypeLayout {
	return DataTypeLayout{Buffers: []BufferSpec{SpecBitmap(), SpecFixedWidth(Int32SizeBytes), SpecFixedWidth(Int32SizeBytes)}}
}

func (*ListViewType) () OffsetTraits { return Int32Traits }

type LargeListViewType struct {
	elem Field
}

func ( Field) *LargeListViewType {
	if .Type == nil {
		panic("arrow: nil DataType")
	}
	return &LargeListViewType{elem: }
}

// LargeListViewOf returns the list-view type with element type t.
// For example, if t represents int32, LargeListViewOf(t) represents []int32.
//
// LargeListViewOf panics if t is nil or invalid. NullableElem defaults to true
func ( DataType) *LargeListViewType {
	if  == nil {
		panic("arrow: nil DataType")
	}
	return &LargeListViewType{elem: Field{Name: "item", Type: , Nullable: true}}
}

// LargeListViewOfNonNullable is like LargeListViewOf but NullableElem defaults
// to false, indicating that the child type should be marked as non-nullable.
func ( DataType) *LargeListViewType {
	if  == nil {
		panic("arrow: nil DataType")
	}
	return &LargeListViewType{elem: Field{Name: "item", Type: , Nullable: false}}
}

func (*LargeListViewType) () Type     { return LARGE_LIST_VIEW }
func (*LargeListViewType) () string { return "large_list_view" }

func ( *LargeListViewType) () string {
	if .elem.Nullable {
		return fmt.Sprintf("large_list_view<%s: %s, nullable>", .elem.Name, .elem.Type)
	}
	return fmt.Sprintf("large_list_view<%s: %s>", .elem.Name, .elem.Type)
}

func ( *LargeListViewType) () string {
	 := .elem.Type.Fingerprint()
	if len() > 0 {
		return typeFingerprint() + "{" +  + "}"
	}
	return ""
}

func ( *LargeListViewType) ( Metadata) { .elem.Metadata =  }

func ( *LargeListViewType) ( bool) { .elem.Nullable =  }

// Elem returns the LargeListViewType's element type.
func ( *LargeListViewType) () DataType { return .elem.Type }

func ( *LargeListViewType) () Field {
	return .elem
}

func ( *LargeListViewType) () []Field { return []Field{.ElemField()} }

func ( *LargeListViewType) () int { return 1 }

func (*LargeListViewType) () DataTypeLayout {
	return DataTypeLayout{Buffers: []BufferSpec{SpecBitmap(), SpecFixedWidth(Int64SizeBytes), SpecFixedWidth(Int64SizeBytes)}}
}

func (*LargeListViewType) () OffsetTraits { return Int64Traits }

// StructType describes a nested type parameterized by an ordered sequence
// of relative types, called its fields.
type StructType struct {
	fields []Field
	index  map[string][]int
	meta   Metadata
}

// StructOf returns the struct type with fields fs.
//
// StructOf panics if there is a field with an invalid DataType.
func ( ...Field) *StructType {
	 := len()
	if  == 0 {
		return &StructType{}
	}

	 := &StructType{
		fields: make([]Field, ),
		index:  make(map[string][]int, ),
	}
	for ,  := range  {
		if .Type == nil {
			panic("arrow: field with nil DataType")
		}
		.fields[] = Field{
			Name:     .Name,
			Type:     .Type,
			Nullable: .Nullable,
			Metadata: .Metadata.clone(),
		}
		if ,  := .index[.Name];  {
			.index[.Name] = append(, )
		} else {
			.index[.Name] = []int{}
		}
	}

	return 
}

func (*StructType) () Type     { return STRUCT }
func (*StructType) () string { return "struct" }

func ( *StructType) () string {
	var  strings.Builder
	.WriteString("struct<")
	for ,  := range .fields {
		if  > 0 {
			.WriteString(", ")
		}
		.WriteString(fmt.Sprintf("%s: %v", .Name, .Type))
	}
	.WriteString(">")
	return .String()
}

// Fields method provides a copy of StructType fields
// (so it can be safely mutated and will not result in updating the StructType).
func ( *StructType) () []Field {
	 := make([]Field, len(.fields))
	copy(, .fields)
	return 
}

func ( *StructType) () int { return len(.fields) }

func ( *StructType) ( int) Field { return .fields[] }

// FieldByName gets the field with the given name.
//
// If there are multiple fields with the given name, FieldByName
// returns the first such field.
func ( *StructType) ( string) (Field, bool) {
	,  := .index[]
	if ! {
		return Field{}, false
	}
	return .fields[[0]], true
}

// FieldIdx gets the index of the field with the given name.
//
// If there are multiple fields with the given name, FieldIdx returns
// the index of the first such field.
func ( *StructType) ( string) (int, bool) {
	,  := .index[]
	if  {
		return [0], true
	}
	return -1, false
}

// FieldsByName returns all fields with the given name.
func ( *StructType) ( string) ([]Field, bool) {
	,  := .index[]
	if ! {
		return nil, 
	}
	 := make([]Field, 0, len())
	for ,  := range  {
		 = append(, .fields[])
	}
	return , 
}

// FieldIndices returns indices of all fields with the given name, or nil.
func ( *StructType) ( string) []int {
	return .index[]
}

func ( *StructType) () string {
	var  strings.Builder
	.WriteString(typeFingerprint())
	.WriteByte('{')
	for ,  := range .fields {
		 := .Fingerprint()
		if len() == 0 {
			return ""
		}
		.WriteString()
		.WriteByte(';')
	}
	.WriteByte('}')
	return .String()
}

func (*StructType) () DataTypeLayout {
	return DataTypeLayout{Buffers: []BufferSpec{SpecBitmap()}}
}

type MapType struct {
	value      *ListType
	KeysSorted bool
}

func (,  DataType) *MapType {
	if  == nil ||  == nil {
		panic("arrow: nil key or item type for MapType")
	}

	return &MapType{value: ListOf(StructOf(Field{Name: "key", Type: }, Field{Name: "value", Type: , Nullable: true}))}
}

func (,  Field) *MapType {
	if .Type == nil || .Type == nil {
		panic("arrow: nil key or item type for MapType")
	}

	if .Nullable {
		panic("arrow: key field must be non-nullable")
	}

	.Name = "key"
	.Name = "value"
	return &MapType{value: ListOfField(Field{
		Name: "entries",
		Type: StructOf(, ),
	})}
}

func ( DataType,  Metadata,  DataType,  Metadata) *MapType {
	if  == nil ||  == nil {
		panic("arrow: nil key or item type for MapType")
	}

	return &MapType{value: ListOf(StructOf(Field{
		Name:     "key",
		Type:     ,
		Metadata: ,
	}, Field{
		Name:     "value",
		Type:     ,
		Nullable: true,
		Metadata: ,
	}))}
}

func (*MapType) () Type     { return MAP }
func (*MapType) () string { return "map" }

func ( *MapType) () string {
	var  strings.Builder
	.WriteString(fmt.Sprintf("map<%s, %s",
		.value.Elem().(*StructType).Field(0).Type,
		.value.Elem().(*StructType).Field(1).Type))
	if .KeysSorted {
		.WriteString(", keys_sorted")
	}
	if .ItemField().Nullable {
		.WriteString(", items_nullable")
	} else {
		.WriteString(", items_non_nullable")
	}
	.WriteString(">")
	return .String()
}

func ( *MapType) () Field    { return .value.Elem().(*StructType).Field(0) }
func ( *MapType) () DataType  { return .KeyField().Type }
func ( *MapType) () Field   { return .value.Elem().(*StructType).Field(1) }
func ( *MapType) () DataType { return .ItemField().Type }

// Deprecated: use MapType.Elem().(*StructType) instead
func ( *MapType) () *StructType { return .Elem().(*StructType) }

// Deprecated: use MapType.ElemField() instead
func ( *MapType) () Field { return .ElemField() }

// Elem returns the MapType's element type (if treating MapType as ListLikeType)
func ( *MapType) () DataType { return .value.Elem() }

// ElemField returns the MapType's element field (if treating MapType as ListLikeType)
func ( *MapType) () Field { return Field{Name: "entries", Type: .Elem()} }

func ( *MapType) ( bool) {
	.value.Elem().(*StructType).fields[1].Nullable = 
}

func ( *MapType) () string {
	 := .KeyType().Fingerprint()
	 := .ItemType().Fingerprint()
	if  == "" ||  == "" {
		return ""
	}

	 := typeFingerprint()
	if .KeysSorted {
		 += "s"
	}
	return  + "{" +  +  + "}"
}

func ( *MapType) () []Field { return []Field{.ElemField()} }

func ( *MapType) () int { return 1 }

func ( *MapType) () DataTypeLayout {
	return .value.Layout()
}

func (*MapType) () OffsetTraits { return Int32Traits }

type (
	// UnionTypeCode is an alias to int8 which is the type of the ids
	// used for union arrays.
	UnionTypeCode = int8
	UnionMode     int8
)

const (
	MaxUnionTypeCode    UnionTypeCode = 127
	InvalidUnionChildID int           = -1

	SparseMode UnionMode = iota // SPARSE
	DenseMode                   // DENSE
)

// UnionType is an interface to encompass both Dense and Sparse Union types.
//
// A UnionType is a nested type where each logical value is taken
// from a single child. A buffer of 8-bit type ids (typed as UnionTypeCode)
// indicates which child a given logical value is to be taken from. This is
// represented as the "child id" or "child index", which is the index into the
// list of child fields for a given child.
type UnionType interface {
	NestedType
	// Mode returns either SparseMode or DenseMode depending on the current
	// concrete data type.
	Mode() UnionMode
	// ChildIDs returns a slice of ints to map UnionTypeCode values to
	// the index in the Fields that represents the given Type. It is
	// initialized with all values being InvalidUnionChildID (-1)
	// before being populated based on the TypeCodes and fields of the type.
	// The field for a given type can be retrieved by Fields()[ChildIDs()[typeCode]]
	ChildIDs() []int
	// TypeCodes returns the list of available type codes for this union type
	// which will correspond to indexes into the ChildIDs slice to locate the
	// appropriate child. A union Array contains a buffer of these type codes
	// which indicate for a given index, which child has the value for that index.
	TypeCodes() []UnionTypeCode
	// MaxTypeCode returns the value of the largest TypeCode in the list of typecodes
	// that are defined by this Union type
	MaxTypeCode() UnionTypeCode
}

// UnionOf returns an appropriate union type for the given Mode (Sparse or Dense),
// child fields, and type codes. len(fields) == len(typeCodes) must be true, or else
// this will panic. len(fields) can be 0.
func ( UnionMode,  []Field,  []UnionTypeCode) UnionType {
	switch  {
	case SparseMode:
		return SparseUnionOf(, )
	case DenseMode:
		return DenseUnionOf(, )
	default:
		panic("arrow: invalid union mode")
	}
}

type unionType struct {
	children  []Field
	typeCodes []UnionTypeCode
	childIDs  [int(MaxUnionTypeCode) + 1]int
}

func ( *unionType) ( []Field,  []UnionTypeCode) {
	// initialize all child IDs to -1
	.childIDs[0] = InvalidUnionChildID
	for  := 1;  < len(.childIDs);  *= 2 {
		copy(.childIDs[:], .childIDs[:])
	}

	.children = 
	.typeCodes = 

	for ,  := range .typeCodes {
		.childIDs[] = 
	}
}

// Fields method provides a copy of union type fields
// (so it can be safely mutated and will not result in updating the union type).
func ( *unionType) () []Field {
	 := make([]Field, len(.children))
	copy(, .children)
	return 
}

func ( *unionType) () int { return len(.children) }

func ( *unionType) () []UnionTypeCode { return .typeCodes }
func ( *unionType) () []int            { return .childIDs[:] }

func ( *unionType) ( []Field,  []UnionTypeCode,  UnionMode) error {
	if len() != len() {
		return errors.New("arrow: union types should have the same number of fields as type codes")
	}

	for ,  := range  {
		if  < 0 ||  > MaxUnionTypeCode {
			return errors.New("arrow: union type code out of bounds")
		}
	}
	return nil
}

func ( *unionType) () ( UnionTypeCode) {
	if len(.typeCodes) == 0 {
		return
	}

	 = .typeCodes[0]
	for ,  := range .typeCodes[1:] {
		if  >  {
			 = 
		}
	}
	return
}

func ( *unionType) () string {
	var  strings.Builder
	.WriteByte('<')
	for  := range .typeCodes {
		if  != 0 {
			.WriteString(", ")
		}
		fmt.Fprintf(&, "%s=%d", .children[], .typeCodes[])
	}
	.WriteByte('>')
	return .String()
}

func ( *unionType) () string {
	var  strings.Builder
	for ,  := range .typeCodes {
		fmt.Fprintf(&, ":%d", )
	}
	.WriteString("]{")
	for ,  := range .children {
		 := .Fingerprint()
		if len() == 0 {
			return ""
		}
		.WriteString()
		.WriteByte(';')
	}
	.WriteByte('}')
	return .String()
}

func fieldsFromArrays( []Array,  ...string) ( []Field) {
	 = make([]Field, len())
	if len() == 0 {
		for ,  := range  {
			[] = Field{Name: strconv.Itoa(), Type: .DataType(), Nullable: true}
		}
	} else {
		debug.Assert(len() == len(), "mismatch of arrays and names")
		for ,  := range  {
			[] = Field{Name: [], Type: .DataType(), Nullable: true}
		}
	}
	return
}

// SparseUnionType is the concrete type for Sparse union data.
//
// A sparse union is a nested type where each logical value is taken
// from a single child. A buffer of 8-bit type ids indicates which child
// a given logical value is to be taken from.
//
// In a sparse union, each child array will have the same length as the
// union array itself, regardless of the actual number of union values which
// refer to it.
//
// Unlike most other types, unions do not have a top-level validity bitmap.
type SparseUnionType struct {
	unionType
}

// SparseUnionFromArrays enables creating a union type from a list of Arrays,
// field names, and type codes. len(fields) should be either 0 or equal to len(children).
// len(codes) should also be either 0, or equal to len(children).
//
// If len(fields) == 0, then the fields will be named numerically as "0", "1", "2"...
// and so on. If len(codes) == 0, then the type codes will be constructed as
// [0, 1, 2, ..., n].
func ( []Array,  []string,  []UnionTypeCode) *SparseUnionType {
	if len() == 0 {
		 = make([]UnionTypeCode, len())
		for  := range  {
			[] = UnionTypeCode()
		}
	}
	return SparseUnionOf(fieldsFromArrays(, ...), )
}

// SparseUnionOf is equivalent to UnionOf(arrow.SparseMode, fields, typeCodes),
// constructing a SparseUnionType from a list of fields and type codes.
//
// If len(fields) != len(typeCodes) this will panic. They are allowed to be
// of length 0.
func ( []Field,  []UnionTypeCode) *SparseUnionType {
	 := &SparseUnionType{}
	if  := .validate(, , .Mode());  != nil {
		panic()
	}
	.init(, )
	return 
}

func (SparseUnionType) () Type        { return SPARSE_UNION }
func (SparseUnionType) () string    { return "sparse_union" }
func (SparseUnionType) () UnionMode { return SparseMode }
func ( *SparseUnionType) () string {
	return typeFingerprint() + "[s" + .fingerprint()
}
func (SparseUnionType) () DataTypeLayout {
	return DataTypeLayout{Buffers: []BufferSpec{SpecFixedWidth(Uint8SizeBytes)}}
}
func ( *SparseUnionType) () string {
	return .Name() + .unionType.String()
}

// DenseUnionType is the concrete type for dense union data.
//
// A dense union is a nested type where each logical value is taken from a
// single child, at a specific offset. A buffer of 8-bit type ids (typed
// as UnionTypeCode) indicates which child a given logical value is to be
// taken from and a buffer of 32-bit offsets indicating which physical position
// in the given child array has the logical value for that index.
//
// Unlike a sparse union, a dense union allows encoding only the child values
// which are actually referred to by the union array. This is counterbalanced
// by the additional footprint of the offsets buffer, and the additional
// indirection cost when looking up values.
//
// Unlike most other types, unions don't have a top-level validity bitmap
type DenseUnionType struct {
	unionType
}

// DenseUnionFromArrays enables creating a union type from a list of Arrays,
// field names, and type codes. len(fields) should be either 0 or equal to len(children).
// len(codes) should also be either 0, or equal to len(children).
//
// If len(fields) == 0, then the fields will be named numerically as "0", "1", "2"...
// and so on. If len(codes) == 0, then the type codes will be constructed as
// [0, 1, 2, ..., n].
func ( []Array,  []string,  []UnionTypeCode) *DenseUnionType {
	if len() == 0 {
		 = make([]UnionTypeCode, len())
		for  := range  {
			[] = UnionTypeCode()
		}
	}
	return DenseUnionOf(fieldsFromArrays(, ...), )
}

// DenseUnionOf is equivalent to UnionOf(arrow.DenseMode, fields, typeCodes),
// constructing a DenseUnionType from a list of fields and type codes.
//
// If len(fields) != len(typeCodes) this will panic. They are allowed to be
// of length 0.
func ( []Field,  []UnionTypeCode) *DenseUnionType {
	 := &DenseUnionType{}
	if  := .validate(, , .Mode());  != nil {
		panic()
	}
	.init(, )
	return 
}

func (DenseUnionType) () Type        { return DENSE_UNION }
func (DenseUnionType) () string    { return "dense_union" }
func (DenseUnionType) () UnionMode { return DenseMode }
func ( *DenseUnionType) () string {
	return typeFingerprint() + "[s" + .fingerprint()
}

func (DenseUnionType) () DataTypeLayout {
	return DataTypeLayout{Buffers: []BufferSpec{SpecFixedWidth(Uint8SizeBytes), SpecFixedWidth(Int32SizeBytes)}}
}

func (DenseUnionType) () OffsetTraits { return Int32Traits }

func ( *DenseUnionType) () string {
	return .Name() + .unionType.String()
}

type Field struct {
	Name     string   // Field name
	Type     DataType // The field's data type
	Nullable bool     // Fields can be nullable
	Metadata Metadata // The field's metadata, if any
}

func ( Field) () string {
	 := .Type.Fingerprint()
	if  == "" {
		return ""
	}

	var  strings.Builder
	.WriteByte('F')
	if .Nullable {
		.WriteByte('n')
	} else {
		.WriteByte('N')
	}
	.WriteString(.Name)
	.WriteByte('{')
	.WriteString()
	.WriteByte('}')
	return .String()
}

func ( Field) () bool { return .Metadata.Len() != 0 }

func ( Field) ( Field) bool {
	switch {
	case .Name != .Name:
		return false
	case .Nullable != .Nullable:
		return false
	case !TypeEqual(.Type, .Type, CheckMetadata()):
		return false
	case !.Metadata.Equal(.Metadata):
		return false
	}
	return true
}

func ( Field) () string {
	var  strings.Builder
	 := ""
	if .Nullable {
		 = ", nullable"
	}
	fmt.Fprintf(&, "%s: type=%v%v", .Name, .Type, )
	if .HasMetadata() {
		fmt.Fprintf(&, "\n%*.smetadata: %v", len(.Name)+2, "", .Metadata)
	}
	return .String()
}

var (
	_ DataType = (*ListType)(nil)
	_ DataType = (*LargeListType)(nil)
	_ DataType = (*FixedSizeListType)(nil)
	_ DataType = (*StructType)(nil)
	_ DataType = (*MapType)(nil)
	_ DataType = (*DenseUnionType)(nil)
	_ DataType = (*SparseUnionType)(nil)

	_ NestedType = (*ListType)(nil)
	_ NestedType = (*LargeListType)(nil)
	_ NestedType = (*FixedSizeListType)(nil)
	_ NestedType = (*MapType)(nil)
	_ NestedType = (*DenseUnionType)(nil)
	_ NestedType = (*SparseUnionType)(nil)

	_ ListLikeType = (*ListType)(nil)
	_ ListLikeType = (*LargeListType)(nil)
	_ ListLikeType = (*FixedSizeListType)(nil)
	_ ListLikeType = (*MapType)(nil)

	_ VarLenListLikeType = (*ListType)(nil)
	_ VarLenListLikeType = (*LargeListType)(nil)
	_ VarLenListLikeType = (*ListViewType)(nil)
	_ VarLenListLikeType = (*LargeListViewType)(nil)
	_ VarLenListLikeType = (*FixedSizeListType)(nil)
	_ VarLenListLikeType = (*MapType)(nil)
)